Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

use ObjectStore for dataframe writes #6987

Merged
merged 11 commits into from
Jul 24, 2023
92 changes: 92 additions & 0 deletions datafusion-examples/examples/dataframe-to-s3.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,92 @@
// 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 datafusion::datasource::file_format::file_type::{FileType, GetExt};
use datafusion::datasource::file_format::parquet::ParquetFormat;
use datafusion::datasource::listing::ListingOptions;
use datafusion::error::Result;
use datafusion::prelude::*;

//use datafusion::prelude::data;
devinjdangelo marked this conversation as resolved.
Show resolved Hide resolved
use object_store::aws::AmazonS3Builder;
use std::env;
use std::sync::Arc;
use url::Url;

/// This example demonstrates executing a simple query against an Arrow data source (a directory
/// with multiple Parquet files) and fetching results
#[tokio::main]
async fn main() -> Result<()> {
// create local execution context
let ctx = SessionContext::new();

//enter region and bucket to which your credentials have GET and PUT access
let region = "<bucket-region-here>";
let bucket_name = "<bucket-name-here>";

let s3 = AmazonS3Builder::new()
.with_bucket_name(bucket_name)
.with_region(region)
.with_access_key_id(env::var("AWS_ACCESS_KEY_ID").unwrap())
.with_secret_access_key(env::var("AWS_SECRET_ACCESS_KEY").unwrap())
.build()?;

let path = format!("s3://{bucket_name}");
let s3_url = Url::parse(&path).unwrap();
let arc_s3 = Arc::new(s3);
ctx.runtime_env()
.register_object_store(&s3_url, arc_s3.clone());

let path = format!("s3://{bucket_name}/test_data/");
let file_format = ParquetFormat::default().with_enable_pruning(Some(true));
let listing_options = ListingOptions::new(Arc::new(file_format))
.with_file_extension(FileType::PARQUET.get_ext());
ctx.register_listing_table("test", &path, listing_options, None, None)
.await?;

// execute the query
let df = ctx.sql("SELECT * from test").await?;

let out_path = format!("s3://{bucket_name}/test_write/");
df.clone().write_parquet(&out_path, None).await?;

//write as JSON to s3
let json_out = format!("s3://{bucket_name}/json_out");
df.clone().write_json(&json_out).await?;

//write as csv to s3
let csv_out = format!("s3://{bucket_name}/csv_out");
df.write_csv(&csv_out).await?;

let file_format = ParquetFormat::default().with_enable_pruning(Some(true));
let listing_options = ListingOptions::new(Arc::new(file_format))
.with_file_extension(FileType::PARQUET.get_ext());
ctx.register_listing_table("test2", &out_path, listing_options, None, None)
.await?;

let df = ctx
.sql(
"SELECT * \
FROM test2 \
",
)
.await?;

df.show_limit(20).await?;

Ok(())
}
65 changes: 38 additions & 27 deletions datafusion/core/src/datasource/physical_plan/csv.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,12 @@
//! Execution plan for reading CSV files

