risingwave_frontend/optimizer/
mod.rs

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
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
// Copyright 2024 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::num::NonZeroU32;
use std::ops::DerefMut;
use std::sync::Arc;

pub mod plan_node;

pub use plan_node::{Explain, PlanRef};

pub mod property;

mod delta_join_solver;
mod heuristic_optimizer;
mod plan_rewriter;

pub use plan_rewriter::PlanRewriter;

mod plan_visitor;

pub use plan_visitor::{
    ExecutionModeDecider, PlanVisitor, ReadStorageTableVisitor, RelationCollectorVisitor,
    SysTableVisitor,
};
use risingwave_sqlparser::ast::OnConflict;

mod logical_optimization;
mod optimizer_context;
pub mod plan_expr_rewriter;
mod plan_expr_visitor;
mod rule;

use std::assert_matches::assert_matches;
use std::collections::{BTreeMap, HashMap};

use fixedbitset::FixedBitSet;
use itertools::Itertools as _;
pub use logical_optimization::*;
pub use optimizer_context::*;
use plan_expr_rewriter::ConstEvalRewriter;
use property::Order;
use risingwave_common::bail;
use risingwave_common::catalog::{
    ColumnCatalog, ColumnDesc, ColumnId, ConflictBehavior, Field, Schema,
};
use risingwave_common::types::DataType;
use risingwave_common::util::column_index_mapping::ColIndexMapping;
use risingwave_common::util::iter_util::ZipEqDebug;
use risingwave_connector::sink::catalog::SinkFormatDesc;
use risingwave_pb::catalog::WatermarkDesc;
use risingwave_pb::stream_plan::StreamScanType;

use self::heuristic_optimizer::ApplyOrder;
use self::plan_node::generic::{self, PhysicalPlanRef};
use self::plan_node::{
    stream_enforce_eowc_requirement, BatchProject, Convention, LogicalProject, LogicalSource,
    PartitionComputeInfo, StreamDml, StreamMaterialize, StreamProject, StreamRowIdGen, StreamSink,
    StreamWatermarkFilter, ToStreamContext,
};
#[cfg(debug_assertions)]
use self::plan_visitor::InputRefValidator;
use self::plan_visitor::{has_batch_exchange, CardinalityVisitor, StreamKeyChecker};
use self::property::{Cardinality, RequiredDist};
use self::rule::*;
use crate::catalog::table_catalog::{TableType, TableVersion};
use crate::error::{ErrorCode, Result};
use crate::expr::TimestamptzExprFinder;
use crate::optimizer::plan_node::generic::{SourceNodeKind, Union};
use crate::optimizer::plan_node::{
    BatchExchange, PlanNodeType, PlanTreeNode, RewriteExprsRecursive, StreamExchange, StreamUnion,
    ToStream, VisitExprsRecursive,
};
use crate::optimizer::plan_visitor::{RwTimestampValidator, TemporalJoinValidator};
use crate::optimizer::property::Distribution;
use crate::utils::{ColIndexMappingRewriteExt, WithOptionsSecResolved};
use crate::TableCatalog;

/// `PlanRoot` is used to describe a plan. planner will construct a `PlanRoot` with `LogicalNode`.
/// and required distribution and order. And `PlanRoot` can generate corresponding streaming or
/// batch plan with optimization. the required Order and Distribution columns might be more than the
/// output columns. for example:
/// ```sql
///    select v1 from t order by id;
/// ```
/// the plan will return two columns (id, v1), and the required order column is id. the id
/// column is required in optimization, but the final generated plan will remove the unnecessary
/// column in the result.
#[derive(Debug, Clone)]
pub struct PlanRoot {
    // The current plan node.
    plan: PlanRef,
    // The phase of the plan.
    phase: PlanPhase,
    required_dist: RequiredDist,
    required_order: Order,
    out_fields: FixedBitSet,
    out_names: Vec<String>,
}

/// `PlanPhase` is used to track the phase of the `PlanRoot`.
/// Usually, it begins from `Logical` and ends with `Batch` or `Stream`, unless we want to construct a `PlanRoot` from an intermediate phase.
/// Typical phase transformation are:
/// - `Logical` -> `OptimizedLogicalForBatch` -> `Batch`
/// - `Logical` -> `OptimizedLogicalForStream` -> `Stream`
#[derive(Debug, Clone, PartialEq)]
pub enum PlanPhase {
    Logical,
    OptimizedLogicalForBatch,
    OptimizedLogicalForStream,
    Batch,
    Stream,
}

impl PlanRoot {
    pub fn new_with_logical_plan(
        plan: PlanRef,
        required_dist: RequiredDist,
        required_order: Order,
        out_fields: FixedBitSet,
        out_names: Vec<String>,
    ) -> Self {
        assert_eq!(plan.convention(), Convention::Logical);
        Self::new_inner(
            plan,
            PlanPhase::Logical,
            required_dist,
            required_order,
            out_fields,
            out_names,
        )
    }

