Skip to main content

flow/batching_mode/
task.rs

1// Copyright 2023 Greptime Team
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::{BTreeSet, HashMap, HashSet};
16use std::sync::{Arc, RwLock};
17use std::time::{Duration, SystemTime, UNIX_EPOCH};
18
19use api::v1::CreateTableExpr;
20use catalog::CatalogManagerRef;
21use common_error::ext::BoxedError;
22use common_query::logical_plan::breakup_insert_plan;
23use common_telemetry::tracing::warn;
24use common_telemetry::{debug, info};
25use common_time::Timestamp;
26use datafusion::datasource::DefaultTableSource;
27use datafusion::sql::unparser::expr_to_sql;
28use datafusion_common::DFSchemaRef;
29use datafusion_common::tree_node::{Transformed, TreeNode};
30use datafusion_expr::{DmlStatement, LogicalPlan, WriteOp, col, lit};
31use datatypes::schema::Schema;
32use query::QueryEngineRef;
33use query::options::FLOW_INCREMENTAL_MODE;
34use query::query_engine::DefaultSerializer;
35use session::context::QueryContextRef;
36use snafu::{OptionExt, ResultExt};
37use sql::parsers::utils::is_tql;
38use store_api::mito_engine_options::MERGE_MODE_KEY;
39use substrait::{DFLogicalSubstraitConvertor, SubstraitPlan};
40use table::table::adapter::DfTableProviderAdapter;
41use tokio::sync::oneshot::error::TryRecvError;
42use tokio::sync::{Mutex, oneshot};
43use tokio::time::Instant;
44
45use crate::batching_mode::BatchingModeOptions;
46use crate::batching_mode::checkpoint::checkpoint_mode_label;
47use crate::batching_mode::frontend_client::{FrontendClient, PeerDesc};
48use crate::batching_mode::state::{
49    CheckpointMode, DirtyTimeWindows, FilterExprInfo, TaskState, to_df_literal,
50};
51use crate::batching_mode::table_creator::{QueryType, create_table_with_expr};
52use crate::batching_mode::time_window::TimeWindowExpr;
53use crate::batching_mode::utils::{
54    AddFilterRewriter, ColumnMatcherRewriter, gen_plan_with_matching_schema,
55    get_table_info_df_schema, sql_to_df_plan,
56};
57use crate::df_optimizer::apply_df_optimizer;
58use crate::error::{
59    DatafusionSnafu, ExternalSnafu, InvalidQuerySnafu, SubstraitEncodeLogicalPlanSnafu,
60    UnexpectedSnafu,
61};
62use crate::metrics::{
63    METRIC_FLOW_BATCHING_ENGINE_ERROR_CNT, METRIC_FLOW_BATCHING_ENGINE_QUERY_TIME,
64    METRIC_FLOW_BATCHING_ENGINE_SLOW_QUERY, METRIC_FLOW_BATCHING_ENGINE_START_QUERY_CNT,
65    METRIC_FLOW_ROWS,
66};
67use crate::{Error, FlowId};
68
69mod ckpt;
70mod inc;
71
72/// The task's config, immutable once created
73#[derive(Clone)]
74pub struct TaskConfig {
75    pub flow_id: FlowId,
76    pub query: String,
77    /// output schema of the query
78    pub output_schema: DFSchemaRef,
79    pub time_window_expr: Option<TimeWindowExpr>,
80    /// in seconds
81    pub expire_after: Option<i64>,
82    pub sink_table_name: [String; 3],
83    pub source_table_names: HashSet<[String; 3]>,
84    pub catalog_manager: CatalogManagerRef,
85    pub query_type: QueryType,
86    pub batch_opts: Arc<BatchingModeOptions>,
87    pub flow_eval_interval: Option<Duration>,
88}
89
90fn determine_query_type(query: &str, query_ctx: &QueryContextRef) -> Result<QueryType, Error> {
91    let is_tql = is_tql(query_ctx.sql_dialect(), query)
92        .map_err(BoxedError::new)
93        .context(ExternalSnafu)?;
94    Ok(if is_tql {
95        QueryType::Tql
96    } else {
97        QueryType::Sql
98    })
99}
100
101fn is_merge_mode_last_non_null(options: &HashMap<String, String>) -> bool {
102    options
103        .get(MERGE_MODE_KEY)
104        .map(|mode| mode.eq_ignore_ascii_case("last_non_null"))
105        .unwrap_or(false)
106}
107
108#[derive(Clone)]
109pub struct BatchingTask {
110    pub config: Arc<TaskConfig>,
111    pub state: Arc<RwLock<TaskState>>,
112    /// Serializes plan generation, execution, checkpoint advancement, and dirty
113    /// window restoration for this flow. Without this, a manual flush and the
114    /// background loop can process the same checkpoint range concurrently.
115    execution_lock: Arc<Mutex<()>>,
116}
117
118/// Arguments for creating batching task
119pub struct TaskArgs<'a> {
120    pub flow_id: FlowId,
121    pub query: &'a str,
122    pub plan: LogicalPlan,
123    pub time_window_expr: Option<TimeWindowExpr>,
124    pub expire_after: Option<i64>,
125    pub sink_table_name: [String; 3],
126    pub source_table_names: Vec<[String; 3]>,
127    pub query_ctx: QueryContextRef,
128    pub catalog_manager: CatalogManagerRef,
129    pub shutdown_rx: oneshot::Receiver<()>,
130    pub batch_opts: Arc<BatchingModeOptions>,
131    pub flow_eval_interval: Option<Duration>,
132}
133
134pub struct PlanInfo {
135    pub plan: LogicalPlan,
136    pub dirty_restore: DirtyRestore,
137    pub can_advance_checkpoints: bool,
138}
139
140pub enum DirtyRestore {
141    /// The query was scoped to dirty time ranges; restore those ranges if the
142    /// run fails.
143    Scoped(FilterExprInfo),
144    /// The query could not be scoped to dirty time ranges, so the dirty-window
145    /// state is only a dirty signal. Restore the consumed signal if the full
146    /// run fails.
147    ///
148    /// TODO(discord9): Full-query runs only need a dirty bool flag. Refactor
149    /// the unscoped path to stop reusing `DirtyTimeWindows` for this signal.
150    Unscoped(DirtyTimeWindows),
151}
152
153struct ExecuteOnceOutcome {
154    new_query: Option<PlanInfo>,
155    /// Execution result of the generated insert plan.
156    ///
157    /// `Ok(Some((affected_rows, elapsed)))` means a query was executed.
158    /// `Ok(None)` means no query was generated because there was no dirty signal.
159    /// `Err(_)` means plan generation or execution failed.
160    result: Result<Option<(usize, Duration)>, Error>,
161}
162
163impl BatchingTask {
164    #[allow(clippy::too_many_arguments)]
165    pub fn try_new(
166        TaskArgs {
167            flow_id,
168            query,
169            plan,
170            time_window_expr,
171            expire_after,
172            sink_table_name,
173            source_table_names,
174            query_ctx,
175            catalog_manager,
176            shutdown_rx,
177            batch_opts,
178            flow_eval_interval,
179        }: TaskArgs<'_>,
180    ) -> Result<Self, Error> {
181        let mut state = TaskState::with_dirty_time_windows(
182            query_ctx.clone(),
183            shutdown_rx,
184            DirtyTimeWindows::new(
185                batch_opts.experimental_max_filter_num_per_query,
186                batch_opts.experimental_time_window_merge_threshold,
187            ),
188        );
189        if !batch_opts.experimental_enable_incremental_read {
190            state.disable_incremental();
191        }
192
193        Ok(Self {
194            config: Arc::new(TaskConfig {
195                flow_id,
196                query: query.to_string(),
197                time_window_expr,
198                expire_after,
199                sink_table_name,
200                source_table_names: source_table_names.into_iter().collect(),
201                catalog_manager,
202                output_schema: plan.schema().clone(),
203                query_type: determine_query_type(query, &query_ctx)?,
204                batch_opts,
205                flow_eval_interval,
206            }),
207            state: Arc::new(RwLock::new(state)),
208            execution_lock: Arc::new(Mutex::new(())),
209        })
210    }
211
212    pub fn last_execution_time_millis(&self) -> Option<i64> {
213        self.state.read().unwrap().last_execution_time_millis()
214    }
215
216    /// mark time window range (now - expire_after, now) as dirty (or (0, now) if expire_after not set)
217    ///
218    /// useful for flush_flow to flush dirty time windows range
219    pub fn mark_all_windows_as_dirty(&self) -> Result<(), Error> {
220        let now = SystemTime::now();
221        let now = Timestamp::new_second(
222            now.duration_since(UNIX_EPOCH)
223                .expect("Time went backwards")
224                .as_secs() as _,
225        );
226        let lower_bound = self
227            .config
228            .expire_after
229            .map(|e| now.sub_duration(Duration::from_secs(e as _)))
230            .transpose()
231            .map_err(BoxedError::new)
232            .context(ExternalSnafu)?
233            .unwrap_or(Timestamp::new_second(0));
234        debug!(
235            "Flow {} mark range ({:?}, {:?}) as dirty",
236            self.config.flow_id, lower_bound, now
237        );
238        self.state
239            .write()
240            .unwrap()
241            .dirty_time_windows
242            .add_window(lower_bound, Some(now));
243        Ok(())
244    }
245
246    /// Create sink table if not exists
247    pub async fn check_or_create_sink_table(
248        &self,
249        engine: &QueryEngineRef,
250        frontend_client: &Arc<FrontendClient>,
251    ) -> Result<Option<(usize, Duration)>, Error> {
252        if !self.is_table_exist(&self.config.sink_table_name).await? {
253            let create_table = self.gen_create_table_expr(engine.clone()).await?;
254            info!(
255                "Try creating sink table(if not exists) with expr: {:?}",
256                create_table
257            );
258            self.create_table(frontend_client, create_table).await?;
259            info!(
260                "Sink table {}(if not exists) created",
261                self.config.sink_table_name.join(".")
262            );
263        }
264
265        Ok(None)
266    }
267
268    /// Validates that the sink table schema can accept this flow's output.
269    ///
270    /// This is a dry-run of the same schema matching logic used by runtime insert-plan
271    /// generation, but without adding dirty-window filters or executing the query. It is used
272    /// during CREATE FLOW to catch existing sink table mismatches early.
273    pub async fn validate_sink_table_schema(&self, engine: &QueryEngineRef) -> Result<(), Error> {
274        let (table, _) = get_table_info_df_schema(
275            self.config.catalog_manager.clone(),
276            self.config.sink_table_name.clone(),
277        )
278        .await?;
279
280        let table_meta = &table.table_info().meta;
281        let merge_mode_last_non_null =
282            is_merge_mode_last_non_null(&table_meta.options.extra_options);
283        let primary_key_indices = table_meta.primary_key_indices.clone();
284        let query_ctx = self.state.read().unwrap().query_ctx.clone();
285
286        gen_plan_with_matching_schema(
287            &self.config.query,
288            query_ctx,
289            engine.clone(),
290            table_meta.schema.clone(),
291            &primary_key_indices,
292            merge_mode_last_non_null,
293        )
294        .await
295        .map(|_| ())
296    }
297
298    async fn is_table_exist(&self, table_name: &[String; 3]) -> Result<bool, Error> {
299        self.config
300            .catalog_manager
301            .table_exists(&table_name[0], &table_name[1], &table_name[2], None)
302            .await
303            .map_err(BoxedError::new)
304            .context(ExternalSnafu)
305    }
306
307    pub(crate) async fn execute_once_serialized(
308        &self,
309        engine: &QueryEngineRef,
310        frontend_client: &Arc<FrontendClient>,
311        max_window_cnt: Option<usize>,
312    ) -> Result<Option<(usize, Duration)>, Error> {
313        let outcome = self
314            .execute_once_serialized_with_outcome(engine, frontend_client, max_window_cnt)
315            .await;
316        outcome.result
317    }
318
319    /// Executes one flow evaluation under `execution_lock` and keeps the
320    /// generated query context for the background loop's error logging/backoff.
321    async fn execute_once_serialized_with_outcome(
322        &self,
323        engine: &QueryEngineRef,
324        frontend_client: &Arc<FrontendClient>,
325        max_window_cnt: Option<usize>,
326    ) -> ExecuteOnceOutcome {
327        let _execution_guard = self.execution_lock.lock().await;
328        self.execute_once_unlocked(engine, frontend_client, max_window_cnt)
329            .await
330    }
331
332    /// Executes one flow evaluation. Caller must hold `execution_lock`.
333    async fn execute_once_unlocked(
334        &self,
335        engine: &QueryEngineRef,
336        frontend_client: &Arc<FrontendClient>,
337        max_window_cnt: Option<usize>,
338    ) -> ExecuteOnceOutcome {
339        let new_query = match self.gen_insert_plan_unlocked(engine, max_window_cnt).await {
340            Ok(new_query) => new_query,
341            Err(err) => {
342                return ExecuteOnceOutcome {
343                    new_query: None,
344                    result: Err(err),
345                };
346            }
347        };
348
349        if let Some(new_query) = new_query {
350            debug!("Generate new query: {}", new_query.plan);
351            let res = self
352                .execute_logical_plan_unlocked(
353                    frontend_client,
354                    &new_query.plan,
355                    new_query.can_advance_checkpoints,
356                )
357                .await;
358            if res.is_err() {
359                self.handle_executed_query_failure(Some(&new_query));
360            }
361            ExecuteOnceOutcome {
362                new_query: Some(new_query),
363                result: res,
364            }
365        } else {
366            debug!("Generate no query");
367            ExecuteOnceOutcome {
368                new_query: None,
369                result: Ok(None),
370            }
371        }
372    }
373
374    /// Generates the insert plan. Caller must reach this through the serialized path.
375    async fn gen_insert_plan_unlocked(
376        &self,
377        engine: &QueryEngineRef,
378        max_window_cnt: Option<usize>,
379    ) -> Result<Option<PlanInfo>, Error> {
380        let (table, df_schema) = get_table_info_df_schema(
381            self.config.catalog_manager.clone(),
382            self.config.sink_table_name.clone(),
383        )
384        .await?;
385
386        let table_meta = &table.table_info().meta;
387        let merge_mode_last_non_null =
388            is_merge_mode_last_non_null(&table_meta.options.extra_options);
389        let primary_key_indices = table_meta.primary_key_indices.clone();
390
391        let new_query = self
392            .gen_query_with_time_window(
393                engine.clone(),
394                &table.table_info().meta.schema,
395                &primary_key_indices,
396                merge_mode_last_non_null,
397                max_window_cnt,
398            )
399            .await?;
400
401        let Some(new_query) = new_query else {
402            return Ok(None);
403        };
404
405        // first check if all columns in input query exists in sink table
406        // since insert into ref to names in record batch generate by given query
407        let table_columns = df_schema
408            .columns()
409            .into_iter()
410            .map(|c| c.name)
411            .collect::<BTreeSet<_>>();
412        for column in new_query.plan.schema().columns() {
413            if !table_columns.contains(column.name()) {
414                self.restore_dirty_windows_after_failure(&new_query);
415                return InvalidQuerySnafu {
416                    reason: format!(
417                        "Column {} not found in sink table with columns {:?}",
418                        column, table_columns
419                    ),
420                }
421                .fail();
422            }
423        }
424
425        let table_provider = Arc::new(DfTableProviderAdapter::new(table));
426        let table_source = Arc::new(DefaultTableSource::new(table_provider));
427
428        // update_at& time index placeholder (if exists) should have default value
429        let plan = LogicalPlan::Dml(DmlStatement::new(
430            datafusion_common::TableReference::Full {
431                catalog: self.config.sink_table_name[0].clone().into(),
432                schema: self.config.sink_table_name[1].clone().into(),
433                table: self.config.sink_table_name[2].clone().into(),
434            },
435            table_source,
436            WriteOp::Insert(datafusion_expr::dml::InsertOp::Append),
437            Arc::new(new_query.plan.clone()),
438        ));
439        let insert_into_info = PlanInfo {
440            plan,
441            dirty_restore: new_query.dirty_restore,
442            can_advance_checkpoints: new_query.can_advance_checkpoints,
443        };
444        let insert_into =
445            match insert_into_info
446                .plan
447                .clone()
448                .recompute_schema()
449                .context(DatafusionSnafu {
450                    context: "Failed to recompute schema",
451                }) {
452                Ok(insert_into) => insert_into,
453                Err(err) => {
454                    self.restore_dirty_windows_after_failure(&insert_into_info);
455                    return Err(err);
456                }
457            };
458
459        Ok(Some(PlanInfo {
460            plan: insert_into,
461            dirty_restore: insert_into_info.dirty_restore,
462            can_advance_checkpoints: insert_into_info.can_advance_checkpoints,
463        }))
464    }
465
466    pub async fn create_table(
467        &self,
468        frontend_client: &Arc<FrontendClient>,
469        expr: CreateTableExpr,
470    ) -> Result<(), Error> {
471        let catalog = &self.config.sink_table_name[0];
472        let schema = &self.config.sink_table_name[1];
473        frontend_client
474            .create(expr.clone(), catalog, schema)
475            .await?;
476        Ok(())
477    }
478
479    /// Executes the insert plan. Caller must reach this through the serialized path.
480    async fn execute_logical_plan_unlocked(
481        &self,
482        frontend_client: &Arc<FrontendClient>,
483        plan: &LogicalPlan,
484        can_advance_checkpoints: bool,
485    ) -> Result<Option<(usize, Duration)>, Error> {
486        let instant = Instant::now();
487        let flow_id = self.config.flow_id;
488
489        debug!(
490            "Executing flow {flow_id}(expire_after={:?} secs) with query {}",
491            self.config.expire_after, &plan
492        );
493
494        let catalog = &self.config.sink_table_name[0];
495        let schema = &self.config.sink_table_name[1];
496
497        // fix all table ref by make it fully qualified, i.e. "table_name" => "catalog_name.schema_name.table_name"
498        let plan = plan
499            .clone()
500            .transform_down_with_subqueries(|p| {
501                if let LogicalPlan::TableScan(mut table_scan) = p {
502                    let resolved = table_scan.table_name.resolve(catalog, schema);
503                    table_scan.table_name = resolved.into();
504                    Ok(Transformed::yes(LogicalPlan::TableScan(table_scan)))
505                } else {
506                    Ok(Transformed::no(p))
507                }
508            })
509            .with_context(|_| DatafusionSnafu {
510                context: format!("Failed to fix table ref in logical plan, plan={:?}", plan),
511            })?
512            .data;
513
514        // For incremental-mode SQL queries, attempt to rewrite the delta aggregate
515        // plan into a safe delta-LEFT-JOIN-sink form before deciding on extensions.
516        let incremental_plan = if can_advance_checkpoints {
517            self.prepare_plan_for_incremental(&plan).await?
518        } else {
519            None
520        };
521        let incremental_safe = incremental_plan.is_some();
522        let plan = incremental_plan.unwrap_or_else(|| plan.clone());
523
524        let extensions = self
525            .build_flow_query_extensions(incremental_safe, can_advance_checkpoints)
526            .await?;
527        let extension_refs = extensions
528            .iter()
529            .map(|(key, value)| (*key, value.as_str()))
530            .collect::<Vec<_>>();
531        let query_mode = if extensions
532            .iter()
533            .any(|(key, _)| *key == FLOW_INCREMENTAL_MODE)
534        {
535            CheckpointMode::Incremental
536        } else {
537            CheckpointMode::FullSnapshot
538        };
539        Self::record_query_mode(flow_id, query_mode);
540        debug!(
541            "Flow {flow_id} executing batching query with checkpoint_mode={}, extension_count={}",
542            checkpoint_mode_label(query_mode),
543            extensions.len()
544        );
545
546        let mut peer_desc = None;
547        let res = {
548            let _timer = METRIC_FLOW_BATCHING_ENGINE_QUERY_TIME
549                .with_label_values(&[flow_id.to_string().as_str()])
550                .start_timer();
551
552            let req = if let Some((insert_to, insert_plan)) =
553                breakup_insert_plan(&plan, catalog, schema)
554            {
555                let message = DFLogicalSubstraitConvertor {}
556                    .encode(&insert_plan, DefaultSerializer)
557                    .context(SubstraitEncodeLogicalPlanSnafu)?;
558                api::v1::QueryRequest {
559                    query: Some(api::v1::query_request::Query::InsertIntoPlan(
560                        api::v1::InsertIntoPlan {
561                            table_name: Some(insert_to),
562                            logical_plan: message.to_vec(),
563                        },
564                    )),
565                }
566            } else {
567                let message = DFLogicalSubstraitConvertor {}
568                    .encode(&plan, DefaultSerializer)
569                    .context(SubstraitEncodeLogicalPlanSnafu)?;
570
571                api::v1::QueryRequest {
572                    query: Some(api::v1::query_request::Query::LogicalPlan(message.to_vec())),
573                }
574            };
575
576            frontend_client
577                .query_with_terminal_metrics(catalog, schema, req, &extension_refs, &mut peer_desc)
578                .await
579        };
580
581        let elapsed = instant.elapsed();
582        let peer_label = peer_desc
583            .as_ref()
584            .map(ToString::to_string)
585            .unwrap_or_else(|| PeerDesc::default().to_string());
586        if let Err(err) = &res {
587            warn!(
588                "Failed to execute Flow {flow_id} on frontend {peer_label}, result: {err:?}, elapsed: {:?} with query: {}",
589                elapsed, &plan
590            );
591            let decision = {
592                let mut state = self.state.write().unwrap();
593                let reason = Self::query_failure_reason(err);
594                Self::apply_query_failure_to_state(&mut state, elapsed, reason)
595            };
596            if let Some(decision) = decision {
597                Self::record_checkpoint_decision(flow_id, decision);
598            }
599        }
600
601        // record slow query
602        if elapsed >= self.config.batch_opts.slow_query_threshold {
603            warn!(
604                "Flow {flow_id} on frontend {peer_label} executed for {:?} before complete, query: {}",
605                elapsed, &plan
606            );
607            let flow_id = flow_id.to_string();
608            METRIC_FLOW_BATCHING_ENGINE_SLOW_QUERY
609                .with_label_values(&[flow_id.as_str(), peer_label.as_str()])
610                .observe(elapsed.as_secs_f64());
611        }
612
613        let res = res?;
614        let (affected_rows, _) = res.output.extract_rows_and_cost();
615        debug!(
616            "Flow {flow_id} executed, affected_rows: {affected_rows:?}, elapsed: {:?}, watermark: {:?}",
617            elapsed,
618            res.region_watermark_map()
619        );
620        METRIC_FLOW_ROWS
621            .with_label_values(&[format!("{}-out-batching", flow_id).as_str()])
622            .inc_by(affected_rows as _);
623        {
624            let mut state = self.state.write().unwrap();
625            let decision = Self::apply_query_result_to_state(
626                &mut state,
627                &res,
628                elapsed,
629                can_advance_checkpoints,
630            );
631            Self::record_checkpoint_decision(flow_id, decision);
632        }
633
634        Ok(Some((affected_rows, elapsed)))
635    }
636
637    /// Restore dirty windows consumed by a failed query so they are retried on
638    /// the next execution.
639    ///
640    fn restore_dirty_windows_after_failure(&self, query: &PlanInfo) {
641        match &query.dirty_restore {
642            DirtyRestore::Scoped(filter) => self.restore_scoped_dirty_windows(filter),
643            DirtyRestore::Unscoped(dirty_windows) => self
644                .state
645                .write()
646                .unwrap()
647                .dirty_time_windows
648                .add_dirty_windows(dirty_windows),
649        }
650    }
651
652    fn restore_scoped_dirty_windows(&self, filter: &FilterExprInfo) {
653        self.state
654            .write()
655            .unwrap()
656            .dirty_time_windows
657            .add_windows(filter.time_ranges.clone());
658    }
659
660    fn restore_scoped_dirty_windows_on_err<T>(
661        &self,
662        filter: &FilterExprInfo,
663        result: Result<T, Error>,
664    ) -> Result<T, Error> {
665        result.inspect_err(|_| {
666            self.restore_scoped_dirty_windows(filter);
667        })
668    }
669
670    fn restore_unscoped_dirty_windows(&self, dirty_windows: &DirtyTimeWindows) {
671        self.state
672            .write()
673            .unwrap()
674            .dirty_time_windows
675            .add_dirty_windows(dirty_windows);
676    }
677
678    fn restore_unscoped_dirty_windows_on_err<T>(
679        &self,
680        dirty_windows: &DirtyTimeWindows,
681        result: Result<T, Error>,
682    ) -> Result<T, Error> {
683        result.inspect_err(|_| {
684            self.restore_unscoped_dirty_windows(dirty_windows);
685        })
686    }
687
688    fn drain_dirty_windows_signal(&self) -> (bool, DirtyTimeWindows) {
689        let mut state = self.state.write().unwrap();
690        let dirty_windows_to_restore = state.dirty_time_windows.clone();
691        let is_dirty = !dirty_windows_to_restore.is_empty();
692        state.dirty_time_windows.clean();
693        (is_dirty, dirty_windows_to_restore)
694    }
695
696    #[allow(clippy::too_many_arguments)]
697    async fn gen_unfiltered_plan_info(
698        &self,
699        engine: QueryEngineRef,
700        query_ctx: QueryContextRef,
701        sink_table_schema: Arc<Schema>,
702        primary_key_indices: &[usize],
703        allow_partial: bool,
704        dirty_windows_to_restore: DirtyTimeWindows,
705        retention_filter: Option<(&str, Timestamp, &'static str)>,
706    ) -> Result<PlanInfo, Error> {
707        let mut plan = self.restore_unscoped_dirty_windows_on_err(
708            &dirty_windows_to_restore,
709            gen_plan_with_matching_schema(
710                &self.config.query,
711                query_ctx,
712                engine,
713                sink_table_schema,
714                primary_key_indices,
715                allow_partial,
716            )
717            .await,
718        )?;
719
720        if let Some((col_name, lower_bound, context)) = retention_filter {
721            let lower = self.restore_unscoped_dirty_windows_on_err(
722                &dirty_windows_to_restore,
723                to_df_literal(lower_bound),
724            )?;
725            let retention_filter = col(col_name).gt_eq(lit(lower));
726            let mut add_filter = AddFilterRewriter::new(retention_filter);
727            plan = self.restore_unscoped_dirty_windows_on_err(
728                &dirty_windows_to_restore,
729                plan.clone()
730                    .rewrite(&mut add_filter)
731                    .with_context(|_| DatafusionSnafu {
732                        context: format!(
733                            "Failed to apply {context} expire_after filter to plan:\n {}\n",
734                            plan
735                        ),
736                    })
737                    .map(|rewrite| rewrite.data),
738            )?;
739        }
740
741        Ok(PlanInfo {
742            plan,
743            dirty_restore: DirtyRestore::Unscoped(dirty_windows_to_restore),
744            can_advance_checkpoints: true,
745        })
746    }
747
748    async fn gen_unfiltered_plan_info_if_dirty(
749        &self,
750        engine: QueryEngineRef,
751        query_ctx: QueryContextRef,
752        sink_table_schema: Arc<Schema>,
753        primary_key_indices: &[usize],
754        allow_partial: bool,
755        retention_filter: Option<(&str, Timestamp, &'static str)>,
756    ) -> Result<Option<PlanInfo>, Error> {
757        let (is_dirty, dirty_windows_to_restore) = self.drain_dirty_windows_signal();
758        if !is_dirty {
759            debug!("Flow id={:?}, no new data, not update", self.config.flow_id);
760            return Ok(None);
761        }
762
763        self.gen_unfiltered_plan_info(
764            engine,
765            query_ctx,
766            sink_table_schema,
767            primary_key_indices,
768            allow_partial,
769            dirty_windows_to_restore,
770            retention_filter,
771        )
772        .await
773        .map(Some)
774    }
775
776    fn handle_executed_query_failure(&self, query: Option<&PlanInfo>) {
777        if let Some(query) = query {
778            self.restore_dirty_windows_after_failure(query);
779        }
780    }
781
782    /// start executing query in a loop, break when receive shutdown signal
783    ///
784    /// any error will be logged when executing query
785    pub async fn start_executing_loop(
786        &self,
787        engine: QueryEngineRef,
788        frontend_client: Arc<FrontendClient>,
789    ) {
790        let flow_id_str = self.config.flow_id.to_string();
791        let mut max_window_cnt = None;
792        let mut interval = self
793            .config
794            .flow_eval_interval
795            .map(|d| tokio::time::interval(d));
796        if let Some(tick) = &mut interval {
797            tick.tick().await; // pass the first tick immediately
798        }
799        loop {
800            // first check if shutdown signal is received
801            // if so, break the loop
802            {
803                let mut state = self.state.write().unwrap();
804                match state.shutdown_rx.try_recv() {
805                    Ok(()) => break,
806                    Err(TryRecvError::Closed) => {
807                        warn!(
808                            "Unexpected shutdown flow {}, shutdown anyway",
809                            self.config.flow_id
810                        );
811                        break;
812                    }
813                    Err(TryRecvError::Empty) => (),
814                }
815            }
816            METRIC_FLOW_BATCHING_ENGINE_START_QUERY_CNT
817                .with_label_values(&[&flow_id_str])
818                .inc();
819
820            let min_refresh = self.config.batch_opts.experimental_min_refresh_duration;
821
822            let outcome = self
823                .execute_once_serialized_with_outcome(&engine, &frontend_client, max_window_cnt)
824                .await;
825
826            match outcome.result {
827                // normal execute, sleep for some time before doing next query
828                Ok(Some(_)) => {
829                    // can increase max_window_cnt to query more windows next time
830                    max_window_cnt = max_window_cnt.map(|cnt| {
831                        (cnt + 1).min(self.config.batch_opts.experimental_max_filter_num_per_query)
832                    });
833
834                    // here use proper ticking if set eval interval
835                    if let Some(eval_interval) = &mut interval {
836                        eval_interval.tick().await;
837                    } else {
838                        // if not explicitly set, just automatically calculate next start time
839                        // using time window size and more args
840                        let sleep_until = {
841                            let state = self.state.write().unwrap();
842
843                            let time_window_size = self
844                                .config
845                                .time_window_expr
846                                .as_ref()
847                                .and_then(|t| *t.time_window_size());
848
849                            let prefer_short_incremental_cadence = state.checkpoint_mode()
850                                == CheckpointMode::Incremental
851                                && !state.is_incremental_disabled();
852
853                            state.get_next_start_query_time(
854                                self.config.flow_id,
855                                &time_window_size,
856                                min_refresh,
857                                Some(self.config.batch_opts.query_timeout),
858                                self.config.batch_opts.experimental_max_filter_num_per_query,
859                                prefer_short_incremental_cadence,
860                            )
861                        };
862
863                        tokio::time::sleep_until(sleep_until).await;
864                    };
865                }
866                // no new data, sleep for some time before checking for new data
867                Ok(None) => {
868                    debug!(
869                        "Flow id = {:?} found no new data, sleep for {:?} then continue",
870                        self.config.flow_id, min_refresh
871                    );
872                    tokio::time::sleep(min_refresh).await;
873                    continue;
874                }
875                // TODO(discord9): this error should have better place to go, but for now just print error, also more context is needed
876                Err(err) => {
877                    METRIC_FLOW_BATCHING_ENGINE_ERROR_CNT
878                        .with_label_values(&[&flow_id_str])
879                        .inc();
880                    match outcome.new_query {
881                        Some(query) => {
882                            common_telemetry::error!(err; "Failed to execute query for flow={} with query: {}", self.config.flow_id, query.plan);
883                            // TODO(discord9): add some backoff here? half the query time window or what
884                            // backoff meaning use smaller `max_window_cnt` for next query
885
886                            // since last query failed, we should not try to query too many windows
887                            max_window_cnt = Some(1);
888                        }
889                        None => {
890                            common_telemetry::error!(err; "Failed to generate query for flow={}", self.config.flow_id)
891                        }
892                    }
893                    // also sleep for a little while before try again to prevent flooding logs
894                    tokio::time::sleep(min_refresh).await;
895                }
896            }
897        }
898    }
899
900    /// Generate the create table SQL
901    ///
902    /// the auto created table will automatically added a `update_at` Milliseconds DEFAULT now() column in the end
903    /// (for compatibility with flow streaming mode)
904    ///
905    /// and it will use first timestamp column as time index, all other columns will be added as normal columns and nullable
906    async fn gen_create_table_expr(
907        &self,
908        engine: QueryEngineRef,
909    ) -> Result<CreateTableExpr, Error> {
910        let query_ctx = self.state.read().unwrap().query_ctx.clone();
911        let plan =
912            sql_to_df_plan(query_ctx.clone(), engine.clone(), &self.config.query, true).await?;
913        create_table_with_expr(&plan, &self.config.sink_table_name, &self.config.query_type)
914    }
915
916    fn should_use_unfiltered_incremental_delta(&self) -> bool {
917        let state = self.state.read().unwrap();
918        state.checkpoint_mode() == CheckpointMode::Incremental
919            && !state.is_incremental_disabled()
920            && matches!(self.config.query_type, QueryType::Sql)
921    }
922
923    fn should_use_unfiltered_full_snapshot_seeding(&self) -> bool {
924        let state = self.state.read().unwrap();
925        state.checkpoint_mode() == CheckpointMode::FullSnapshot
926            && !state.is_incremental_disabled()
927            && matches!(self.config.query_type, QueryType::Sql)
928    }
929
930    /// will merge and use the first ten time window in query
931    async fn gen_query_with_time_window(
932        &self,
933        engine: QueryEngineRef,
934        sink_table_schema: &Arc<Schema>,
935        primary_key_indices: &[usize],
936        allow_partial: bool,
937        max_window_cnt: Option<usize>,
938    ) -> Result<Option<PlanInfo>, Error> {
939        let query_ctx = self.state.read().unwrap().query_ctx.clone();
940        let start = SystemTime::now();
941        let since_the_epoch = start
942            .duration_since(UNIX_EPOCH)
943            .expect("Time went backwards");
944        let low_bound = self
945            .config
946            .expire_after
947            .map(|e| since_the_epoch.as_secs() - e as u64)
948            .unwrap_or(u64::MIN);
949
950        let low_bound = Timestamp::new_second(low_bound as i64);
951
952        let expire_time_window_bound = self
953            .config
954            .time_window_expr
955            .as_ref()
956            .map(|expr| expr.eval(low_bound))
957            .transpose()?;
958
959        let (expire_lower_bound, expire_upper_bound) =
960            match (expire_time_window_bound, &self.config.query_type) {
961                (Some((Some(l), Some(u))), QueryType::Sql) => (l, u),
962                (None, QueryType::Sql) => {
963                    // if it's sql query and no time window lower/upper bound is found, just return the original query(with auto columns)
964                    // use sink_table_meta to add to query the `update_at` and `__ts_placeholder` column's value too for compatibility reason
965                    debug!(
966                        "Flow id = {:?}, no time window, using the same query",
967                        self.config.flow_id
968                    );
969                    // clean dirty time window too, this could be from create flow's check_execute
970                    return self
971                        .gen_unfiltered_plan_info_if_dirty(
972                            engine,
973                            query_ctx,
974                            sink_table_schema.clone(),
975                            primary_key_indices,
976                            allow_partial,
977                            None,
978                        )
979                        .await;
980                }
981                _ => {
982                    // Clean dirty windows for full-query/non-scoped paths,
983                    // such as TQL, that cannot use a time-window filter.
984                    let (_, dirty_windows_to_restore) = self.drain_dirty_windows_signal();
985
986                    let plan_info = self
987                        .gen_unfiltered_plan_info(
988                            engine,
989                            query_ctx,
990                            sink_table_schema.clone(),
991                            primary_key_indices,
992                            allow_partial,
993                            dirty_windows_to_restore,
994                            None,
995                        )
996                        .await?;
997
998                    return Ok(Some(plan_info));
999                }
1000            };
1001
1002        debug!(
1003            "Flow id = {:?}, found time window: precise_lower_bound={:?}, precise_upper_bound={:?} with dirty time windows: {:?}",
1004            self.config.flow_id,
1005            expire_lower_bound,
1006            expire_upper_bound,
1007            self.state.read().unwrap().dirty_time_windows
1008        );
1009        let window_size = expire_upper_bound
1010            .sub(&expire_lower_bound)
1011            .with_context(|| UnexpectedSnafu {
1012                reason: format!(
1013                    "Can't get window size from {expire_upper_bound:?} - {expire_lower_bound:?}"
1014                ),
1015            })?;
1016        let col_name = self
1017            .config
1018            .time_window_expr
1019            .as_ref()
1020            .map(|expr| expr.column_name.clone())
1021            .with_context(|| UnexpectedSnafu {
1022                reason: format!(
1023                    "Flow id={:?}, Failed to get column name from time window expr",
1024                    self.config.flow_id
1025                ),
1026            })?;
1027
1028        if self.should_use_unfiltered_full_snapshot_seeding() {
1029            // A full-snapshot query that can seed/refresh incremental
1030            // checkpoints must not use dirty-window predicates. Rows can be
1031            // written after dirty windows are drained but before the source scan
1032            // snapshot opens; a stale dirty-window filter could exclude those
1033            // rows while the returned watermark includes them, causing the next
1034            // incremental read to skip them forever. Execute an unfiltered full
1035            // snapshot instead, and keep dirty windows only as the scheduling and
1036            // failure-restoration signal.
1037            let retention_filter = self
1038                .config
1039                .expire_after
1040                .map(|_| (col_name.as_str(), expire_lower_bound, "full-snapshot"));
1041            return self
1042                .gen_unfiltered_plan_info_if_dirty(
1043                    engine,
1044                    query_ctx,
1045                    sink_table_schema.clone(),
1046                    primary_key_indices,
1047                    allow_partial,
1048                    retention_filter,
1049                )
1050                .await;
1051        }
1052
1053        if self.should_use_unfiltered_incremental_delta() {
1054            // In incremental mode, source correctness is defined by the
1055            // per-region sequence range `(checkpoint, scan-open snapshot]`, not
1056            // by dirty-window predicates. Dirty windows are only a scheduling
1057            // signal here. Applying a stale dirty-window filter to the source can
1058            // exclude rows that are inside the returned watermark and make a
1059            // checkpoint advance skip them forever. The sink side is also left
1060            // unfiltered by dirty windows; the incremental rewrite joins the
1061            // delta groups with the full sink state for correctness. Future
1062            // dynamic filters can prune sink reads as a pure optimization.
1063            let retention_filter = self
1064                .config
1065                .expire_after
1066                .map(|_| (col_name.as_str(), expire_lower_bound, "incremental"));
1067            return self
1068                .gen_unfiltered_plan_info_if_dirty(
1069                    engine,
1070                    query_ctx,
1071                    sink_table_schema.clone(),
1072                    primary_key_indices,
1073                    allow_partial,
1074                    retention_filter,
1075                )
1076                .await;
1077        }
1078
1079        let (expr, can_advance_checkpoints) = {
1080            let mut state = self.state.write().unwrap();
1081            let window_cnt = max_window_cnt
1082                .unwrap_or(self.config.batch_opts.experimental_max_filter_num_per_query);
1083            let expr = state.dirty_time_windows.gen_filter_exprs(
1084                &col_name,
1085                Some(expire_lower_bound),
1086                window_size,
1087                window_cnt,
1088                self.config.flow_id,
1089                Some(self),
1090            )?;
1091            let can_advance_checkpoints = state.dirty_time_windows.is_empty();
1092            (expr, can_advance_checkpoints)
1093        };
1094
1095        let Some(expr) = expr else {
1096            // no new data, hence no need to update
1097            debug!("Flow id={:?}, no new data, not update", self.config.flow_id);
1098            return Ok(None);
1099        };
1100
1101        let filter_sql = expr_to_sql(&expr.expr)
1102            .map(|sql| sql.to_string())
1103            .unwrap_or_else(|err| format!("<failed to format filter expr: {err}>"));
1104
1105        debug!(
1106            "Flow id={:?}, Generated filter expr: {:?}",
1107            self.config.flow_id, filter_sql
1108        );
1109
1110        let mut add_filter = AddFilterRewriter::new(expr.expr.clone());
1111        let mut add_auto_column = ColumnMatcherRewriter::new(
1112            sink_table_schema.clone(),
1113            primary_key_indices.to_vec(),
1114            allow_partial,
1115        );
1116
1117        let plan = self.restore_scoped_dirty_windows_on_err(
1118            &expr,
1119            sql_to_df_plan(query_ctx.clone(), engine.clone(), &self.config.query, false).await,
1120        )?;
1121        let rewrite = self.restore_scoped_dirty_windows_on_err(
1122            &expr,
1123            plan.clone()
1124                .rewrite(&mut add_filter)
1125                .and_then(|p| p.data.rewrite(&mut add_auto_column))
1126                .with_context(|_| DatafusionSnafu {
1127                    context: format!("Failed to rewrite plan:\n {}\n", plan),
1128                })
1129                .map(|rewrite| rewrite.data),
1130        )?;
1131        // only apply optimize after complex rewrite is done
1132        let new_plan = self.restore_scoped_dirty_windows_on_err(
1133            &expr,
1134            apply_df_optimizer(rewrite, &query_ctx).await,
1135        )?;
1136
1137        let info = PlanInfo {
1138            plan: new_plan.clone(),
1139            dirty_restore: DirtyRestore::Scoped(expr),
1140            can_advance_checkpoints,
1141        };
1142
1143        Ok(Some(info))
1144    }
1145}
1146
1147#[cfg(test)]
1148mod test;