spark_connect_core/
session.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
//! Spark Session containing the remote gRPC client

use std::collections::HashMap;
use std::sync::Arc;

use crate::catalog::Catalog;
use crate::conf::RunTimeConfig;
use crate::dataframe::{DataFrame, DataFrameReader};
use crate::plan::LogicalPlanBuilder;
use crate::spark;
use crate::streaming::DataStreamReader;

use crate::client::{ChannelBuilder, MetadataInterceptor, SparkConnectClient};
use crate::errors::SparkError;
use spark::spark_connect_service_client::SparkConnectServiceClient;

use arrow::record_batch::RecordBatch;

use tokio::sync::RwLock;

#[cfg(not(feature = "wasm"))]
use tonic::transport::{Channel, Endpoint};

#[cfg(feature = "wasm")]
use tonic_web_wasm_client::Client;

use tonic::service::interceptor::InterceptedService;

/// SparkSessionBuilder creates a remote Spark Session a connection string.
///
/// The connection string is define based on the requirements from [Spark Documentation](https://github.com/apache/spark/blob/master/connector/connect/docs/client-connection-string.md)
#[derive(Clone, Debug)]
pub struct SparkSessionBuilder {
    pub channel_builder: ChannelBuilder,
    configs: HashMap<String, String>,
}

/// Default connects a Spark cluster running at `sc://127.0.0.1:15002/`
impl Default for SparkSessionBuilder {
    fn default() -> Self {
        let channel_builder = ChannelBuilder::default();

        Self {
            channel_builder,
            configs: HashMap::new(),
        }
    }
}

impl SparkSessionBuilder {
    fn new(connection: &str) -> Self {
        let channel_builder = ChannelBuilder::create(connection).unwrap();

        Self {
            channel_builder,
            configs: HashMap::new(),
        }
    }

    /// Validate a connect string for a remote Spark Session
    ///
    /// String must conform to the [Spark Documentation](https://github.com/apache/spark/blob/master/connector/connect/docs/client-connection-string.md)
    pub fn remote(connection: &str) -> Self {
        Self::new(connection)
    }

    /// Sets a config option.
    pub fn config(mut self, key: &str, value: &str) -> Self {
        self.configs.insert(key.into(), value.into());
        self
    }

    /// Sets a name for the application, which will be shown in the Spark web UI.
    pub fn app_name(mut self, name: &str) -> Self {
        self.configs
            .insert("spark.app.name".to_string(), name.into());
        self
    }

    #[cfg(not(feature = "wasm"))]
    async fn create_client(&self) -> Result<SparkSession, SparkError> {
        let channel = Endpoint::from_shared(self.channel_builder.endpoint())
            .expect("Failed to create endpoint")
            .connect()
            .await
            .expect("Failed to create channel");

        let service_client = SparkConnectServiceClient::with_interceptor(
            channel,
            MetadataInterceptor::new(
                self.channel_builder.token().to_owned(),
                self.channel_builder.headers().to_owned(),
            ),
        );

        let client = Arc::new(RwLock::new(service_client));

        let spark_connnect_client = SparkConnectClient::new(client, self.channel_builder.clone());

        let mut rt_config = RunTimeConfig::new(&spark_connnect_client);

        rt_config.set_configs(&self.configs).await?;

        Ok(SparkSession::new(spark_connnect_client))
    }

    #[cfg(feature = "wasm")]
    async fn create_client(&self) -> Result<SparkSession, SparkError> {
        let inner = Client::new(self.channel_builder.endpoint());

        let service_client = SparkConnectServiceClient::with_interceptor(
            inner,
            MetadataInterceptor::new(
                self.channel_builder.token().to_owned(),
                self.channel_builder.headers().to_owned(),
            ),
        );

        let client = Arc::new(RwLock::new(service_client));

        let spark_connnect_client = SparkConnectClient::new(client, self.channel_builder.clone());

        let mut rt_config = RunTimeConfig::new(&spark_connnect_client);

        rt_config.set_configs(&self.configs).await?;

        Ok(SparkSession::new(spark_connnect_client))
    }

    /// Attempt to connect to a remote Spark Session
    ///
    /// and return a [SparkSession]
    pub async fn build(&self) -> Result<SparkSession, SparkError> {
        self.create_client().await
    }
}

/// The entry point to connecting to a Spark Cluster
/// using the Spark Connection gRPC protocol.
#[derive(Clone, Debug)]
pub struct SparkSession {
    #[cfg(not(feature = "wasm"))]
    client: SparkConnectClient<InterceptedService<Channel, MetadataInterceptor>>,

    #[cfg(feature = "wasm")]
    client: SparkConnectClient<InterceptedService<Client, MetadataInterceptor>>,

    session_id: String,
}

impl SparkSession {
    #[cfg(not(feature = "wasm"))]
    pub fn new(
        client: SparkConnectClient<InterceptedService<Channel, MetadataInterceptor>>,
    ) -> Self {
        Self {
            session_id: client.session_id(),
            client,
        }
    }

