risingwave_sqlsmith/sqlreduce/reducer.rs
1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7// http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15//! SQL reduction framework for `RisingWave`.
16//!
17//! This module provides path-based SQL query reduction using systematic
18//! path enumeration and rules-based transformations to systematically
19//! reduce SQL queries with better coverage and control.
20
21use std::collections::HashSet;
22
23use anyhow::{Result, anyhow};
24
25use crate::parse_sql;
26use crate::sqlreduce::checker::Checker;
27use crate::sqlreduce::path::{
28 ast_node_to_statement, enumerate_reduction_paths, statement_to_ast_node,
29};
30use crate::sqlreduce::rules::{
31 ReductionRules, apply_reduction_operation, generate_reduction_candidates,
32};
33
34pub struct Reducer {
35 rules: ReductionRules,
36 checker: Checker,
37}
38
39impl Reducer {
40 pub fn new(checker: Checker) -> Self {
41 Self {
42 rules: ReductionRules::default(),
43 checker,
44 }
45 }
46
47 /// Perform reduction on a SQL input containing multiple statements,
48 /// where only the **last** statement is considered the failing one.
49 ///
50 /// The reducer:
51 /// 1. Executes all preceding statements using the checker client.
52 /// 2. Verifies that the last statement indeed fails (self-check).
53 /// 3. Applies transformation passes to simplify the failing query
54 /// while preserving the failure behavior.
55 /// 4. Returns the reduced failing SQL query as a string.
56 ///
57 /// # Arguments
58 /// - `sql`: A SQL string with multiple statements (e.g., setup + failing query).
59 ///
60 /// # Returns
61 /// - A simplified version of the last statement that still fails in the same way.
62 /// - The preceding statements are also returned as a string.
63 ///
64 /// # Errors
65 /// - Returns an error if SQL parsing fails or if no statements are found.
66 /// - Panics if the checker fails to validate failure preservation on the original failing query.
67 pub async fn reduce(&mut self, sql: &str) -> Result<String> {
68 tracing::info!("Preparing schema");
69 self.checker.prepare_schema().await;
70
71 tracing::info!("Starting reduction");
72 let sql_statements = parse_sql(sql);
73
74 let (failing_query, proceeding_stmts) = sql_statements
75 .split_last()
76 .ok_or_else(|| anyhow!("No SQL statements found"))?;
77
78 for s in proceeding_stmts {
79 tracing::info!("Executing preceding statement: {}", s);
80 self.checker.client.simple_query(&s.to_string()).await?;
81 }
82
83 if !self
84 .checker
85 .is_failure_preserved(&failing_query.to_string(), &failing_query.to_string())
86 .await
87 {
88 tracing::error!("Checker failed: failing query does not fail on itself");
89 panic!("There is a bug in the checker!")
90 }
91
92 tracing::info!("Beginning path-based reduction");
93 let reduced_sql = self.reduce_path_based(&failing_query.to_string()).await;
94
95 tracing::info!("Reduction complete");
96
97 let mut reduced_sqls = String::new();
98 for s in proceeding_stmts {
99 reduced_sqls.push_str(&s.to_string());
100 reduced_sqls.push_str(";\n");
101 }
102 reduced_sqls.push_str(&reduced_sql);
103 reduced_sqls.push_str(";\n");
104
105 // Drop the schema after the reduction is complete.
106 self.checker.drop_schema().await;
107
108 Ok(reduced_sqls)
109 }
110
111 /// Path-based reduction approach using systematic AST traversal.
112 ///
113 /// This method:
114 /// 1. Enumerates all reduction paths in the AST
115 /// 2. Generates reduction candidates based on rules
116 /// 3. Applies candidates one-by-one, validating after each successful application
117 /// 4. Uses a seen-query cache to avoid redundant checks
118 async fn reduce_path_based(&mut self, sql: &str) -> String {
119 let sql_statements = parse_sql(sql);
120 let mut ast_node = statement_to_ast_node(&sql_statements[0]);
121 let mut seen_queries = HashSet::new();
122 let mut iteration = 0;
123 let mut sql_len = sql.len();
124 let mut candidate_index = 0;
125
126 // Track the original query
127 seen_queries.insert(sql.to_owned());
128
129 tracing::info!(
130 "Starting path-based reduction with initial SQL length: {}",
131 sql_len
132 );
133
134 loop {
135 iteration += 1;
136 tracing::info!("Path-based iteration {} starting", iteration);
137 let mut found_reduction = false;
138
139 // Enumerate all paths in the current AST
140 let paths = enumerate_reduction_paths(&ast_node, vec![]);
141 tracing::debug!("Found {} reduction paths in AST", paths.len());
142
143 // Generate reduction candidates
144 let candidates = generate_reduction_candidates(&ast_node, &self.rules, &paths);
145 tracing::debug!("Generated {} reduction candidates", candidates.len());
146
147 // Try applying each candidate in order, with scoped multi-step removal for same-path operations
148 let mut i = 0usize;
149 while i < candidates.len() {
150 let candidate = &candidates[i];
151 candidate_index += 1;
152 tracing::debug!(
153 "Trying candidate {} of {} (global #{}): {:?}",
154 i + 1,
155 candidates.len(),
156 candidate_index,
157 candidate
158 );
159
160 // Check if we can batch multiple operations of the same type at the same path
161 let base_path = &candidate.path;
162 let mut batch_applied = false;
163
164 // Try batching based on operation type
165 match &candidate.operation {
166 crate::sqlreduce::rules::ReductionOperation::RemoveListElement(_) => {
167 // Collect consecutive RemoveListElement operations on the same list path
168 let mut j = i;
169 let mut group_indices = Vec::new();
170 while j < candidates.len() {
171 if let crate::sqlreduce::rules::ReductionOperation::RemoveListElement(
172 _,
173 ) = &candidates[j].operation
174 {
175 if candidates[j].path == *base_path {
176 group_indices.push(j);
177 j += 1;
178 } else {
179 break; // Different path
180 }
181 } else {
182 break; // Different operation type
183 }
184 }
185
186 // Use binary search to find the maximum batch size that works
187 if group_indices.len() > 1
188 && let Some((success_ast, success_sql, applied_count)) = self
189 .try_batch_with_binary_search(
190 &ast_node,
191 &candidates,
192 &group_indices,
193 sql,
194 sql_len,
195 &mut seen_queries,
196 "List-batch",
197 base_path,
198 )
199 .await
200 {
201 tracing::info!(
202 "✓ Valid list-batch reduction! Removed {} items, SQL len {} → {}",
203 applied_count,
204 sql_len,
205 success_sql.len()
206 );
207 ast_node = success_ast;
208 sql_len = success_sql.len();
209 found_reduction = true;
210 batch_applied = true;
211 }
212 }
213
214 crate::sqlreduce::rules::ReductionOperation::Remove(_) => {
215 // Collect consecutive Remove operations on the same node path
216 let mut j = i;
217 let mut group_indices = Vec::new();
218 while j < candidates.len() {
219 if let crate::sqlreduce::rules::ReductionOperation::Remove(_) =
220 &candidates[j].operation
221 {
222 if candidates[j].path == *base_path {
223 group_indices.push(j);
224 j += 1;
225 } else {
226 break;
227 }
228 } else {
229 break;
230 }
231 }
232
233 // Use binary search to find the maximum batch size that works
234 if group_indices.len() > 1
235 && let Some((success_ast, success_sql, applied_count)) = self
236 .try_batch_with_binary_search(
237 &ast_node,
238 &candidates,
239 &group_indices,
240 sql,
241 sql_len,
242 &mut seen_queries,
243 "Attr-batch",
244 base_path,
245 )
246 .await
247 {
248 tracing::info!(
249 "✓ Valid attr-batch reduction! Removed {} attributes, SQL len {} → {}",
250 applied_count,
251 sql_len,
252 success_sql.len()
253 );
254 ast_node = success_ast;
255 sql_len = success_sql.len();
256 found_reduction = true;
257 batch_applied = true;
258 }
259 }
260
261 crate::sqlreduce::rules::ReductionOperation::Replace(_) => {
262 // Collect consecutive Replace operations on the same node path
263 let mut j = i;
264 let mut group_indices = Vec::new();
265 while j < candidates.len() {
266 if let crate::sqlreduce::rules::ReductionOperation::Replace(_) =
267 &candidates[j].operation
268 {
269 if candidates[j].path == *base_path {
270 group_indices.push(j);
271 j += 1;
272 } else {
273 break;
274 }
275 } else {
276 break;
277 }
278 }
279
280 // Use binary search to find the maximum batch size that works
281 if group_indices.len() > 1
282 && let Some((success_ast, success_sql, applied_count)) = self
283 .try_batch_with_binary_search(
284 &ast_node,
285 &candidates,
286 &group_indices,
287 sql,
288 sql_len,
289 &mut seen_queries,
290 "Replace-batch",
291 base_path,
292 )
293 .await
294 {
295 tracing::info!(
296 "✓ Valid replace-batch reduction! Applied {} replacements, SQL len {} → {}",
297 applied_count,
298 sql_len,
299 success_sql.len()
300 );
301 ast_node = success_ast;
302 sql_len = success_sql.len();
303 found_reduction = true;
304 batch_applied = true;
305 }
306 }
307
308 crate::sqlreduce::rules::ReductionOperation::Pullup(_) => {
309 // Collect consecutive Pullup operations on the same node path
310 let mut j = i;
311 let mut group_indices = Vec::new();
312 while j < candidates.len() {
313 if let crate::sqlreduce::rules::ReductionOperation::Pullup(_) =
314 &candidates[j].operation
315 {
316 if candidates[j].path == *base_path {
317 group_indices.push(j);
318 j += 1;
319 } else {
320 break;
321 }
322 } else {
323 break;
324 }
325 }
326
327 // Use binary search to find the maximum batch size that works
328 if group_indices.len() > 1
329 && let Some((success_ast, success_sql, applied_count)) = self
330 .try_batch_with_binary_search(
331 &ast_node,
332 &candidates,
333 &group_indices,
334 sql,
335 sql_len,
336 &mut seen_queries,
337 "Pullup-batch",
338 base_path,
339 )
340 .await
341 {
342 tracing::info!(
343 "✓ Valid pullup-batch reduction! Applied {} pullups, SQL len {} → {}",
344 applied_count,
345 sql_len,
346 success_sql.len()
347 );
348 ast_node = success_ast;
349 sql_len = success_sql.len();
350 found_reduction = true;
351 batch_applied = true;
352 }
353 }
354
355 _ => {
356 // TryNull and other operations: no batching
357 }
358 }
359
360 // If batch was applied successfully, restart iteration with new AST
361 if batch_applied {
362 break;
363 }
364
365 // Fallback: try single candidate
366 let Some(new_ast) = apply_reduction_operation(&ast_node, candidate) else {
367 tracing::debug!("Failed to apply reduction operation");
368 i += 1;
369 continue;
370 };
371
372 let Some(new_stmt) = ast_node_to_statement(&new_ast) else {
373 tracing::debug!("Failed to convert reduced AST back to statement");
374 i += 1;
375 continue;
376 };
377
378 let new_sql = new_stmt.to_string();
379 let new_len = new_sql.len();
380
381 tracing::debug!(
382 "Generated candidate SQL with length: {} (reduction: {})",
383 new_len,
384 sql_len as i32 - new_len as i32
385 );
386
387 // Only consider if it's actually smaller and we haven't seen it
388 if new_len >= sql_len {
389 tracing::debug!(
390 "Candidate not smaller ({} >= {}), skipping. Generated SQL: {}",
391 new_len,
392 sql_len,
393 new_sql
394 );
395 i += 1;
396 continue;
397 }
398
399 if seen_queries.contains(&new_sql) {
400 tracing::debug!("Candidate already seen, skipping");
401 i += 1;
402 continue;
403 }
404
405 tracing::debug!(
406 "SQL changes from:\n{}\nto:\n{}",
407 ast_node_to_statement(&ast_node)
408 .map(|s| s.to_string())
409 .unwrap_or_else(|| "<failed to convert AST to statement>".to_owned()),
410 new_sql
411 );
412
413 seen_queries.insert(new_sql.clone());
414
415 // Check if the failure is preserved
416 tracing::debug!("Checking if failure is preserved");
417 if !self.checker.is_failure_preserved(sql, &new_sql).await {
418 tracing::debug!("Reduction not valid; failure not preserved");
419 i += 1;
420 continue;
421 }
422
423 tracing::info!("✓ Valid reduction found! SQL len {} → {}", sql_len, new_len);
424 tracing::info!("Applying candidate and continuing to next iteration");
425 ast_node = new_ast;
426 sql_len = new_len;
427 found_reduction = true;
428 break;
429 }
430
431 if !found_reduction {
432 tracing::info!(
433 "Path-based iteration {} complete: no valid reductions found",
434 iteration
435 );
436 tracing::info!(
437 "Path-based reduction finished after {} iterations",
438 iteration
439 );
440 tracing::info!(
441 "Final SQL length: {} (reduced by {} characters)",
442 sql_len,
443 sql.len() as i32 - sql_len as i32
444 );
445 break;
446 } else {
447 tracing::debug!(
448 "Path-based iteration {} complete: found valid reduction, continuing",
449 iteration
450 );
451 }
452 }
453
454 let final_sql = ast_node_to_statement(&ast_node)
455 .map(|s| s.to_string())
456 .unwrap_or_else(|| sql.to_owned());
457
458 tracing::info!(
459 "Path-based reduction complete. Processed {} total candidates across {} iterations",
460 candidate_index,
461 iteration
462 );
463
464 final_sql
465 }
466
467 /// Try to apply a batch of operations using binary search to find the maximum working batch size.
468 ///
469 /// Binary search strategy:
470 /// - Start with the full batch size
471 /// - If it works, return success immediately
472 /// - If it fails, binary search for the largest working subset
473 ///
474 /// Returns: (AST, SQL, `applied_count`) if any batch succeeds, None otherwise
475 #[allow(clippy::too_many_arguments)]
476 async fn try_batch_with_binary_search(
477 &mut self,
478 ast_node: &crate::sqlreduce::path::AstNode,
479 candidates: &[crate::sqlreduce::rules::ReductionCandidate],
480 group_indices: &[usize],
481 original_sql: &str,
482 sql_len: usize,
483 seen_queries: &mut HashSet<String>,
484 batch_type: &str,
485 base_path: &crate::sqlreduce::path::AstPath,
486 ) -> Option<(crate::sqlreduce::path::AstNode, String, usize)> {
487 let total = group_indices.len();
488
489 tracing::debug!(
490 "{}: Found {} candidates at same path, trying binary search",
491 batch_type,
492 total
493 );
494
495 // Binary search for the maximum working batch size
496 let mut left = 2; // Minimum batch size
497 let mut right = total;
498 let mut best_result: Option<(crate::sqlreduce::path::AstNode, String, usize)> = None;
499
500 while left <= right {
501 let mid = (left + right) / 2;
502
503 tracing::debug!(
504 "{}: Trying batch size {} (range: {}-{})",
505 batch_type,
506 mid,
507 left,
508 right
509 );
510
511 // Try to apply this batch size
512 let mut tmp_ast = ast_node.clone();
513 let mut applied = 0usize;
514
515 for &idx in &group_indices[..mid] {
516 if let Some(next_ast) = apply_reduction_operation(&tmp_ast, &candidates[idx]) {
517 tmp_ast = next_ast;
518 applied += 1;
519 } else {
520 break;
521 }
522 }
523
524 if applied >= 2
525 && let Some(new_stmt) = ast_node_to_statement(&tmp_ast)
526 {
527 let new_sql = new_stmt.to_string();
528 let new_len = new_sql.len();
529
530 if new_len < sql_len && !seen_queries.contains(&new_sql) {
531 // Check if the failure is preserved
532 if self
533 .checker
534 .is_failure_preserved(original_sql, &new_sql)
535 .await
536 {
537 tracing::debug!(
538 "{}: Batch size {} succeeded, trying larger",
539 batch_type,
540 mid
541 );
542 seen_queries.insert(new_sql.clone());
543 best_result = Some((tmp_ast, new_sql, applied));
544 left = mid + 1; // Try larger batch
545 continue;
546 }
547 }
548 }
549
550 // If we reach here, this batch size didn't work
551 tracing::debug!("{}: Batch size {} failed, trying smaller", batch_type, mid);
552 right = mid - 1;
553 }
554
555 if let Some((_, ref sql, count)) = best_result {
556 tracing::debug!(
557 "{}: Binary search found optimal batch size {} at path {} (len: {})",
558 batch_type,
559 count,
560 crate::sqlreduce::path::display_ast_path(base_path),
561 sql.len()
562 );
563 } else {
564 tracing::debug!(
565 "{}: Binary search found no valid batch at path {}",
566 batch_type,
567 crate::sqlreduce::path::display_ast_path(base_path)
568 );
569 }
570
571 best_result
572 }
573}