    pub fn new_with_batch_plan(
        plan: PlanRef,
        required_dist: RequiredDist,
        required_order: Order,
        out_fields: FixedBitSet,
        out_names: Vec<String>,
    ) -> Self {
        assert_eq!(plan.convention(), Convention::Batch);
        Self::new_inner(
            plan,
            PlanPhase::Batch,
            required_dist,
            required_order,
            out_fields,
            out_names,
        )
    }

    fn new_inner(
        plan: PlanRef,
        phase: PlanPhase,
        required_dist: RequiredDist,
        required_order: Order,
        out_fields: FixedBitSet,
        out_names: Vec<String>,
    ) -> Self {
        let input_schema = plan.schema();
        assert_eq!(input_schema.fields().len(), out_fields.len());
        assert_eq!(out_fields.count_ones(..), out_names.len());

        Self {
            plan,
            phase,
            required_dist,
            required_order,
            out_fields,
            out_names,
        }
    }

    /// Set customized names of the output fields, used for `CREATE [MATERIALIZED VIEW | SINK] r(a,
    /// b, ..)`.
    ///
    /// If the number of names does not match the number of output fields, an error is returned.
    pub fn set_out_names(&mut self, out_names: Vec<String>) -> Result<()> {
        if out_names.len() != self.out_fields.count_ones(..) {
            Err(ErrorCode::InvalidInputSyntax(
                "number of column names does not match number of columns".to_string(),
            ))?
        }
        self.out_names = out_names;
        Ok(())
    }

    /// Get the plan root's schema, only including the fields to be output.
    pub fn schema(&self) -> Schema {
        // The schema can be derived from the `out_fields` and `out_names`, so we don't maintain it
        // as a field and always construct one on demand here to keep it in sync.
        Schema {
            fields: self
                .out_fields
                .ones()
                .map(|i| self.plan.schema().fields()[i].clone())
                .zip_eq_debug(&self.out_names)
                .map(|(field, name)| Field {
                    name: name.clone(),
                    ..field
                })
                .collect(),
        }
    }

    /// Transform the [`PlanRoot`] back to a [`PlanRef`] suitable to be used as a subplan, for
    /// example as insert source or subquery. This ignores Order but retains post-Order pruning
    /// (`out_fields`).
    pub fn into_unordered_subplan(self) -> PlanRef {
        assert_eq!(self.phase, PlanPhase::Logical);
        assert_eq!(self.plan.convention(), Convention::Logical);
        if self.out_fields.count_ones(..) == self.out_fields.len() {
            return self.plan;
        }
        LogicalProject::with_out_fields(self.plan, &self.out_fields).into()
    }

    /// Transform the [`PlanRoot`] wrapped in an array-construction subquery to a [`PlanRef`]
    /// supported by `ARRAY_AGG`. Similar to the unordered version, this abstracts away internal
    /// `self.plan` which is further modified by `self.required_order` then `self.out_fields`.
    pub fn into_array_agg(self) -> Result<PlanRef> {
        use generic::Agg;
        use plan_node::PlanAggCall;
        use risingwave_common::types::ListValue;
        use risingwave_expr::aggregate::PbAggKind;

        use crate::expr::{ExprImpl, ExprType, FunctionCall, InputRef};
        use crate::utils::{Condition, IndexSet};

        assert_eq!(self.phase, PlanPhase::Logical);
        assert_eq!(self.plan.convention(), Convention::Logical);
        let Ok(select_idx) = self.out_fields.ones().exactly_one() else {
            bail!("subquery must return only one column");
        };
        let input_column_type = self.plan.schema().fields()[select_idx].data_type();
        let return_type = DataType::List(input_column_type.clone().into());
        let agg = Agg::new(
            vec![PlanAggCall {
                agg_type: PbAggKind::ArrayAgg.into(),
                return_type: return_type.clone(),
                inputs: vec![InputRef::new(select_idx, input_column_type.clone())],
                distinct: false,
                order_by: self.required_order.column_orders,
                filter: Condition::true_cond(),
                direct_args: vec![],
            }],
            IndexSet::empty(),
            self.plan,
        );
        Ok(LogicalProject::create(
            agg.into(),
            vec![FunctionCall::new(
                ExprType::Coalesce,
                vec![
                    InputRef::new(0, return_type).into(),
                    ExprImpl::literal_list(ListValue::empty(&input_column_type), input_column_type),
                ],
            )
            .unwrap()
            .into()],
        ))
    }

    /// Apply logical optimization to the plan for stream.
    pub fn gen_optimized_logical_plan_for_stream(&mut self) -> Result<PlanRef> {
        assert_eq!(self.phase, PlanPhase::Logical);
        assert_eq!(self.plan.convention(), Convention::Logical);
        self.plan = LogicalOptimizer::gen_optimized_logical_plan_for_stream(self.plan.clone())?;
        self.phase = PlanPhase::OptimizedLogicalForStream;
        assert_eq!(self.plan.convention(), Convention::Logical);
        Ok(self.plan.clone())
    }