    #[cfg(feature = "wasm")]
    pub fn new(
        client: SparkConnectClient<InterceptedService<Client, MetadataInterceptor>>,
    ) -> Self {
        Self {
            session_id: client.session_id(),
            client,
        }
    }

    pub fn session(&self) -> SparkSession {
        self.clone()
    }

    /// Create a [DataFrame] with a spingle column named `id`,
    /// containing elements in a range from `start` (default 0) to
    /// `end` (exclusive) with a step value `step`, and control the number
    /// of partitions with `num_partitions`
    pub fn range(
        &self,
        start: Option<i64>,
        end: i64,
        step: i64,
        num_partitions: Option<i32>,
    ) -> DataFrame {
        let range_relation = spark::relation::RelType::Range(spark::Range {
            start,
            end,
            step,
            num_partitions,
        });

        DataFrame::new(self.session(), LogicalPlanBuilder::from(range_relation))
    }

    /// Returns a [DataFrameReader] that can be used to read datra in as a [DataFrame]
    pub fn read(&self) -> DataFrameReader {
        DataFrameReader::new(self.session())
    }

    /// Returns a [DataFrameReader] that can be used to read datra in as a [DataFrame]
    pub fn read_stream(&self) -> DataStreamReader {
        DataStreamReader::new(self.session())
    }

    pub fn table(&self, name: &str) -> Result<DataFrame, SparkError> {
        DataFrameReader::new(self.session()).table(name, None)
    }

    /// Interface through which the user may create, drop, alter or query underlying databases,
    /// tables, functions, etc.
    pub fn catalog(&self) -> Catalog {
        Catalog::new(self.session())
    }

    /// Returns a [DataFrame] representing the result of the given query
    pub async fn sql(&self, sql_query: &str) -> Result<DataFrame, SparkError> {
        let sql_cmd = spark::command::CommandType::SqlCommand(spark::SqlCommand {
            sql: sql_query.to_string(),
            args: HashMap::default(),
            pos_args: vec![],
        });

        let plan = LogicalPlanBuilder::plan_cmd(sql_cmd);

        let resp = self
            .clone()
            .client()
            .execute_command_and_fetch(plan)
            .await?;

        let relation = resp.sql_command_result.to_owned().unwrap().relation;

        let logical_plan = LogicalPlanBuilder::new(relation.unwrap());

        Ok(DataFrame::new(self.session(), logical_plan))
    }

    pub fn create_dataframe(&self, data: &RecordBatch) -> Result<DataFrame, SparkError> {
        let logical_plan = LogicalPlanBuilder::local_relation(data)?;

        Ok(DataFrame::new(self.session(), logical_plan))
    }

    /// Return the session ID
    pub fn session_id(&self) -> &str {
        &self.session_id
    }

    /// Spark Connection gRPC client interface
    #[cfg(not(feature = "wasm"))]
    pub fn client(self) -> SparkConnectClient<InterceptedService<Channel, MetadataInterceptor>> {
        self.client
    }

    #[cfg(feature = "wasm")]
    pub fn client(self) -> SparkConnectClient<InterceptedService<Client, MetadataInterceptor>> {
        self.client
    }

    /// Interrupt all operations of this session currently running on the connected server.
    pub async fn interrupt_all(&self) -> Result<Vec<String>, SparkError> {
        let resp = self
            .client
            .interrupt_request(spark::interrupt_request::InterruptType::All, None)
            .await?;

        Ok(resp.interrupted_ids)
    }

    /// Interrupt all operations of this session with the given operation tag.
    pub async fn interrupt_tag(&self, tag: &str) -> Result<Vec<String>, SparkError> {
        let resp = self
            .client
            .interrupt_request(
                spark::interrupt_request::InterruptType::Tag,
                Some(tag.to_string()),
            )
            .await?;

        Ok(resp.interrupted_ids)
    }

    /// Interrupt an operation of this session with the given operationId.
    pub async fn interrupt_operation(&self, op_id: &str) -> Result<Vec<String>, SparkError> {
        let resp = self
            .client
            .interrupt_request(
                spark::interrupt_request::InterruptType::OperationId,
                Some(op_id.to_string()),
            )
            .await?;

        Ok(resp.interrupted_ids)
    }

    /// Add a tag to be assigned to all the operations started by this thread in this session.
    pub fn add_tag(&mut self, tag: &str) -> Result<(), SparkError> {
        self.client.add_tag(tag)
    }

    /// Remove a tag previously added to be assigned to all the operations started by this thread in this session.
    pub fn remove_tag(&mut self, tag: &str) -> Result<(), SparkError> {
        self.client.remove_tag(tag)
    }

    /// Get the tags that are currently set to be assigned to all the operations started by this thread.
    pub fn get_tags(&mut self) -> &Vec<String> {
        self.client.get_tags()
    }

    /// Clear the current thread’s operation tags.
    pub fn clear_tags(&mut self) {
        self.client.clear_tags()
    }

