spark-connect 0.2.2

Rust client for Apache Spark Connect.
Documentation
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
//! Internal gRPC client abstraction for the Spark Connect protocol.
//!
//! This module defines [`SparkClient`], the low-level asynchronous client that
//! manages communication with a Spark Connect server over gRPC.
//!
//! <div class="warning">
//! 
//! End users are advised <b>not</b> to construct or use `SparkClient` directly — use
//! [`SparkSession`](crate::SparkSession) instead, which provides a high-level API.
//! 
//! </div>
//!
//! # Overview
//!
//! `SparkClient` wraps the generated [`SparkConnectServiceClient`] and provides:
//!
//! - Connection setup (via [`ChannelBuilder`]);
//! - Metadata injection (via [`HeaderInterceptor`]);
//! - Structured request/response handling for:
//!   - [`Analyze` plan](crate::spark::analyze_plan_request::Analyze);
//!   - [Execution `Plan`](crate::spark::Plan);
//!   - [`Interrupt`](crate::spark::interrupt_request::Interrupt);
//!   - Reattach/release semantics.
//!
//! Each call validates the active Spark session and maps server responses into
//! safe Rust types or [`SparkError`] values.
mod builder;
mod error;
mod handlers;
mod middleware;

pub use self::builder::ChannelBuilder;
pub(crate) use self::error::{ClientError, ClientErrorKind};
pub use self::middleware::HeaderInterceptor;
use self::handlers::{AnalyzeHandler, ExecuteHandler, InterruptHandler};
use crate::spark;
use crate::spark::spark_connect_service_client::SparkConnectServiceClient;
use crate::spark::execute_plan_response::ResponseType;

use arrow::array::RecordBatch;
use std::sync::Arc;
use tokio::sync::RwLock;
use tonic::codec::Streaming;
use tonic::transport::Channel;
use uuid;

/// Utility type alias for a gRPC channel with an attached interceptor.
type InterceptedChannel = tonic::service::interceptor::InterceptedService<Channel, HeaderInterceptor>;

/// Asynchronous gRPC client for Spark Connect.
///
/// `SparkClient` manages RPC calls, session validation, and response
/// interpretation. It is used internally by [`SparkSession`](crate::SparkSession)
/// to execute plans and perform analysis or interrupt operations.
///
/// <div class="warning">
/// This struct is <b>not</b> intended for direct use; it exposes low-level details
/// of the Spark Connect protocol.
/// </div>
///
/// # Lifecycle
///
/// - Constructed indirectly through [`SparkSessionBuilder`](crate::SparkSessionBuilder);
/// - Maintains session context (e.g. `session_id`, `user_context`);
/// - Automatically attaches metadata headers.
#[derive(Clone, Debug)]
pub struct SparkClient {
    pub(crate) builder: ChannelBuilder,
    stub: Arc<RwLock<SparkConnectServiceClient<InterceptedChannel>>>,
    user_context: Option<spark::UserContext>,
    use_reattachable_execute: bool,
    session_id: String,
    operation_id: Option<String>,
    response_id: Option<String>,
    handler_analyze: AnalyzeHandler,
    handler_execute: ExecuteHandler,
    handler_interrupt: InterruptHandler,
}

impl SparkClient {
    /// Creates a new client from a gRPC stub and a configured [`ChannelBuilder`].
    ///
    /// Typically called internally by [`SparkSessionBuilder`](crate::SparkSessionBuilder).
    pub(crate) fn new(
        stub: Arc<RwLock<SparkConnectServiceClient<InterceptedChannel>>>,
        builder: ChannelBuilder,
    ) -> Self {
        let user_ref = builder.user_id.clone().unwrap_or("".to_string());
        let session_id = builder.session_id.to_string();

        Self {
            stub,
            builder,
            user_context: Some(spark::UserContext {
                user_id: user_ref.clone(),
                user_name: user_ref,
                extensions: vec![],
            }),
            session_id,
            operation_id: None,
            response_id: None,
            handler_analyze: AnalyzeHandler::default(),
            handler_execute: ExecuteHandler::default(),
            handler_interrupt: InterruptHandler::default(),
            use_reattachable_execute: true,
        }
    }

    /// Returns the session ID associated with this client.
    pub(crate) fn session_id(&self) -> String {
        self.session_id.to_string()
    }

    /// Returns the Spark version obtained from the last analyze request.
    pub(crate) fn spark_version(&self) -> Result<String, ClientError> {
        self.handler_analyze
            .spark_version
            .to_owned()
            .ok_or_else(|| ClientError::new(ClientErrorKind::AnalyzeResponseNotFound(
                "Spark version response is empty".to_string()
            )))
    }

