-
Notifications
You must be signed in to change notification settings - Fork 1.3k
/
Copy pathdataframe.rs
343 lines (306 loc) · 11.5 KB
/
dataframe.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
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
use arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray};
use datafusion::arrow::datatypes::{DataType, Field, Schema};
use datafusion::dataframe::DataFrameWriteOptions;
use datafusion::error::Result;
use datafusion::functions_aggregate::average::avg;
use datafusion::functions_aggregate::min_max::max;
use datafusion::prelude::*;
use datafusion_common::config::CsvOptions;
use datafusion_common::parsers::CompressionTypeVariant;
use datafusion_common::DataFusionError;
use datafusion_common::ScalarValue;
use std::fs::File;
use std::io::Write;
use std::sync::Arc;
use tempfile::tempdir;
/// This example demonstrates using DataFusion's DataFrame API
///
/// # Reading from different formats
///
/// * [read_parquet]: execute queries against parquet files
/// * [read_csv]: execute queries against csv files
/// * [read_memory]: execute queries against in-memory arrow data
///
/// # Writing out to local storage
///
/// The following examples demonstrate how to write a DataFrame to local
/// storage. See `external_dependency/dataframe-to-s3.rs` for an example writing
/// to a remote object store.
///
/// * [write_out]: write out a DataFrame to a table, parquet file, csv file, or json file
///
/// # Executing subqueries
///
/// * [where_scalar_subquery]: execute a scalar subquery
/// * [where_in_subquery]: execute a subquery with an IN clause
/// * [where_exist_subquery]: execute a subquery with an EXISTS clause
///
/// # Querying data
///
/// * [query_to_date]: execute queries against parquet files
#[tokio::main]
async fn main() -> Result<()> {
// The SessionContext is the main high level API for interacting with DataFusion
let ctx = SessionContext::new();
read_parquet(&ctx).await?;
read_csv(&ctx).await?;
read_memory(&ctx).await?;
write_out(&ctx).await?;
query_to_date().await?;
register_aggregate_test_data("t1", &ctx).await?;
register_aggregate_test_data("t2", &ctx).await?;
where_scalar_subquery(&ctx).await?;
where_in_subquery(&ctx).await?;
where_exist_subquery(&ctx).await?;
Ok(())
}
/// Use DataFrame API to
/// 1. Read parquet files,
/// 2. Show the schema
/// 3. Select columns and rows
async fn read_parquet(ctx: &SessionContext) -> Result<()> {
// Find the local path of "alltypes_plain.parquet"
let testdata = datafusion::test_util::parquet_test_data();
let filename = &format!("{testdata}/alltypes_plain.parquet");
// Read the parquet files and show its schema using 'describe'
let parquet_df = ctx
.read_parquet(filename, ParquetReadOptions::default())
.await?;
// show its schema using 'describe'
parquet_df.clone().describe().await?.show().await?;
// Select three columns and filter the results
// so that only rows where id > 1 are returned
parquet_df
.select_columns(&["id", "bool_col", "timestamp_col"])?
.filter(col("id").gt(lit(1)))?
.show()
.await?;
Ok(())
}
/// Use the DataFrame API to
/// 1. Read CSV files
/// 2. Optionally specify schema
async fn read_csv(ctx: &SessionContext) -> Result<()> {
// create example.csv file in a temporary directory
let dir = tempdir()?;
let file_path = dir.path().join("example.csv");
{
let mut file = File::create(&file_path)?;
// write CSV data
file.write_all(
r#"id,time,vote,unixtime,rating
a1,"10 6, 2013",3,1381017600,5.0
a2,"08 9, 2013",2,1376006400,4.5"#
.as_bytes(),
)?;
} // scope closes the file
let file_path = file_path.to_str().unwrap();
// You can read a CSV file and DataFusion will infer the schema automatically
let csv_df = ctx.read_csv(file_path, CsvReadOptions::default()).await?;
csv_df.show().await?;
// If you know the types of your data you can specify them explicitly
let schema = Schema::new(vec![
Field::new("id", DataType::Utf8, false),
Field::new("time", DataType::Utf8, false),
Field::new("vote", DataType::Int32, true),
Field::new("unixtime", DataType::Int64, false),
Field::new("rating", DataType::Float32, true),
]);
// Create a csv option provider with the desired schema
let csv_read_option = CsvReadOptions {
// Update the option provider with the defined schema
schema: Some(&schema),
..Default::default()
};
let csv_df = ctx.read_csv(file_path, csv_read_option).await?;
csv_df.show().await?;
// You can also create DataFrames from the result of sql queries
// and using the `enable_url_table` refer to local files directly
let dyn_ctx = ctx.clone().enable_url_table();
let csv_df = dyn_ctx
.sql(&format!("SELECT rating, unixtime FROM '{}'", file_path))
.await?;
csv_df.show().await?;
Ok(())
}
/// Use the DataFrame API to:
/// 1. Read in-memory data.
async fn read_memory(ctx: &SessionContext) -> Result<()> {
// define data in memory
let a: ArrayRef = Arc::new(StringArray::from(vec!["a", "b", "c", "d"]));
let b: ArrayRef = Arc::new(Int32Array::from(vec![1, 10, 10, 100]));
let batch = RecordBatch::try_from_iter(vec![("a", a), ("b", b)])?;
// declare a table in memory. In Apache Spark API, this corresponds to createDataFrame(...).
ctx.register_batch("t", batch)?;
let df = ctx.table("t").await?;
// construct an expression corresponding to "SELECT a, b FROM t WHERE b = 10" in SQL
let filter = col("b").eq(lit(10));
let df = df.select_columns(&["a", "b"])?.filter(filter)?;
// print the results
df.show().await?;
Ok(())
}
/// Use the DataFrame API to:
/// 1. Write out a DataFrame to a table
/// 2. Write out a DataFrame to a parquet file
/// 3. Write out a DataFrame to a csv file
/// 4. Write out a DataFrame to a json file
async fn write_out(ctx: &SessionContext) -> std::result::Result<(), DataFusionError> {
let mut df = ctx.sql("values ('a'), ('b'), ('c')").await.unwrap();
// Ensure the column names and types match the target table
df = df.with_column_renamed("column1", "tablecol1").unwrap();
ctx.sql(
"create external table
test(tablecol1 varchar)
stored as parquet
location './datafusion-examples/test_table/'",
)
.await?
.collect()
.await?;
// This is equivalent to INSERT INTO test VALUES ('a'), ('b'), ('c').
// The behavior of write_table depends on the TableProvider's implementation
// of the insert_into method.
df.clone()
.write_table("test", DataFrameWriteOptions::new())
.await?;
df.clone()
.write_parquet(
"./datafusion-examples/test_parquet/",
DataFrameWriteOptions::new(),
None,
)
.await?;
df.clone()
.write_csv(
"./datafusion-examples/test_csv/",
// DataFrameWriteOptions contains options which control how data is written
// such as compression codec
DataFrameWriteOptions::new(),
Some(CsvOptions::default().with_compression(CompressionTypeVariant::GZIP)),
)
.await?;
df.clone()
.write_json(
"./datafusion-examples/test_json/",
DataFrameWriteOptions::new(),
None,
)
.await?;
Ok(())
}
/// This example demonstrates how to use the to_date series
/// of functions in the DataFrame API as well as via sql.
async fn query_to_date() -> Result<()> {
// define a schema.
let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, false)]));
// define data.
let batch = RecordBatch::try_new(
schema,
vec![Arc::new(StringArray::from(vec![
"2020-09-08T13:42:29Z",
"2020-09-08T13:42:29.190855-05:00",
"2020-08-09 12:13:29",
"2020-01-02",
]))],
)?;
// declare a new context. In spark API, this corresponds to a new spark SQLsession
let ctx = SessionContext::new();
// declare a table in memory. In spark API, this corresponds to createDataFrame(...).
ctx.register_batch("t", batch)?;
let df = ctx.table("t").await?;
// use to_date function to convert col 'a' to timestamp type using the default parsing
let df = df.with_column("a", to_date(vec![col("a")]))?;
let df = df.select_columns(&["a"])?;
// print the results
df.show().await?;
Ok(())
}
/// Use the DataFrame API to execute the following subquery:
/// select c1,c2 from t1 where (select avg(t2.c2) from t2 where t1.c1 = t2.c1)>0 limit 3;
async fn where_scalar_subquery(ctx: &SessionContext) -> Result<()> {
ctx.table("t1")
.await?
.filter(
scalar_subquery(Arc::new(
ctx.table("t2")
.await?
.filter(out_ref_col(DataType::Utf8, "t1.c1").eq(col("t2.c1")))?
.aggregate(vec![], vec![avg(col("t2.c2"))])?
.select(vec![avg(col("t2.c2"))])?
.into_unoptimized_plan(),
))
.gt(lit(0u8)),
)?
.select(vec![col("t1.c1"), col("t1.c2")])?
.limit(0, Some(3))?
.show()
.await?;
Ok(())
}
/// Use the DataFrame API to execute the following subquery:
/// select t1.c1, t1.c2 from t1 where t1.c2 in (select max(t2.c2) from t2 where t2.c1 > 0 ) limit 3;
async fn where_in_subquery(ctx: &SessionContext) -> Result<()> {
ctx.table("t1")
.await?
.filter(in_subquery(
col("t1.c2"),
Arc::new(
ctx.table("t2")
.await?
.filter(col("t2.c1").gt(lit(ScalarValue::UInt8(Some(0)))))?
.aggregate(vec![], vec![max(col("t2.c2"))])?
.select(vec![max(col("t2.c2"))])?
.into_unoptimized_plan(),
),
))?
.select(vec![col("t1.c1"), col("t1.c2")])?
.limit(0, Some(3))?
.show()
.await?;
Ok(())
}
/// Use the DataFrame API to execute the following subquery:
/// select t1.c1, t1.c2 from t1 where exists (select t2.c2 from t2 where t1.c1 = t2.c1) limit 3;
async fn where_exist_subquery(ctx: &SessionContext) -> Result<()> {
ctx.table("t1")
.await?
.filter(exists(Arc::new(
ctx.table("t2")
.await?
.filter(out_ref_col(DataType::Utf8, "t1.c1").eq(col("t2.c1")))?
.select(vec![col("t2.c2")])?
.into_unoptimized_plan(),
)))?
.select(vec![col("t1.c1"), col("t1.c2")])?
.limit(0, Some(3))?
.show()
.await?;
Ok(())
}
async fn register_aggregate_test_data(name: &str, ctx: &SessionContext) -> Result<()> {
let testdata = datafusion::test_util::arrow_test_data();
ctx.register_csv(
name,
&format!("{testdata}/csv/aggregate_test_100.csv"),
CsvReadOptions::default(),
)
.await?;
Ok(())
}