forked from lance-format/lance-graph
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathquery.rs
More file actions
1918 lines (1677 loc) · 68.5 KB
/
query.rs
File metadata and controls
1918 lines (1677 loc) · 68.5 KB
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
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright The Lance Authors
//! High-level Cypher query interface for Lance datasets
use crate::ast::CypherQuery as CypherAST;
use crate::config::GraphConfig;
use crate::error::{GraphError, Result};
use crate::logical_plan::LogicalPlanner;
use crate::parser::parse_cypher_query;
use crate::simple_executor::{
to_df_boolean_expr_simple, to_df_order_by_expr_simple, to_df_value_expr_simple, PathExecutor,
};
use std::collections::HashMap;
/// Execution strategy for Cypher queries
#[derive(Debug, Clone, Copy, PartialEq, Eq, Default)]
pub enum ExecutionStrategy {
/// Use DataFusion query planner (default, full feature support)
#[default]
DataFusion,
/// Use simple single-table executor (legacy, limited features)
Simple,
/// Use Lance native executor (not yet implemented)
LanceNative,
}
/// A Cypher query that can be executed against Lance datasets
#[derive(Debug, Clone)]
pub struct CypherQuery {
/// The original Cypher query string
query_text: String,
/// Parsed AST representation
ast: CypherAST,
/// Graph configuration for mapping
config: Option<GraphConfig>,
/// Query parameters
parameters: HashMap<String, serde_json::Value>,
}
impl CypherQuery {
/// Create a new Cypher query from a query string
pub fn new(query: &str) -> Result<Self> {
let ast = parse_cypher_query(query)?;
Ok(Self {
query_text: query.to_string(),
ast,
config: None,
parameters: HashMap::new(),
})
}
/// Set the graph configuration for this query
pub fn with_config(mut self, config: GraphConfig) -> Self {
self.config = Some(config);
self
}
/// Add a parameter to the query
pub fn with_parameter<K, V>(mut self, key: K, value: V) -> Self
where
K: Into<String>,
V: Into<serde_json::Value>,
{
self.parameters.insert(key.into(), value.into());
self
}
/// Add multiple parameters to the query
pub fn with_parameters(mut self, params: HashMap<String, serde_json::Value>) -> Self {
self.parameters.extend(params);
self
}
/// Get the original query text
pub fn query_text(&self) -> &str {
&self.query_text
}
/// Get the parsed AST
pub fn ast(&self) -> &CypherAST {
&self.ast
}
/// Get the graph configuration
pub fn config(&self) -> Option<&GraphConfig> {
self.config.as_ref()
}
/// Get query parameters
pub fn parameters(&self) -> &HashMap<String, serde_json::Value> {
&self.parameters
}
/// Get the required config, returning an error if not set
fn require_config(&self) -> Result<&GraphConfig> {
self.config.as_ref().ok_or_else(|| GraphError::ConfigError {
message: "Graph configuration is required for query execution".to_string(),
location: snafu::Location::new(file!(), line!(), column!()),
})
}
/// Execute the query against provided in-memory datasets
///
/// This method uses the DataFusion planner by default for comprehensive query support
/// including joins, aggregations, and complex patterns. You can optionally specify
/// a different execution strategy.
///
/// # Arguments
/// * `datasets` - HashMap of table name to RecordBatch (nodes and relationships)
/// * `strategy` - Optional execution strategy (defaults to DataFusion)
///
/// # Returns
/// A single RecordBatch containing the query results
///
/// # Errors
/// Returns error if query parsing, planning, or execution fails
///
/// # Example
/// ```ignore
/// use std::collections::HashMap;
/// use arrow::record_batch::RecordBatch;
/// use lance_graph::query::CypherQuery;
///
/// // Create in-memory datasets
/// let mut datasets = HashMap::new();
/// datasets.insert("Person".to_string(), person_batch);
/// datasets.insert("KNOWS".to_string(), knows_batch);
///
/// // Parse and execute query
/// let query = CypherQuery::parse("MATCH (p:Person)-[:KNOWS]->(f) RETURN p.name, f.name")?
/// .with_config(config);
/// // Use the default DataFusion strategy
/// let result = query.execute(datasets, None).await?;
/// // Use the Simple strategy explicitly
/// let result = query.execute(datasets, Some(ExecutionStrategy::Simple)).await?;
/// ```
pub async fn execute(
&self,
datasets: HashMap<String, arrow::record_batch::RecordBatch>,
strategy: Option<ExecutionStrategy>,
) -> Result<arrow::record_batch::RecordBatch> {
let strategy = strategy.unwrap_or_default();
match strategy {
ExecutionStrategy::DataFusion => self.execute_datafusion(datasets).await,
ExecutionStrategy::Simple => self.execute_simple(datasets).await,
ExecutionStrategy::LanceNative => Err(GraphError::UnsupportedFeature {
feature: "Lance native execution strategy is not yet implemented".to_string(),
location: snafu::Location::new(file!(), line!(), column!()),
}),
}
}
/// Explain the query execution plan using in-memory datasets
///
/// Returns a formatted string showing the query execution plan at different stages:
/// - Graph Logical Plan (graph-specific operators)
/// - DataFusion Logical Plan (optimized relational plan)
/// - DataFusion Physical Plan (execution plan with optimizations)
///
/// This is useful for understanding query performance, debugging, and optimization.
///
/// # Arguments
/// * `datasets` - HashMap of table name to RecordBatch (nodes and relationships)
///
/// # Returns
/// A formatted string containing the execution plan at multiple levels
///
/// # Errors
/// Returns error if planning fails
///
/// # Example
/// ```ignore
/// use std::collections::HashMap;
/// use arrow::record_batch::RecordBatch;
/// use lance_graph::query::CypherQuery;
///
/// // Create in-memory datasets
/// let mut datasets = HashMap::new();
/// datasets.insert("Person".to_string(), person_batch);
/// datasets.insert("KNOWS".to_string(), knows_batch);
///
/// let query = CypherQuery::parse("MATCH (p:Person) WHERE p.age > 30 RETURN p.name")?
/// .with_config(config);
///
/// let plan = query.explain(datasets).await?;
/// println!("{}", plan);
/// ```
pub async fn explain(
&self,
datasets: HashMap<String, arrow::record_batch::RecordBatch>,
) -> Result<String> {
use std::sync::Arc;
// Build catalog and context from datasets
let (catalog, ctx) = self
.build_catalog_and_context_from_datasets(datasets)
.await?;
// Delegate to the internal explain method
self.explain_internal(Arc::new(catalog), ctx).await
}
/// Convert the Cypher query to a DataFusion SQL string
///
/// This method generates a SQL string that corresponds to the DataFusion logical plan
/// derived from the Cypher query. It uses the `datafusion-sql` unparser.
///
/// **WARNING**: This method is experimental and the generated SQL dialect may change.
///
/// **Case Sensitivity Limitation**: All table names in the generated SQL are lowercased
/// (e.g., `Person` becomes `person`, `Company` becomes `company`), due to the internal
/// handling of DataFusion's SQL unparser. Note that this only affects the SQL string
/// representation - actual query execution with `execute()` handles case-sensitive labels
/// correctly.
///
/// If you need case-sensitive table names in the SQL output, consider:
/// - Using lowercase labels consistently in your Cypher queries and table names
/// - Post-processing the SQL string to replace table names with the correct case
///
/// # Arguments
/// * `datasets` - HashMap of table name to RecordBatch (nodes and relationships)
///
/// # Returns
/// A SQL string representing the query
pub async fn to_sql(
&self,
datasets: HashMap<String, arrow::record_batch::RecordBatch>,
) -> Result<String> {
use datafusion_sql::unparser::plan_to_sql;
use std::sync::Arc;
let _config = self.require_config()?;
// Build catalog and context from datasets using the helper
let (catalog, ctx) = self
.build_catalog_and_context_from_datasets(datasets)
.await?;
// Generate Logical Plan
let (_, df_plan) = self.create_logical_plans(Arc::new(catalog))?;
// Optimize the plan using DataFusion's default optimizer rules
// This helps simplify the plan (e.g., merging projections) to produce cleaner SQL
let optimized_plan = ctx
.state()
.optimize(&df_plan)
.map_err(|e| GraphError::PlanError {
message: format!("Failed to optimize plan: {}", e),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
// Unparse to SQL
let sql_ast = plan_to_sql(&optimized_plan).map_err(|e| GraphError::PlanError {
message: format!("Failed to unparse plan to SQL: {}", e),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
Ok(sql_ast.to_string())
}
/// Execute query with a DataFusion SessionContext, automatically building the catalog
///
/// This is a convenience method that builds the graph catalog by querying the
/// SessionContext for table schemas. The GraphConfig determines which tables to
/// look up (node labels and relationship types).
///
/// This method is ideal for integrating with DataFusion's rich data source ecosystem
/// (CSV, Parquet, Delta Lake, Iceberg, etc.) without manually building a catalog.
///
/// # Arguments
/// * `ctx` - DataFusion SessionContext with pre-registered tables
///
/// # Returns
/// Query results as an Arrow RecordBatch
///
/// # Errors
/// Returns error if:
/// - GraphConfig is not set (use `.with_config()` first)
/// - Required tables are not registered in the SessionContext
/// - Query execution fails
///
/// # Example
/// ```ignore
/// use datafusion::execution::context::SessionContext;
/// use datafusion::prelude::CsvReadOptions;
/// use lance_graph::{CypherQuery, GraphConfig};
///
/// // Step 1: Create GraphConfig
/// let config = GraphConfig::builder()
/// .with_node_label("Person", "person_id")
/// .with_relationship("KNOWS", "src_id", "dst_id")
/// .build()?;
///
/// // Step 2: Register data sources in DataFusion
/// let ctx = SessionContext::new();
/// ctx.register_csv("Person", "data/persons.csv", CsvReadOptions::default()).await?;
/// ctx.register_parquet("KNOWS", "s3://bucket/knows.parquet", Default::default()).await?;
///
/// // Step 3: Execute query (catalog is built automatically)
/// let query = CypherQuery::parse("MATCH (p:Person)-[:KNOWS]->(f) RETURN p.name")?
/// .with_config(config);
/// let result = query.execute_with_context(ctx).await?;
/// ```
///
/// # Note
/// The catalog is built by querying the SessionContext for schemas of tables
/// mentioned in the GraphConfig. Table names must match between GraphConfig
/// (node labels/relationship types) and SessionContext (registered table names).
pub async fn execute_with_context(
&self,
ctx: datafusion::execution::context::SessionContext,
) -> Result<arrow::record_batch::RecordBatch> {
use crate::source_catalog::InMemoryCatalog;
use datafusion::datasource::DefaultTableSource;
use std::sync::Arc;
let config = self.require_config()?;
// Build catalog by querying SessionContext for table providers
let mut catalog = InMemoryCatalog::new();
// Register node sources
for label in config.node_mappings.keys() {
let table_provider =
ctx.table_provider(label)
.await
.map_err(|e| GraphError::ConfigError {
message: format!(
"Node label '{}' not found in SessionContext: {}",
label, e
),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
let table_source = Arc::new(DefaultTableSource::new(table_provider));
catalog = catalog.with_node_source(label, table_source);
}
// Register relationship sources
for rel_type in config.relationship_mappings.keys() {
let table_provider =
ctx.table_provider(rel_type)
.await
.map_err(|e| GraphError::ConfigError {
message: format!(
"Relationship type '{}' not found in SessionContext: {}",
rel_type, e
),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
let table_source = Arc::new(DefaultTableSource::new(table_provider));
catalog = catalog.with_relationship_source(rel_type, table_source);
}
// Execute using the built catalog
self.execute_with_catalog_and_context(Arc::new(catalog), ctx)
.await
}
/// Execute query with an explicit catalog and session context
///
/// This is the most flexible API for advanced users who want to provide their own
/// catalog implementation or have fine-grained control over both the catalog and
/// session context.
///
/// # Arguments
/// * `catalog` - Graph catalog containing node and relationship schemas for planning
/// * `ctx` - DataFusion SessionContext with registered data sources for execution
///
/// # Returns
/// Query results as an Arrow RecordBatch
///
/// # Errors
/// Returns error if query parsing, planning, or execution fails
///
/// # Example
/// ```ignore
/// use std::sync::Arc;
/// use datafusion::execution::context::SessionContext;
/// use lance_graph::source_catalog::InMemoryCatalog;
/// use lance_graph::query::CypherQuery;
///
/// // Create custom catalog
/// let catalog = InMemoryCatalog::new()
/// .with_node_source("Person", custom_table_source);
///
/// // Create SessionContext
/// let ctx = SessionContext::new();
/// ctx.register_table("Person", custom_table).unwrap();
///
/// // Execute with explicit catalog and context
/// let query = CypherQuery::parse("MATCH (p:Person) RETURN p.name")?
/// .with_config(config);
/// let result = query.execute_with_catalog_and_context(Arc::new(catalog), ctx).await?;
/// ```
pub async fn execute_with_catalog_and_context(
&self,
catalog: std::sync::Arc<dyn crate::source_catalog::GraphSourceCatalog>,
ctx: datafusion::execution::context::SessionContext,
) -> Result<arrow::record_batch::RecordBatch> {
use arrow::compute::concat_batches;
// Create logical plans (phases 1-3)
let (_logical_plan, df_logical_plan) = self.create_logical_plans(catalog)?;
// Execute the DataFusion plan (phase 4)
let df = ctx
.execute_logical_plan(df_logical_plan)
.await
.map_err(|e| GraphError::ExecutionError {
message: format!("Failed to execute DataFusion plan: {}", e),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
// Get schema before collecting (in case result is empty)
let result_schema = df.schema().inner().clone();
// Collect results
let batches = df.collect().await.map_err(|e| GraphError::ExecutionError {
message: format!("Failed to collect query results: {}", e),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
if batches.is_empty() {
// Return empty batch with the schema from the DataFrame
// This preserves column structure even when there are no rows
return Ok(arrow::record_batch::RecordBatch::new_empty(result_schema));
}
// Combine all batches
let schema = batches[0].schema();
concat_batches(&schema, &batches).map_err(|e| GraphError::ExecutionError {
message: format!("Failed to concatenate result batches: {}", e),
location: snafu::Location::new(file!(), line!(), column!()),
})
}
/// Execute using the DataFusion planner with in-memory datasets
///
/// # Overview
/// This convenience method creates both a catalog and session context from the provided
/// in-memory RecordBatches. It's ideal for testing and small datasets that fit in memory.
///
/// For production use with external data sources (CSV, Parquet, databases), use
/// `execute_with_context` instead, which automatically builds the catalog
/// from the SessionContext.
///
/// # Arguments
/// * `datasets` - HashMap of table name to RecordBatch (nodes and relationships)
///
/// # Returns
/// A single RecordBatch containing the query results
async fn execute_datafusion(
&self,
datasets: HashMap<String, arrow::record_batch::RecordBatch>,
) -> Result<arrow::record_batch::RecordBatch> {
use std::sync::Arc;
// Build catalog and context from datasets
let (catalog, ctx) = self
.build_catalog_and_context_from_datasets(datasets)
.await?;
// Delegate to common execution logic
self.execute_with_catalog_and_context(Arc::new(catalog), ctx)
.await
}
/// Helper to build catalog and context from in-memory datasets
async fn build_catalog_and_context_from_datasets(
&self,
datasets: HashMap<String, arrow::record_batch::RecordBatch>,
) -> Result<(
crate::source_catalog::InMemoryCatalog,
datafusion::execution::context::SessionContext,
)> {
use crate::source_catalog::InMemoryCatalog;
use datafusion::datasource::{DefaultTableSource, MemTable};
use datafusion::execution::context::SessionContext;
use std::sync::Arc;
if datasets.is_empty() {
return Err(GraphError::ConfigError {
message: "No input datasets provided".to_string(),
location: snafu::Location::new(file!(), line!(), column!()),
});
}
// Create session context and catalog
let ctx = SessionContext::new();
let mut catalog = InMemoryCatalog::new();
// Register all datasets as tables
for (name, batch) in &datasets {
let mem_table = Arc::new(
MemTable::try_new(batch.schema(), vec![vec![batch.clone()]]).map_err(|e| {
GraphError::PlanError {
message: format!("Failed to create MemTable for {}: {}", name, e),
location: snafu::Location::new(file!(), line!(), column!()),
}
})?,
);
// Register in session context for execution
ctx.register_table(name, mem_table.clone())
.map_err(|e| GraphError::PlanError {
message: format!("Failed to register table {}: {}", name, e),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
let table_source = Arc::new(DefaultTableSource::new(mem_table));
// Register as both node and relationship source
// The planner will use whichever is appropriate based on the query
catalog = catalog
.with_node_source(name, table_source.clone())
.with_relationship_source(name, table_source);
}
Ok((catalog, ctx))
}
/// Internal helper to explain the query execution plan with explicit catalog and session context
async fn explain_internal(
&self,
catalog: std::sync::Arc<dyn crate::source_catalog::GraphSourceCatalog>,
ctx: datafusion::execution::context::SessionContext,
) -> Result<String> {
// Create all plans (phases 1-4)
let (logical_plan, df_logical_plan, physical_plan) =
self.create_plans(catalog, &ctx).await?;
// Format the explain output
self.format_explain_output(&logical_plan, &df_logical_plan, physical_plan.as_ref())
}
/// Helper to create logical plans (graph logical, DataFusion logical)
///
/// This performs phases 1-3 of query execution (semantic analysis, graph logical planning,
/// DataFusion logical planning) without creating the physical plan.
fn create_logical_plans(
&self,
catalog: std::sync::Arc<dyn crate::source_catalog::GraphSourceCatalog>,
) -> Result<(
crate::logical_plan::LogicalOperator,
datafusion::logical_expr::LogicalPlan,
)> {
use crate::datafusion_planner::{DataFusionPlanner, GraphPhysicalPlanner};
use crate::semantic::SemanticAnalyzer;
let config = self.require_config()?;
// Phase 1: Semantic Analysis
let mut analyzer = SemanticAnalyzer::new(config.clone());
analyzer.analyze(&self.ast)?;
// Phase 2: Graph Logical Plan
let mut logical_planner = LogicalPlanner::new();
let logical_plan = logical_planner.plan(&self.ast)?;
// Phase 3: DataFusion Logical Plan
let df_planner = DataFusionPlanner::with_catalog(config.clone(), catalog);
let df_logical_plan = df_planner.plan(&logical_plan)?;
Ok((logical_plan, df_logical_plan))
}
/// Helper to create all plans (graph logical, DataFusion logical, physical)
async fn create_plans(
&self,
catalog: std::sync::Arc<dyn crate::source_catalog::GraphSourceCatalog>,
ctx: &datafusion::execution::context::SessionContext,
) -> Result<(
crate::logical_plan::LogicalOperator,
datafusion::logical_expr::LogicalPlan,
std::sync::Arc<dyn datafusion::physical_plan::ExecutionPlan>,
)> {
// Phases 1-3: Create logical plans
let (logical_plan, df_logical_plan) = self.create_logical_plans(catalog)?;
// Phase 4: DataFusion Physical Plan
let df = ctx
.execute_logical_plan(df_logical_plan.clone())
.await
.map_err(|e| GraphError::ExecutionError {
message: format!("Failed to execute DataFusion plan: {}", e),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
let physical_plan =
df.create_physical_plan()
.await
.map_err(|e| GraphError::ExecutionError {
message: format!("Failed to create physical plan: {}", e),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
Ok((logical_plan, df_logical_plan, physical_plan))
}
/// Format explain output as a table
fn format_explain_output(
&self,
logical_plan: &crate::logical_plan::LogicalOperator,
df_logical_plan: &datafusion::logical_expr::LogicalPlan,
physical_plan: &dyn datafusion::physical_plan::ExecutionPlan,
) -> Result<String> {
// Format output with query first, then table
let mut output = String::new();
// Show Cypher query before the table
output.push_str("Cypher Query:\n");
output.push_str(&format!(" {}\n\n", self.query_text));
// Build table rows (without the query)
let mut rows = vec![];
// Row 1: Graph Logical Plan
let graph_plan_str = format!("{:#?}", logical_plan);
rows.push(("graph_logical_plan", graph_plan_str));
// Row 2: DataFusion Logical Plan
let df_logical_str = format!("{}", df_logical_plan.display_indent());
rows.push(("logical_plan", df_logical_str));
// Row 3: DataFusion Physical Plan
let df_physical_str = format!(
"{}",
datafusion::physical_plan::displayable(physical_plan).indent(true)
);
rows.push(("physical_plan", df_physical_str));
// Calculate column widths
let plan_type_width = rows.iter().map(|(t, _)| t.len()).max().unwrap_or(10);
let plan_width = rows
.iter()
.map(|(_, p)| p.lines().map(|l| l.len()).max().unwrap_or(0))
.max()
.unwrap_or(50);
// Build table
let separator = format!(
"+{}+{}+",
"-".repeat(plan_type_width + 2),
"-".repeat(plan_width + 2)
);
output.push_str(&separator);
output.push('\n');
// Header
output.push_str(&format!(
"| {:<width$} | {:<plan_width$} |\n",
"plan_type",
"plan",
width = plan_type_width,
plan_width = plan_width
));
output.push_str(&separator);
output.push('\n');
// Data rows
for (plan_type, plan_content) in rows {
let lines: Vec<&str> = plan_content.lines().collect();
if lines.is_empty() {
output.push_str(&format!(
"| {:<width$} | {:<plan_width$} |\n",
plan_type,
"",
width = plan_type_width,
plan_width = plan_width
));
} else {
// First line with plan_type
output.push_str(&format!(
"| {:<width$} | {:<plan_width$} |\n",
plan_type,
lines[0],
width = plan_type_width,
plan_width = plan_width
));
// Remaining lines with empty plan_type
for line in &lines[1..] {
output.push_str(&format!(
"| {:<width$} | {:<plan_width$} |\n",
"",
line,
width = plan_type_width,
plan_width = plan_width
));
}
}
}
output.push_str(&separator);
output.push('\n');
Ok(output)
}
/// Execute simple single-table queries (legacy implementation)
///
/// This method supports basic projection/filter/limit workflows on a single table.
/// For full query support including joins and complex patterns, use `execute()` instead.
///
/// Note: This implementation is retained for backward compatibility and simple use cases.
pub async fn execute_simple(
&self,
datasets: HashMap<String, arrow::record_batch::RecordBatch>,
) -> Result<arrow::record_batch::RecordBatch> {
use arrow::compute::concat_batches;
use datafusion::datasource::MemTable;
use datafusion::prelude::*;
use std::sync::Arc;
// Require a config for now, even if we don't fully exploit it yet
let _config = self.require_config()?;
if datasets.is_empty() {
return Err(GraphError::PlanError {
message: "No input datasets provided".to_string(),
location: snafu::Location::new(file!(), line!(), column!()),
});
}
// Create DataFusion context and register all provided tables
let ctx = SessionContext::new();
for (name, batch) in &datasets {
let table =
MemTable::try_new(batch.schema(), vec![vec![batch.clone()]]).map_err(|e| {
GraphError::PlanError {
message: format!("Failed to create DataFusion table: {}", e),
location: snafu::Location::new(file!(), line!(), column!()),
}
})?;
ctx.register_table(name, Arc::new(table))
.map_err(|e| GraphError::PlanError {
message: format!("Failed to register table '{}': {}", name, e),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
}
// Try to execute a path (1+ hops) if the query is a simple pattern
if let Some(df) = self.try_execute_path_generic(&ctx).await? {
let batches = df.collect().await.map_err(|e| GraphError::PlanError {
message: format!("Failed to collect results: {}", e),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
if batches.is_empty() {
let schema = datasets.values().next().unwrap().schema();
return Ok(arrow_array::RecordBatch::new_empty(schema));
}
let merged = concat_batches(&batches[0].schema(), &batches).map_err(|e| {
GraphError::PlanError {
message: format!("Failed to concatenate result batches: {}", e),
location: snafu::Location::new(file!(), line!(), column!()),
}
})?;
return Ok(merged);
}
// Fallback: single-table style query on the first provided table
let (table_name, batch) = datasets.iter().next().unwrap();
let schema = batch.schema();
// Start a DataFrame from the registered table
let mut df = ctx
.table(table_name)
.await
.map_err(|e| GraphError::PlanError {
message: format!("Failed to create DataFrame for '{}': {}", table_name, e),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
// Apply WHERE if present (limited support: simple comparisons on a single column)
if let Some(where_clause) = &self.ast.where_clause {
if let Some(filter_expr) = to_df_boolean_expr_simple(&where_clause.expression) {
df = df.filter(filter_expr).map_err(|e| GraphError::PlanError {
message: format!("Failed to apply filter: {}", e),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
}
}
// Build projection from RETURN clause
let proj_exprs: Vec<Expr> = self
.ast
.return_clause
.items
.iter()
.map(|item| to_df_value_expr_simple(&item.expression))
.collect();
if !proj_exprs.is_empty() {
df = df.select(proj_exprs).map_err(|e| GraphError::PlanError {
message: format!("Failed to project: {}", e),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
}
// Apply DISTINCT
if self.ast.return_clause.distinct {
df = df.distinct().map_err(|e| GraphError::PlanError {
message: format!("Failed to apply DISTINCT: {}", e),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
}
// Apply ORDER BY if present
if let Some(order_by) = &self.ast.order_by {
let sort_expr = to_df_order_by_expr_simple(&order_by.items);
df = df.sort(sort_expr).map_err(|e| GraphError::PlanError {
message: format!("Failed to apply ORDER BY: {}", e),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
}
// Apply SKIP/OFFSET and LIMIT if present
if self.ast.skip.is_some() || self.ast.limit.is_some() {
let offset = self.ast.skip.unwrap_or(0) as usize;
let fetch = self.ast.limit.map(|l| l as usize);
df = df.limit(offset, fetch).map_err(|e| GraphError::PlanError {
message: format!("Failed to apply SKIP/LIMIT: {}", e),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
}
// Collect results and concat into a single RecordBatch
let batches = df.collect().await.map_err(|e| GraphError::PlanError {
message: format!("Failed to collect results: {}", e),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
if batches.is_empty() {
// Return an empty batch with the source schema
return Ok(arrow_array::RecordBatch::new_empty(schema));
}
let merged =
concat_batches(&batches[0].schema(), &batches).map_err(|e| GraphError::PlanError {
message: format!("Failed to concatenate result batches: {}", e),
location: snafu::Location::new(file!(), line!(), column!()),
})?;
Ok(merged)
}
/// Get all node labels referenced in this query
pub fn referenced_node_labels(&self) -> Vec<String> {
let mut labels = Vec::new();
for match_clause in &self.ast.match_clauses {
for pattern in &match_clause.patterns {
self.collect_node_labels_from_pattern(pattern, &mut labels);
}
}
labels.sort();
labels.dedup();
labels
}
/// Get all relationship types referenced in this query
pub fn referenced_relationship_types(&self) -> Vec<String> {
let mut types = Vec::new();
for match_clause in &self.ast.match_clauses {
for pattern in &match_clause.patterns {
self.collect_relationship_types_from_pattern(pattern, &mut types);
}
}
types.sort();
types.dedup();
types
}
/// Get all variables used in this query
pub fn variables(&self) -> Vec<String> {
let mut variables = Vec::new();
for match_clause in &self.ast.match_clauses {
for pattern in &match_clause.patterns {
self.collect_variables_from_pattern(pattern, &mut variables);
}
}
variables.sort();
variables.dedup();
variables
}
// Collection helper methods
fn collect_node_labels_from_pattern(
&self,
pattern: &crate::ast::GraphPattern,
labels: &mut Vec<String>,
) {
match pattern {
crate::ast::GraphPattern::Node(node) => {
labels.extend(node.labels.clone());
}
crate::ast::GraphPattern::Path(path) => {
labels.extend(path.start_node.labels.clone());
for segment in &path.segments {
labels.extend(segment.end_node.labels.clone());
}
}
}
}
fn collect_relationship_types_from_pattern(
&self,
pattern: &crate::ast::GraphPattern,
types: &mut Vec<String>,
) {
if let crate::ast::GraphPattern::Path(path) = pattern {
for segment in &path.segments {
types.extend(segment.relationship.types.clone());
}
}
}
fn collect_variables_from_pattern(
&self,
pattern: &crate::ast::GraphPattern,
variables: &mut Vec<String>,
) {
match pattern {
crate::ast::GraphPattern::Node(node) => {
if let Some(var) = &node.variable {
variables.push(var.clone());
}
}
crate::ast::GraphPattern::Path(path) => {
if let Some(var) = &path.start_node.variable {
variables.push(var.clone());
}
for segment in &path.segments {
if let Some(var) = &segment.relationship.variable {
variables.push(var.clone());
}
if let Some(var) = &segment.end_node.variable {
variables.push(var.clone());
}
}
}
}
}
}
impl CypherQuery {
// Generic path executor (N-hop) entrypoint.
async fn try_execute_path_generic(
&self,
ctx: &datafusion::prelude::SessionContext,
) -> Result<Option<datafusion::dataframe::DataFrame>> {
use crate::ast::GraphPattern;
let [mc] = self.ast.match_clauses.as_slice() else {
return Ok(None);
};
let match_clause = mc;
let path = match match_clause.patterns.as_slice() {
[GraphPattern::Path(p)] if !p.segments.is_empty() => p,
_ => return Ok(None),
};
let cfg = self.require_config()?;
// Handle single-segment variable-length paths by unrolling ranges (*1..N, capped)
if path.segments.len() == 1 {
if let Some(length_range) = &path.segments[0].relationship.length {
let cap: u32 = crate::MAX_VARIABLE_LENGTH_HOPS;
let min_len = length_range.min.unwrap_or(1).max(1);
let max_len = length_range.max.unwrap_or(cap);
if min_len > max_len {
return Err(GraphError::InvalidPattern {
message: format!(
"Invalid variable-length range: min {:?} greater than max {:?}",
length_range.min, length_range.max
),
location: snafu::Location::new(file!(), line!(), column!()),
});
}
if max_len > cap {
return Err(GraphError::UnsupportedFeature {
feature: format!(
"Variable-length paths with length > {} are not supported (got {:?}..{:?})",
cap, length_range.min, length_range.max
),
location: snafu::Location::new(file!(), line!(), column!()),
});
}
use datafusion::dataframe::DataFrame;
let mut union_df: Option<DataFrame> = None;