    /// Returns the list of operation IDs that were interrupted.
    pub(crate) fn interrupted_ids(&self) -> Vec<String> {
        self.handler_interrupt.interrupted_ids.to_owned()
    }

    /// Returns the last relation received in an [`ExecutePlanResponse`](crate::spark::ExecutePlanResponse).
    pub(crate) fn relation(&self) -> Result<spark::Relation, ClientError> {
        self.handler_execute
            .relation
            .to_owned()
            .ok_or_else(||  ClientError::new(ClientErrorKind::AnalyzeResponseNotFound(
                "relation response is empty".to_string()
            )))
    }

    /// Returns all record batches accumulated during the last execution.
    pub(crate) fn batches(&self) -> Vec<RecordBatch> {
        self.handler_execute.batches.to_owned()
    }

    /// Compares a session_id to the current session's id.
    fn validate_session(&self, session_id: &str) -> Result<(), ClientError> {
        if self.session_id() != session_id {
            return Err(ClientError::new(ClientErrorKind::SessionIDMismatch {
                client_session_id: self.builder.session_id.to_string(),
                request_session_id: session_id.to_string()
            }));
        }
        Ok(())
    }

    /// Sends an [`AnalyzePlanRequest`](crate::spark::AnalyzePlanRequest)
    /// to the Spark Connect server and updates the internal analysis handler.
    pub(crate) async fn analyze(
        &mut self,
        analyze: spark::analyze_plan_request::Analyze,
    ) -> Result<&mut Self, ClientError> {
        let request = spark::AnalyzePlanRequest {
            session_id: self.session_id(),
            user_context: self.user_context.clone(),
            client_type: self.builder.user_agent.clone(),
            analyze: Some(analyze),
        };
        
        let mut client = self.stub.write().await;
        let resp = client.analyze_plan(request.clone())
            .await
            .map_err(|status| {
                ClientError::new(ClientErrorKind::AnalyzeRequest { status, request })
            })?
            .into_inner();
        drop(client);
        
        self.handle_analyze_response(resp)?;
        
        Ok(self)
    }

    fn handle_analyze_response(
        &mut self,
        resp: spark::AnalyzePlanResponse,
    ) -> Result<(), ClientError> {
        self.validate_session(&resp.session_id)?;

        // clear out any prior responses
        self.handler_analyze = AnalyzeHandler::default();
        
        if let Some(result) = resp.result {
            match result {
                spark::analyze_plan_response::Result::Schema(schema) => {
                    self.handler_analyze.schema = schema.schema
                }
                // spark::analyze_plan_response::Result::Explain(explain) => {
                //     self.handler_analyze.explain = Some(explain.explain_string)
                // }
                // spark::analyze_plan_response::Result::TreeString(tree_string) => {
                //     self.handler_analyze.tree_string = Some(tree_string.tree_string)
                // }
                // spark::analyze_plan_response::Result::IsLocal(is_local) => {
                //     self.handler_analyze.is_local = Some(is_local.is_local)
                // }
                // spark::analyze_plan_response::Result::IsStreaming(is_streaming) => {
                //     self.handler_analyze.is_streaming = Some(is_streaming.is_streaming)
                // }
                // spark::analyze_plan_response::Result::InputFiles(input_files) => {
                //     self.handler_analyze.input_files = Some(input_files.files)
                // }
                spark::analyze_plan_response::Result::SparkVersion(spark_version) => {
                    self.handler_analyze.spark_version = Some(spark_version.version)
                }
                // spark::analyze_plan_response::Result::DdlParse(ddl_parse) => {
                //     self.handler_analyze.ddl_parse = ddl_parse.parsed
                // }
                // spark::analyze_plan_response::Result::SameSemantics(same_semantics) => {
                //     self.handler_analyze.same_semantics = Some(same_semantics.result)
                // }
                // spark::analyze_plan_response::Result::SemanticHash(semantic_hash) => {
                //     self.handler_analyze.semantic_hash = Some(semantic_hash.result)
                // }
                // spark::analyze_plan_response::Result::Persist(_) => {}
                // spark::analyze_plan_response::Result::Unpersist(_) => {}
                // spark::analyze_plan_response::Result::GetStorageLevel(level) => {
                //     self.handler_analyze.get_storage_level = level.storage_level
                // }
                _ => return Err(ClientError::new(ClientErrorKind::Unimplemented(format!(
                    "Handling of analyze response {result:?} not implemented!"
                ))))
            }
        }

        Ok(())
    }