    /// Apply logical optimization to the plan for batch.
    pub fn gen_optimized_logical_plan_for_batch(&mut self) -> Result<PlanRef> {
        assert_eq!(self.phase, PlanPhase::Logical);
        assert_eq!(self.plan.convention(), Convention::Logical);
        self.plan = LogicalOptimizer::gen_optimized_logical_plan_for_batch(self.plan.clone())?;
        self.phase = PlanPhase::OptimizedLogicalForBatch;
        assert_eq!(self.plan.convention(), Convention::Logical);
        Ok(self.plan.clone())
    }

    /// Optimize and generate a singleton batch physical plan without exchange nodes.
    pub fn gen_batch_plan(&mut self) -> Result<PlanRef> {
        assert_eq!(self.plan.convention(), Convention::Logical);
        let mut plan = match self.phase {
            PlanPhase::Logical => {
                // Logical optimization
                self.gen_optimized_logical_plan_for_batch()?
            }
            PlanPhase::OptimizedLogicalForBatch => self.plan.clone(),
            PlanPhase::Batch | PlanPhase::OptimizedLogicalForStream | PlanPhase::Stream => {
                panic!("unexpected phase")
            }
        };

        if TemporalJoinValidator::exist_dangling_temporal_scan(plan.clone()) {
            return Err(ErrorCode::NotSupported(
                "do not support temporal join for batch queries".to_string(),
                "please use temporal join in streaming queries".to_string(),
            )
            .into());
        }

        let ctx = plan.ctx();
        // Inline session timezone mainly for rewriting now()
        plan = inline_session_timezone_in_exprs(ctx.clone(), plan)?;

        // Const eval of exprs at the last minute, but before `to_batch` to make functional index selection happy.
        plan = const_eval_exprs(plan)?;

        if ctx.is_explain_trace() {
            ctx.trace("Const eval exprs:");
            ctx.trace(plan.explain_to_string());
        }

        // Convert to physical plan node
        plan = plan.to_batch_with_order_required(&self.required_order)?;
        if ctx.is_explain_trace() {
            ctx.trace("To Batch Plan:");
            ctx.trace(plan.explain_to_string());
        }

        plan = plan.optimize_by_rules(&OptimizationStage::new(
            "Merge BatchProject",
            vec![BatchProjectMergeRule::create()],
            ApplyOrder::BottomUp,
        ))?;

        // Inline session timezone
        plan = inline_session_timezone_in_exprs(ctx.clone(), plan)?;

        if ctx.is_explain_trace() {
            ctx.trace("Inline Session Timezone:");
            ctx.trace(plan.explain_to_string());
        }

        #[cfg(debug_assertions)]
        InputRefValidator.validate(plan.clone());
        assert!(
            *plan.distribution() == Distribution::Single,
            "{}",
            plan.explain_to_string()
        );
        assert!(
            !has_batch_exchange(plan.clone()),
            "{}",
            plan.explain_to_string()
        );

        let ctx = plan.ctx();
        if ctx.is_explain_trace() {
            ctx.trace("To Batch Physical Plan:");
            ctx.trace(plan.explain_to_string());
        }

        self.plan = plan;
        self.phase = PlanPhase::Batch;
        assert_eq!(self.plan.convention(), Convention::Batch);
        Ok(self.plan.clone())
    }

    /// Optimize and generate a batch query plan for distributed execution.
    pub fn gen_batch_distributed_plan(mut self) -> Result<PlanRef> {
        assert_eq!(self.phase, PlanPhase::Batch);
        assert_eq!(self.plan.convention(), Convention::Batch);
        self.required_dist = RequiredDist::single();
        let mut plan = self.plan;

        // Convert to distributed plan
        plan = plan.to_distributed_with_required(&self.required_order, &self.required_dist)?;

        // Add Project if the any position of `self.out_fields` is set to zero.
        if self.out_fields.count_ones(..) != self.out_fields.len() {
            plan =
                BatchProject::new(generic::Project::with_out_fields(plan, &self.out_fields)).into();
        }

        let ctx = plan.ctx();
        if ctx.is_explain_trace() {
            ctx.trace("To Batch Distributed Plan:");
            ctx.trace(plan.explain_to_string());
        }
        if require_additional_exchange_on_root_in_distributed_mode(plan.clone()) {
            plan =
                BatchExchange::new(plan, self.required_order.clone(), Distribution::Single).into();
        }

        // Both two phase limit and topn could generate limit on top of the scan, so we push limit here.
        let plan = plan.optimize_by_rules(&OptimizationStage::new(
            "Push Limit To Scan",
            vec![BatchPushLimitToScanRule::create()],
            ApplyOrder::BottomUp,
        ))?;

        // For iceberg scan, we do iceberg predicate pushdown
        // BatchFilter -> BatchIcebergScan
        let plan = plan.optimize_by_rules(&OptimizationStage::new(
            "Iceberg Predicate Pushdown",
            vec![BatchIcebergPredicatePushDownRule::create()],
            ApplyOrder::BottomUp,
        ))?;

        assert_eq!(plan.convention(), Convention::Batch);
        Ok(plan)
    }

