spark_connect_rs/client/
mod.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
//! Implementation of the SparkConnectServiceClient

use std::sync::Arc;

use tokio::sync::RwLock;

use tonic::codec::Streaming;
use tonic::codegen::{Body, Bytes, StdError};
use tonic::transport::Channel;

use crate::spark;
use spark::execute_plan_response::ResponseType;
use spark::spark_connect_service_client::SparkConnectServiceClient;

use arrow::compute::concat_batches;
use arrow::error::ArrowError;
use arrow::record_batch::RecordBatch;
use arrow_ipc::reader::StreamReader;

use crate::errors::SparkError;

mod builder;
mod middleware;

pub use builder::ChannelBuilder;
pub use middleware::{HeadersLayer, HeadersMiddleware};

pub type SparkClient = SparkConnectClient<HeadersMiddleware<Channel>>;

#[allow(dead_code)]
#[derive(Default, Debug, Clone)]
pub(crate) struct ResponseHandler {
    session_id: Option<String>,
    operation_id: Option<String>,
    response_id: Option<String>,
    metrics: Option<spark::execute_plan_response::Metrics>,
    observed_metrics: Option<spark::execute_plan_response::ObservedMetrics>,
    pub(crate) schema: Option<spark::DataType>,
    batches: Vec<RecordBatch>,
    pub(crate) sql_command_result: Option<spark::execute_plan_response::SqlCommandResult>,
    pub(crate) write_stream_operation_start_result: Option<spark::WriteStreamOperationStartResult>,
    pub(crate) streaming_query_command_result: Option<spark::StreamingQueryCommandResult>,
    pub(crate) get_resources_command_result: Option<spark::GetResourcesCommandResult>,
    pub(crate) streaming_query_manager_command_result:
        Option<spark::StreamingQueryManagerCommandResult>,
    pub(crate) result_complete: bool,
    total_count: isize,
}

#[derive(Default, Debug, Clone)]
pub(crate) struct AnalyzeHandler {
    pub(crate) schema: Option<spark::DataType>,
    pub(crate) explain: Option<String>,
    pub(crate) tree_string: Option<String>,
    pub(crate) is_local: Option<bool>,
    pub(crate) is_streaming: Option<bool>,
    pub(crate) input_files: Option<Vec<String>>,
    pub(crate) spark_version: Option<String>,
    pub(crate) ddl_parse: Option<spark::DataType>,
    pub(crate) same_semantics: Option<bool>,
    pub(crate) semantic_hash: Option<i32>,
    pub(crate) get_storage_level: Option<spark::StorageLevel>,
}

/// Client wrapper to handle submitting requests and handling responses from the [SparkConnectServiceClient]
#[derive(Clone, Debug)]
pub struct SparkConnectClient<T> {
    stub: Arc<RwLock<SparkConnectServiceClient<T>>>,
    builder: ChannelBuilder,
    pub(crate) handler: ResponseHandler,
    pub(crate) analyzer: AnalyzeHandler,
    pub(crate) user_context: Option<spark::UserContext>,
    pub(crate) tags: Vec<String>,
    pub(crate) use_reattachable_execute: bool,
}