    /// Sends an [`InterruptRequest`](crate::spark::InterruptRequest) to Spark.
    ///
    /// Used to stop long-running operations or cancel all running executions.
    pub(crate) async fn interrupt(
        &mut self,
        interrupt_type: spark::interrupt_request::InterruptType,
        id_or_tag: Option<String>,
    ) -> Result<&mut Self, ClientError> {
        let mut request = spark::InterruptRequest {
            session_id: self.session_id(),
            user_context: self.user_context.clone(),
            client_type: self.builder.user_agent.clone(),
            interrupt_type: 0,
            interrupt: None,
        };

        match interrupt_type {
            spark::interrupt_request::InterruptType::All => {
                request.interrupt_type = interrupt_type.into();
            }
            spark::interrupt_request::InterruptType::Tag => {
                return Err(ClientError::new(ClientErrorKind::Unimplemented(
                    "Tag interrupts are not implemented!".to_string()
                )))
            }
            spark::interrupt_request::InterruptType::OperationId => {
                let op_id = id_or_tag.expect("Operation ID can not be empty");
                let interrupt = spark::interrupt_request::Interrupt::OperationId(op_id);
                request.interrupt_type = interrupt_type.into();
                request.interrupt = Some(interrupt);
            }
            spark::interrupt_request::InterruptType::Unspecified => {
                return Err(ClientError::new(ClientErrorKind::UnspecifiedInterruptRequest))
            }
        };

        let mut client = self.stub.write().await;
        let resp = client
            .interrupt(request.clone())
            .await
            .map_err(|status| {
                ClientError::new(ClientErrorKind::InterruptRequest { status, request })
            })?
            .into_inner();
        drop(client);
        
        self.handler_interrupt = InterruptHandler::default();
        self.handler_interrupt.interrupted_ids = resp.interrupted_ids;
        
        Ok(self)
    }
    

    /// Executes a query plan and streams results from Spark.
    ///
    /// This method handles deserialization, streaming,
    /// and optional *reattachment* for fault-tolerant execution.
    ///
    /// The resulting record batches can be retrieved with [`batches()`](Self::batches).
    pub(crate) async fn execute_plan(
        &mut self,
        plan: spark::Plan
    ) -> Result<&mut Self, ClientError> {
        let mut request = self.new_execute_plan_request();
        request.plan = Some(plan);

        let mut client = self.stub.write().await;
        let mut stream = client
            .execute_plan(request.clone())
            .await
            .map_err(|status| {
                ClientError::new(ClientErrorKind::ExecutePlanRequest { status, request })
            })?
            .into_inner();
        drop(client);

        self.handler_execute = ExecuteHandler::default();
        self.process_stream(&mut stream).await?;
        
        if self.use_reattachable_execute && self.handler_execute.result_complete {
            self.release_all().await?;
        }
        
        Ok(self)
    }

    fn new_execute_plan_request(&mut self) -> spark::ExecutePlanRequest {
        let operation_id = uuid::Uuid::new_v4().to_string();

        self.operation_id = Some(operation_id.clone());

        spark::ExecutePlanRequest {
            session_id: self.session_id(),
            user_context: self.user_context.clone(),
            operation_id: Some(operation_id),
            plan: None,
            client_type: self.builder.user_agent.clone(),
            request_options: vec![spark::execute_plan_request::RequestOption {
                request_option: Some(
                    spark::execute_plan_request::request_option::RequestOption::ReattachOptions(
                        spark::ReattachOptions { reattachable: self.use_reattachable_execute },
                    ),
                ),
            }],
            tags: vec![],
        }
    }
    
    fn handle_execute_response(
        &mut self,
        resp: spark::ExecutePlanResponse
    ) -> Result<(), ClientError> {
        self.validate_session(&resp.session_id)?;

        self.operation_id = Some(resp.operation_id);
        self.response_id = Some(resp.response_id);

        if let Some(data) = resp.response_type {
            match data {
                ResponseType::ArrowBatch(res) => {
                    let (batches, total_count) = crate::io::deserialize(
                        res.data.as_slice(), res.row_count
                    )?;

                    self.handler_execute.batches.extend(batches);
                    self.handler_execute.total_count += total_count;
                }
                ResponseType::SqlCommandResult(sql_cmd) => {
                    self.handler_execute.relation = sql_cmd.clone().relation
                }
                // ResponseType::WriteStreamOperationStartResult(write_stream_op) => {
                //     self.handler.write_stream_operation_start_result = Some(write_stream_op)
                // }
                // ResponseType::StreamingQueryCommandResult(stream_qry_cmd) => {
                //     self.handler.streaming_query_command_result = Some(stream_qry_cmd)
                // }
                // ResponseType::GetResourcesCommandResult(resource_cmd) => {
                //     self.handler.get_resources_command_result = Some(resource_cmd)
                // }
                // ResponseType::StreamingQueryManagerCommandResult(stream_qry_mngr_cmd) => {
                //     self.handler.streaming_query_manager_command_result = Some(stream_qry_mngr_cmd)
                // }
                ResponseType::ResultComplete(_) => self.handler_execute.result_complete = true,
                _ => return Err(ClientError::new(ClientErrorKind::Unimplemented(
                    format!("Handling of plan response {data:?} not implemented!")
                )))
            }
        }
        Ok(())
    }