    /// Optimize and generate a batch query plan for local execution.
    pub fn gen_batch_local_plan(self) -> Result<PlanRef> {
        assert_eq!(self.phase, PlanPhase::Batch);
        assert_eq!(self.plan.convention(), Convention::Batch);
        let mut plan = self.plan;

        // Convert to local plan node
        plan = plan.to_local_with_order_required(&self.required_order)?;

        // 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 => require_additional_exchange_on_root_in_local_mode(plan.clone()),
            _ => true,
        };
        if insert_exchange {
            plan =
                BatchExchange::new(plan, self.required_order.clone(), Distribution::Single).into()
        }

        // Add Project if the any position of `self.out_fields` is set to zero.
        if self.out_fields.count_ones(..) != self.out_fields.len() {
            plan =
                BatchProject::new(generic::Project::with_out_fields(plan, &self.out_fields)).into();
        }

        let ctx = plan.ctx();
        if ctx.is_explain_trace() {
            ctx.trace("To Batch Local Plan:");
            ctx.trace(plan.explain_to_string());
        }

        // Both two phase limit and topn could generate limit on top of the scan, so we push limit here.
        let plan = plan.optimize_by_rules(&OptimizationStage::new(
            "Push Limit To Scan",
            vec![BatchPushLimitToScanRule::create()],
            ApplyOrder::BottomUp,
        ))?;