impl<T> SparkConnectClient<T>
where
    T: tonic::client::GrpcService<tonic::body::BoxBody>,
    T::Error: Into<StdError>,
    T::ResponseBody: Body<Data = Bytes> + Send + 'static,
    <T::ResponseBody as Body>::Error: Into<StdError> + Send,
{
    pub fn new(stub: Arc<RwLock<SparkConnectServiceClient<T>>>, builder: ChannelBuilder) -> Self {
        let user_ref = builder.user_id.clone().unwrap_or("".to_string());

        SparkConnectClient {
            stub,
            builder,
            handler: ResponseHandler::default(),
            analyzer: AnalyzeHandler::default(),
            user_context: Some(spark::UserContext {
                user_id: user_ref.clone(),
                user_name: user_ref,
                extensions: vec![],
            }),
            tags: vec![],
            use_reattachable_execute: true,
        }
    }

    /// Session ID
    pub fn session_id(&self) -> String {
        self.builder.session_id.to_string()
    }

    /// Change the reattachable execute value
    pub fn set_reattachable_execute(&mut self, setting: bool) -> Result<(), SparkError> {
        self.use_reattachable_execute = setting;
        Ok(())
    }

    fn request_options(&self) -> Vec<spark::execute_plan_request::RequestOption> {
        if self.use_reattachable_execute {
            let reattach_opt = spark::ReattachOptions { reattachable: true };
            let request_opt = spark::execute_plan_request::RequestOption {
                request_option: Some(
                    spark::execute_plan_request::request_option::RequestOption::ReattachOptions(
                        reattach_opt,
                    ),
                ),
            };

            return vec![request_opt];
        };

        vec![]
    }

    pub fn execute_plan_request_with_metadata(&self) -> spark::ExecutePlanRequest {
        spark::ExecutePlanRequest {
            session_id: self.session_id(),
            user_context: self.user_context.clone(),
            operation_id: None,
            plan: None,
            client_type: self.builder.user_agent.clone(),
            request_options: self.request_options(),
            tags: self.tags.clone(),
        }
    }

    pub fn analyze_plan_request_with_metadata(&self) -> spark::AnalyzePlanRequest {
        spark::AnalyzePlanRequest {
            session_id: self.session_id(),
            user_context: self.user_context.clone(),
            client_type: self.builder.user_agent.clone(),
            analyze: None,
        }
    }

    pub async fn execute_and_fetch(
        &mut self,
        req: spark::ExecutePlanRequest,
    ) -> Result<(), SparkError> {
        let mut client = self.stub.write().await;

        let mut stream = client.execute_plan(req).await?.into_inner();
        drop(client);

        // clear out any prior responses
        self.handler = ResponseHandler::default();

        self.process_stream(&mut stream).await?;

        if self.use_reattachable_execute && self.handler.result_complete {
            self.release_all().await?
        }

        Ok(())
    }

    async fn reattach_execute(&mut self) -> Result<(), SparkError> {
        let mut client = self.stub.write().await;

        let req = spark::ReattachExecuteRequest {
            session_id: self.handler.session_id.clone().unwrap(),
            user_context: self.user_context.clone(),
            operation_id: self.handler.operation_id.clone().unwrap(),
            client_type: self.builder.user_agent.clone(),
            last_response_id: self.handler.response_id.clone(),
        };

        let mut stream = client.reattach_execute(req).await?.into_inner();
        drop(client);

        self.process_stream(&mut stream).await?;

        if self.use_reattachable_execute && self.handler.result_complete {
            self.release_all().await?
        }

        Ok(())
    }

    async fn process_stream(
        &mut self,
        stream: &mut Streaming<spark::ExecutePlanResponse>,
    ) -> Result<(), SparkError> {
        while let Some(_resp) = match stream.message().await {
            Ok(Some(msg)) => {
                self.handle_response(msg.clone())?;
                Some(msg)
            }
            Ok(None) => {
                if self.use_reattachable_execute && !self.handler.result_complete {
                    Box::pin(self.reattach_execute()).await?;
                }
                None
            }
            Err(err) => {
                if self.use_reattachable_execute && self.handler.response_id.is_some() {
                    self.release_until().await?;
                }
                return Err(err.into());
            }
        } {}

        Ok(())
    }

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

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

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

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

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

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

        let _resp = client.release_execute(req).await?.into_inner();

        Ok(())
    }

    pub async fn analyze(
        &mut self,
        analyze: spark::analyze_plan_request::Analyze,
    ) -> Result<&mut Self, SparkError> {
        let mut req = self.analyze_plan_request_with_metadata();

        req.analyze = Some(analyze);

        // clear out any prior responses
        self.analyzer = AnalyzeHandler::default();

        let mut client = self.stub.write().await;
        let resp = client.analyze_plan(req).await?.into_inner();
        drop(client);

        self.handle_analyze(resp)
    }

    fn validate_tag(&self, tag: &str) -> Result<(), SparkError> {
        if tag.contains(',') {
            return Err(SparkError::AnalysisException(
                "Spark Connect tag can not contain ',' ".to_string(),
            ));
        };

        if tag.is_empty() {
            return Err(SparkError::AnalysisException(
                "Spark Connect tag can not an empty string ".to_string(),
            ));
        };

        Ok(())
    }

    pub fn add_tag(&mut self, tag: &str) -> Result<(), SparkError> {
        self.validate_tag(tag)?;
        self.tags.push(tag.to_string());
        Ok(())
    }

    pub fn remove_tag(&mut self, tag: &str) -> Result<(), SparkError> {
        self.validate_tag(tag)?;
        self.tags.retain(|t| t != tag);
        Ok(())
    }

    pub fn get_tags(&self) -> &Vec<String> {
        &self.tags
    }

    pub fn clear_tags(&mut self) {
        self.tags = vec![];
    }

    pub async fn config_request(
        &self,
        operation: spark::config_request::Operation,
    ) -> Result<spark::ConfigResponse, SparkError> {
        let operation = spark::ConfigRequest {
            session_id: self.session_id(),
            user_context: self.user_context.clone(),
            client_type: self.builder.user_agent.clone(),
            operation: Some(operation),
        };

        let mut client = self.stub.write().await;

        let resp = client.config(operation).await?.into_inner();

        Ok(resp)
    }

    pub async fn interrupt_request(
        &self,
        interrupt_type: spark::interrupt_request::InterruptType,
        id_or_tag: Option<String>,
    ) -> Result<spark::InterruptResponse, SparkError> {
        let mut req = 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 => {
                req.interrupt_type = interrupt_type.into();
            }
            spark::interrupt_request::InterruptType::Tag => {
                let tag = id_or_tag.expect("Tag can not be empty");
                let interrupt = spark::interrupt_request::Interrupt::OperationTag(tag);
                req.interrupt_type = interrupt_type.into();
                req.interrupt = Some(interrupt);
            }
            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);
                req.interrupt_type = interrupt_type.into();
                req.interrupt = Some(interrupt);
            }
            spark::interrupt_request::InterruptType::Unspecified => {
                return Err(SparkError::AnalysisException(
                    "Interrupt Type was not specified".to_string(),
                ))
            }
        };

        let mut client = self.stub.write().await;

        let resp = client.interrupt(req).await?.into_inner();

        Ok(resp)
    }

    fn handle_response(&mut self, resp: spark::ExecutePlanResponse) -> Result<(), SparkError> {
        self.validate_session(&resp.session_id)?;

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

        if let Some(schema) = &resp.schema {
            self.handler.schema = Some(schema.clone());
        }
        if let Some(metrics) = &resp.metrics {
            self.handler.metrics = Some(metrics.clone());
        }
        if let Some(data) = resp.response_type {
            match data {
                ResponseType::ArrowBatch(res) => {
                    self.deserialize(res.data.as_slice(), res.row_count)?
                }
                ResponseType::SqlCommandResult(sql_cmd) => {
                    self.handler.sql_command_result = Some(sql_cmd.clone())
                }
                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.result_complete = true,
                ResponseType::Extension(_) => {
                    unimplemented!("extension response types are not implemented")
                }
            }
        }
        Ok(())
    }

    fn handle_analyze(
        &mut self,
        resp: spark::AnalyzePlanResponse,
    ) -> Result<&mut Self, SparkError> {
        self.validate_session(&resp.session_id)?;
        if let Some(result) = resp.result {
            match result {
                spark::analyze_plan_response::Result::Schema(schema) => {
                    self.analyzer.schema = schema.schema
                }
                spark::analyze_plan_response::Result::Explain(explain) => {
                    self.analyzer.explain = Some(explain.explain_string)
                }
                spark::analyze_plan_response::Result::TreeString(tree_string) => {
                    self.analyzer.tree_string = Some(tree_string.tree_string)
                }
                spark::analyze_plan_response::Result::IsLocal(is_local) => {
                    self.analyzer.is_local = Some(is_local.is_local)
                }
                spark::analyze_plan_response::Result::IsStreaming(is_streaming) => {
                    self.analyzer.is_streaming = Some(is_streaming.is_streaming)
                }
                spark::analyze_plan_response::Result::InputFiles(input_files) => {
                    self.analyzer.input_files = Some(input_files.files)
                }
                spark::analyze_plan_response::Result::SparkVersion(spark_version) => {
                    self.analyzer.spark_version = Some(spark_version.version)
                }
                spark::analyze_plan_response::Result::DdlParse(ddl_parse) => {
                    self.analyzer.ddl_parse = ddl_parse.parsed
                }
                spark::analyze_plan_response::Result::SameSemantics(same_semantics) => {
                    self.analyzer.same_semantics = Some(same_semantics.result)
                }
                spark::analyze_plan_response::Result::SemanticHash(semantic_hash) => {
                    self.analyzer.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.analyzer.get_storage_level = level.storage_level
                }
            }
        }

        Ok(self)
    }

    fn validate_session(&self, session_id: &str) -> Result<(), SparkError> {
        if self.builder.session_id.to_string() != session_id {
            return Err(SparkError::AnalysisException(format!(
                "Received incorrect session identifier for request: {0} != {1}",
                self.builder.session_id, session_id
            )));
        }
        Ok(())
    }

    fn deserialize(&mut self, res: &[u8], row_count: i64) -> Result<(), SparkError> {
        let reader = StreamReader::try_new(res, None)?;
        for batch in reader {
            let record = batch?;
            if record.num_rows() != row_count as usize {
                return Err(SparkError::ArrowError(ArrowError::IpcError(format!(
                    "Expected {} rows in arrow batch but got {}",
                    row_count,
                    record.num_rows()
                ))));
            };
            self.handler.batches.push(record);
            self.handler.total_count += row_count as isize;
        }
        Ok(())
    }

    pub async fn execute_command(&mut self, plan: spark::Plan) -> Result<(), SparkError> {
        let mut req = self.execute_plan_request_with_metadata();

        req.plan = Some(plan);

        self.execute_and_fetch(req).await?;

        Ok(())
    }

    pub(crate) async fn execute_command_and_fetch(
        &mut self,
        plan: spark::Plan,
    ) -> Result<ResponseHandler, SparkError> {
        let mut req = self.execute_plan_request_with_metadata();

        req.plan = Some(plan);

        self.execute_and_fetch(req).await?;

        Ok(self.handler.clone())
    }

    #[allow(clippy::wrong_self_convention)]
    pub async fn to_arrow(&mut self, plan: spark::Plan) -> Result<RecordBatch, SparkError> {
        let mut req = self.execute_plan_request_with_metadata();

        req.plan = Some(plan);

        self.execute_and_fetch(req).await?;

        Ok(concat_batches(
            &self.handler.batches[0].schema(),
            &self.handler.batches,
        )?)
    }

    #[allow(clippy::wrong_self_convention)]
    pub(crate) async fn to_first_value(&mut self, plan: spark::Plan) -> Result<String, SparkError> {
        let rows = self.to_arrow(plan).await?;
        let col = rows.column(0);

        let data: &arrow::array::StringArray = match col.data_type() {
            arrow::datatypes::DataType::Utf8 => col.as_any().downcast_ref().unwrap(),
            _ => unimplemented!("only Utf8 data types are currently handled currently."),
        };

        Ok(data.value(0).to_string())
    }

    pub fn schema(&self) -> Result<spark::DataType, SparkError> {
        self.analyzer
            .schema
            .to_owned()
            .ok_or_else(|| SparkError::AnalysisException("Schema response is empty".to_string()))
    }

    pub fn explain(&self) -> Result<String, SparkError> {
        self.analyzer
            .explain
            .to_owned()
            .ok_or_else(|| SparkError::AnalysisException("Explain response is empty".to_string()))
    }

    pub fn tree_string(&self) -> Result<String, SparkError> {
        self.analyzer.tree_string.to_owned().ok_or_else(|| {
            SparkError::AnalysisException("Tree String response is empty".to_string())
        })
    }

    pub fn is_local(&self) -> Result<bool, SparkError> {
        self.analyzer
            .is_local
            .to_owned()
            .ok_or_else(|| SparkError::AnalysisException("Is Local response is empty".to_string()))
    }

    pub fn is_streaming(&self) -> Result<bool, SparkError> {
        self.analyzer.is_streaming.to_owned().ok_or_else(|| {
            SparkError::AnalysisException("Is Streaming response is empty".to_string())
        })
    }

    pub fn input_files(&self) -> Result<Vec<String>, SparkError> {
        self.analyzer.input_files.to_owned().ok_or_else(|| {
            SparkError::AnalysisException("Input Files response is empty".to_string())
        })
    }

    pub fn spark_version(&mut self) -> Result<String, SparkError> {
        self.analyzer.spark_version.to_owned().ok_or_else(|| {
            SparkError::AnalysisException("Spark Version resonse is empty".to_string())
        })
    }

    pub fn ddl_parse(&self) -> Result<spark::DataType, SparkError> {
        self.analyzer
            .ddl_parse
            .to_owned()
            .ok_or_else(|| SparkError::AnalysisException("DDL parse response is empty".to_string()))
    }

    pub fn same_semantics(&self) -> Result<bool, SparkError> {
        self.analyzer.same_semantics.to_owned().ok_or_else(|| {
            SparkError::AnalysisException("Same Semantics response is empty".to_string())
        })
    }

    pub fn semantic_hash(&self) -> Result<i32, SparkError> {
        self.analyzer.semantic_hash.to_owned().ok_or_else(|| {
            SparkError::AnalysisException("Semantic Hash response is empty".to_string())
        })
    }

    pub fn get_storage_level(&self) -> Result<spark::StorageLevel, SparkError> {
        self.analyzer.get_storage_level.to_owned().ok_or_else(|| {
            SparkError::AnalysisException("Storage Level response is empty".to_string())
        })
    }
}