    /// Execute an [execution reattachment request](crate::spark::ReattachExecuteRequest).
    async fn reattach(&mut self) -> Result<(), ClientError> {
        let request = spark::ReattachExecuteRequest {
            session_id: self.session_id(),
            user_context: self.user_context.clone(),
            operation_id: self.operation_id.clone().unwrap(),
            client_type: self.builder.user_agent.clone(),
            last_response_id: self.response_id.clone(),
        };

        let mut client = self.stub.write().await;
        let mut stream = client
            .reattach_execute(request.clone())
            .await
            .map_err(|status| {
                ClientError::new(ClientErrorKind::ReattachExecuteRequest { status, request })
            })?
            .into_inner();
        drop(client);

        self.process_stream(&mut stream).await?;
        
        if self.use_reattachable_execute && self.handler_execute.result_complete {
            self.release_all().await?;
        }

        Ok(())
    }
    
    async fn process_stream(
        &mut self, stream: &mut Streaming<spark::ExecutePlanResponse>
    ) -> Result<(), ClientError> {
        while let Some(_resp) = match stream.message().await {
            Ok(Some(msg)) => {
                self.handle_execute_response(msg.clone())?;
                Some(msg)
            }
            Ok(None) => {
                if self.use_reattachable_execute && !self.handler_execute.result_complete {
                    Box::pin(self.reattach()).await?;
                }
                None
            }
            Err(status) => {
                if self.use_reattachable_execute && self.response_id.is_some() {
                    self.release_until().await?;
                }
                return Err(ClientError::new(ClientErrorKind::Stream(status)));
            }
        } {}

        Ok(())
    }

    async fn release_until(&mut self) -> Result<(), ClientError> {
        let release_until = spark::release_execute_request::ReleaseUntil {
            response_id: self.response_id.clone().unwrap(),
        };

        self.release_execute(spark::release_execute_request::Release::ReleaseUntil(
            release_until,
        )).await
    }

    async fn release_all(&mut self) -> Result<(), ClientError> {
        let release_all = spark::release_execute_request::ReleaseAll {};

        self.release_execute(spark::release_execute_request::Release::ReleaseAll(
            release_all,
        )).await
    }

    async fn release_execute(
        &mut self,
        release: spark::release_execute_request::Release,
    ) -> Result<(), ClientError> {
        let mut client = self.stub.write().await;

        let request = spark::ReleaseExecuteRequest {
            session_id: self.session_id(),
            user_context: self.user_context.clone(),
            operation_id: self.operation_id.clone().unwrap(),
            client_type: self.builder.user_agent.clone(),
            release: Some(release),
        };

        let _resp = client
            .release_execute(request.clone())
            .await
            .map_err(|status| {
                ClientError::new(ClientErrorKind::ReleaseExecuteRequest { status, request })
            })?
            .into_inner();

        Ok(())
    }
}


#[cfg(test)]
mod tests {
    use crate::test_utils::test_utils::setup_session;
    use crate::spark;
    
    /// Verifies that the client correctly handles and reports errors, such as
    /// a session validation failure.
    #[tokio::test]
    async fn test_validate_session_error() {
        // Arrange: Start server and create a session
        let session = setup_session().await.expect("Failed to create Spark session");

        // Create a clone of the client and manually corrupt its session ID
        let mut client_with_bad_session = session.client().clone();
        client_with_bad_session.session_id = "invalid-session-id".to_string();

        // Act: Attempt to use the corrupted client. This will cause the real server
        // to return an error that Spark Connect may not map directly to a session
        // ID mismatch, but it will be an error nonetheless.
        let result = client_with_bad_session
            .analyze(spark::analyze_plan_request::Analyze::SparkVersion(
                spark::analyze_plan_request::SparkVersion {},
            ))
            .await;

        // Assert: The operation should fail.
        assert!(
            result.is_err(),
            "Expected an error due to invalid session ID"
        );
    }
    
    /// Verifies that the client can send an interrupt request without errors.
    /// This tests the `SparkClient::interrupt_request` method.
    #[tokio::test]
    async fn test_interrupt_all_request() {
        // Arrange: Start server and create a session
        let session = setup_session().await.expect("Failed to create Spark session");
        
        // Act: Send an "interrupt all" request. The server should accept this
        // command gracefully even if nothing is running.
        let mut client = session.client();
        let result = client
            .interrupt(spark::interrupt_request::InterruptType::All, None)
            .await
            .unwrap();
            
        // Assert: The request should succeed. The response may be empty.
        assert_eq!(result.session_id(), session.session_id());
    }
}