use std::collections::HashMap;
use std::{any::Any, str::FromStr, sync::Arc};
use super::helpers::{expr_applicable_for_cols, pruned_partition_list, split_files};
use super::{ListingTableUrl, PartitionedFile};
use crate::datasource::{
create_ordering,
file_format::{
file_compression_type::FileCompressionType, FileFormat, FilePushdownSupport,
},
get_statistics_with_limit,
physical_plan::{FileScanConfig, FileSinkConfig},
};
use crate::execution::context::SessionState;
use datafusion_catalog::TableProvider;
use datafusion_common::{config_err, DataFusionError, Result};
use datafusion_expr::dml::InsertOp;
use datafusion_expr::{utils::conjunction, Expr, TableProviderFilterPushDown};
use datafusion_expr::{SortExpr, TableType};
use datafusion_physical_plan::{empty::EmptyExec, ExecutionPlan, Statistics};
use arrow::datatypes::{DataType, Field, SchemaBuilder, SchemaRef};
use arrow_schema::Schema;
use datafusion_common::{
config_datafusion_err, internal_err, plan_err, project_schema, Constraints,
SchemaExt, ToDFSchema,
};
use datafusion_execution::cache::{
cache_manager::FileStatisticsCache, cache_unit::DefaultFileStatisticsCache,
};
use datafusion_physical_expr::{
create_physical_expr, LexOrdering, PhysicalSortRequirement,
};
use async_trait::async_trait;
use datafusion_catalog::Session;
use datafusion_physical_expr_common::sort_expr::LexRequirement;
use futures::{future, stream, StreamExt, TryStreamExt};
use itertools::Itertools;
use object_store::ObjectStore;
#[derive(Debug, Clone)]
pub struct ListingTableConfig {
pub table_paths: Vec<ListingTableUrl>,
pub file_schema: Option<SchemaRef>,
pub options: Option<ListingOptions>,
}
impl ListingTableConfig {
pub fn new(table_path: ListingTableUrl) -> Self {
let table_paths = vec![table_path];
Self {
table_paths,
file_schema: None,
options: None,
}
}
pub fn new_with_multi_paths(table_paths: Vec<ListingTableUrl>) -> Self {
Self {
table_paths,
file_schema: None,
options: None,
}
}
pub fn with_schema(self, schema: SchemaRef) -> Self {
Self {
table_paths: self.table_paths,
file_schema: Some(schema),
options: self.options,
}
}
pub fn with_listing_options(self, listing_options: ListingOptions) -> Self {
Self {
table_paths: self.table_paths,
file_schema: self.file_schema,
options: Some(listing_options),
}
}
fn infer_file_extension_and_compression_type(
path: &str,
) -> Result<(String, Option<String>)> {
let mut exts = path.rsplit('.');
let splitted = exts.next().unwrap_or("");
let file_compression_type = FileCompressionType::from_str(splitted)
.unwrap_or(FileCompressionType::UNCOMPRESSED);
if file_compression_type.is_compressed() {
let splitted2 = exts.next().unwrap_or("");
Ok((splitted2.to_string(), Some(splitted.to_string())))
} else {
Ok((splitted.to_string(), None))
}
}
pub async fn infer_options(self, state: &SessionState) -> Result<Self> {
let store = if let Some(url) = self.table_paths.first() {
state.runtime_env().object_store(url)?
} else {
return Ok(self);
};
let file = self
.table_paths
.first()
.unwrap()
.list_all_files(state, store.as_ref(), "")
.await?
.next()
.await
.ok_or_else(|| DataFusionError::Internal("No files for table".into()))??;
let (file_extension, maybe_compression_type) =
ListingTableConfig::infer_file_extension_and_compression_type(
file.location.as_ref(),
)?;
let mut format_options = HashMap::new();
if let Some(ref compression_type) = maybe_compression_type {
format_options
.insert("format.compression".to_string(), compression_type.clone());
}
let file_format = state
.get_file_format_factory(&file_extension)
.ok_or(config_datafusion_err!(
"No file_format found with extension {file_extension}"
))?
.create(state, &format_options)?;
let listing_file_extension =
if let Some(compression_type) = maybe_compression_type {
format!("{}.{}", &file_extension, &compression_type)
} else {
file_extension
};
let listing_options = ListingOptions::new(file_format)
.with_file_extension(listing_file_extension)
.with_target_partitions(state.config().target_partitions());
Ok(Self {
table_paths: self.table_paths,
file_schema: self.file_schema,
options: Some(listing_options),
})
}
pub async fn infer_schema(self, state: &SessionState) -> Result<Self> {
match self.options {
Some(options) => {
let schema = if let Some(url) = self.table_paths.first() {
options.infer_schema(state, url).await?
} else {
Arc::new(Schema::empty())
};
Ok(Self {
table_paths: self.table_paths,
file_schema: Some(schema),
options: Some(options),
})
}
None => internal_err!("No `ListingOptions` set for inferring schema"),
}
}
pub async fn infer(self, state: &SessionState) -> Result<Self> {
self.infer_options(state).await?.infer_schema(state).await
}
pub async fn infer_partitions_from_path(self, state: &SessionState) -> Result<Self> {
match self.options {
Some(options) => {
let Some(url) = self.table_paths.first() else {
return config_err!("No table path found");
};
let partitions = options
.infer_partitions(state, url)
.await?
.into_iter()
.map(|col_name| {
(
col_name,
DataType::Dictionary(
Box::new(DataType::UInt16),
Box::new(DataType::Utf8),
),
)
})
.collect::<Vec<_>>();
let options = options.with_table_partition_cols(partitions);
Ok(Self {
table_paths: self.table_paths,
file_schema: self.file_schema,
options: Some(options),
})
}
None => config_err!("No `ListingOptions` set for inferring schema"),
}
}
}
#[derive(Clone, Debug)]
pub struct ListingOptions {
pub file_extension: String,
pub format: Arc<dyn FileFormat>,
pub table_partition_cols: Vec<(String, DataType)>,
pub collect_stat: bool,
pub target_partitions: usize,
pub file_sort_order: Vec<Vec<SortExpr>>,
}
impl ListingOptions {
pub fn new(format: Arc<dyn FileFormat>) -> Self {
Self {
file_extension: format.get_ext(),
format,
table_partition_cols: vec![],
collect_stat: true,
target_partitions: 1,
file_sort_order: vec![],
}
}
pub fn with_file_extension(mut self, file_extension: impl Into<String>) -> Self {
self.file_extension = file_extension.into();
self
}
pub fn with_file_extension_opt<S>(mut self, file_extension: Option<S>) -> Self
where
S: Into<String>,
{
if let Some(file_extension) = file_extension {
self.file_extension = file_extension.into();
}
self
}
pub fn with_table_partition_cols(
mut self,
table_partition_cols: Vec<(String, DataType)>,
) -> Self {
self.table_partition_cols = table_partition_cols;
self
}
pub fn with_collect_stat(mut self, collect_stat: bool) -> Self {
self.collect_stat = collect_stat;
self
}
pub fn with_target_partitions(mut self, target_partitions: usize) -> Self {
self.target_partitions = target_partitions;
self
}
pub fn with_file_sort_order(mut self, file_sort_order: Vec<Vec<SortExpr>>) -> Self {
self.file_sort_order = file_sort_order;
self
}
pub async fn infer_schema<'a>(
&'a self,
state: &SessionState,
table_path: &'a ListingTableUrl,
) -> Result<SchemaRef> {
let store = state.runtime_env().object_store(table_path)?;
let files: Vec<_> = table_path
.list_all_files(state, store.as_ref(), &self.file_extension)
.await?
.try_filter(|object_meta| future::ready(object_meta.size > 0))
.try_collect()
.await?;
let schema = self.format.infer_schema(state, &store, &files).await?;
Ok(schema)
}
pub async fn validate_partitions(
&self,
state: &SessionState,
table_path: &ListingTableUrl,
) -> Result<()> {
if self.table_partition_cols.is_empty() {
return Ok(());
}
if !table_path.is_collection() {
return plan_err!(
"Can't create a partitioned table backed by a single file, \
perhaps the URL is missing a trailing slash?"
);
}
let inferred = self.infer_partitions(state, table_path).await?;
if inferred.is_empty() {
return Ok(());
}
let table_partition_names = self
.table_partition_cols
.iter()
.map(|(col_name, _)| col_name.clone())
.collect_vec();
if inferred.len() < table_partition_names.len() {
return plan_err!(
"Inferred partitions to be {:?}, but got {:?}",
inferred,
table_partition_names
);
}
for (idx, col) in table_partition_names.iter().enumerate() {
if &inferred[idx] != col {
return plan_err!(
"Inferred partitions to be {:?}, but got {:?}",
inferred,
table_partition_names
);
}
}
Ok(())
}
pub(crate) async fn infer_partitions(
&self,
state: &SessionState,
table_path: &ListingTableUrl,
) -> Result<Vec<String>> {
let store = state.runtime_env().object_store(table_path)?;
let files: Vec<_> = table_path
.list_all_files(state, store.as_ref(), &self.file_extension)
.await?
.take(10)
.try_collect()
.await?;
let stripped_path_parts = files.iter().map(|file| {
table_path
.strip_prefix(&file.location)
.unwrap()
.collect_vec()
});
let partition_keys = stripped_path_parts
.map(|path_parts| {
path_parts
.into_iter()
.rev()
.skip(1) .rev()
.map(|s| s.split('=').take(1).collect())
.collect_vec()
})
.collect_vec();
match partition_keys.into_iter().all_equal_value() {
Ok(v) => Ok(v),
Err(None) => Ok(vec![]),
Err(Some(diff)) => {
let mut sorted_diff = [diff.0, diff.1];
sorted_diff.sort();
plan_err!("Found mixed partition values on disk {:?}", sorted_diff)
}
}
}
}
#[derive(Debug)]
pub struct ListingTable {
table_paths: Vec<ListingTableUrl>,
file_schema: SchemaRef,
table_schema: SchemaRef,
options: ListingOptions,
definition: Option<String>,
collected_statistics: FileStatisticsCache,
constraints: Constraints,
column_defaults: HashMap<String, Expr>,
}
impl ListingTable {
pub fn try_new(config: ListingTableConfig) -> Result<Self> {
let file_schema = config
.file_schema
.ok_or_else(|| DataFusionError::Internal("No schema provided.".into()))?;
let options = config.options.ok_or_else(|| {
DataFusionError::Internal("No ListingOptions provided".into())
})?;
let mut builder = SchemaBuilder::from(file_schema.as_ref().to_owned());
for (part_col_name, part_col_type) in &options.table_partition_cols {
builder.push(Field::new(part_col_name, part_col_type.clone(), false));
}
let table_schema = Arc::new(
builder
.finish()
.with_metadata(file_schema.metadata().clone()),
);
let table = Self {
table_paths: config.table_paths,
file_schema,
table_schema,
options,
definition: None,
collected_statistics: Arc::new(DefaultFileStatisticsCache::default()),
constraints: Constraints::empty(),
column_defaults: HashMap::new(),
};
Ok(table)
}
pub fn with_constraints(mut self, constraints: Constraints) -> Self {
self.constraints = constraints;
self
}
pub fn with_column_defaults(
mut self,
column_defaults: HashMap<String, Expr>,
) -> Self {
self.column_defaults = column_defaults;
self
}
pub fn with_cache(mut self, cache: Option<FileStatisticsCache>) -> Self {
self.collected_statistics =
cache.unwrap_or(Arc::new(DefaultFileStatisticsCache::default()));
self
}
pub fn with_definition(mut self, definition: Option<String>) -> Self {
self.definition = definition;
self
}
pub fn table_paths(&self) -> &Vec<ListingTableUrl> {
&self.table_paths
}
pub fn options(&self) -> &ListingOptions {
&self.options
}
fn try_create_output_ordering(&self) -> Result<Vec<LexOrdering>> {
create_ordering(&self.table_schema, &self.options.file_sort_order)
}
}
fn can_be_evaluted_for_partition_pruning(
partition_column_names: &[&str],
expr: &Expr,
) -> bool {
!partition_column_names.is_empty()
&& expr_applicable_for_cols(partition_column_names, expr)
}
#[async_trait]
impl TableProvider for ListingTable {
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
Arc::clone(&self.table_schema)
}
fn constraints(&self) -> Option<&Constraints> {
Some(&self.constraints)
}
fn table_type(&self) -> TableType {
TableType::Base
}
async fn scan(
&self,
state: &dyn Session,
projection: Option<&Vec<usize>>,
filters: &[Expr],
limit: Option<usize>,
) -> Result<Arc<dyn ExecutionPlan>> {
let table_partition_cols = self
.options
.table_partition_cols
.iter()
.map(|col| Ok(self.table_schema.field_with_name(&col.0)?.clone()))
.collect::<Result<Vec<_>>>()?;
let table_partition_col_names = table_partition_cols
.iter()
.map(|field| field.name().as_str())
.collect::<Vec<_>>();
let (partition_filters, filters): (Vec<_>, Vec<_>) =
filters.iter().cloned().partition(|filter| {
can_be_evaluted_for_partition_pruning(&table_partition_col_names, filter)
});
let session_state = state.as_any().downcast_ref::<SessionState>().unwrap();
let statistic_file_limit = if filters.is_empty() { limit } else { None };
let (mut partitioned_file_lists, statistics) = self
.list_files_for_scan(session_state, &partition_filters, statistic_file_limit)
.await?;
if partitioned_file_lists.is_empty() {
let projected_schema = project_schema(&self.schema(), projection)?;
return Ok(Arc::new(EmptyExec::new(projected_schema)));
}
let output_ordering = self.try_create_output_ordering()?;
match state
.config_options()
.execution
.split_file_groups_by_statistics
.then(|| {
output_ordering.first().map(|output_ordering| {
FileScanConfig::split_groups_by_statistics(
&self.table_schema,
&partitioned_file_lists,
output_ordering,
)
})
})
.flatten()
{
Some(Err(e)) => log::debug!("failed to split file groups by statistics: {e}"),
Some(Ok(new_groups)) => {
if new_groups.len() <= self.options.target_partitions {
partitioned_file_lists = new_groups;
} else {
log::debug!("attempted to split file groups by statistics, but there were more file groups than target_partitions; falling back to unordered")
}
}
None => {} };
let filters = match conjunction(filters.to_vec()) {
Some(expr) => {
let table_df_schema = self.table_schema.as_ref().clone().to_dfschema()?;
let filters = create_physical_expr(
&expr,
&table_df_schema,
state.execution_props(),
)?;
Some(filters)
}
None => None,
};
let Some(object_store_url) =
self.table_paths.first().map(ListingTableUrl::object_store)
else {
return Ok(Arc::new(EmptyExec::new(Arc::new(Schema::empty()))));
};
self.options
.format
.create_physical_plan(
session_state,
FileScanConfig::new(object_store_url, Arc::clone(&self.file_schema))
.with_file_groups(partitioned_file_lists)
.with_constraints(self.constraints.clone())
.with_statistics(statistics)
.with_projection(projection.cloned())
.with_limit(limit)
.with_output_ordering(output_ordering)
.with_table_partition_cols(table_partition_cols),
filters.as_ref(),
)
.await
}
fn supports_filters_pushdown(
&self,
filters: &[&Expr],
) -> Result<Vec<TableProviderFilterPushDown>> {
let partition_column_names = self
.options
.table_partition_cols
.iter()
.map(|col| col.0.as_str())
.collect::<Vec<_>>();
filters
.iter()
.map(|filter| {
if can_be_evaluted_for_partition_pruning(&partition_column_names, filter)
{
return Ok(TableProviderFilterPushDown::Exact);
}
let supports_pushdown = self.options.format.supports_filters_pushdown(
&self.file_schema,
&self.table_schema,
&[filter],
)?;
if supports_pushdown == FilePushdownSupport::Supported {
return Ok(TableProviderFilterPushDown::Exact);
}
Ok(TableProviderFilterPushDown::Inexact)
})
.collect()
}
fn get_table_definition(&self) -> Option<&str> {
self.definition.as_deref()
}
async fn insert_into(
&self,
state: &dyn Session,
input: Arc<dyn ExecutionPlan>,
insert_op: InsertOp,
) -> Result<Arc<dyn ExecutionPlan>> {
if !self
.schema()
.logically_equivalent_names_and_types(&input.schema())
{
return plan_err!(
"Inserting query must have the same schema with the table. \
Expected: {:?}, got: {:?}",
self.schema()
.fields()
.iter()
.map(|field| field.data_type())
.collect::<Vec<_>>(),
input
.schema()
.fields()
.iter()
.map(|field| field.data_type())
.collect::<Vec<_>>()
);
}
let table_path = &self.table_paths()[0];
if !table_path.is_collection() {
return plan_err!(
"Inserting into a ListingTable backed by a single file is not supported, URL is possibly missing a trailing `/`. \
To append to an existing file use StreamTable, e.g. by using CREATE UNBOUNDED EXTERNAL TABLE"
);
}
let store = state.runtime_env().object_store(table_path)?;
let session_state = state.as_any().downcast_ref::<SessionState>().unwrap();
let file_list_stream = pruned_partition_list(
session_state,
store.as_ref(),
table_path,
&[],
&self.options.file_extension,
&self.options.table_partition_cols,
)
.await?;
let file_groups = file_list_stream.try_collect::<Vec<_>>().await?;
let keep_partition_by_columns =
state.config_options().execution.keep_partition_by_columns;
let config = FileSinkConfig {
object_store_url: self.table_paths()[0].object_store(),
table_paths: self.table_paths().clone(),
file_groups,
output_schema: self.schema(),
table_partition_cols: self.options.table_partition_cols.clone(),
insert_op,
keep_partition_by_columns,
file_extension: self.options().format.get_ext(),
};
let order_requirements = if !self.options().file_sort_order.is_empty() {
let orderings = self.try_create_output_ordering()?;
let Some(ordering) = orderings.first() else {
return internal_err!(
"Expected ListingTable to have a sort order, but none found!"
);
};
Some(LexRequirement::new(
ordering
.into_iter()
.cloned()
.map(PhysicalSortRequirement::from)
.collect::<Vec<_>>(),
))
} else {
None
};
self.options()
.format
.create_writer_physical_plan(input, session_state, config, order_requirements)
.await
}
fn get_column_default(&self, column: &str) -> Option<&Expr> {
self.column_defaults.get(column)
}
}
impl ListingTable {
async fn list_files_for_scan<'a>(
&'a self,
ctx: &'a SessionState,
filters: &'a [Expr],
limit: Option<usize>,
) -> Result<(Vec<Vec<PartitionedFile>>, Statistics)> {
let store = if let Some(url) = self.table_paths.first() {
ctx.runtime_env().object_store(url)?
} else {
return Ok((vec![], Statistics::new_unknown(&self.file_schema)));
};
let file_list = future::try_join_all(self.table_paths.iter().map(|table_path| {
pruned_partition_list(
ctx,
store.as_ref(),
table_path,
filters,
&self.options.file_extension,
&self.options.table_partition_cols,
)
}))
.await?;
let file_list = stream::iter(file_list).flatten();
let files = file_list
.map(|part_file| async {
let part_file = part_file?;
if self.options.collect_stat {
let statistics =
self.do_collect_statistics(ctx, &store, &part_file).await?;
Ok((part_file, statistics))
} else {
Ok((
part_file,
Arc::new(Statistics::new_unknown(&self.file_schema)),
))
}
})
.boxed()
.buffered(ctx.config_options().execution.meta_fetch_concurrency);
let (files, statistics) = get_statistics_with_limit(
files,
self.schema(),
limit,
self.options.collect_stat,
)
.await?;
Ok((
split_files(files, self.options.target_partitions),
statistics,
))
}
async fn do_collect_statistics(
&self,
ctx: &SessionState,
store: &Arc<dyn ObjectStore>,
part_file: &PartitionedFile,
) -> Result<Arc<Statistics>> {
match self
.collected_statistics
.get_with_extra(&part_file.object_meta.location, &part_file.object_meta)
{
Some(statistics) => Ok(statistics),
None => {
let statistics = self
.options
.format
.infer_stats(
ctx,
store,
Arc::clone(&self.file_schema),
&part_file.object_meta,
)
.await?;
let statistics = Arc::new(statistics);
self.collected_statistics.put_with_extra(
&part_file.object_meta.location,
Arc::clone(&statistics),
&part_file.object_meta,
);
Ok(statistics)
}
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::datasource::file_format::avro::AvroFormat;
use crate::datasource::file_format::csv::CsvFormat;
use crate::datasource::file_format::json::JsonFormat;
#[cfg(feature = "parquet")]
use crate::datasource::file_format::parquet::ParquetFormat;
use crate::datasource::{provider_as_source, MemTable};
use crate::execution::options::ArrowReadOptions;
use crate::prelude::*;
use crate::{
assert_batches_eq,
test::{columns, object_store::register_test_store},
};
use datafusion_physical_plan::collect;
use arrow::record_batch::RecordBatch;
use arrow_schema::SortOptions;
use datafusion_common::stats::Precision;
use datafusion_common::{assert_contains, ScalarValue};
use datafusion_expr::{BinaryExpr, LogicalPlanBuilder, Operator};
use datafusion_physical_expr::PhysicalSortExpr;
use datafusion_physical_plan::ExecutionPlanProperties;
use tempfile::TempDir;
#[tokio::test]
async fn read_single_file() -> Result<()> {
let ctx = SessionContext::new();
let table = load_table(&ctx, "alltypes_plain.parquet").await?;
let projection = None;
let exec = table
.scan(&ctx.state(), projection, &[], None)
.await
.expect("Scan table");
assert_eq!(exec.children().len(), 0);
assert_eq!(exec.output_partitioning().partition_count(), 1);
assert_eq!(exec.statistics()?.num_rows, Precision::Exact(8));
assert_eq!(exec.statistics()?.total_byte_size, Precision::Exact(671));
Ok(())
}
#[cfg(feature = "parquet")]
#[tokio::test]
async fn load_table_stats_by_default() -> Result<()> {
use crate::datasource::file_format::parquet::ParquetFormat;
let testdata = crate::test_util::parquet_test_data();
let filename = format!("{}/{}", testdata, "alltypes_plain.parquet");
let table_path = ListingTableUrl::parse(filename).unwrap();
let ctx = SessionContext::new();
let state = ctx.state();
let opt = ListingOptions::new(Arc::new(ParquetFormat::default()));
let schema = opt.infer_schema(&state, &table_path).await?;
let config = ListingTableConfig::new(table_path)
.with_listing_options(opt)
.with_schema(schema);
let table = ListingTable::try_new(config)?;
let exec = table.scan(&state, None, &[], None).await?;
assert_eq!(exec.statistics()?.num_rows, Precision::Exact(8));
assert_eq!(exec.statistics()?.total_byte_size, Precision::Exact(671));
Ok(())
}
#[cfg(feature = "parquet")]
#[tokio::test]
async fn load_table_stats_when_no_stats() -> Result<()> {
use crate::datasource::file_format::parquet::ParquetFormat;
let testdata = crate::test_util::parquet_test_data();
let filename = format!("{}/{}", testdata, "alltypes_plain.parquet");
let table_path = ListingTableUrl::parse(filename).unwrap();
let ctx = SessionContext::new();
let state = ctx.state();
let opt = ListingOptions::new(Arc::new(ParquetFormat::default()))
.with_collect_stat(false);
let schema = opt.infer_schema(&state, &table_path).await?;
let config = ListingTableConfig::new(table_path)
.with_listing_options(opt)
.with_schema(schema);
let table = ListingTable::try_new(config)?;
let exec = table.scan(&state, None, &[], None).await?;
assert_eq!(exec.statistics()?.num_rows, Precision::Absent);
assert_eq!(exec.statistics()?.total_byte_size, Precision::Absent);
Ok(())
}
#[cfg(feature = "parquet")]
#[tokio::test]
async fn test_try_create_output_ordering() {
let testdata = crate::test_util::parquet_test_data();
let filename = format!("{}/{}", testdata, "alltypes_plain.parquet");
let table_path = ListingTableUrl::parse(filename).unwrap();
let ctx = SessionContext::new();
let state = ctx.state();
let options = ListingOptions::new(Arc::new(ParquetFormat::default()));
let schema = options.infer_schema(&state, &table_path).await.unwrap();
use crate::datasource::file_format::parquet::ParquetFormat;
use datafusion_physical_plan::expressions::col as physical_col;
use std::ops::Add;
let cases = vec![
(vec![], Ok(vec![])),
(
vec![vec![
col("int_col").add(lit(1)).sort(true, true),
]],
Err("Expected single column references in output_ordering, got int_col + Int32(1)"),
),
(
vec![vec![col("string_col").sort(true, false)]],
Ok(vec![LexOrdering::new(
vec![PhysicalSortExpr {
expr: physical_col("string_col", &schema).unwrap(),
options: SortOptions {
descending: false,
nulls_first: false,
},
}],
)
])
),
(
vec![vec![
col("string_col").sort(true, false),
col("int_col").sort(false, true),
]],
Ok(vec![LexOrdering::new(
vec![
PhysicalSortExpr::new_default(physical_col("string_col", &schema).unwrap())
.asc()
.nulls_last(),
PhysicalSortExpr::new_default(physical_col("int_col", &schema).unwrap())
.desc()
.nulls_first()
],
)
])
),
];
for (file_sort_order, expected_result) in cases {
let options = options.clone().with_file_sort_order(file_sort_order);
let config = ListingTableConfig::new(table_path.clone())
.with_listing_options(options)
.with_schema(schema.clone());
let table =
ListingTable::try_new(config.clone()).expect("Creating the table");
let ordering_result = table.try_create_output_ordering();
match (expected_result, ordering_result) {
(Ok(expected), Ok(result)) => {
assert_eq!(expected, result);
}
(Err(expected), Err(result)) => {
let result = result.to_string();
let expected = expected.to_string();
assert_contains!(result.to_string(), expected);
}
(expected_result, ordering_result) => {
panic!(
"expected: {expected_result:#?}\n\nactual:{ordering_result:#?}"
);
}
}
}
}
#[tokio::test]
async fn read_empty_table() -> Result<()> {
let ctx = SessionContext::new();
let path = String::from("table/p1=v1/file.avro");
register_test_store(&ctx, &[(&path, 100)]);
let opt = ListingOptions::new(Arc::new(AvroFormat {}))
.with_file_extension(AvroFormat.get_ext())
.with_table_partition_cols(vec![(String::from("p1"), DataType::Utf8)])
.with_target_partitions(4);
let table_path = ListingTableUrl::parse("test:///table/").unwrap();
let file_schema =
Arc::new(Schema::new(vec![Field::new("a", DataType::Boolean, false)]));
let config = ListingTableConfig::new(table_path)
.with_listing_options(opt)
.with_schema(file_schema);
let table = ListingTable::try_new(config)?;
assert_eq!(
columns(&table.schema()),
vec!["a".to_owned(), "p1".to_owned()]
);
let filter = Expr::not_eq(col("p1"), lit("v1"));
let scan = table
.scan(&ctx.state(), None, &[filter], None)
.await
.expect("Empty execution plan");
assert!(scan.as_any().is::<EmptyExec>());
assert_eq!(
columns(&scan.schema()),
vec!["a".to_owned(), "p1".to_owned()]
);
Ok(())
}
#[tokio::test]
async fn test_assert_list_files_for_scan_grouping() -> Result<()> {
assert_list_files_for_scan_grouping(
&[
"bucket/key-prefix/file0",
"bucket/key-prefix/file1",
"bucket/key-prefix/file2",
"bucket/key-prefix/file3",
"bucket/key-prefix/file4",
],
"test:///bucket/key-prefix/",
12,
5,
Some(""),
)
.await?;
assert_list_files_for_scan_grouping(
&[
"bucket/key-prefix/file0",
"bucket/key-prefix/file1",
"bucket/key-prefix/file2",
"bucket/key-prefix/file3",
],
"test:///bucket/key-prefix/",
4,
4,
Some(""),
)
.await?;
assert_list_files_for_scan_grouping(
&[
"bucket/key-prefix/file0",
"bucket/key-prefix/file1",
"bucket/key-prefix/file2",
"bucket/key-prefix/file3",
"bucket/key-prefix/file4",
],
"test:///bucket/key-prefix/",
2,
2,
Some(""),
)
.await?;
assert_list_files_for_scan_grouping(
&[],
"test:///bucket/key-prefix/",
2,
0,
Some(""),
)
.await?;
assert_list_files_for_scan_grouping(
&[
"bucket/key-prefix/file0",
"bucket/key-prefix/file1",
"bucket/other-prefix/roguefile",
],
"test:///bucket/key-prefix/",
10,
2,
Some(""),
)
.await?;
assert_list_files_for_scan_grouping(
&[
"bucket/key-prefix/file0.avro",
"bucket/key-prefix/file1.parquet",
"bucket/other-prefix/roguefile.avro",
],
"test:///bucket/key-prefix/",
10,
1,
None,
)
.await?;
Ok(())
}
#[tokio::test]
async fn test_assert_list_files_for_multi_path() -> Result<()> {
assert_list_files_for_multi_paths(
&[
"bucket/key1/file0",
"bucket/key1/file1",
"bucket/key1/file2",
"bucket/key2/file3",
"bucket/key2/file4",
"bucket/key3/file5",
],
&["test:///bucket/key1/", "test:///bucket/key2/"],
12,
5,
Some(""),
)
.await?;
assert_list_files_for_multi_paths(
&[
"bucket/key1/file0",
"bucket/key1/file1",
"bucket/key1/file2",
"bucket/key2/file3",
"bucket/key2/file4",
"bucket/key3/file5",
],
&["test:///bucket/key1/", "test:///bucket/key2/"],
5,
5,
Some(""),
)
.await?;
assert_list_files_for_multi_paths(
&[
"bucket/key1/file0",
"bucket/key1/file1",
"bucket/key1/file2",
"bucket/key2/file3",
"bucket/key2/file4",
"bucket/key3/file5",
],
&["test:///bucket/key1/"],
2,
2,
Some(""),
)
.await?;
assert_list_files_for_multi_paths(&[], &["test:///bucket/key1/"], 2, 0, Some(""))
.await?;
assert_list_files_for_multi_paths(
&[
"bucket/key1/file0",
"bucket/key1/file1",
"bucket/key1/file2",
"bucket/key2/file3",
"bucket/key2/file4",
"bucket/key3/file5",
],
&["test:///bucket/key3/"],
2,
1,
Some(""),
)
.await?;
assert_list_files_for_multi_paths(
&[
"bucket/key1/file0.avro",
"bucket/key1/file1.csv",
"bucket/key1/file2.avro",
"bucket/key2/file3.csv",
"bucket/key2/file4.avro",
"bucket/key3/file5.csv",
],
&["test:///bucket/key1/", "test:///bucket/key3/"],
2,
2,
None,
)
.await?;
Ok(())
}
async fn load_table(
ctx: &SessionContext,
name: &str,
) -> Result<Arc<dyn TableProvider>> {
let testdata = crate::test_util::parquet_test_data();
let filename = format!("{testdata}/{name}");
let table_path = ListingTableUrl::parse(filename).unwrap();
let config = ListingTableConfig::new(table_path)
.infer(&ctx.state())
.await?;
let table = ListingTable::try_new(config)?;
Ok(Arc::new(table))
}
async fn assert_list_files_for_scan_grouping(
files: &[&str],
table_prefix: &str,
target_partitions: usize,
output_partitioning: usize,
file_ext: Option<&str>,
) -> Result<()> {
let ctx = SessionContext::new();
register_test_store(&ctx, &files.iter().map(|f| (*f, 10)).collect::<Vec<_>>());
let format = AvroFormat {};
let opt = ListingOptions::new(Arc::new(format))
.with_file_extension_opt(file_ext)
.with_target_partitions(target_partitions);
let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]);
let table_path = ListingTableUrl::parse(table_prefix).unwrap();
let config = ListingTableConfig::new(table_path)
.with_listing_options(opt)
.with_schema(Arc::new(schema));
let table = ListingTable::try_new(config)?;
let (file_list, _) = table.list_files_for_scan(&ctx.state(), &[], None).await?;
assert_eq!(file_list.len(), output_partitioning);
Ok(())
}
async fn assert_list_files_for_multi_paths(
files: &[&str],
table_prefix: &[&str],
target_partitions: usize,
output_partitioning: usize,
file_ext: Option<&str>,
) -> Result<()> {
let ctx = SessionContext::new();
register_test_store(&ctx, &files.iter().map(|f| (*f, 10)).collect::<Vec<_>>());
let format = AvroFormat {};
let opt = ListingOptions::new(Arc::new(format))
.with_file_extension_opt(file_ext)
.with_target_partitions(target_partitions);
let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]);
let table_paths = table_prefix
.iter()
.map(|t| ListingTableUrl::parse(t).unwrap())
.collect();
let config = ListingTableConfig::new_with_multi_paths(table_paths)
.with_listing_options(opt)
.with_schema(Arc::new(schema));
let table = ListingTable::try_new(config)?;
let (file_list, _) = table.list_files_for_scan(&ctx.state(), &[], None).await?;
assert_eq!(file_list.len(), output_partitioning);
Ok(())
}
#[tokio::test]
async fn test_insert_into_append_new_json_files() -> Result<()> {
let mut config_map: HashMap<String, String> = HashMap::new();
config_map.insert("datafusion.execution.batch_size".into(), "10".into());
config_map.insert(
"datafusion.execution.soft_max_rows_per_output_file".into(),
"10".into(),
);
helper_test_append_new_files_to_table(
JsonFormat::default().get_ext(),
FileCompressionType::UNCOMPRESSED,
Some(config_map),
2,
)
.await?;
Ok(())
}
#[tokio::test]
async fn test_insert_into_append_new_csv_files() -> Result<()> {
let mut config_map: HashMap<String, String> = HashMap::new();
config_map.insert("datafusion.execution.batch_size".into(), "10".into());
config_map.insert(
"datafusion.execution.soft_max_rows_per_output_file".into(),
"10".into(),
);
helper_test_append_new_files_to_table(
CsvFormat::default().get_ext(),
FileCompressionType::UNCOMPRESSED,
Some(config_map),
2,
)
.await?;
Ok(())
}
#[tokio::test]
async fn test_insert_into_append_2_new_parquet_files_defaults() -> Result<()> {
let mut config_map: HashMap<String, String> = HashMap::new();
config_map.insert("datafusion.execution.batch_size".into(), "10".into());
config_map.insert(
"datafusion.execution.soft_max_rows_per_output_file".into(),
"10".into(),
);
helper_test_append_new_files_to_table(
ParquetFormat::default().get_ext(),
FileCompressionType::UNCOMPRESSED,
Some(config_map),
2,
)
.await?;
Ok(())
}
#[tokio::test]
async fn test_insert_into_append_1_new_parquet_files_defaults() -> Result<()> {
let mut config_map: HashMap<String, String> = HashMap::new();
config_map.insert("datafusion.execution.batch_size".into(), "20".into());
config_map.insert(
"datafusion.execution.soft_max_rows_per_output_file".into(),
"20".into(),
);
helper_test_append_new_files_to_table(
ParquetFormat::default().get_ext(),
FileCompressionType::UNCOMPRESSED,
Some(config_map),
1,
)
.await?;
Ok(())
}
#[tokio::test]
async fn test_insert_into_sql_csv_defaults() -> Result<()> {
helper_test_insert_into_sql("csv", FileCompressionType::UNCOMPRESSED, "", None)
.await?;
Ok(())
}
#[tokio::test]
async fn test_insert_into_sql_csv_defaults_header_row() -> Result<()> {
helper_test_insert_into_sql(
"csv",
FileCompressionType::UNCOMPRESSED,
"",
Some(HashMap::from([("has_header".into(), "true".into())])),
)
.await?;
Ok(())
}
#[tokio::test]
async fn test_insert_into_sql_json_defaults() -> Result<()> {
helper_test_insert_into_sql("json", FileCompressionType::UNCOMPRESSED, "", None)
.await?;
Ok(())
}
#[tokio::test]
async fn test_insert_into_sql_parquet_defaults() -> Result<()> {
helper_test_insert_into_sql(
"parquet",
FileCompressionType::UNCOMPRESSED,
"",
None,
)
.await?;
Ok(())
}
#[tokio::test]
async fn test_insert_into_sql_parquet_session_overrides() -> Result<()> {
let mut config_map: HashMap<String, String> = HashMap::new();
config_map.insert(
"datafusion.execution.parquet.compression".into(),
"zstd(5)".into(),
);
config_map.insert(
"datafusion.execution.parquet.dictionary_enabled".into(),
"false".into(),
);
config_map.insert(
"datafusion.execution.parquet.dictionary_page_size_limit".into(),
"100".into(),
);
config_map.insert(
"datafusion.execution.parquet.statistics_enabled".into(),
"none".into(),
);
config_map.insert(
"datafusion.execution.parquet.max_statistics_size".into(),
"10".into(),
);
config_map.insert(
"datafusion.execution.parquet.max_row_group_size".into(),
"5".into(),
);
config_map.insert(
"datafusion.execution.parquet.created_by".into(),
"datafusion test".into(),
);
config_map.insert(
"datafusion.execution.parquet.column_index_truncate_length".into(),
"50".into(),
);
config_map.insert(
"datafusion.execution.parquet.data_page_row_count_limit".into(),
"50".into(),
);
config_map.insert(
"datafusion.execution.parquet.bloom_filter_on_write".into(),
"true".into(),
);
config_map.insert(
"datafusion.execution.parquet.bloom_filter_fpp".into(),
"0.01".into(),
);
config_map.insert(
"datafusion.execution.parquet.bloom_filter_ndv".into(),
"1000".into(),
);
config_map.insert(
"datafusion.execution.parquet.writer_version".into(),
"2.0".into(),
);
config_map.insert(
"datafusion.execution.parquet.write_batch_size".into(),
"5".into(),
);
helper_test_insert_into_sql(
"parquet",
FileCompressionType::UNCOMPRESSED,
"",
Some(config_map),
)
.await?;
Ok(())
}
#[tokio::test]
async fn test_insert_into_append_new_parquet_files_session_overrides() -> Result<()> {
let mut config_map: HashMap<String, String> = HashMap::new();
config_map.insert("datafusion.execution.batch_size".into(), "10".into());
config_map.insert(
"datafusion.execution.soft_max_rows_per_output_file".into(),
"10".into(),
);
config_map.insert(
"datafusion.execution.parquet.compression".into(),
"zstd(5)".into(),
);
config_map.insert(
"datafusion.execution.parquet.dictionary_enabled".into(),
"false".into(),
);
config_map.insert(
"datafusion.execution.parquet.dictionary_page_size_limit".into(),
"100".into(),
);
config_map.insert(
"datafusion.execution.parquet.statistics_enabled".into(),
"none".into(),
);
config_map.insert(
"datafusion.execution.parquet.max_statistics_size".into(),
"10".into(),
);
config_map.insert(
"datafusion.execution.parquet.max_row_group_size".into(),
"5".into(),
);
config_map.insert(
"datafusion.execution.parquet.created_by".into(),
"datafusion test".into(),
);
config_map.insert(
"datafusion.execution.parquet.column_index_truncate_length".into(),
"50".into(),
);
config_map.insert(
"datafusion.execution.parquet.data_page_row_count_limit".into(),
"50".into(),
);
config_map.insert(
"datafusion.execution.parquet.encoding".into(),
"delta_binary_packed".into(),
);
config_map.insert(
"datafusion.execution.parquet.bloom_filter_on_write".into(),
"true".into(),
);
config_map.insert(
"datafusion.execution.parquet.bloom_filter_fpp".into(),
"0.01".into(),
);
config_map.insert(
"datafusion.execution.parquet.bloom_filter_ndv".into(),
"1000".into(),
);
config_map.insert(
"datafusion.execution.parquet.writer_version".into(),
"2.0".into(),
);
config_map.insert(
"datafusion.execution.parquet.write_batch_size".into(),
"5".into(),
);
config_map.insert("datafusion.execution.batch_size".into(), "1".into());
helper_test_append_new_files_to_table(
ParquetFormat::default().get_ext(),
FileCompressionType::UNCOMPRESSED,
Some(config_map),
2,
)
.await?;
Ok(())
}
#[tokio::test]
async fn test_insert_into_append_new_parquet_files_invalid_session_fails(
) -> Result<()> {
let mut config_map: HashMap<String, String> = HashMap::new();
config_map.insert(
"datafusion.execution.parquet.compression".into(),
"zstd".into(),
);
let e = helper_test_append_new_files_to_table(
ParquetFormat::default().get_ext(),
FileCompressionType::UNCOMPRESSED,
Some(config_map),
2,
)
.await
.expect_err("Example should fail!");
assert_eq!(e.strip_backtrace(), "Invalid or Unsupported Configuration: zstd compression requires specifying a level such as zstd(4)");
Ok(())
}
async fn helper_test_append_new_files_to_table(
file_type_ext: String,
file_compression_type: FileCompressionType,
session_config_map: Option<HashMap<String, String>>,
expected_n_files_per_insert: usize,
) -> Result<()> {
let session_ctx = match session_config_map {
Some(cfg) => {
let config = SessionConfig::from_string_hash_map(&cfg)?;
SessionContext::new_with_config(config)
}
None => SessionContext::new(),
};
let schema = Arc::new(Schema::new(vec![Field::new(
"column1",
DataType::Int32,
false,
)]));
let filter_predicate = Expr::BinaryExpr(BinaryExpr::new(
Box::new(Expr::Column("column1".into())),
Operator::GtEq,
Box::new(Expr::Literal(ScalarValue::Int32(Some(0)))),
));
let batch = RecordBatch::try_new(
schema.clone(),
vec![Arc::new(arrow_array::Int32Array::from(vec![
1, 2, 3, 4, 5, 6, 7, 8, 9, 10,
]))],
)?;
let tmp_dir = TempDir::new()?;
match file_type_ext.as_str() {
"csv" => {
session_ctx
.register_csv(
"t",
tmp_dir.path().to_str().unwrap(),
CsvReadOptions::new()
.schema(schema.as_ref())
.file_compression_type(file_compression_type),
)
.await?;
}
"json" => {
session_ctx
.register_json(
"t",
tmp_dir.path().to_str().unwrap(),
NdJsonReadOptions::default()
.schema(schema.as_ref())
.file_compression_type(file_compression_type),
)
.await?;
}
"parquet" => {
session_ctx
.register_parquet(
"t",
tmp_dir.path().to_str().unwrap(),
ParquetReadOptions::default().schema(schema.as_ref()),
)
.await?;
}
"avro" => {
session_ctx
.register_avro(
"t",
tmp_dir.path().to_str().unwrap(),
AvroReadOptions::default().schema(schema.as_ref()),
)
.await?;
}
"arrow" => {
session_ctx
.register_arrow(
"t",
tmp_dir.path().to_str().unwrap(),
ArrowReadOptions::default().schema(schema.as_ref()),
)
.await?;
}
_ => panic!("Unrecognized file extension {file_type_ext}"),
}
let source_table = Arc::new(MemTable::try_new(
schema.clone(),
vec![vec![batch.clone(), batch.clone()]],
)?);
session_ctx.register_table("source", source_table.clone())?;
let source = provider_as_source(source_table);
let scan_plan = LogicalPlanBuilder::scan("source", source, None)?
.filter(filter_predicate)?
.build()?;
let insert_into_table =
LogicalPlanBuilder::insert_into(scan_plan, "t", &schema, InsertOp::Append)?
.build()?;
let plan = session_ctx
.state()
.create_physical_plan(&insert_into_table)
.await?;
let res = collect(plan, session_ctx.task_ctx()).await?;
let expected = [
"+-------+",
"| count |",
"+-------+",
"| 20 |",
"+-------+",
];
assert_batches_eq!(expected, &res);
let batches = session_ctx
.sql("select count(*) as count from t")
.await?
.collect()
.await?;
let expected = [
"+-------+",
"| count |",
"+-------+",
"| 20 |",
"+-------+",
];
assert_batches_eq!(expected, &batches);
let num_files = tmp_dir.path().read_dir()?.count();
assert_eq!(num_files, expected_n_files_per_insert);
let plan = session_ctx
.state()
.create_physical_plan(&insert_into_table)
.await?;
let res = collect(plan, session_ctx.task_ctx()).await?;
let expected = [
"+-------+",
"| count |",
"+-------+",
"| 20 |",
"+-------+",
];
assert_batches_eq!(expected, &res);
let batches = session_ctx
.sql("select count(*) AS count from t")
.await?
.collect()
.await?;
let expected = [
"+-------+",
"| count |",
"+-------+",
"| 40 |",
"+-------+",
];
assert_batches_eq!(expected, &batches);
let num_files = tmp_dir.path().read_dir()?.count();
assert_eq!(num_files, expected_n_files_per_insert * 2);
Ok(())
}
async fn helper_test_insert_into_sql(
file_type: &str,
_file_compression_type: FileCompressionType,
external_table_options: &str,
session_config_map: Option<HashMap<String, String>>,
) -> Result<()> {
let session_ctx = match session_config_map {
Some(cfg) => {
let config = SessionConfig::from_string_hash_map(&cfg)?;
SessionContext::new_with_config(config)
}
None => SessionContext::new(),
};
let tmp_dir = TempDir::new()?;
let tmp_path = tmp_dir.into_path();
let str_path = tmp_path.to_str().expect("Temp path should convert to &str");
session_ctx
.sql(&format!(
"create external table foo(a varchar, b varchar, c int) \
stored as {file_type} \
location '{str_path}' \
{external_table_options}"
))
.await?
.collect()
.await?;
session_ctx.sql("insert into foo values ('foo', 'bar', 1),('foo', 'bar', 2), ('foo', 'bar', 3)")
.await?
.collect()
.await?;
let batches = session_ctx
.sql("select * from foo")
.await?
.collect()
.await?;
let expected = [
"+-----+-----+---+",
"| a | b | c |",
"+-----+-----+---+",
"| foo | bar | 1 |",
"| foo | bar | 2 |",
"| foo | bar | 3 |",
"+-----+-----+---+",
];
assert_batches_eq!(expected, &batches);
Ok(())
}
#[tokio::test]
async fn test_infer_options_compressed_csv() -> Result<()> {
let testdata = crate::test_util::arrow_test_data();
let filename = format!("{}/csv/aggregate_test_100.csv.gz", testdata);
let table_path = ListingTableUrl::parse(filename).unwrap();
let ctx = SessionContext::new();
let config = ListingTableConfig::new(table_path);
let config_with_opts = config.infer_options(&ctx.state()).await?;
let config_with_schema = config_with_opts.infer_schema(&ctx.state()).await?;
let schema = config_with_schema.file_schema.unwrap();
assert_eq!(schema.fields.len(), 13);
Ok(())
}
}