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

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

pub use crate::client::SparkSessionBuilder;
use crate::dataframe::{DataFrame, DataFrameReader};
use crate::handler::ResponseHandler;
use crate::plan::LogicalPlanBuilder;
use crate::spark;

use arrow::error::ArrowError;
use arrow::record_batch::RecordBatch;

use spark::spark_connect_service_client::SparkConnectServiceClient;
use spark::ExecutePlanResponse;

use tokio::sync::Mutex;
use tonic::transport::Channel;
use tonic::Streaming;

/// The entry point to connecting to a Spark Cluster
/// using the Spark Connection gRPC protocol.
#[allow(dead_code)]
#[derive(Clone, Debug)]
pub struct SparkSession {
    /// Spark Connection gRPC client interface
    pub client: Arc<
        Mutex<
            SparkConnectServiceClient<
                tonic::service::interceptor::InterceptedService<
                    Channel,
                    crate::client::MetadataInterceptor,
                >,
            >,
        >,
    >,

    /// Spark Session ID
    pub session_id: String,

    /// gRPC metadata collected from the connection string
    pub metadata: Option<HashMap<String, String>>,
    pub user_id: Option<String>,

    pub token: Option<&'static str>,
}

impl SparkSession {
    /// 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, 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)
    }

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

        let plan = LogicalPlanBuilder::build_plan_cmd(sql_cmd);

        // !TODO this is gross and needs to be handled WAY better
        let resp = self
            .execute_plan(Some(plan))
            .await
            .unwrap()
            .message()
            .await
            .unwrap()
            .unwrap();

        match resp.response_type {
            Some(spark::execute_plan_response::ResponseType::SqlCommandResult(sql_result)) => {
                let logical_plan = LogicalPlanBuilder::new(sql_result.relation.unwrap());
                DataFrame::new(self.clone(), logical_plan)
            }
            Some(_) => todo!("not implemented"),
            None => todo!("got none as a response for SQL Command"),
        }
    }

    fn build_execute_plan_request(&self, plan: Option<spark::Plan>) -> spark::ExecutePlanRequest {
        spark::ExecutePlanRequest {
            session_id: self.session_id.clone(),
            user_context: Some(spark::UserContext {
                user_id: self.user_id.clone().unwrap_or("NA".to_string()),
                user_name: self.user_id.clone().unwrap_or("NA".to_string()),
                extensions: vec![],
            }),
            operation_id: None,
            plan,
            client_type: Some("_SPARK_CONNECT_RUST".to_string()),
            request_options: vec![],
            tags: vec![],
        }
    }

    fn build_analyze_plan_request(
        &self,
        analyze: Option<spark::analyze_plan_request::Analyze>,
    ) -> spark::AnalyzePlanRequest {
        spark::AnalyzePlanRequest {
            session_id: self.session_id.clone(),
            user_context: Some(spark::UserContext {
                user_id: self.user_id.clone().unwrap_or("NA".to_string()),
                user_name: self.user_id.clone().unwrap_or("NA".to_string()),
                extensions: vec![],
            }),
            client_type: Some("_SPARK_CONNECT_RUST".to_string()),
            analyze,
        }
    }

    pub async fn execute_plan(
        &mut self,
        plan: Option<spark::Plan>,
    ) -> Result<Streaming<ExecutePlanResponse>, tonic::Status> {
        let exc_plan = self.build_execute_plan_request(plan);

        let mut client = self.client.lock().await;

        let value = client.execute_plan(exc_plan).await?.into_inner();

        Ok(value)
    }

    /// Call a service on the remote Spark Connect server by running
    /// a provided [spark::Plan].
    ///
    /// A [spark::Plan] produces a vector of [RecordBatch] records
    pub async fn consume_plan(
        &mut self,
        plan: Option<spark::Plan>,
    ) -> Result<Vec<RecordBatch>, ArrowError> {
        let mut stream = self.execute_plan(plan).await.map_err(|err| {
            ArrowError::IoError(
                err.to_string(),
                Error::new(std::io::ErrorKind::Other, err.to_string()),
            )
        })?;

        let mut handler = ResponseHandler::new();

        while let Some(resp) = stream.message().await.map_err(|err| {
            ArrowError::IoError(
                err.to_string(),
                Error::new(std::io::ErrorKind::Other, err.to_string()),
            )
        })? {
            let _ = handler.handle_response(&resp);
        }
        handler.records()
    }

    pub async fn analyze_plan(
        &mut self,
        analyze: Option<spark::analyze_plan_request::Analyze>,
    ) -> Option<spark::analyze_plan_response::Result> {
        let request = self.build_analyze_plan_request(analyze);
        let mut client = self.client.lock().await;

        let stream = client.analyze_plan(request).await.unwrap();

        stream.into_inner().result
    }
}