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
// 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 std::sync::Arc;
use crate::datasource::physical_plan::plan_to_csv;
use super::super::options::{CsvReadOptions, ReadOptions};
use super::{DataFilePaths, DataFrame, ExecutionPlan, Result, SessionContext};
impl SessionContext {
/// Creates a [`DataFrame`] for reading a CSV data source.
///
/// For more control such as reading multiple files, you can use
/// [`read_table`](Self::read_table) with a [`super::ListingTable`].
///
/// Example usage is given below:
///
/// ```
/// use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// // You can read a single file using `read_csv`
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// // you can also read multiple files:
/// let df = ctx.read_csv(vec!["tests/data/example.csv", "tests/data/example.csv"], CsvReadOptions::new()).await?;
/// # Ok(())
/// # }
/// ```
pub async fn read_csv<P: DataFilePaths>(
&self,
table_paths: P,
options: CsvReadOptions<'_>,
) -> Result<DataFrame> {
self._read_type(table_paths, options).await
}
/// Registers a CSV file as a table which can referenced from SQL
/// statements executed against this context.
pub async fn register_csv(
&self,
name: &str,
table_path: &str,
options: CsvReadOptions<'_>,
) -> Result<()> {
let listing_options = options.to_listing_options(&self.copied_config());
self.register_listing_table(
name,
table_path,
listing_options,
options.schema.map(|s| Arc::new(s.to_owned())),
None,
)
.await?;
Ok(())
}
/// Executes a query and writes the results to a partitioned CSV file.
pub async fn write_csv(
&self,
plan: Arc<dyn ExecutionPlan>,
path: impl AsRef<str>,
) -> Result<()> {
plan_to_csv(self.task_ctx(), plan, path).await
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::assert_batches_eq;
use crate::test_util::{plan_and_collect, populate_csv_partitions};
use async_trait::async_trait;
use tempfile::TempDir;
#[tokio::test]
async fn query_csv_with_custom_partition_extension() -> Result<()> {
let tmp_dir = TempDir::new()?;
// The main stipulation of this test: use a file extension that isn't .csv.
let file_extension = ".tst";
let ctx = SessionContext::new();
let schema = populate_csv_partitions(&tmp_dir, 2, file_extension)?;
ctx.register_csv(
"test",
tmp_dir.path().to_str().unwrap(),
CsvReadOptions::new()
.schema(&schema)
.file_extension(file_extension),
)
.await?;
let results =
plan_and_collect(&ctx, "SELECT SUM(c1), SUM(c2), COUNT(*) FROM test").await?;
assert_eq!(results.len(), 1);
let expected = [
"+--------------+--------------+----------+",
"| SUM(test.c1) | SUM(test.c2) | COUNT(*) |",
"+--------------+--------------+----------+",
"| 10 | 110 | 20 |",
"+--------------+--------------+----------+",
];
assert_batches_eq!(expected, &results);
Ok(())
}
// Test for compilation error when calling read_* functions from an #[async_trait] function.
// See https://github.com/apache/arrow-datafusion/issues/1154
#[async_trait]
trait CallReadTrait {
async fn call_read_csv(&self) -> DataFrame;
}
struct CallRead {}
#[async_trait]
impl CallReadTrait for CallRead {
async fn call_read_csv(&self) -> DataFrame {
let ctx = SessionContext::new();
ctx.read_csv("dummy", CsvReadOptions::new()).await.unwrap()
}
}
}