    /// The version of Spark on which this application is running.
    pub async fn version(&self) -> Result<String, SparkError> {
        let version = spark::analyze_plan_request::Analyze::SparkVersion(
            spark::analyze_plan_request::SparkVersion {},
        );

        let mut client = self.client.clone();

        client.analyze(version).await?.spark_version()
    }

    /// [RunTimeConfig] configuration interface for Spark.
    pub fn conf(&self) -> RunTimeConfig {
        RunTimeConfig::new(&self.client)
    }
}

#[cfg(test)]
mod tests {
    use super::*;

    use arrow::{
        array::{ArrayRef, StringArray},
        record_batch::RecordBatch,
    };

    async fn setup() -> SparkSession {
        println!("SparkSession Setup");

        let connection = "sc://127.0.0.1:15002/;user_id=rust_test;session_id=0d2af2a9-cc3c-4d4b-bf27-e2fefeaca233";

        SparkSessionBuilder::remote(connection)
            .build()
            .await
            .unwrap()
    }

    #[test]
    fn test_session_builder() {
        let connection = "sc://myhost.com:443/;token=ABCDEFG;user_agent=some_agent;user_id=user123";

        let ssbuilder = SparkSessionBuilder::remote(connection);

        assert_eq!(
            "http://myhost.com:443".to_string(),
            ssbuilder.channel_builder.endpoint()
        );
        assert_eq!(
            "Bearer ABCDEFG".to_string(),
            ssbuilder.channel_builder.token().unwrap()
        );
    }

    #[tokio::test]
    async fn test_spark_range() -> Result<(), SparkError> {
        let spark = setup().await;

        let df = spark.range(None, 100, 1, Some(8));

        let records = df.collect().await?;

        assert_eq!(records.num_rows(), 100);
        Ok(())
    }

    #[tokio::test]
    async fn test_spark_create_dataframe() -> Result<(), SparkError> {
        let spark = setup().await;

        let a: ArrayRef = Arc::new(StringArray::from(vec!["hello", "world"]));

        let record_batch = RecordBatch::try_from_iter(vec![("a", a)])?;

        let df = spark.create_dataframe(&record_batch)?;

        let rows = df.collect().await?;

        assert_eq!(record_batch, rows);
        Ok(())
    }

    #[tokio::test]
    async fn test_spark_session_create() {
        let connection =
            "sc://localhost:15002/;token=ABCDEFG;user_agent=some_agent;user_id=user123";

        let spark = SparkSessionBuilder::remote(connection).build().await;

        assert!(spark.is_ok());
    }

    #[tokio::test]
    async fn test_session_tags() -> Result<(), SparkError> {
        let mut spark = SparkSessionBuilder::default().build().await?;

        spark.add_tag("hello-tag")?;

        spark.add_tag("hello-tag-2")?;

        let expected = vec!["hello-tag".to_string(), "hello-tag-2".to_string()];

        let res = spark.get_tags();

        assert_eq!(&expected, res);

        spark.clear_tags();
        let res = spark.get_tags();

        let expected: Vec<String> = vec![];

        assert_eq!(&expected, res);

        Ok(())
    }

    #[tokio::test]
    async fn test_session_tags_panic() -> Result<(), SparkError> {
        let mut spark = SparkSessionBuilder::default().build().await?;

        assert!(spark.add_tag("bad,tag").is_err());
        assert!(spark.add_tag("").is_err());

        assert!(spark.remove_tag("bad,tag").is_err());
        assert!(spark.remove_tag("").is_err());

        Ok(())
    }

    #[tokio::test]
    async fn test_session_version() -> Result<(), SparkError> {
        let spark = SparkSessionBuilder::default().build().await?;

        let version = spark.version().await?;

        assert_eq!("3.5.1".to_string(), version);
        Ok(())
    }

    #[tokio::test]
    async fn test_session_config() -> Result<(), SparkError> {
        let value = "rust-test-app";

        let spark = SparkSessionBuilder::default()
            .app_name("rust-test-app")
            .build()
            .await?;

        let name = spark.conf().get("spark.app.name", None).await?;

        assert_eq!(value, &name);

        // validate set
        spark
            .conf()
            .set("spark.sql.shuffle.partitions", "42")
            .await?;

        // validate get
        let val = spark
            .conf()
            .get("spark.sql.shuffle.partitions", None)
            .await?;

        assert_eq!("42", &val);

        // validate unset
        spark.conf().unset("spark.sql.shuffle.partitions").await?;

        let val = spark
            .conf()
            .get("spark.sql.shuffle.partitions", None)
            .await?;

        assert_eq!("200", &val);

        // not a modifable setting
        let val = spark
            .conf()
            .is_modifable("spark.executor.instances")
            .await?;
        assert!(!val);

        // a modifable setting
        let val = spark
            .conf()
            .is_modifable("spark.sql.shuffle.partitions")
            .await?;
        assert!(val);

        Ok(())
    }
}