        assert_eq!(plan.convention(), Convention::Batch);
        Ok(plan)
    }

    /// Generate optimized stream plan
    fn gen_optimized_stream_plan(
        &mut self,
        emit_on_window_close: bool,
        allow_snapshot_backfill: bool,
    ) -> Result<PlanRef> {
        assert_eq!(self.phase, PlanPhase::Logical);
        assert_eq!(self.plan.convention(), Convention::Logical);
        let stream_scan_type = if allow_snapshot_backfill && self.should_use_snapshot_backfill() {
            StreamScanType::SnapshotBackfill
        } else if self.should_use_arrangement_backfill() {
            StreamScanType::ArrangementBackfill
        } else {
            StreamScanType::Backfill
        };
        self.gen_optimized_stream_plan_inner(emit_on_window_close, stream_scan_type)
    }

    fn gen_optimized_stream_plan_inner(
        &mut self,
        emit_on_window_close: bool,
        stream_scan_type: StreamScanType,
    ) -> Result<PlanRef> {
        assert_eq!(self.phase, PlanPhase::Logical);
        assert_eq!(self.plan.convention(), Convention::Logical);
        let ctx = self.plan.ctx();
        let _explain_trace = ctx.is_explain_trace();

        let mut plan = self.gen_stream_plan(emit_on_window_close, stream_scan_type)?;

        plan = plan.optimize_by_rules(&OptimizationStage::new(
            "Merge StreamProject",
            vec![StreamProjectMergeRule::create()],
            ApplyOrder::BottomUp,
        ))?;

        if ctx.session_ctx().config().streaming_enable_delta_join() {
            // TODO: make it a logical optimization.
            // Rewrite joins with index to delta join
            plan = plan.optimize_by_rules(&OptimizationStage::new(
                "To IndexDeltaJoin",
                vec![IndexDeltaJoinRule::create()],
                ApplyOrder::BottomUp,
            ))?;
        }

        // Inline session timezone
        plan = inline_session_timezone_in_exprs(ctx.clone(), plan)?;

        if ctx.is_explain_trace() {
            ctx.trace("Inline session timezone:");
            ctx.trace(plan.explain_to_string());
        }

        // Const eval of exprs at the last minute
        plan = const_eval_exprs(plan)?;

        if ctx.is_explain_trace() {
            ctx.trace("Const eval exprs:");
            ctx.trace(plan.explain_to_string());
        }

        #[cfg(debug_assertions)]
        InputRefValidator.validate(plan.clone());

        if TemporalJoinValidator::exist_dangling_temporal_scan(plan.clone()) {
            return Err(ErrorCode::NotSupported(
                "exist dangling temporal scan".to_string(),
                "please check your temporal join syntax e.g. consider removing the right outer join if it is being used.".to_string(),
            ).into());
        }

        if RwTimestampValidator::select_rw_timestamp_in_stream_query(plan.clone()) {
            return Err(ErrorCode::NotSupported(
                "selecting `_rw_timestamp` in a streaming query is not allowed".to_string(),
                "please run the sql in batch mode or remove the column `_rw_timestamp` from the streaming query".to_string(),
            ).into());
        }

        self.plan = plan;
        self.phase = PlanPhase::Stream;
        assert_eq!(self.plan.convention(), Convention::Stream);
        Ok(self.plan.clone())
    }

    /// Generate create index or create materialize view plan.
    fn gen_stream_plan(
        &mut self,
        emit_on_window_close: bool,
        stream_scan_type: StreamScanType,
    ) -> Result<PlanRef> {
        assert_eq!(self.phase, PlanPhase::Logical);
        assert_eq!(self.plan.convention(), Convention::Logical);
        let ctx = self.plan.ctx();
        let explain_trace = ctx.is_explain_trace();

        let plan = match self.plan.convention() {
            Convention::Logical => {
                if !ctx
                    .session_ctx()
                    .config()
                    .streaming_allow_jsonb_in_stream_key()
                    && let Some(err) = StreamKeyChecker.visit(self.plan.clone())
                {
                    return Err(ErrorCode::NotSupported(
                        err,
                        "Using JSONB columns as part of the join or aggregation keys can severely impair performance. \
                        If you intend to proceed, force to enable it with: `set rw_streaming_allow_jsonb_in_stream_key to true`".to_string(),
                    ).into());
                }
                let plan = self.gen_optimized_logical_plan_for_stream()?;
                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::new(map, target_size);
                        (plan.into(), out_col_change)
                    }
                };

                if explain_trace {
                    ctx.trace("Logical Rewrite For Stream:");
                    ctx.trace(plan.explain_to_string());
                }

                self.required_dist =
                    out_col_change.rewrite_required_distribution(&self.required_dist);
                self.required_order = out_col_change
                    .rewrite_required_order(&self.required_order)
                    .unwrap();
                self.out_fields = out_col_change.rewrite_bitset(&self.out_fields);
                let plan = plan.to_stream_with_dist_required(
                    &self.required_dist,
                    &mut ToStreamContext::new_with_stream_scan_type(
                        emit_on_window_close,
                        stream_scan_type,
                    ),
                )?;
                stream_enforce_eowc_requirement(ctx.clone(), plan, emit_on_window_close)
            }
            _ => unreachable!(),
        }?;

        if explain_trace {
            ctx.trace("To Stream Plan:");
            ctx.trace(plan.explain_to_string());
        }
        Ok(plan)
    }

    /// Visit the plan root and compute the cardinality.
    ///
    /// Panics if not called on a logical plan.
    fn compute_cardinality(&self) -> Cardinality {
        assert_matches!(self.plan.convention(), Convention::Logical);
        CardinalityVisitor.visit(self.plan.clone())
    }

    /// Optimize and generate a create table plan.
    #[allow(clippy::too_many_arguments)]
    pub fn gen_table_plan(
        mut self,
        context: OptimizerContextRef,
        table_name: String,
        columns: Vec<ColumnCatalog>,
        definition: String,
        pk_column_ids: Vec<ColumnId>,
        row_id_index: Option<usize>,
        append_only: bool,
        on_conflict: Option<OnConflict>,
        with_version_column: Option<String>,
        watermark_descs: Vec<WatermarkDesc>,
        version: Option<TableVersion>,
        with_external_source: bool,
        retention_seconds: Option<NonZeroU32>,
        cdc_table_id: Option<String>,
    ) -> Result<StreamMaterialize> {
        assert_eq!(self.phase, PlanPhase::Logical);
        assert_eq!(self.plan.convention(), Convention::Logical);
        // Snapshot backfill is not allowed for create table
        let stream_plan = self.gen_optimized_stream_plan(false, false)?;
        assert_eq!(self.phase, PlanPhase::Stream);
        assert_eq!(stream_plan.convention(), Convention::Stream);

        assert!(!pk_column_ids.is_empty() || row_id_index.is_some());

        let pk_column_indices = {
            let mut id_to_idx = HashMap::new();

            columns.iter().enumerate().for_each(|(idx, c)| {
                id_to_idx.insert(c.column_id(), idx);
            });
            pk_column_ids
                .iter()
                .map(|c| id_to_idx.get(c).copied().unwrap()) // pk column id must exist in table columns.
                .collect_vec()
        };

        fn inject_project_for_generated_column_if_needed(
            columns: &[ColumnCatalog],
            node: PlanRef,
        ) -> Result<PlanRef> {
            let exprs = LogicalSource::derive_output_exprs_from_generated_columns(columns)?;
            if let Some(exprs) = exprs {
                let logical_project = generic::Project::new(exprs, node);
                return Ok(StreamProject::new(logical_project).into());
            }
            Ok(node)
        }

        #[derive(PartialEq, Debug, Copy, Clone)]
        enum PrimaryKeyKind {
            UserDefinedPrimaryKey,
            NonAppendOnlyRowIdPk,
            AppendOnlyRowIdPk,
        }

        fn inject_dml_node(
            columns: &[ColumnCatalog],
            append_only: bool,
            stream_plan: PlanRef,
            pk_column_indices: &[usize],
            kind: PrimaryKeyKind,
            column_descs: Vec<ColumnDesc>,
        ) -> Result<PlanRef> {
            let mut dml_node = StreamDml::new(stream_plan, append_only, column_descs).into();

            // Add generated columns.
            dml_node = inject_project_for_generated_column_if_needed(columns, dml_node)?;

            dml_node = match kind {
                PrimaryKeyKind::UserDefinedPrimaryKey | PrimaryKeyKind::NonAppendOnlyRowIdPk => {
                    RequiredDist::hash_shard(pk_column_indices)
                        .enforce_if_not_satisfies(dml_node, &Order::any())?
                }
                PrimaryKeyKind::AppendOnlyRowIdPk => {
                    StreamExchange::new_no_shuffle(dml_node).into()
                }
            };

            Ok(dml_node)
        }

        let kind = if let Some(row_id_index) = row_id_index {
            assert_eq!(
                pk_column_indices.iter().exactly_one().copied().unwrap(),
                row_id_index
            );
            if append_only {
                PrimaryKeyKind::AppendOnlyRowIdPk
            } else {
                PrimaryKeyKind::NonAppendOnlyRowIdPk
            }
        } else {
            PrimaryKeyKind::UserDefinedPrimaryKey
        };

        let column_descs = columns
            .iter()
            .filter(|&c| c.can_dml())
            .map(|c| c.column_desc.clone())
            .collect();

        let version_column_index = if let Some(version_column) = with_version_column {
            find_version_column_index(&columns, version_column)?
        } else {
            None
        };

        let union_inputs = if with_external_source {
            let mut external_source_node = stream_plan;
            external_source_node =
                inject_project_for_generated_column_if_needed(&columns, external_source_node)?;
            external_source_node = match kind {
                PrimaryKeyKind::UserDefinedPrimaryKey => {
                    RequiredDist::hash_shard(&pk_column_indices)
                        .enforce_if_not_satisfies(external_source_node, &Order::any())?
                }

                PrimaryKeyKind::NonAppendOnlyRowIdPk | PrimaryKeyKind::AppendOnlyRowIdPk => {
                    StreamExchange::new_no_shuffle(external_source_node).into()
                }
            };

            let dummy_source_node = LogicalSource::new(
                None,
                columns.clone(),
                row_id_index,
                SourceNodeKind::CreateTable,
                context.clone(),
                None,
            )
            .and_then(|s| s.to_stream(&mut ToStreamContext::new(false)))?;

            let dml_node = inject_dml_node(
                &columns,
                append_only,
                dummy_source_node,
                &pk_column_indices,
                kind,
                column_descs,
            )?;

            vec![external_source_node, dml_node]
        } else {
            let dml_node = inject_dml_node(
                &columns,
                append_only,
                stream_plan,
                &pk_column_indices,
                kind,
                column_descs,
            )?;

            vec![dml_node]
        };

        let dists = union_inputs
            .iter()
            .map(|input| input.distribution())
            .unique()
            .collect_vec();

        let dist = match &dists[..] {
            &[Distribution::SomeShard, Distribution::HashShard(_)]
            | &[Distribution::HashShard(_), Distribution::SomeShard] => Distribution::SomeShard,
            &[dist @ Distribution::SomeShard] | &[dist @ Distribution::HashShard(_)] => {
                dist.clone()
            }
            _ => {
                unreachable!()
            }
        };

        let mut stream_plan = StreamUnion::new_with_dist(
            Union {
                all: true,
                inputs: union_inputs,
                source_col: None,
            },
            dist.clone(),
        )
        .into();

        // Add WatermarkFilter node.
        if !watermark_descs.is_empty() {
            stream_plan = StreamWatermarkFilter::new(stream_plan, watermark_descs).into();
        }

        // Add RowIDGen node if needed.
        if let Some(row_id_index) = row_id_index {
            match kind {
                PrimaryKeyKind::UserDefinedPrimaryKey => {
                    unreachable!()
                }
                PrimaryKeyKind::NonAppendOnlyRowIdPk | PrimaryKeyKind::AppendOnlyRowIdPk => {
                    stream_plan = StreamRowIdGen::new_with_dist(
                        stream_plan,
                        row_id_index,
                        Distribution::HashShard(vec![row_id_index]),
                    )
                    .into();
                }
            }
        }

        let conflict_behavior = match on_conflict {
            Some(on_conflict) => match on_conflict {
                OnConflict::UpdateFull => ConflictBehavior::Overwrite,
                OnConflict::Nothing => ConflictBehavior::IgnoreConflict,
                OnConflict::UpdateIfNotNull => ConflictBehavior::DoUpdateIfNotNull,
            },
            None => match append_only {
                true => ConflictBehavior::NoCheck,
                false => ConflictBehavior::Overwrite,
            },
        };

        if let ConflictBehavior::IgnoreConflict = conflict_behavior
            && version_column_index.is_some()
        {
            Err(ErrorCode::InvalidParameterValue(
                "The with version column syntax cannot be used with the ignore behavior of on conflict".to_string(),
            ))?
        }

        let table_required_dist = {
            let mut bitset = FixedBitSet::with_capacity(columns.len());
            for idx in &pk_column_indices {
                bitset.insert(*idx);
            }
            RequiredDist::ShardByKey(bitset)
        };

        let stream_plan = inline_session_timezone_in_exprs(context, stream_plan)?;

        StreamMaterialize::create_for_table(
            stream_plan,
            table_name,
            table_required_dist,
            Order::any(),
            columns,
            definition,
            conflict_behavior,
            version_column_index,
            pk_column_indices,
            row_id_index,
            version,
            retention_seconds,
            cdc_table_id,
        )
    }

    /// Optimize and generate a create materialized view plan.
    pub fn gen_materialize_plan(
        mut self,
        mv_name: String,
        definition: String,
        emit_on_window_close: bool,
    ) -> Result<StreamMaterialize> {
        let cardinality = self.compute_cardinality();
        assert_eq!(self.phase, PlanPhase::Logical);
        assert_eq!(self.plan.convention(), Convention::Logical);
        let stream_plan = self.gen_optimized_stream_plan(emit_on_window_close, true)?;
        assert_eq!(self.phase, PlanPhase::Stream);
        assert_eq!(stream_plan.convention(), Convention::Stream);
        StreamMaterialize::create(
            stream_plan,
            mv_name,
            self.required_dist.clone(),
            self.required_order.clone(),
            self.out_fields.clone(),
            self.out_names.clone(),
            definition,
            TableType::MaterializedView,
            cardinality,
            None,
        )
    }

    /// Optimize and generate a create index plan.
    pub fn gen_index_plan(
        mut self,
        index_name: String,
        definition: String,
        retention_seconds: Option<NonZeroU32>,
    ) -> Result<StreamMaterialize> {
        let cardinality = self.compute_cardinality();
        assert_eq!(self.phase, PlanPhase::Logical);
        assert_eq!(self.plan.convention(), Convention::Logical);
        let stream_plan = self.gen_optimized_stream_plan(false, false)?;
        assert_eq!(self.phase, PlanPhase::Stream);
        assert_eq!(stream_plan.convention(), Convention::Stream);

        StreamMaterialize::create(
            stream_plan,
            index_name,
            self.required_dist.clone(),
            self.required_order.clone(),
            self.out_fields.clone(),
            self.out_names.clone(),
            definition,
            TableType::Index,
            cardinality,
            retention_seconds,
        )
    }

    /// Optimize and generate a create sink plan.
    #[allow(clippy::too_many_arguments)]
    pub fn gen_sink_plan(
        &mut self,
        sink_name: String,
        definition: String,
        properties: WithOptionsSecResolved,
        emit_on_window_close: bool,
        db_name: String,
        sink_from_table_name: String,
        format_desc: Option<SinkFormatDesc>,
        without_backfill: bool,
        target_table: Option<Arc<TableCatalog>>,
        partition_info: Option<PartitionComputeInfo>,
        user_specified_columns: bool,
    ) -> Result<StreamSink> {
        let stream_scan_type = if without_backfill {
            StreamScanType::UpstreamOnly
        } else if target_table.is_none() && self.should_use_snapshot_backfill() {
            // Snapshot backfill on sink-into-table is not allowed
            StreamScanType::SnapshotBackfill
        } else if self.should_use_arrangement_backfill() {
            StreamScanType::ArrangementBackfill
        } else {
            StreamScanType::Backfill
        };
        assert_eq!(self.phase, PlanPhase::Logical);
        assert_eq!(self.plan.convention(), Convention::Logical);
        let stream_plan =
            self.gen_optimized_stream_plan_inner(emit_on_window_close, stream_scan_type)?;
        assert_eq!(self.phase, PlanPhase::Stream);
        assert_eq!(stream_plan.convention(), Convention::Stream);
        let target_columns_to_plan_mapping = target_table.as_ref().map(|t| {
            let columns = t.columns_without_rw_timestamp();
            self.target_columns_to_plan_mapping(&columns, user_specified_columns)
        });
        StreamSink::create(
            stream_plan,
            sink_name,
            db_name,
            sink_from_table_name,
            target_table,
            target_columns_to_plan_mapping,
            self.required_dist.clone(),
            self.required_order.clone(),
            self.out_fields.clone(),
            self.out_names.clone(),
            definition,
            properties,
            format_desc,
            partition_info,
        )
    }

    pub fn should_use_arrangement_backfill(&self) -> bool {
        let ctx = self.plan.ctx();
        let session_ctx = ctx.session_ctx();
        let arrangement_backfill_enabled = session_ctx
            .env()
            .streaming_config()
            .developer
            .enable_arrangement_backfill;
        arrangement_backfill_enabled && session_ctx.config().streaming_use_arrangement_backfill()
    }

    pub fn should_use_snapshot_backfill(&self) -> bool {
        self.plan
            .ctx()
            .session_ctx()
            .config()
            .streaming_use_snapshot_backfill()
    }

    /// used when the plan has a target relation such as DML and sink into table, return the mapping from table's columns to the plan's schema
    pub fn target_columns_to_plan_mapping(
        &self,
        tar_cols: &[ColumnCatalog],
        user_specified_columns: bool,
    ) -> Vec<Option<usize>> {
        #[allow(clippy::disallowed_methods)]
        let visible_cols: Vec<(usize, String)> = self
            .out_fields
            .ones()
            .zip_eq(self.out_names.iter().cloned())
            .collect_vec();

        let visible_col_idxes = visible_cols.iter().map(|(i, _)| *i).collect_vec();
        let visible_col_idxes_by_name = visible_cols
            .iter()
            .map(|(i, name)| (name.as_ref(), *i))
            .collect::<BTreeMap<_, _>>();

        tar_cols
            .iter()
            .enumerate()
            .filter(|(_, tar_col)| tar_col.can_dml())
            .map(|(tar_i, tar_col)| {
                if user_specified_columns {
                    visible_col_idxes_by_name.get(tar_col.name()).cloned()
                } else {
                    (tar_i < visible_col_idxes.len()).then(|| visible_cols[tar_i].0)
                }
            })
            .collect()
    }
}

