|
17 | 17 |
|
18 | 18 | //! Re-exports the [`datafusion_datasource_arrow::file_format`] module, and contains tests for it. |
19 | 19 | pub use datafusion_datasource_arrow::file_format::*; |
| 20 | + |
| 21 | +#[cfg(test)] |
| 22 | +mod tests { |
| 23 | + use futures::StreamExt; |
| 24 | + use std::sync::Arc; |
| 25 | + |
| 26 | + use arrow::array::{Int64Array, StringArray}; |
| 27 | + use arrow::datatypes::{DataType, Field, Schema}; |
| 28 | + use arrow::record_batch::RecordBatch; |
| 29 | + use datafusion_common::Result; |
| 30 | + |
| 31 | + use crate::execution::options::ArrowReadOptions; |
| 32 | + use crate::prelude::SessionContext; |
| 33 | + |
| 34 | + #[tokio::test] |
| 35 | + async fn test_write_empty_arrow_from_sql() -> Result<()> { |
| 36 | + let ctx = SessionContext::new(); |
| 37 | + |
| 38 | + let tmp_dir = tempfile::TempDir::new()?; |
| 39 | + let path = format!("{}/empty_sql.arrow", tmp_dir.path().to_string_lossy()); |
| 40 | + |
| 41 | + ctx.sql(&format!( |
| 42 | + "COPY (SELECT CAST(1 AS BIGINT) AS id LIMIT 0) TO '{path}' STORED AS ARROW", |
| 43 | + )) |
| 44 | + .await? |
| 45 | + .collect() |
| 46 | + .await?; |
| 47 | + |
| 48 | + assert!(std::path::Path::new(&path).exists()); |
| 49 | + |
| 50 | + let read_df = ctx.read_arrow(&path, ArrowReadOptions::default()).await?; |
| 51 | + let stream = read_df.execute_stream().await?; |
| 52 | + |
| 53 | + assert_eq!(stream.schema().fields().len(), 1); |
| 54 | + assert_eq!(stream.schema().field(0).name(), "id"); |
| 55 | + |
| 56 | + let results: Vec<_> = stream.collect().await; |
| 57 | + let total_rows: usize = results |
| 58 | + .iter() |
| 59 | + .filter_map(|r| r.as_ref().ok()) |
| 60 | + .map(|b| b.num_rows()) |
| 61 | + .sum(); |
| 62 | + assert_eq!(total_rows, 0); |
| 63 | + |
| 64 | + Ok(()) |
| 65 | + } |
| 66 | + |
| 67 | + #[tokio::test] |
| 68 | + async fn test_write_empty_arrow_from_record_batch() -> Result<()> { |
| 69 | + let ctx = SessionContext::new(); |
| 70 | + |
| 71 | + let schema = Arc::new(Schema::new(vec![ |
| 72 | + Field::new("id", DataType::Int64, false), |
| 73 | + Field::new("name", DataType::Utf8, true), |
| 74 | + ])); |
| 75 | + let empty_batch = RecordBatch::try_new( |
| 76 | + schema.clone(), |
| 77 | + vec![ |
| 78 | + Arc::new(Int64Array::from(Vec::<i64>::new())), |
| 79 | + Arc::new(StringArray::from(Vec::<Option<&str>>::new())), |
| 80 | + ], |
| 81 | + )?; |
| 82 | + |
| 83 | + let tmp_dir = tempfile::TempDir::new()?; |
| 84 | + let path = format!("{}/empty_batch.arrow", tmp_dir.path().to_string_lossy()); |
| 85 | + |
| 86 | + ctx.register_batch("empty_table", empty_batch)?; |
| 87 | + |
| 88 | + ctx.sql(&format!("COPY empty_table TO '{path}' STORED AS ARROW")) |
| 89 | + .await? |
| 90 | + .collect() |
| 91 | + .await?; |
| 92 | + |
| 93 | + assert!(std::path::Path::new(&path).exists()); |
| 94 | + |
| 95 | + let read_df = ctx.read_arrow(&path, ArrowReadOptions::default()).await?; |
| 96 | + let stream = read_df.execute_stream().await?; |
| 97 | + |
| 98 | + assert_eq!(stream.schema().fields().len(), 2); |
| 99 | + assert_eq!(stream.schema().field(0).name(), "id"); |
| 100 | + assert_eq!(stream.schema().field(1).name(), "name"); |
| 101 | + |
| 102 | + let results: Vec<_> = stream.collect().await; |
| 103 | + let total_rows: usize = results |
| 104 | + .iter() |
| 105 | + .filter_map(|r| r.as_ref().ok()) |
| 106 | + .map(|b| b.num_rows()) |
| 107 | + .sum(); |
| 108 | + assert_eq!(total_rows, 0); |
| 109 | + |
| 110 | + Ok(()) |
| 111 | + } |
| 112 | +} |
0 commit comments