use crate::datasource::file_format::file_type::FileCompressionType;
use crate::datasource::listing::FileRange;
use crate::datasource::listing::{FileRange, ListingTableUrl};
use crate::datasource::physical_plan::file_stream::{
FileOpenFuture, FileOpener, FileStream,
};
use crate::datasource::physical_plan::FileMeta;
use crate::datasource::physical_plan::RecordBatchMultiPartWriter;
use crate::error::{DataFusionError, Result};
use crate::physical_plan::expressions::PhysicalSortExpr;
use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet};
Expand All @@ -43,10 +44,8 @@ use futures::{StreamExt, TryStreamExt};
use object_store::local::LocalFileSystem;
use object_store::{GetOptions, GetResult, ObjectStore};
use std::any::Any;
use std::fs;
use std::io::Cursor;
use std::ops::Range;
use std::path::Path;
use std::sync::Arc;
use std::task::Poll;
use tokio::task::JoinSet;
Expand Down Expand Up @@ -566,30 +565,28 @@ pub async fn plan_to_csv(
path: impl AsRef<str>,
) -> Result<()> {
let path = path.as_ref();
// create directory to contain the CSV files (one per partition)
let fs_path = Path::new(path);
if let Err(e) = fs::create_dir(fs_path) {
return Err(DataFusionError::Execution(format!(
"Could not create directory {path}: {e:?}"
)));
}

let parsed = ListingTableUrl::parse(path)?;
let object_store_url = parsed.object_store();
let store = task_ctx.runtime_env().object_store(&object_store_url)?;
let mut join_set = JoinSet::new();
for i in 0..plan.output_partitioning().partition_count() {
let plan = plan.clone();
let filename = format!("part-{i}.csv");
let path = fs_path.join(filename);
let file = fs::File::create(path)?;
let mut writer = csv::Writer::new(file);
let stream = plan.execute(i, task_ctx.clone())?;
let storeref = store.clone();
let plan: Arc<dyn ExecutionPlan> = plan.clone();
let filename = format!("{}/part-{i}.csv", parsed.prefix());
let file = object_store::path::Path::parse(filename)?;

let mut stream = plan.execute(i, task_ctx.clone())?;

join_set.spawn(async move {
let result: Result<()> = stream
.map(|batch| writer.write(&batch?))
.try_collect()
.await
.map_err(DataFusionError::from);
result
let (_, multipart_writer) = storeref.put_multipart(&file).await?;
let mut multipart_rb_writer =
RecordBatchMultiPartWriter::new(csv::Writer::new, multipart_writer, None);
while let Some(next_batch) = stream.next().await {
let batch = next_batch?;
multipart_rb_writer.write_rb(batch).await?;
}

multipart_rb_writer.shutdown().await
});
}

Expand Down Expand Up @@ -1033,14 +1030,20 @@ mod tests {
#[tokio::test]
async fn write_csv_results_error_handling() -> Result<()> {
let ctx = SessionContext::new();

// register a local file system object store
let tmp_dir = TempDir::new()?;
let local = Arc::new(LocalFileSystem::new_with_prefix(&tmp_dir)?);
let local_url = Url::parse("file://local").unwrap();
ctx.runtime_env().register_object_store(&local_url, local);
let options = CsvReadOptions::default()
.schema_infer_max_records(2)
.has_header(true);
let df = ctx.read_csv("tests/data/corrupt.csv", options).await?;
let tmp_dir = TempDir::new()?;
let out_dir = tmp_dir.as_ref().to_str().unwrap().to_string() + "/out";

let out_dir_url = "file://local/out";
let e = df
.write_csv(&out_dir)
.write_csv(out_dir_url)
.await
.expect_err("should fail because input file does not match inferred schema");
assert_eq!("Arrow error: Parser error: Error while parsing value d for column 0 at line 4", format!("{e}"));
Expand All @@ -1064,10 +1067,18 @@ mod tests {
)
.await?;

// register a local file system object store
let tmp_dir = TempDir::new()?;
let local = Arc::new(LocalFileSystem::new_with_prefix(&tmp_dir)?);
let local_url = Url::parse("file://local").unwrap();

ctx.runtime_env().register_object_store(&local_url, local);

// execute a simple query and write the results to CSV
let out_dir = tmp_dir.as_ref().to_str().unwrap().to_string() + "/out";
let out_dir_url = "file://local/out";
let df = ctx.sql("SELECT c1, c2 FROM test").await?;
df.write_csv(&out_dir).await?;
df.write_csv(out_dir_url).await?;

// create a new context and verify that the results were saved to a partitioned csv file
let ctx = SessionContext::new();
Expand Down
68 changes: 41 additions & 27 deletions datafusion/core/src/datasource/physical_plan/json.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,12 @@

//! Execution plan for reading line-delimited JSON files
use crate::datasource::file_format::file_type::FileCompressionType;
use crate::datasource::listing::ListingTableUrl;
use crate::datasource::physical_plan::file_stream::{
FileOpenFuture, FileOpener, FileStream,
};
use crate::datasource::physical_plan::FileMeta;
use crate::datasource::physical_plan::RecordBatchMultiPartWriter;
use crate::error::{DataFusionError, Result};
use crate::physical_plan::expressions::PhysicalSortExpr;
use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet};
Expand All @@ -36,11 +38,10 @@ use datafusion_physical_expr::{LexOrdering, OrderingEquivalenceProperties};

use bytes::{Buf, Bytes};
use futures::{ready, stream, StreamExt, TryStreamExt};
use object_store;
use object_store::{GetResult, ObjectStore};
use std::any::Any;
use std::fs;
use std::io::BufReader;
use std::path::Path;
use std::sync::Arc;
use std::task::Poll;
use tokio::task::JoinSet;
Expand Down Expand Up @@ -259,29 +260,31 @@ pub async fn plan_to_json(
path: impl AsRef<str>,
) -> Result<()> {
let path = path.as_ref();
// create directory to contain the CSV files (one per partition)
let fs_path = Path::new(path);
if let Err(e) = fs::create_dir(fs_path) {
return Err(DataFusionError::Execution(format!(
"Could not create directory {path}: {e:?}"
)));
}

let parsed = ListingTableUrl::parse(path)?;
let object_store_url = parsed.object_store();
let store = task_ctx.runtime_env().object_store(&object_store_url)?;
let mut join_set = JoinSet::new();
for i in 0..plan.output_partitioning().partition_count() {
let plan = plan.clone();
let filename = format!("part-{i}.json");
let path = fs_path.join(filename);
let file = fs::File::create(path)?;
let mut writer = json::LineDelimitedWriter::new(file);
let stream = plan.execute(i, task_ctx.clone())?;
let storeref = store.clone();
let plan: Arc<dyn ExecutionPlan> = plan.clone();
let filename = format!("{}/part-{i}.json", parsed.prefix());
let file = object_store::path::Path::parse(filename)?;

let mut stream = plan.execute(i, task_ctx.clone())?;

join_set.spawn(async move {
let result: Result<()> = stream
.map(|batch| writer.write(&batch?))
.try_collect()
.await
.map_err(DataFusionError::from);
result
let (_, multipart_writer) = storeref.put_multipart(&file).await?;
let mut multipart_rb_writer = RecordBatchMultiPartWriter::new(
json::LineDelimitedWriter::new,
multipart_writer,
None,
);
while let Some(next_batch) = stream.next().await {
let batch = next_batch?;
multipart_rb_writer.write_rb(batch).await?;
}

multipart_rb_writer.shutdown().await
});
}

Expand Down Expand Up @@ -320,6 +323,7 @@ mod tests {
use crate::test::partitioned_file_groups;
use datafusion_common::cast::{as_int32_array, as_int64_array, as_string_array};
use rstest::*;
use std::path::Path;
use tempfile::TempDir;
use url::Url;

Expand Down Expand Up @@ -649,7 +653,6 @@ mod tests {
#[tokio::test]
async fn write_json_results() -> Result<()> {
// create partitioned input file and context
let tmp_dir = TempDir::new()?;
let ctx =
SessionContext::with_config(SessionConfig::new().with_target_partitions(8));

Expand All @@ -659,10 +662,17 @@ mod tests {
ctx.register_json("test", path.as_str(), NdJsonReadOptions::default())
.await?;

// register a local file system object store for /tmp directory
let tmp_dir = TempDir::new()?;
let local = Arc::new(LocalFileSystem::new_with_prefix(&tmp_dir)?);
let local_url = Url::parse("file://local").unwrap();
ctx.runtime_env().register_object_store(&local_url, local);

// execute a simple query and write the results to CSV
let out_dir = tmp_dir.as_ref().to_str().unwrap().to_string() + "/out";
let out_dir_url = "file://local/out";
let df = ctx.sql("SELECT a, b FROM test").await?;
df.write_json(&out_dir).await?;
df.write_json(out_dir_url).await?;

// create a new context and verify that the results were saved to a partitioned csv file
let ctx = SessionContext::new();
Expand Down Expand Up @@ -720,14 +730,18 @@ mod tests {
#[tokio::test]
async fn write_json_results_error_handling() -> Result<()> {
let ctx = SessionContext::new();
// register a local file system object store for /tmp directory
let tmp_dir = TempDir::new()?;
let local = Arc::new(LocalFileSystem::new_with_prefix(&tmp_dir)?);
let local_url = Url::parse("file://local").unwrap();
ctx.runtime_env().register_object_store(&local_url, local);
let options = CsvReadOptions::default()
.schema_infer_max_records(2)
.has_header(true);
let df = ctx.read_csv("tests/data/corrupt.csv", options).await?;
let tmp_dir = TempDir::new()?;
let out_dir = tmp_dir.as_ref().to_str().unwrap().to_string() + "/out";
let out_dir_url = "file://local/out";
let e = df
.write_json(&out_dir)
.write_json(out_dir_url)
.await
.expect_err("should fail because input file does not match inferred schema");
assert_eq!("Arrow error: Parser error: Error while parsing value d for column 0 at line 4", format!("{e}"));
Expand Down
Loading