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
//! Spark Connection Client for Rust
//!
//! Currently, the Spark Connect client for Rust is **highly experimental** and **should
//! not be used in any production setting**. This is currently a "proof of concept" to identify the methods
//! of interacting with Spark cluster from rust.
//!
//! # Usage
//!
//! Create a Spark Session and create a DataFrame from a SQL statement:
//!
//! ```rust
//! use spark_connect_rs::{SparkSession, SparkSessionBuilder};
//!
//! #[tokio::main]
//! async fn main() -> Result<(), Box<dyn std::error::Error>> {
//!
//! let spark: SparkSession = SparkSessionBuilder::remote("sc://127.0.0.1:15002/;user_id=example_rs")
//! .build()
//! .await?;
//!
//! let mut df = spark.sql("SELECT * FROM json.`/opt/spark/examples/src/main/resources/employees.json`").await?;
//!
//! df.filter("salary > 3000").show(Some(5), None, None).await?;
//!
//! Ok(())
//! };
//!```
//!
//! Create a Spark Session, create a DataFrame from a CSV file, apply function transformations, and write the results:
//!
//! ```rust
//! use spark_connect_rs::{SparkSession, SparkSessionBuilder};
//!
//! use spark_connect_rs::functions as F;
//!
//! #[tokio::main]
//! async fn main() -> Result<(), Box<dyn std::error::Error>> {
//!
//! let spark: SparkSession = SparkSessionBuilder::remote("sc://127.0.0.1:15002/;user_id=example_rs")
//! .build()
//! .await?;
//!
//! let paths = vec!["/opt/spark/examples/src/main/resources/people.csv".to_string()];
//!
//! let mut df = spark
//! .read()
//! .format("csv")
//! .option("header", "True")
//! .option("delimiter", ";")
//! .load(paths);
//!
//! let mut df = df
//! .filter("age > 30")
//! .select(vec![
//! F::col("name"),
//! F::col("age").cast("int")
//! ]);
//!
//! df.write()
//! .format("csv")
//! .option("header", "true")
//! .save("/opt/spark/examples/src/main/rust/people/")
//! .await?;
//!
//! Ok(())
//! };
//!```
//!
/// Spark Connect gRPC protobuf translated using [tonic]
pub mod spark {
tonic::include_proto!("spark.connect");
}
pub mod dataframe;
pub mod plan;
pub mod readwriter;
pub mod session;
mod catalog;
mod client;
pub mod column;
mod errors;
mod expressions;
pub mod functions;
pub mod storage;
mod types;
mod utils;
pub use arrow;
pub use dataframe::{DataFrame, DataFrameReader, DataFrameWriter};
pub use session::{SparkSession, SparkSessionBuilder};
#[cfg(test)]
mod tests {
use std::sync::Arc;
use arrow::{
array::Int64Array,
datatypes::{DataType, Field, Schema},
record_batch::RecordBatch,
};
use super::*;
use super::functions::*;
async fn setup() -> SparkSession {
println!("SparkSession Setup");
let connection = "sc://127.0.0.1:15002/;user_id=rust_test";
SparkSessionBuilder::remote(connection)
.build()
.await
.unwrap()
}
#[tokio::test]
async fn test_dataframe_range() {
let spark = setup().await;
let mut df = spark.range(None, 100, 1, Some(8));
let records = df.collect().await.unwrap();
assert_eq!(records.num_rows(), 100)
}
#[tokio::test]
async fn test_dataframe_sort() {
let spark = setup().await;
let mut df = spark
.range(None, 100, 1, Some(1))
.sort(vec![col("id").desc()]);
let rows = df.limit(1).collect().await.unwrap();
let schema = Schema::new(vec![Field::new("id", DataType::Int64, false)]);
let value = Int64Array::from(vec![99]);
let expected_batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(value)]).unwrap();
assert_eq!(expected_batch, rows)
}
#[tokio::test]
async fn test_dataframe_read() {
let spark = setup().await;
let path = ["/opt/spark/examples/src/main/resources/people.csv"];
let mut df = spark
.read()
.format("csv")
.option("header", "True")
.option("delimiter", ";")
.load(path);
let rows = df
.filter("age > 30")
.select(vec![col("name")])
.collect()
.await
.unwrap();
assert_eq!(rows.num_rows(), 1);
}
#[tokio::test]
async fn test_dataframe_write() {
let spark = setup().await;
let df = spark
.clone()
.range(None, 1000, 1, Some(16))
.selectExpr(vec!["id AS range_id"]);
let path = "/opt/spark/examples/src/main/rust/employees/";
df.write()
.format("csv")
.option("header", "true")
.save(path)
.await
.unwrap();
let mut df = spark
.clone()
.read()
.format("csv")
.option("header", "true")
.load([path]);
let records = df.select(vec![col("range_id")]).collect().await.unwrap();
assert_eq!(records.num_rows(), 1000)
}
#[tokio::test]
async fn test_dataframe_write_table() {
let spark = setup().await;
let df = spark
.clone()
.range(None, 1000, 1, Some(16))
.selectExpr(vec!["id AS range_id"]);
df.write()
.mode(dataframe::SaveMode::Overwrite)
.saveAsTable("test_table")
.await
.unwrap();
let mut df = spark.clone().read().table("test_table", None);
let records = df.select(vec![col("range_id")]).collect().await.unwrap();
assert_eq!(records.num_rows(), 1000)
}
}