fn find_version_column_index(
    column_catalog: &Vec<ColumnCatalog>,
    version_column_name: String,
) -> Result<Option<usize>> {
    for (index, column) in column_catalog.iter().enumerate() {
        if column.column_desc.name == version_column_name {
            if let &DataType::Jsonb
            | &DataType::List(_)
            | &DataType::Struct(_)
            | &DataType::Bytea
            | &DataType::Boolean = column.data_type()
            {
                Err(ErrorCode::InvalidParameterValue(
                    "The specified version column data type is invalid.".to_string(),
                ))?
            }
            return Ok(Some(index));
        }
    }
    Err(ErrorCode::InvalidParameterValue(
        "The specified version column name is not in the current columns.".to_string(),
    ))?
}

fn const_eval_exprs(plan: PlanRef) -> Result<PlanRef> {
    let mut const_eval_rewriter = ConstEvalRewriter { error: None };

    let plan = plan.rewrite_exprs_recursive(&mut const_eval_rewriter);
    if let Some(error) = const_eval_rewriter.error {
        return Err(error);
    }
    Ok(plan)
}

fn inline_session_timezone_in_exprs(ctx: OptimizerContextRef, plan: PlanRef) -> Result<PlanRef> {
    let mut v = TimestamptzExprFinder::default();
    plan.visit_exprs_recursive(&mut v);
    if v.has() {
        Ok(plan.rewrite_exprs_recursive(ctx.session_timezone().deref_mut()))
    } else {
        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.node_type() == PlanNodeType::BatchSeqScan
    }

    fn is_log_table(plan: &PlanRef) -> bool {
        plan.node_type() == PlanNodeType::BatchLogSeqScan
    }

    fn is_source(plan: &PlanRef) -> bool {
        plan.node_type() == PlanNodeType::BatchSource
            || plan.node_type() == PlanNodeType::BatchKafkaScan
            || plan.node_type() == PlanNodeType::BatchIcebergScan
    }

    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)
        || exist_and_no_exchange_before(&plan, is_log_table)
}

/// 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.node_type() == PlanNodeType::BatchSeqScan
    }

    fn is_source(plan: &PlanRef) -> bool {
        plan.node_type() == PlanNodeType::BatchSource
            || plan.node_type() == PlanNodeType::BatchKafkaScan
            || plan.node_type() == PlanNodeType::BatchIcebergScan
    }

    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 super::*;
    use crate::optimizer::plan_node::LogicalValues;

    #[tokio::test]
    async fn test_as_subplan() {
        let ctx = OptimizerContext::mock().await;
        let values = LogicalValues::new(
            vec![],
            Schema::new(vec![
                Field::with_name(DataType::Int32, "v1"),
                Field::with_name(DataType::Varchar, "v2"),
            ]),
            ctx,
        )
        .into();
        let out_fields = FixedBitSet::with_capacity_and_blocks(2, [1]);
        let out_names = vec!["v1".into()];
        let root = PlanRoot::new_with_logical_plan(
            values,
            RequiredDist::Any,
            Order::any(),
            out_fields,
            out_names,
        );
        let subplan = root.into_unordered_subplan();
        assert_eq!(
            subplan.schema(),
            &Schema::new(vec![Field::with_name(DataType::Int32, "v1")])
        );
    }
}