risingwave_frontend/optimizer/plan_node/
utils.rs

1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use 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,
27    OBJECT_ID_PLACEHOLDER, Schema, 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_pb::plan_common::as_of::AsOfType;
42use risingwave_pb::plan_common::{PbAsOf, as_of};
43use risingwave_sqlparser::ast::AsOf;
44
45use super::generic::{self, GenericPlanRef, PhysicalPlanRef};
46use super::{BatchPlanRef, StreamPlanRef, pretty_config};
47use crate::catalog::table_catalog::TableType;
48use crate::catalog::{ColumnId, TableCatalog, TableId};
49use crate::error::{ErrorCode, Result};
50use crate::expr::InputRef;
51use crate::optimizer::StreamScanType;
52use crate::optimizer::plan_node::generic::Agg;
53use crate::optimizer::plan_node::{BatchSimpleAgg, PlanAggCall};
54use crate::optimizer::property::{Cardinality, Order, RequiredDist, WatermarkColumns};
55use crate::utils::{Condition, IndexSet};
56
57#[derive(Default)]
58pub struct TableCatalogBuilder {
59    /// All columns in this table
60    columns: Vec<ColumnCatalog>,
61    pk: Vec<ColumnOrder>,
62    value_indices: Option<Vec<usize>>,
63    vnode_col_idx: Option<usize>,
64    column_names: HashMap<String, i32>,
65    watermark_columns: Option<FixedBitSet>,
66    dist_key_in_pk: Option<Vec<usize>>,
67}
68
69/// For DRY, mainly used for construct internal table catalog in stateful streaming executors.
70/// Be careful of the order of add column.
71impl TableCatalogBuilder {
72    /// Add a column from Field info, return the column index of the table
73    pub fn add_column(&mut self, field: &Field) -> usize {
74        let column_idx = self.columns.len();
75        let column_id = column_idx as i32;
76        // Add column desc.
77        let mut column_desc = ColumnDesc::from_field_with_column_id(field, column_id);
78
79        // Replace dot of the internal table column name with underline.
80        column_desc.name = column_desc.name.replace('.', "_");
81        // Avoid column name duplicate.
82        self.avoid_duplicate_col_name(&mut column_desc);
83
84        self.columns.push(ColumnCatalog {
85            column_desc,
86            // All columns in internal table are invisible to batch query.
87            is_hidden: false,
88        });
89        column_idx
90    }
91
92    /// Extend the columns with column ids reset. The input columns should NOT have duplicate names.
93    ///
94    /// Returns the indices of the extended columns.
95    pub fn extend_columns(&mut self, columns: &[ColumnCatalog]) -> Vec<usize> {
96        let base_idx = self.columns.len();
97        columns.iter().enumerate().for_each(|(i, col)| {
98            assert!(!self.column_names.contains_key(col.name()));
99            self.column_names.insert(col.name().to_owned(), 0);
100
101            // Reset the column id for the columns.
102            let mut new_col = col.clone();
103            new_col.column_desc.column_id = ColumnId::new((base_idx + i) as _);
104            self.columns.push(new_col);
105        });
106        Vec::from_iter(base_idx..(base_idx + columns.len()))
107    }
108
109    /// Check whether need to add a ordered column. Different from value, order desc equal pk in
110    /// semantics and they are encoded as storage key.
111    pub fn add_order_column(&mut self, column_index: usize, order_type: OrderType) {
112        self.pk.push(ColumnOrder::new(column_index, order_type));
113    }
114
115    /// get the current exist field number of the primary key.
116    pub fn get_current_pk_len(&self) -> usize {
117        self.pk.len()
118    }
119
120    pub fn set_vnode_col_idx(&mut self, vnode_col_idx: usize) {
121        self.vnode_col_idx = Some(vnode_col_idx);
122    }
123
124    pub fn set_value_indices(&mut self, value_indices: Vec<usize>) {
125        self.value_indices = Some(value_indices);
126    }
127
128    pub fn set_dist_key_in_pk(&mut self, dist_key_in_pk: Vec<usize>) {
129        self.dist_key_in_pk = Some(dist_key_in_pk);
130    }
131
132    /// Check the column name whether exist before. if true, record occurrence and change the name
133    /// to avoid duplicate.
134    fn avoid_duplicate_col_name(&mut self, column_desc: &mut ColumnDesc) {
135        if let Some(old_identity) = self.column_names.get(&column_desc.name) {
136            let column_name = column_desc.name.clone();
137            let mut identity = *old_identity;
138            loop {
139                column_desc.name = format!("{}_{}", column_name, identity);
140                identity += 1;
141                if !self.column_names.contains_key(&column_desc.name) {
142                    break;
143                }
144            }
145            *self.column_names.get_mut(&column_name).unwrap() = identity;
146        }
147        self.column_names.insert(column_desc.name.clone(), 0);
148    }
149
150    /// Consume builder and create `TableCatalog` (for proto). The `read_prefix_len_hint` is the
151    /// anticipated read prefix pattern (number of fields) for the table, which can be utilized for
152    /// implementing the table's bloom filter or other storage optimization techniques.
153    pub fn build(self, distribution_key: Vec<usize>, read_prefix_len_hint: usize) -> TableCatalog {
154        assert!(read_prefix_len_hint <= self.pk.len());
155        let watermark_columns = match self.watermark_columns {
156            Some(w) => w,
157            None => FixedBitSet::with_capacity(self.columns.len()),
158        };
159
160        // If `dist_key_in_pk` is set, check if it matches with `distribution_key`.
161        // Note that we cannot derive in the opposite direction, because there can be a column
162        // appearing multiple times in the PK.
163        if let Some(dist_key_in_pk) = &self.dist_key_in_pk {
164            let derived_dist_key = dist_key_in_pk
165                .iter()
166                .map(|idx| self.pk[*idx].column_index)
167                .collect_vec();
168            assert_eq!(
169                derived_dist_key, distribution_key,
170                "dist_key mismatch with dist_key_in_pk"
171            );
172        }
173
174        TableCatalog {
175            id: TableId::placeholder(),
176            schema_id: 0,
177            database_id: 0,
178            associated_source_id: None,
179            name: String::new(),
180            columns: self.columns.clone(),
181            pk: self.pk,
182            stream_key: vec![],
183            distribution_key,
184            // NOTE: This should be altered if `TableCatalogBuilder` is used to build something
185            // other than internal tables.
186            table_type: TableType::Internal,
187            append_only: false,
188            owner: risingwave_common::catalog::DEFAULT_SUPER_USER_ID,
189            fragment_id: OBJECT_ID_PLACEHOLDER,
190            dml_fragment_id: None,
191            vnode_col_index: self.vnode_col_idx,
192            row_id_index: None,
193            value_indices: self
194                .value_indices
195                .unwrap_or_else(|| (0..self.columns.len()).collect_vec()),
196            definition: "".into(),
197            conflict_behavior: ConflictBehavior::NoCheck,
198            version_column_index: None,
199            read_prefix_len_hint,
200            version: None, // the internal table is not versioned and can't be schema changed
201            watermark_columns,
202            dist_key_in_pk: self.dist_key_in_pk.unwrap_or_default(),
203            cardinality: Cardinality::unknown(), // TODO(card): cardinality of internal table
204            created_at_epoch: None,
205            initialized_at_epoch: None,
206            cleaned_by_watermark: false,
207            // NOTE(kwannoel): This may not match the create type of the materialized table.
208            // It should be ignored for internal tables.
209            create_type: CreateType::Foreground,
210            stream_job_status: StreamJobStatus::Creating,
211            description: None,
212            incoming_sinks: vec![],
213            initialized_at_cluster_version: None,
214            created_at_cluster_version: None,
215            retention_seconds: None,
216            cdc_table_id: None,
217            vnode_count: VnodeCount::Placeholder, // will be filled in by the meta service later
218            webhook_info: None,
219            job_id: None,
220            engine: Engine::Hummock,
221            clean_watermark_index_in_pk: None, // TODO: fill this field
222            refreshable: false,                // Internal tables are not refreshable
223            vector_index_info: None,
224        }
225    }
226
227    pub fn columns(&self) -> &[ColumnCatalog] {
228        &self.columns
229    }
230}
231
232/// See also [`super::generic::DistillUnit`].
233pub trait Distill {
234    fn distill<'a>(&self) -> XmlNode<'a>;
235
236    fn distill_to_string(&self) -> String {
237        let mut config = pretty_config();
238        let mut output = String::with_capacity(2048);
239        config.unicode(&mut output, &Pretty::Record(self.distill()));
240        output
241    }
242}
243
244pub(super) fn childless_record<'a>(
245    name: impl Into<Str<'a>>,
246    fields: StrAssocArr<'a>,
247) -> XmlNode<'a> {
248    XmlNode::simple_record(name, fields, Default::default())
249}
250
251macro_rules! impl_distill_by_unit {
252    ($ty:ty, $core:ident, $name:expr) => {
253        use pretty_xmlish::XmlNode;
254        use $crate::optimizer::plan_node::generic::DistillUnit;
255        use $crate::optimizer::plan_node::utils::Distill;
256        impl Distill for $ty {
257            fn distill<'a>(&self) -> XmlNode<'a> {
258                self.$core.distill_with_name($name)
259            }
260        }
261    };
262}
263pub(crate) use impl_distill_by_unit;
264
265pub(crate) fn column_names_pretty<'a>(schema: &Schema) -> Pretty<'a> {
266    let columns = (schema.fields.iter())
267        .map(|f| f.name.clone())
268        .map(Pretty::from)
269        .collect();
270    Pretty::Array(columns)
271}
272
273pub(crate) fn watermark_pretty<'a>(
274    watermark_columns: &WatermarkColumns,
275    schema: &Schema,
276) -> Option<Pretty<'a>> {
277    if watermark_columns.is_empty() {
278        None
279    } else {
280        let groups = watermark_columns.grouped();
281        let pretty_groups = groups
282            .values()
283            .map(|cols| {
284                Pretty::Array(
285                    cols.indices()
286                        .map(|idx| FieldDisplay(schema.fields.get(idx).unwrap()))
287                        .map(|d| Pretty::display(&d))
288                        .collect::<Vec<_>>(),
289                )
290            })
291            .collect();
292        Some(Pretty::Array(pretty_groups))
293    }
294}
295
296#[derive(Clone, Copy)]
297pub struct IndicesDisplay<'a> {
298    pub indices: &'a [usize],
299    pub schema: &'a Schema,
300}
301
302impl<'a> IndicesDisplay<'a> {
303    /// Returns `None` means all
304    pub fn from_join<'b>(
305        join: &'a generic::Join<impl GenericPlanRef>,
306        input_schema: &'a Schema,
307    ) -> Pretty<'b> {
308        let col_num = join.internal_column_num();
309        let id = Self::from(&join.output_indices, col_num, input_schema);
310        id.map_or_else(|| Pretty::from("all"), Self::distill)
311    }
312
313    /// Returns `None` means all
314    fn from(indices: &'a [usize], col_num: usize, schema: &'a Schema) -> Option<Self> {
315        if indices.iter().copied().eq(0..col_num) {
316            return None;
317        }
318        Some(Self { indices, schema })
319    }
320
321    pub fn distill<'b>(self) -> Pretty<'b> {
322        let vec = self.indices.iter().map(|&i| {
323            let name = self.schema.fields.get(i).unwrap().name.clone();
324            Pretty::from(name)
325        });
326        Pretty::Array(vec.collect())
327    }
328}
329
330pub(crate) fn sum_affected_row(dml: BatchPlanRef) -> Result<BatchPlanRef> {
331    let dml = RequiredDist::single().batch_enforce_if_not_satisfies(dml, &Order::any())?;
332    // Accumulate the affected rows.
333    let sum_agg = PlanAggCall {
334        agg_type: PbAggKind::Sum.into(),
335        return_type: DataType::Int64,
336        inputs: vec![InputRef::new(0, DataType::Int64)],
337        distinct: false,
338        order_by: vec![],
339        filter: Condition::true_cond(),
340        direct_args: vec![],
341    };
342    let agg = Agg::new(vec![sum_agg], IndexSet::empty(), dml);
343    let batch_agg = BatchSimpleAgg::new(agg);
344    Ok(batch_agg.into())
345}
346
347/// Call `debug_struct` on the given formatter to create a debug struct builder.
348/// If a property list is provided, properties in it will be added to the struct name according to
349/// the condition of that property.
350macro_rules! plan_node_name {
351    ($name:literal $(, { $prop:literal, $cond:expr } )* $(,)?) => {
352        {
353            #[allow(unused_mut)]
354            let mut properties: Vec<&str> = vec![];
355            $( if $cond { properties.push($prop); } )*
356            let mut name = $name.to_string();
357            if !properties.is_empty() {
358                name += " [";
359                name += &properties.join(", ");
360                name += "]";
361            }
362            name
363        }
364    };
365}
366pub(crate) use plan_node_name;
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    // Modify distribution key indices based on the pre-defined columns.
409    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    // Modify distribution key indices based on the pre-defined columns.
454    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
464/// Check that all leaf nodes must be stream table scan,
465/// since that plan node maps to `backfill` executor, which supports recovery.
466/// Some other leaf nodes like `StreamValues` do not support recovery, and they
467/// cannot use background ddl.
468pub(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 {
484        assert!(!plan.inputs().is_empty());
485        plan.inputs().iter().all(plan_can_use_background_ddl)
486    }
487}
488
489pub fn to_pb_time_travel_as_of(a: &Option<AsOf>) -> Result<Option<PbAsOf>> {
490    let Some(a) = a else {
491        return Ok(None);
492    };
493    Feature::TimeTravel.check_available()?;
494    let as_of_type = match a {
495        AsOf::ProcessTime => {
496            return Err(ErrorCode::NotSupported(
497                "do not support as of proctime".to_owned(),
498                "please use as of timestamp".to_owned(),
499            )
500            .into());
501        }
502        AsOf::TimestampNum(ts) => AsOfType::Timestamp(as_of::Timestamp { timestamp: *ts }),
503        AsOf::TimestampString(ts) => {
504            let date_time = speedate::DateTime::parse_str_rfc3339(ts)
505                .map_err(|_e| anyhow!("fail to parse timestamp"))?;
506            let timestamp = if date_time.time.tz_offset.is_none() {
507                // If the input does not specify a time zone, use the time zone set by the "SET TIME ZONE" command.
508                risingwave_expr::expr_context::TIME_ZONE::try_with(|set_time_zone| {
509                    let tz =
510                        Timestamptz::lookup_time_zone(set_time_zone).map_err(|e| anyhow!(e))?;
511                    match tz.with_ymd_and_hms(
512                        date_time.date.year.into(),
513                        date_time.date.month.into(),
514                        date_time.date.day.into(),
515                        date_time.time.hour.into(),
516                        date_time.time.minute.into(),
517                        date_time.time.second.into(),
518                    ) {
519                        MappedLocalTime::Single(d) => Ok(d.timestamp()),
520                        MappedLocalTime::Ambiguous(_, _) | MappedLocalTime::None => {
521                            Err(anyhow!(format!(
522                                "failed to parse the timestamp {ts} with the specified time zone {tz}"
523                            )))
524                        }
525                    }
526                })??
527            } else {
528                date_time.timestamp_tz()
529            };
530            AsOfType::Timestamp(as_of::Timestamp { timestamp })
531        }
532        AsOf::VersionNum(_) | AsOf::VersionString(_) => {
533            return Err(ErrorCode::NotSupported(
534                "do not support as of version".to_owned(),
535                "please use as of timestamp".to_owned(),
536            )
537            .into());
538        }
539        AsOf::ProcessTimeWithInterval((value, leading_field)) => {
540            let interval = Interval::parse_with_fields(
541                value,
542                Some(crate::Binder::bind_date_time_field(*leading_field)),
543            )
544            .map_err(|_| anyhow!("fail to parse interval"))?;
545            let interval_sec = (interval.epoch_in_micros() / 1_000_000) as i64;
546            let timestamp = chrono::Utc::now()
547                .timestamp()
548                .checked_sub(interval_sec)
549                .ok_or_else(|| anyhow!("invalid timestamp"))?;
550            AsOfType::Timestamp(as_of::Timestamp { timestamp })
551        }
552    };
553    Ok(Some(PbAsOf {
554        as_of_type: Some(as_of_type),
555    }))
556}
557
558pub fn to_iceberg_time_travel_as_of(
559    a: &Option<AsOf>,
560    timezone: &String,
561) -> Result<Option<IcebergTimeTravelInfo>> {
562    Ok(match a {
563        Some(AsOf::VersionNum(v)) => Some(IcebergTimeTravelInfo::Version(*v)),
564        Some(AsOf::TimestampNum(ts)) => Some(IcebergTimeTravelInfo::TimestampMs(ts * 1000)),
565        Some(AsOf::VersionString(_)) => {
566            bail!("Unsupported version string in iceberg time travel")
567        }
568        Some(AsOf::TimestampString(ts)) => {
569            let date_time = speedate::DateTime::parse_str_rfc3339(ts)
570                .map_err(|_e| anyhow!("fail to parse timestamp"))?;
571            let timestamp = if date_time.time.tz_offset.is_none() {
572                // If the input does not specify a time zone, use the time zone set by the "SET TIME ZONE" command.
573                let tz = Timestamptz::lookup_time_zone(timezone).map_err(|e| anyhow!(e))?;
574                match tz.with_ymd_and_hms(
575                    date_time.date.year.into(),
576                    date_time.date.month.into(),
577                    date_time.date.day.into(),
578                    date_time.time.hour.into(),
579                    date_time.time.minute.into(),
580                    date_time.time.second.into(),
581                ) {
582                    MappedLocalTime::Single(d) => Ok(d.timestamp()),
583                    MappedLocalTime::Ambiguous(_, _) | MappedLocalTime::None => {
584                        Err(anyhow!(format!(
585                            "failed to parse the timestamp {ts} with the specified time zone {tz}"
586                        )))
587                    }
588                }?
589            } else {
590                date_time.timestamp_tz()
591            };
592
593            Some(IcebergTimeTravelInfo::TimestampMs(
594                timestamp * 1000 + date_time.time.microsecond as i64 / 1000,
595            ))
596        }
597        Some(AsOf::ProcessTime) | Some(AsOf::ProcessTimeWithInterval(_)) => {
598            unreachable!()
599        }
600        None => None,
601    })
602}
603
604pub fn scan_ranges_as_strs(order_names: Vec<String>, scan_ranges: &Vec<ScanRange>) -> Vec<String> {
605    let mut range_strs = vec![];
606
607    let explain_max_range = 20;
608    for scan_range in scan_ranges.iter().take(explain_max_range) {
609        #[expect(clippy::disallowed_methods)]
610        let mut range_str = scan_range
611            .eq_conds
612            .iter()
613            .zip(order_names.iter())
614            .map(|(v, name)| match v {
615                Some(v) => format!("{} = {:?}", name, v),
616                None => format!("{} IS NULL", name),
617            })
618            .collect_vec();
619
620        let len = scan_range.eq_conds.len();
621        if !is_full_range(&scan_range.range) {
622            let bound_range_str = match (&scan_range.range.0, &scan_range.range.1) {
623                (Bound::Unbounded, Bound::Unbounded) => unreachable!(),
624                (Bound::Unbounded, ub) => ub_to_string(&order_names[len..], ub),
625                (lb, Bound::Unbounded) => lb_to_string(&order_names[len..], lb),
626                (lb, ub) => format!(
627                    "{} AND {}",
628                    lb_to_string(&order_names[len..], lb),
629                    ub_to_string(&order_names[len..], ub)
630                ),
631            };
632            range_str.push(bound_range_str);
633        }
634        range_strs.push(range_str.join(" AND "));
635    }
636    if scan_ranges.len() > explain_max_range {
637        range_strs.push("...".to_owned());
638    }
639    range_strs
640}
641
642pub fn ub_to_string(order_names: &[String], ub: &Bound<Vec<Option<ScalarImpl>>>) -> String {
643    match ub {
644        Bound::Included(v) => {
645            let (name, value) = row_to_string(order_names, v);
646            format!("{} <= {}", name, value)
647        }
648        Bound::Excluded(v) => {
649            let (name, value) = row_to_string(order_names, v);
650            format!("{} < {}", name, value)
651        }
652        Bound::Unbounded => unreachable!(),
653    }
654}
655
656pub fn lb_to_string(order_names: &[String], lb: &Bound<Vec<Option<ScalarImpl>>>) -> String {
657    match lb {
658        Bound::Included(v) => {
659            let (name, value) = row_to_string(order_names, v);
660            format!("{} >= {}", name, value)
661        }
662        Bound::Excluded(v) => {
663            let (name, value) = row_to_string(order_names, v);
664            format!("{} > {}", name, value)
665        }
666        Bound::Unbounded => unreachable!(),
667    }
668}
669
670pub fn row_to_string(
671    order_names: &[String],
672    struct_values: &Vec<Option<ScalarImpl>>,
673) -> (String, String) {
674    let mut names = vec![];
675    let mut values = vec![];
676    #[expect(clippy::disallowed_methods)]
677    for (name, value) in order_names.iter().zip(struct_values.iter()) {
678        names.push(name);
679        match value {
680            Some(v) => values.push(format!("{:?}", v)),
681            None => values.push("null".to_owned()),
682        }
683    }
684    if names.len() == 1 {
685        (names[0].clone(), values[0].clone())
686    } else {
687        (
688            format!("({})", names.iter().join(", ")),
689            format!("({})", values.iter().join(", ")),
690        )
691    }
692}