1use std::collections::HashMap;
16use std::default::Default;
17use std::ops::Bound;
18use std::vec;
19
20use anyhow::anyhow;
21use chrono::{MappedLocalTime, TimeZone};
22use fixedbitset::FixedBitSet;
23use itertools::Itertools;
24use pretty_xmlish::{Pretty, Str, StrAssocArr, XmlNode};
25use risingwave_common::catalog::{
26 ColumnCatalog, ColumnDesc, ConflictBehavior, CreateType, Engine, Field, FieldDisplay, Schema,
27 StreamJobStatus,
28};
29use risingwave_common::constants::log_store::v2::{
30 KV_LOG_STORE_PREDEFINED_COLUMNS, PK_ORDERING, VNODE_COLUMN_INDEX,
31};
32use risingwave_common::hash::VnodeCount;
33use risingwave_common::license::Feature;
34use risingwave_common::types::{DataType, Interval, ScalarImpl, Timestamptz};
35use risingwave_common::util::iter_util::ZipEqFast;
36use risingwave_common::util::scan_range::{ScanRange, is_full_range};
37use risingwave_common::util::sort_util::{ColumnOrder, OrderType};
38use risingwave_connector::source::iceberg::IcebergTimeTravelInfo;
39use risingwave_expr::aggregate::PbAggKind;
40use risingwave_expr::bail;
41use risingwave_sqlparser::ast::AsOf;
42
43use super::generic::{self, GenericPlanRef, PhysicalPlanRef};
44use super::{BatchPlanRef, StreamPlanRef, pretty_config};
45use crate::catalog::table_catalog::TableType;
46use crate::catalog::{ColumnId, FragmentId, TableCatalog, TableId};
47use crate::error::{ErrorCode, Result};
48use crate::expr::InputRef;
49use crate::optimizer::StreamScanType;
50use crate::optimizer::plan_node::generic::Agg;
51use crate::optimizer::plan_node::{BatchSimpleAgg, PlanAggCall};
52use crate::optimizer::property::{Cardinality, Order, RequiredDist, WatermarkColumns};
53use crate::utils::{Condition, IndexSet};
54
55#[derive(Default)]
56pub struct TableCatalogBuilder {
57 columns: Vec<ColumnCatalog>,
59 pk: Vec<ColumnOrder>,
60 value_indices: Option<Vec<usize>>,
61 vnode_col_idx: Option<usize>,
62 column_names: HashMap<String, i32>,
63 watermark_columns: Option<FixedBitSet>,
64 dist_key_in_pk: Option<Vec<usize>>,
65}
66
67impl TableCatalogBuilder {
70 pub fn add_column(&mut self, field: &Field) -> usize {
72 let column_idx = self.columns.len();
73 let column_id = column_idx as i32;
74 let mut column_desc = ColumnDesc::from_field_with_column_id(field, column_id);
76
77 column_desc.name = column_desc.name.replace('.', "_");
79 self.avoid_duplicate_col_name(&mut column_desc);
81
82 self.columns.push(ColumnCatalog {
83 column_desc,
84 is_hidden: false,
86 });
87 column_idx
88 }
89
90 pub fn extend_columns(&mut self, columns: &[ColumnCatalog]) -> Vec<usize> {
94 let base_idx = self.columns.len();
95 columns.iter().enumerate().for_each(|(i, col)| {
96 assert!(!self.column_names.contains_key(col.name()));
97 self.column_names.insert(col.name().to_owned(), 0);
98
99 let mut new_col = col.clone();
101 new_col.column_desc.column_id = ColumnId::new((base_idx + i) as _);
102 self.columns.push(new_col);
103 });
104 Vec::from_iter(base_idx..(base_idx + columns.len()))
105 }
106
107 pub fn add_order_column(&mut self, column_index: usize, order_type: OrderType) {
110 self.pk.push(ColumnOrder::new(column_index, order_type));
111 }
112
113 pub fn get_current_pk_len(&self) -> usize {
115 self.pk.len()
116 }
117
118 pub fn set_vnode_col_idx(&mut self, vnode_col_idx: usize) {
119 self.vnode_col_idx = Some(vnode_col_idx);
120 }
121
122 pub fn set_value_indices(&mut self, value_indices: Vec<usize>) {
123 self.value_indices = Some(value_indices);
124 }
125
126 pub fn set_dist_key_in_pk(&mut self, dist_key_in_pk: Vec<usize>) {
127 self.dist_key_in_pk = Some(dist_key_in_pk);
128 }
129
130 fn avoid_duplicate_col_name(&mut self, column_desc: &mut ColumnDesc) {
133 if let Some(old_identity) = self.column_names.get(&column_desc.name) {
134 let column_name = column_desc.name.clone();
135 let mut identity = *old_identity;
136 loop {
137 column_desc.name = format!("{}_{}", column_name, identity);
138 identity += 1;
139 if !self.column_names.contains_key(&column_desc.name) {
140 break;
141 }
142 }
143 *self.column_names.get_mut(&column_name).unwrap() = identity;
144 }
145 self.column_names.insert(column_desc.name.clone(), 0);
146 }
147
148 pub fn build(self, distribution_key: Vec<usize>, read_prefix_len_hint: usize) -> TableCatalog {
152 assert!(read_prefix_len_hint <= self.pk.len());
153 let watermark_columns = match self.watermark_columns {
154 Some(w) => w,
155 None => FixedBitSet::with_capacity(self.columns.len()),
156 };
157
158 if let Some(dist_key_in_pk) = &self.dist_key_in_pk {
162 let derived_dist_key = dist_key_in_pk
163 .iter()
164 .map(|idx| self.pk[*idx].column_index)
165 .collect_vec();
166 assert_eq!(
167 derived_dist_key, distribution_key,
168 "dist_key mismatch with dist_key_in_pk"
169 );
170 }
171
172 TableCatalog {
173 id: TableId::placeholder(),
174 schema_id: 0.into(),
175 database_id: 0.into(),
176 associated_source_id: None,
177 name: String::new(),
178 columns: self.columns.clone(),
179 pk: self.pk,
180 stream_key: vec![],
181 distribution_key,
182 table_type: TableType::Internal,
185 append_only: false,
186 owner: risingwave_common::catalog::DEFAULT_SUPER_USER_ID,
187 fragment_id: FragmentId::placeholder(),
188 dml_fragment_id: None,
189 vnode_col_index: self.vnode_col_idx,
190 row_id_index: None,
191 value_indices: self
192 .value_indices
193 .unwrap_or_else(|| (0..self.columns.len()).collect_vec()),
194 definition: "".into(),
195 conflict_behavior: ConflictBehavior::NoCheck,
196 version_column_indices: vec![],
197 read_prefix_len_hint,
198 version: None, watermark_columns,
200 dist_key_in_pk: self.dist_key_in_pk.unwrap_or_default(),
201 cardinality: Cardinality::unknown(), created_at_epoch: None,
203 initialized_at_epoch: None,
204 cleaned_by_watermark: false,
205 create_type: CreateType::Foreground,
208 stream_job_status: StreamJobStatus::Creating,
209 description: None,
210 initialized_at_cluster_version: None,
211 created_at_cluster_version: None,
212 retention_seconds: None,
213 cdc_table_id: None,
214 vnode_count: VnodeCount::Placeholder, webhook_info: None,
216 job_id: None,
217 engine: Engine::Hummock,
218 clean_watermark_index_in_pk: None, clean_watermark_indices: vec![], refreshable: false, vector_index_info: None,
222 cdc_table_type: None,
223 }
224 }
225
226 pub fn columns(&self) -> &[ColumnCatalog] {
227 &self.columns
228 }
229}
230
231pub trait Distill {
233 fn distill<'a>(&self) -> XmlNode<'a>;
234
235 fn distill_to_string(&self) -> String {
236 let mut config = pretty_config();
237 let mut output = String::with_capacity(2048);
238 config.unicode(&mut output, &Pretty::Record(self.distill()));
239 output
240 }
241}
242
243pub(super) fn childless_record<'a>(
244 name: impl Into<Str<'a>>,
245 fields: StrAssocArr<'a>,
246) -> XmlNode<'a> {
247 XmlNode::simple_record(name, fields, Default::default())
248}
249
250macro_rules! impl_distill_by_unit {
251 ($ty:ty, $core:ident, $name:expr) => {
252 use pretty_xmlish::XmlNode;
253 use $crate::optimizer::plan_node::generic::DistillUnit;
254 use $crate::optimizer::plan_node::utils::Distill;
255 impl Distill for $ty {
256 fn distill<'a>(&self) -> XmlNode<'a> {
257 self.$core.distill_with_name($name)
258 }
259 }
260 };
261}
262pub(crate) use impl_distill_by_unit;
263
264pub(crate) fn column_names_pretty<'a>(schema: &Schema) -> Pretty<'a> {
265 let columns = (schema.fields.iter())
266 .map(|f| f.name.clone())
267 .map(Pretty::from)
268 .collect();
269 Pretty::Array(columns)
270}
271
272pub(crate) fn watermark_pretty<'a>(
273 watermark_columns: &WatermarkColumns,
274 schema: &Schema,
275) -> Option<Pretty<'a>> {
276 if watermark_columns.is_empty() {
277 None
278 } else {
279 let groups = watermark_columns.grouped();
280 let pretty_groups = groups
281 .values()
282 .map(|cols| {
283 Pretty::Array(
284 cols.indices()
285 .map(|idx| FieldDisplay(schema.fields.get(idx).unwrap()))
286 .map(|d| Pretty::display(&d))
287 .collect::<Vec<_>>(),
288 )
289 })
290 .collect();
291 Some(Pretty::Array(pretty_groups))
292 }
293}
294
295#[derive(Clone, Copy)]
296pub struct IndicesDisplay<'a> {
297 pub indices: &'a [usize],
298 pub schema: &'a Schema,
299}
300
301impl<'a> IndicesDisplay<'a> {
302 pub fn from_join<'b>(
304 join: &'a generic::Join<impl GenericPlanRef>,
305 input_schema: &'a Schema,
306 ) -> Pretty<'b> {
307 let col_num = join.internal_column_num();
308 let id = Self::from(&join.output_indices, col_num, input_schema);
309 id.map_or_else(|| Pretty::from("all"), Self::distill)
310 }
311
312 fn from(indices: &'a [usize], col_num: usize, schema: &'a Schema) -> Option<Self> {
314 if indices.iter().copied().eq(0..col_num) {
315 return None;
316 }
317 Some(Self { indices, schema })
318 }
319
320 pub fn distill<'b>(self) -> Pretty<'b> {
321 let vec = self.indices.iter().map(|&i| {
322 let name = self.schema.fields.get(i).unwrap().name.clone();
323 Pretty::from(name)
324 });
325 Pretty::Array(vec.collect())
326 }
327}
328
329pub(crate) fn sum_affected_row(dml: BatchPlanRef) -> Result<BatchPlanRef> {
330 let dml = RequiredDist::single().batch_enforce_if_not_satisfies(dml, &Order::any())?;
331 let sum_agg = PlanAggCall {
333 agg_type: PbAggKind::Sum.into(),
334 return_type: DataType::Int64,
335 inputs: vec![InputRef::new(0, DataType::Int64)],
336 distinct: false,
337 order_by: vec![],
338 filter: Condition::true_cond(),
339 direct_args: vec![],
340 };
341 let agg = Agg::new(vec![sum_agg], IndexSet::empty(), dml);
342 let batch_agg = BatchSimpleAgg::new(agg);
343 Ok(batch_agg.into())
344}
345
346macro_rules! plan_node_name {
350 ($name:literal $(, { $prop:literal, $cond:expr } )* $(,)?) => {
351 {
352 #[allow(unused_mut)]
353 let mut properties: Vec<&str> = vec![];
354 $( if $cond { properties.push($prop); } )*
355 let mut name = $name.to_string();
356 if !properties.is_empty() {
357 name += " [";
358 name += &properties.join(", ");
359 name += "]";
360 }
361 name
362 }
363 };
364}
365pub(crate) use plan_node_name;
366use risingwave_pb::common::{PbBatchQueryEpoch, batch_query_epoch};
367
368pub fn infer_kv_log_store_table_catalog_inner(
369 input: &StreamPlanRef,
370 columns: &[ColumnCatalog],
371) -> TableCatalog {
372 let mut table_catalog_builder = TableCatalogBuilder::default();
373
374 let mut value_indices =
375 Vec::with_capacity(KV_LOG_STORE_PREDEFINED_COLUMNS.len() + input.schema().fields().len());
376
377 for (name, data_type) in KV_LOG_STORE_PREDEFINED_COLUMNS {
378 let indice = table_catalog_builder.add_column(&Field::with_name(data_type, name));
379 value_indices.push(indice);
380 }
381
382 table_catalog_builder.set_vnode_col_idx(VNODE_COLUMN_INDEX);
383
384 for (i, ordering) in PK_ORDERING.iter().enumerate() {
385 table_catalog_builder.add_order_column(i, *ordering);
386 }
387
388 let read_prefix_len_hint = table_catalog_builder.get_current_pk_len();
389
390 if columns.len() != input.schema().fields().len()
391 || columns
392 .iter()
393 .zip_eq_fast(input.schema().fields())
394 .any(|(c, f)| *c.data_type() != f.data_type())
395 {
396 tracing::warn!(
397 "sink schema different with upstream schema: sink columns: {:?}, input schema: {:?}.",
398 columns,
399 input.schema()
400 );
401 }
402 for field in input.schema().fields() {
403 let indice = table_catalog_builder.add_column(field);
404 value_indices.push(indice);
405 }
406 table_catalog_builder.set_value_indices(value_indices);
407
408 let dist_key = input
410 .distribution()
411 .dist_column_indices()
412 .iter()
413 .map(|idx| idx + KV_LOG_STORE_PREDEFINED_COLUMNS.len())
414 .collect_vec();
415
416 table_catalog_builder.build(dist_key, read_prefix_len_hint)
417}
418
419pub fn infer_synced_kv_log_store_table_catalog_inner(
420 input: &StreamPlanRef,
421 columns: &[Field],
422) -> TableCatalog {
423 let mut table_catalog_builder = TableCatalogBuilder::default();
424
425 let mut value_indices =
426 Vec::with_capacity(KV_LOG_STORE_PREDEFINED_COLUMNS.len() + columns.len());
427
428 for (name, data_type) in KV_LOG_STORE_PREDEFINED_COLUMNS {
429 let indice = table_catalog_builder.add_column(&Field::with_name(data_type, name));
430 value_indices.push(indice);
431 }
432
433 table_catalog_builder.set_vnode_col_idx(VNODE_COLUMN_INDEX);
434
435 for (i, ordering) in PK_ORDERING.iter().enumerate() {
436 table_catalog_builder.add_order_column(i, *ordering);
437 }
438
439 let read_prefix_len_hint = table_catalog_builder.get_current_pk_len();
440
441 let payload_indices = {
442 let mut payload_indices = Vec::with_capacity(columns.len());
443 for column in columns {
444 let payload_index = table_catalog_builder.add_column(column);
445 payload_indices.push(payload_index);
446 }
447 payload_indices
448 };
449
450 value_indices.extend(payload_indices);
451 table_catalog_builder.set_value_indices(value_indices);
452
453 let dist_key = input
455 .distribution()
456 .dist_column_indices()
457 .iter()
458 .map(|idx| idx + KV_LOG_STORE_PREDEFINED_COLUMNS.len())
459 .collect_vec();
460
461 table_catalog_builder.build(dist_key, read_prefix_len_hint)
462}
463
464pub(crate) fn plan_can_use_background_ddl(plan: &StreamPlanRef) -> bool {
469 if plan.inputs().is_empty() {
470 if plan.as_stream_source_scan().is_some()
471 || plan.as_stream_now().is_some()
472 || plan.as_stream_source().is_some()
473 {
474 true
475 } else if let Some(scan) = plan.as_stream_table_scan() {
476 scan.stream_scan_type() == StreamScanType::Backfill
477 || scan.stream_scan_type() == StreamScanType::ArrangementBackfill
478 || scan.stream_scan_type() == StreamScanType::CrossDbSnapshotBackfill
479 || scan.stream_scan_type() == StreamScanType::SnapshotBackfill
480 } else {
481 false
482 }
483 } else if plan.as_stream_locality_provider().is_some() {
484 false
487 } else {
488 assert!(!plan.inputs().is_empty());
489 plan.inputs().iter().all(plan_can_use_background_ddl)
490 }
491}
492
493pub fn unix_timestamp_sec_to_epoch(ts: i64) -> risingwave_common::util::epoch::Epoch {
494 let ts = ts.checked_add(1).unwrap();
495 risingwave_common::util::epoch::Epoch::from_unix_millis_or_earliest(
496 u64::try_from(ts).unwrap_or(0).checked_mul(1000).unwrap(),
497 )
498}
499
500pub fn to_batch_query_epoch(a: &Option<AsOf>) -> Result<Option<PbBatchQueryEpoch>> {
501 let Some(a) = a else {
502 return Ok(None);
503 };
504 Feature::TimeTravel.check_available()?;
505 let timestamp = match a {
506 AsOf::ProcessTime => {
507 return Err(ErrorCode::NotSupported(
508 "do not support as of proctime".to_owned(),
509 "please use as of timestamp".to_owned(),
510 )
511 .into());
512 }
513 AsOf::TimestampNum(ts) => *ts,
514 AsOf::TimestampString(ts) => {
515 let date_time = speedate::DateTime::parse_str_rfc3339(ts)
516 .map_err(|_e| anyhow!("fail to parse timestamp"))?;
517 if date_time.time.tz_offset.is_none() {
518 risingwave_expr::expr_context::TIME_ZONE::try_with(|set_time_zone| {
520 let tz =
521 Timestamptz::lookup_time_zone(set_time_zone).map_err(|e| anyhow!(e))?;
522 match tz.with_ymd_and_hms(
523 date_time.date.year.into(),
524 date_time.date.month.into(),
525 date_time.date.day.into(),
526 date_time.time.hour.into(),
527 date_time.time.minute.into(),
528 date_time.time.second.into(),
529 ) {
530 MappedLocalTime::Single(d) => Ok(d.timestamp()),
531 MappedLocalTime::Ambiguous(_, _) | MappedLocalTime::None => {
532 Err(anyhow!(format!(
533 "failed to parse the timestamp {ts} with the specified time zone {tz}"
534 )))
535 }
536 }
537 })??
538 } else {
539 date_time.timestamp_tz()
540 }
541 }
542 AsOf::VersionNum(_) | AsOf::VersionString(_) => {
543 return Err(ErrorCode::NotSupported(
544 "do not support as of version".to_owned(),
545 "please use as of timestamp".to_owned(),
546 )
547 .into());
548 }
549 AsOf::ProcessTimeWithInterval((value, leading_field)) => {
550 let interval = Interval::parse_with_fields(
551 value,
552 Some(crate::Binder::bind_date_time_field(*leading_field)),
553 )
554 .map_err(|_| anyhow!("fail to parse interval"))?;
555 let interval_sec = (interval.epoch_in_micros() / 1_000_000) as i64;
556 chrono::Utc::now()
557 .timestamp()
558 .checked_sub(interval_sec)
559 .ok_or_else(|| anyhow!("invalid timestamp"))?
560 }
561 };
562 Ok(Some(PbBatchQueryEpoch {
563 epoch: Some(batch_query_epoch::PbEpoch::TimeTravel(
564 unix_timestamp_sec_to_epoch(timestamp).0,
565 )),
566 }))
567}
568
569pub fn to_iceberg_time_travel_as_of(
570 a: &Option<AsOf>,
571 timezone: &String,
572) -> Result<Option<IcebergTimeTravelInfo>> {
573 Ok(match a {
574 Some(AsOf::VersionNum(v)) => Some(IcebergTimeTravelInfo::Version(*v)),
575 Some(AsOf::TimestampNum(ts)) => Some(IcebergTimeTravelInfo::TimestampMs(ts * 1000)),
576 Some(AsOf::VersionString(_)) => {
577 bail!("Unsupported version string in iceberg time travel")
578 }
579 Some(AsOf::TimestampString(ts)) => {
580 let date_time = speedate::DateTime::parse_str_rfc3339(ts)
581 .map_err(|_e| anyhow!("fail to parse timestamp"))?;
582 let timestamp = if date_time.time.tz_offset.is_none() {
583 let tz = Timestamptz::lookup_time_zone(timezone).map_err(|e| anyhow!(e))?;
585 match tz.with_ymd_and_hms(
586 date_time.date.year.into(),
587 date_time.date.month.into(),
588 date_time.date.day.into(),
589 date_time.time.hour.into(),
590 date_time.time.minute.into(),
591 date_time.time.second.into(),
592 ) {
593 MappedLocalTime::Single(d) => Ok(d.timestamp()),
594 MappedLocalTime::Ambiguous(_, _) | MappedLocalTime::None => {
595 Err(anyhow!(format!(
596 "failed to parse the timestamp {ts} with the specified time zone {tz}"
597 )))
598 }
599 }?
600 } else {
601 date_time.timestamp_tz()
602 };
603
604 Some(IcebergTimeTravelInfo::TimestampMs(
605 timestamp * 1000 + date_time.time.microsecond as i64 / 1000,
606 ))
607 }
608 Some(AsOf::ProcessTime) | Some(AsOf::ProcessTimeWithInterval(_)) => {
609 unreachable!()
610 }
611 None => None,
612 })
613}
614
615pub fn scan_ranges_as_strs(order_names: Vec<String>, scan_ranges: &Vec<ScanRange>) -> Vec<String> {
616 let mut range_strs = vec![];
617
618 let explain_max_range = 20;
619 for scan_range in scan_ranges.iter().take(explain_max_range) {
620 #[expect(clippy::disallowed_methods)]
621 let mut range_str = scan_range
622 .eq_conds
623 .iter()
624 .zip(order_names.iter())
625 .map(|(v, name)| match v {
626 Some(v) => format!("{} = {:?}", name, v),
627 None => format!("{} IS NULL", name),
628 })
629 .collect_vec();
630
631 let len = scan_range.eq_conds.len();
632 if !is_full_range(&scan_range.range) {
633 let bound_range_str = match (&scan_range.range.0, &scan_range.range.1) {
634 (Bound::Unbounded, Bound::Unbounded) => unreachable!(),
635 (Bound::Unbounded, ub) => ub_to_string(&order_names[len..], ub),
636 (lb, Bound::Unbounded) => lb_to_string(&order_names[len..], lb),
637 (lb, ub) => format!(
638 "{} AND {}",
639 lb_to_string(&order_names[len..], lb),
640 ub_to_string(&order_names[len..], ub)
641 ),
642 };
643 range_str.push(bound_range_str);
644 }
645 range_strs.push(range_str.join(" AND "));
646 }
647 if scan_ranges.len() > explain_max_range {
648 range_strs.push("...".to_owned());
649 }
650 range_strs
651}
652
653pub fn ub_to_string(order_names: &[String], ub: &Bound<Vec<Option<ScalarImpl>>>) -> String {
654 match ub {
655 Bound::Included(v) => {
656 let (name, value) = row_to_string(order_names, v);
657 format!("{} <= {}", name, value)
658 }
659 Bound::Excluded(v) => {
660 let (name, value) = row_to_string(order_names, v);
661 format!("{} < {}", name, value)
662 }
663 Bound::Unbounded => unreachable!(),
664 }
665}
666
667pub fn lb_to_string(order_names: &[String], lb: &Bound<Vec<Option<ScalarImpl>>>) -> String {
668 match lb {
669 Bound::Included(v) => {
670 let (name, value) = row_to_string(order_names, v);
671 format!("{} >= {}", name, value)
672 }
673 Bound::Excluded(v) => {
674 let (name, value) = row_to_string(order_names, v);
675 format!("{} > {}", name, value)
676 }
677 Bound::Unbounded => unreachable!(),
678 }
679}
680
681pub fn row_to_string(
682 order_names: &[String],
683 struct_values: &Vec<Option<ScalarImpl>>,
684) -> (String, String) {
685 let mut names = vec![];
686 let mut values = vec![];
687 #[expect(clippy::disallowed_methods)]
688 for (name, value) in order_names.iter().zip(struct_values.iter()) {
689 names.push(name);
690 match value {
691 Some(v) => values.push(format!("{:?}", v)),
692 None => values.push("null".to_owned()),
693 }
694 }
695 if names.len() == 1 {
696 (names[0].clone(), values[0].clone())
697 } else {
698 (
699 format!("({})", names.iter().join(", ")),
700 format!("({})", values.iter().join(", ")),
701 )
702 }
703}