houqp commented on a change in pull request #1141:
URL: https://github.com/apache/arrow-datafusion/pull/1141#discussion_r735280489



##########
File path: datafusion/src/datasource/file_format/mod.rs
##########
@@ -36,25 +36,25 @@ use async_trait::async_trait;
 use super::object_store::{ObjectReader, ObjectReaderStream, ObjectStore};
 use super::PartitionedFile;
 
-/// The configurations to be passed when creating a physical plan for
-/// a given file format.
+/// The base configurations to provide when creating a physical plan for
+/// any given file format.
 pub struct PhysicalPlanConfig {
     /// Store from which the `files` should be fetched
     pub object_store: Arc<dyn ObjectStore>,
     /// Schema before projection
-    pub schema: SchemaRef,
+    pub file_schema: SchemaRef,
     /// List of files to be processed, grouped into partitions
-    pub files: Vec<Vec<PartitionedFile>>,
+    pub file_groups: Vec<Vec<PartitionedFile>>,
     /// Estimated overall statistics of the plan, taking `filters` into account
     pub statistics: Statistics,
     /// Columns on which to project the data
     pub projection: Option<Vec<usize>>,
     /// The maximum number of records per arrow column
     pub batch_size: usize,
-    /// The filters that were pushed down to this execution plan
-    pub filters: Vec<Expr>,
     /// The minimum number of records required from this source plan
     pub limit: Option<usize>,
+    /// The partitioning column names
+    pub table_partition_dims: Vec<String>,

Review comment:
       curious why not name it as `table_partition_cols` to better align with 
the comment? the type should make it clear that it's storing column names.

##########
File path: datafusion/src/datasource/listing/table.rs
##########
@@ -109,7 +108,8 @@ impl ListingOptions {
 /// or file system listing capability to get the list of files.
 pub struct ListingTable {
     object_store: Arc<dyn ObjectStore>,
-    path: String,
+    table_path: String,

Review comment:
       I think the table_path should contain contain uri scheme as well? if so, 
`table_uri` might be a better name here.

##########
File path: datafusion/src/datasource/listing/helpers.rs
##########
@@ -0,0 +1,682 @@
+// 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.
+
+//! Helper functions for the table implementation
+
+use std::sync::Arc;
+
+use arrow::{
+    array::{
+        Array, ArrayBuilder, ArrayRef, Date64Array, Date64Builder, StringArray,
+        StringBuilder, UInt64Array, UInt64Builder,
+    },
+    datatypes::{DataType, Field, Schema},
+    record_batch::RecordBatch,
+};
+use chrono::{TimeZone, Utc};
+use futures::{
+    stream::{self},
+    StreamExt, TryStreamExt,
+};
+use log::debug;
+
+use crate::{
+    error::Result,
+    execution::context::ExecutionContext,
+    logical_plan::{self, Expr},
+    physical_plan::functions::Volatility,
+    scalar::ScalarValue,
+};
+
+use crate::datasource::{
+    object_store::{FileMeta, ObjectStore, SizedFile},
+    MemTable, PartitionedFile, PartitionedFileStream,
+};
+
+const FILE_SIZE_COLUMN_NAME: &str = "_df_part_file_size_";
+const FILE_PATH_COLUMN_NAME: &str = "_df_part_file_path_";
+const FILE_MODIFIED_COLUMN_NAME: &str = "_df_part_file_modified_";
+
+/// Partition the list of files into `n` groups
+pub fn split_files(
+    partitioned_files: Vec<PartitionedFile>,
+    n: usize,
+) -> Vec<Vec<PartitionedFile>> {
+    if partitioned_files.is_empty() {
+        return vec![];
+    }
+    let mut chunk_size = partitioned_files.len() / n;
+    if partitioned_files.len() % n > 0 {
+        chunk_size += 1;
+    }
+    partitioned_files
+        .chunks(chunk_size)
+        .map(|c| c.to_vec())
+        .collect()
+}
+
+/// Discover the partitions on the given path and prune out files
+/// relative to irrelevant partitions using `filters` expressions
+/// TODO for tables with many files (10k+), it will usually more efficient
+/// to first list the folders relative to the first partition dimension,
+/// prune those, then list only the contain of the remaining folders.
+pub async fn pruned_partition_list(
+    store: &dyn ObjectStore,
+    table_path: &str,
+    filters: &[Expr],
+    file_extension: &str,
+    table_partition_dims: &[String],
+) -> Result<PartitionedFileStream> {
+    if table_partition_dims.is_empty() {
+        Ok(Box::pin(
+            store
+                .list_file_with_suffix(table_path, file_extension)
+                .await?
+                .map(|f| {
+                    Ok(PartitionedFile {
+                        file_meta: f?,
+                        partition_values: vec![],
+                    })
+                }),
+        ))
+    } else {
+        let applicable_filters = filters
+            .iter()
+            .filter(|f| expr_applicable_for_cols(table_partition_dims, f));
+
+        let table_partition_dims = table_partition_dims.to_vec();
+        let stream_path = table_path.to_owned();
+        // TODO avoid collecting but have a streaming memory table instead
+        let batches: Vec<RecordBatch> = store
+            .list_file_with_suffix(table_path, file_extension)
+            .await?
+            .chunks(64)

Review comment:
       any particular reason for picking 64 here?

##########
File path: datafusion/src/datasource/listing/helpers.rs
##########
@@ -0,0 +1,682 @@
+// 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.
+
+//! Helper functions for the table implementation
+
+use std::sync::Arc;
+
+use arrow::{
+    array::{
+        Array, ArrayBuilder, ArrayRef, Date64Array, Date64Builder, StringArray,
+        StringBuilder, UInt64Array, UInt64Builder,
+    },
+    datatypes::{DataType, Field, Schema},
+    record_batch::RecordBatch,
+};
+use chrono::{TimeZone, Utc};
+use futures::{
+    stream::{self},
+    StreamExt, TryStreamExt,
+};
+use log::debug;
+
+use crate::{
+    error::Result,
+    execution::context::ExecutionContext,
+    logical_plan::{self, Expr},
+    physical_plan::functions::Volatility,
+    scalar::ScalarValue,
+};
+
+use crate::datasource::{
+    object_store::{FileMeta, ObjectStore, SizedFile},
+    MemTable, PartitionedFile, PartitionedFileStream,
+};
+
+const FILE_SIZE_COLUMN_NAME: &str = "_df_part_file_size_";
+const FILE_PATH_COLUMN_NAME: &str = "_df_part_file_path_";
+const FILE_MODIFIED_COLUMN_NAME: &str = "_df_part_file_modified_";
+
+/// Partition the list of files into `n` groups
+pub fn split_files(
+    partitioned_files: Vec<PartitionedFile>,
+    n: usize,
+) -> Vec<Vec<PartitionedFile>> {
+    if partitioned_files.is_empty() {
+        return vec![];
+    }
+    let mut chunk_size = partitioned_files.len() / n;
+    if partitioned_files.len() % n > 0 {
+        chunk_size += 1;
+    }
+    partitioned_files
+        .chunks(chunk_size)
+        .map(|c| c.to_vec())
+        .collect()
+}
+
+/// Discover the partitions on the given path and prune out files
+/// relative to irrelevant partitions using `filters` expressions
+/// TODO for tables with many files (10k+), it will usually more efficient
+/// to first list the folders relative to the first partition dimension,
+/// prune those, then list only the contain of the remaining folders.
+pub async fn pruned_partition_list(
+    store: &dyn ObjectStore,
+    table_path: &str,
+    filters: &[Expr],
+    file_extension: &str,
+    table_partition_dims: &[String],
+) -> Result<PartitionedFileStream> {
+    if table_partition_dims.is_empty() {
+        Ok(Box::pin(
+            store
+                .list_file_with_suffix(table_path, file_extension)
+                .await?
+                .map(|f| {
+                    Ok(PartitionedFile {
+                        file_meta: f?,
+                        partition_values: vec![],
+                    })
+                }),
+        ))
+    } else {
+        let applicable_filters = filters
+            .iter()
+            .filter(|f| expr_applicable_for_cols(table_partition_dims, f));
+
+        let table_partition_dims = table_partition_dims.to_vec();
+        let stream_path = table_path.to_owned();
+        // TODO avoid collecting but have a streaming memory table instead
+        let batches: Vec<RecordBatch> = store
+            .list_file_with_suffix(table_path, file_extension)
+            .await?
+            .chunks(64)
+            .map(|v| v.into_iter().collect::<Result<Vec<_>>>())
+            .map(move |metas| {
+                paths_to_batch(&table_partition_dims, &stream_path, &metas?)
+            })
+            .try_collect()
+            .await?;
+
+        let mem_table = MemTable::try_new(batches[0].schema(), vec![batches])?;
+
+        // Filter the partitions using a local datafusion context
+        // TODO having the external context would allow us to resolve 
`Volatility::Stable`
+        // scalar functions (`ScalarFunction` & `ScalarUDF`) and 
`ScalarVariable`s
+        let mut ctx = ExecutionContext::new();
+        let mut df = ctx.read_table(Arc::new(mem_table))?;
+        for filter in applicable_filters {
+            df = df.filter(filter.clone())?;
+        }
+        let filtered_batches = df.collect().await?;

Review comment:
       here we go datafusion in datafusion :D I do think in the future we could 
optimize this to only use the physical execution code to directly evaluate the 
record batches.

##########
File path: datafusion/src/datasource/listing/helpers.rs
##########
@@ -0,0 +1,682 @@
+// 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.
+
+//! Helper functions for the table implementation
+
+use std::sync::Arc;
+
+use arrow::{
+    array::{
+        Array, ArrayBuilder, ArrayRef, Date64Array, Date64Builder, StringArray,
+        StringBuilder, UInt64Array, UInt64Builder,
+    },
+    datatypes::{DataType, Field, Schema},
+    record_batch::RecordBatch,
+};
+use chrono::{TimeZone, Utc};
+use futures::{
+    stream::{self},
+    StreamExt, TryStreamExt,
+};
+use log::debug;
+
+use crate::{
+    error::Result,
+    execution::context::ExecutionContext,
+    logical_plan::{self, Expr},
+    physical_plan::functions::Volatility,
+    scalar::ScalarValue,
+};
+
+use crate::datasource::{
+    object_store::{FileMeta, ObjectStore, SizedFile},
+    MemTable, PartitionedFile, PartitionedFileStream,
+};
+
+const FILE_SIZE_COLUMN_NAME: &str = "_df_part_file_size_";
+const FILE_PATH_COLUMN_NAME: &str = "_df_part_file_path_";
+const FILE_MODIFIED_COLUMN_NAME: &str = "_df_part_file_modified_";
+
+/// Partition the list of files into `n` groups
+pub fn split_files(
+    partitioned_files: Vec<PartitionedFile>,
+    n: usize,
+) -> Vec<Vec<PartitionedFile>> {
+    if partitioned_files.is_empty() {
+        return vec![];
+    }
+    let mut chunk_size = partitioned_files.len() / n;
+    if partitioned_files.len() % n > 0 {
+        chunk_size += 1;
+    }

Review comment:
       minor optimizaton:
   ```suggestion
       let chunk_size = (partitioned_files.len()+n-1) / n;
   ```

##########
File path: datafusion/src/datasource/listing/helpers.rs
##########
@@ -0,0 +1,682 @@
+// 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.
+
+//! Helper functions for the table implementation
+
+use std::sync::Arc;
+
+use arrow::{
+    array::{
+        Array, ArrayBuilder, ArrayRef, Date64Array, Date64Builder, StringArray,
+        StringBuilder, UInt64Array, UInt64Builder,
+    },
+    datatypes::{DataType, Field, Schema},
+    record_batch::RecordBatch,
+};
+use chrono::{TimeZone, Utc};
+use futures::{
+    stream::{self},
+    StreamExt, TryStreamExt,
+};
+use log::debug;
+
+use crate::{
+    error::Result,
+    execution::context::ExecutionContext,
+    logical_plan::{self, Expr},
+    physical_plan::functions::Volatility,
+    scalar::ScalarValue,
+};
+
+use crate::datasource::{
+    object_store::{FileMeta, ObjectStore, SizedFile},
+    MemTable, PartitionedFile, PartitionedFileStream,
+};
+
+const FILE_SIZE_COLUMN_NAME: &str = "_df_part_file_size_";
+const FILE_PATH_COLUMN_NAME: &str = "_df_part_file_path_";
+const FILE_MODIFIED_COLUMN_NAME: &str = "_df_part_file_modified_";
+
+/// Partition the list of files into `n` groups
+pub fn split_files(
+    partitioned_files: Vec<PartitionedFile>,
+    n: usize,
+) -> Vec<Vec<PartitionedFile>> {
+    if partitioned_files.is_empty() {
+        return vec![];
+    }
+    let mut chunk_size = partitioned_files.len() / n;
+    if partitioned_files.len() % n > 0 {
+        chunk_size += 1;
+    }
+    partitioned_files
+        .chunks(chunk_size)
+        .map(|c| c.to_vec())
+        .collect()
+}
+
+/// Discover the partitions on the given path and prune out files
+/// relative to irrelevant partitions using `filters` expressions
+/// TODO for tables with many files (10k+), it will usually more efficient
+/// to first list the folders relative to the first partition dimension,
+/// prune those, then list only the contain of the remaining folders.
+pub async fn pruned_partition_list(
+    store: &dyn ObjectStore,
+    table_path: &str,
+    filters: &[Expr],
+    file_extension: &str,
+    table_partition_dims: &[String],
+) -> Result<PartitionedFileStream> {
+    if table_partition_dims.is_empty() {

Review comment:
       I think we can also avoid the filter when no filter is provided?
   ```suggestion
       if table_partition_dims.is_empty() || filters.empty() {
   ```

##########
File path: datafusion/src/datasource/listing/helpers.rs
##########
@@ -0,0 +1,682 @@
+// 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.
+
+//! Helper functions for the table implementation
+
+use std::sync::Arc;
+
+use arrow::{
+    array::{
+        Array, ArrayBuilder, ArrayRef, Date64Array, Date64Builder, StringArray,
+        StringBuilder, UInt64Array, UInt64Builder,
+    },
+    datatypes::{DataType, Field, Schema},
+    record_batch::RecordBatch,
+};
+use chrono::{TimeZone, Utc};
+use futures::{
+    stream::{self},
+    StreamExt, TryStreamExt,
+};
+use log::debug;
+
+use crate::{
+    error::Result,
+    execution::context::ExecutionContext,
+    logical_plan::{self, Expr},
+    physical_plan::functions::Volatility,
+    scalar::ScalarValue,
+};
+
+use crate::datasource::{
+    object_store::{FileMeta, ObjectStore, SizedFile},
+    MemTable, PartitionedFile, PartitionedFileStream,
+};
+
+const FILE_SIZE_COLUMN_NAME: &str = "_df_part_file_size_";
+const FILE_PATH_COLUMN_NAME: &str = "_df_part_file_path_";
+const FILE_MODIFIED_COLUMN_NAME: &str = "_df_part_file_modified_";
+
+/// Partition the list of files into `n` groups
+pub fn split_files(
+    partitioned_files: Vec<PartitionedFile>,
+    n: usize,
+) -> Vec<Vec<PartitionedFile>> {
+    if partitioned_files.is_empty() {
+        return vec![];
+    }
+    let mut chunk_size = partitioned_files.len() / n;
+    if partitioned_files.len() % n > 0 {
+        chunk_size += 1;
+    }
+    partitioned_files
+        .chunks(chunk_size)
+        .map(|c| c.to_vec())
+        .collect()
+}
+
+/// Discover the partitions on the given path and prune out files
+/// relative to irrelevant partitions using `filters` expressions
+/// TODO for tables with many files (10k+), it will usually more efficient
+/// to first list the folders relative to the first partition dimension,
+/// prune those, then list only the contain of the remaining folders.
+pub async fn pruned_partition_list(
+    store: &dyn ObjectStore,
+    table_path: &str,
+    filters: &[Expr],
+    file_extension: &str,
+    table_partition_dims: &[String],
+) -> Result<PartitionedFileStream> {
+    if table_partition_dims.is_empty() {
+        Ok(Box::pin(
+            store
+                .list_file_with_suffix(table_path, file_extension)
+                .await?
+                .map(|f| {
+                    Ok(PartitionedFile {
+                        file_meta: f?,
+                        partition_values: vec![],
+                    })
+                }),
+        ))
+    } else {
+        let applicable_filters = filters
+            .iter()
+            .filter(|f| expr_applicable_for_cols(table_partition_dims, f));
+
+        let table_partition_dims = table_partition_dims.to_vec();
+        let stream_path = table_path.to_owned();
+        // TODO avoid collecting but have a streaming memory table instead
+        let batches: Vec<RecordBatch> = store
+            .list_file_with_suffix(table_path, file_extension)
+            .await?
+            .chunks(64)
+            .map(|v| v.into_iter().collect::<Result<Vec<_>>>())
+            .map(move |metas| {
+                paths_to_batch(&table_partition_dims, &stream_path, &metas?)
+            })
+            .try_collect()
+            .await?;
+
+        let mem_table = MemTable::try_new(batches[0].schema(), vec![batches])?;
+
+        // Filter the partitions using a local datafusion context
+        // TODO having the external context would allow us to resolve 
`Volatility::Stable`
+        // scalar functions (`ScalarFunction` & `ScalarUDF`) and 
`ScalarVariable`s
+        let mut ctx = ExecutionContext::new();
+        let mut df = ctx.read_table(Arc::new(mem_table))?;
+        for filter in applicable_filters {
+            df = df.filter(filter.clone())?;
+        }
+        let filtered_batches = df.collect().await?;
+
+        Ok(Box::pin(stream::iter(
+            batches_to_paths(&filtered_batches).into_iter().map(Ok),
+        )))
+    }
+}
+
+/// convert the paths of the files to a record batch with the following 
columns:
+/// - one column for the file size named `_df_part_file_size_`
+/// - one column for with the original path named `_df_part_file_path_`
+/// - one column for with the last modified date named 
`_df_part_file_modified_`
+/// - ... one column by partition ...
+/// Note: For the last modified date, this looses precisions higher than 
millisecond.
+fn paths_to_batch(
+    table_partition_dims: &[String],
+    table_path: &str,
+    metas: &[FileMeta],
+) -> Result<RecordBatch> {
+    let mut key_builder = StringBuilder::new(metas.len());
+    let mut length_builder = UInt64Builder::new(metas.len());
+    let mut modified_builder = Date64Builder::new(metas.len());
+    let mut partition_builders = table_partition_dims
+        .iter()
+        .map(|_| StringBuilder::new(metas.len()))
+        .collect::<Vec<_>>();
+    for file_meta in metas {
+        if let Some(partition_values) =
+            parse_partitions_for_path(table_path, file_meta.path(), 
table_partition_dims)
+        {
+            key_builder.append_value(file_meta.path())?;
+            length_builder.append_value(file_meta.size())?;
+            match file_meta.last_modified {
+                Some(lm) => 
modified_builder.append_value(lm.timestamp_millis())?,
+                None => modified_builder.append_null()?,
+            }
+            for (i, part_val) in partition_values.iter().enumerate() {
+                partition_builders[i].append_value(part_val)?;
+            }
+        } else {
+            debug!("No partitioning for path {}", file_meta.path());
+        }
+    }
+
+    // finish all builders
+    let mut col_arrays: Vec<ArrayRef> = vec![
+        ArrayBuilder::finish(&mut key_builder),
+        ArrayBuilder::finish(&mut length_builder),
+        ArrayBuilder::finish(&mut modified_builder),
+    ];
+    for mut partition_builder in partition_builders {
+        col_arrays.push(ArrayBuilder::finish(&mut partition_builder));
+    }
+
+    // put the schema together
+    let mut fields = vec![
+        Field::new(FILE_SIZE_COLUMN_NAME, DataType::Utf8, false),
+        Field::new(FILE_PATH_COLUMN_NAME, DataType::UInt64, false),

Review comment:
       typo?
   ```suggestion
           Field::new(FILE_SIZE_COLUMN_NAME, DataType::Uint64, false),
           Field::new(FILE_PATH_COLUMN_NAME, DataType::Utf8, false),
   ```

##########
File path: datafusion/src/datasource/listing/helpers.rs
##########
@@ -0,0 +1,682 @@
+// 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.
+
+//! Helper functions for the table implementation
+
+use std::sync::Arc;
+
+use arrow::{
+    array::{
+        Array, ArrayBuilder, ArrayRef, Date64Array, Date64Builder, StringArray,
+        StringBuilder, UInt64Array, UInt64Builder,
+    },
+    datatypes::{DataType, Field, Schema},
+    record_batch::RecordBatch,
+};
+use chrono::{TimeZone, Utc};
+use futures::{
+    stream::{self},
+    StreamExt, TryStreamExt,
+};
+use log::debug;
+
+use crate::{
+    error::Result,
+    execution::context::ExecutionContext,
+    logical_plan::{self, Expr},
+    physical_plan::functions::Volatility,
+    scalar::ScalarValue,
+};
+
+use crate::datasource::{
+    object_store::{FileMeta, ObjectStore, SizedFile},
+    MemTable, PartitionedFile, PartitionedFileStream,
+};
+
+const FILE_SIZE_COLUMN_NAME: &str = "_df_part_file_size_";
+const FILE_PATH_COLUMN_NAME: &str = "_df_part_file_path_";
+const FILE_MODIFIED_COLUMN_NAME: &str = "_df_part_file_modified_";
+
+/// Partition the list of files into `n` groups
+pub fn split_files(
+    partitioned_files: Vec<PartitionedFile>,
+    n: usize,
+) -> Vec<Vec<PartitionedFile>> {
+    if partitioned_files.is_empty() {
+        return vec![];
+    }
+    let mut chunk_size = partitioned_files.len() / n;
+    if partitioned_files.len() % n > 0 {
+        chunk_size += 1;
+    }
+    partitioned_files
+        .chunks(chunk_size)
+        .map(|c| c.to_vec())
+        .collect()
+}
+
+/// Discover the partitions on the given path and prune out files
+/// relative to irrelevant partitions using `filters` expressions
+/// TODO for tables with many files (10k+), it will usually more efficient
+/// to first list the folders relative to the first partition dimension,
+/// prune those, then list only the contain of the remaining folders.
+pub async fn pruned_partition_list(
+    store: &dyn ObjectStore,
+    table_path: &str,
+    filters: &[Expr],
+    file_extension: &str,
+    table_partition_dims: &[String],
+) -> Result<PartitionedFileStream> {
+    if table_partition_dims.is_empty() {
+        Ok(Box::pin(
+            store
+                .list_file_with_suffix(table_path, file_extension)
+                .await?
+                .map(|f| {
+                    Ok(PartitionedFile {
+                        file_meta: f?,
+                        partition_values: vec![],
+                    })
+                }),
+        ))
+    } else {
+        let applicable_filters = filters
+            .iter()
+            .filter(|f| expr_applicable_for_cols(table_partition_dims, f));
+
+        let table_partition_dims = table_partition_dims.to_vec();
+        let stream_path = table_path.to_owned();
+        // TODO avoid collecting but have a streaming memory table instead
+        let batches: Vec<RecordBatch> = store
+            .list_file_with_suffix(table_path, file_extension)
+            .await?
+            .chunks(64)
+            .map(|v| v.into_iter().collect::<Result<Vec<_>>>())
+            .map(move |metas| {
+                paths_to_batch(&table_partition_dims, &stream_path, &metas?)
+            })
+            .try_collect()
+            .await?;
+
+        let mem_table = MemTable::try_new(batches[0].schema(), vec![batches])?;
+
+        // Filter the partitions using a local datafusion context
+        // TODO having the external context would allow us to resolve 
`Volatility::Stable`
+        // scalar functions (`ScalarFunction` & `ScalarUDF`) and 
`ScalarVariable`s
+        let mut ctx = ExecutionContext::new();
+        let mut df = ctx.read_table(Arc::new(mem_table))?;
+        for filter in applicable_filters {
+            df = df.filter(filter.clone())?;
+        }
+        let filtered_batches = df.collect().await?;
+
+        Ok(Box::pin(stream::iter(
+            batches_to_paths(&filtered_batches).into_iter().map(Ok),
+        )))
+    }
+}
+
+/// convert the paths of the files to a record batch with the following 
columns:
+/// - one column for the file size named `_df_part_file_size_`
+/// - one column for with the original path named `_df_part_file_path_`
+/// - one column for with the last modified date named 
`_df_part_file_modified_`
+/// - ... one column by partition ...
+/// Note: For the last modified date, this looses precisions higher than 
millisecond.
+fn paths_to_batch(
+    table_partition_dims: &[String],
+    table_path: &str,
+    metas: &[FileMeta],
+) -> Result<RecordBatch> {
+    let mut key_builder = StringBuilder::new(metas.len());
+    let mut length_builder = UInt64Builder::new(metas.len());
+    let mut modified_builder = Date64Builder::new(metas.len());
+    let mut partition_builders = table_partition_dims
+        .iter()
+        .map(|_| StringBuilder::new(metas.len()))
+        .collect::<Vec<_>>();
+    for file_meta in metas {
+        if let Some(partition_values) =
+            parse_partitions_for_path(table_path, file_meta.path(), 
table_partition_dims)
+        {
+            key_builder.append_value(file_meta.path())?;
+            length_builder.append_value(file_meta.size())?;
+            match file_meta.last_modified {
+                Some(lm) => 
modified_builder.append_value(lm.timestamp_millis())?,
+                None => modified_builder.append_null()?,
+            }
+            for (i, part_val) in partition_values.iter().enumerate() {
+                partition_builders[i].append_value(part_val)?;
+            }
+        } else {
+            debug!("No partitioning for path {}", file_meta.path());
+        }
+    }
+
+    // finish all builders
+    let mut col_arrays: Vec<ArrayRef> = vec![
+        ArrayBuilder::finish(&mut key_builder),
+        ArrayBuilder::finish(&mut length_builder),
+        ArrayBuilder::finish(&mut modified_builder),
+    ];
+    for mut partition_builder in partition_builders {
+        col_arrays.push(ArrayBuilder::finish(&mut partition_builder));
+    }
+
+    // put the schema together
+    let mut fields = vec![
+        Field::new(FILE_SIZE_COLUMN_NAME, DataType::Utf8, false),
+        Field::new(FILE_PATH_COLUMN_NAME, DataType::UInt64, false),
+        Field::new(FILE_MODIFIED_COLUMN_NAME, DataType::Date64, false),
+    ];
+    for pn in table_partition_dims {
+        fields.push(Field::new(pn, DataType::Utf8, false));
+    }
+
+    let batch = RecordBatch::try_new(Arc::new(Schema::new(fields)), 
col_arrays)?;
+    Ok(batch)
+}
+
+/// convert a set of record batches created by `paths_to_batch()` back to 
partitioned files.
+fn batches_to_paths(batches: &[RecordBatch]) -> Vec<PartitionedFile> {
+    batches
+        .iter()
+        .flat_map(|batch| {
+            let key_array = batch
+                .column(0)
+                .as_any()
+                .downcast_ref::<StringArray>()
+                .unwrap();
+            let length_array = batch
+                .column(1)
+                .as_any()
+                .downcast_ref::<UInt64Array>()
+                .unwrap();
+            let modified_array = batch
+                .column(2)
+                .as_any()
+                .downcast_ref::<Date64Array>()
+                .unwrap();
+
+            (0..batch.num_rows()).map(move |row| PartitionedFile {
+                file_meta: FileMeta {
+                    last_modified: match modified_array.is_null(row) {
+                        false => 
Some(Utc.timestamp_millis(modified_array.value(row))),
+                        true => None,
+                    },
+                    sized_file: SizedFile {
+                        path: key_array.value(row).to_owned(),
+                        size: length_array.value(row),
+                    },
+                },
+                partition_values: (3..batch.columns().len())
+                    .map(|col| {
+                        ScalarValue::try_from_array(batch.column(col), 
row).unwrap()
+                    })
+                    .collect(),
+            })
+        })
+        .collect()
+}
+
+/// Extract the partition values for the given `file_path` (in the given 
`table_path`)
+/// associated to the partitions defined by `table_partition_dims`
+fn parse_partitions_for_path<'a>(
+    table_path: &str,
+    file_path: &'a str,
+    table_partition_dims: &[String],
+) -> Option<Vec<&'a str>> {
+    let subpath = file_path.strip_prefix(table_path)?;
+
+    // ignore whether table_path ended with "/" or not
+    let subpath = match subpath.strip_prefix('/') {
+        Some(subpath) => subpath,
+        None => subpath,
+    };
+
+    let mut part_values = vec![];
+    for (path, pn) in subpath.split('/').zip(table_partition_dims) {
+        if let Some(val) = path.strip_prefix(&format!("{}=", pn)) {

Review comment:
       I think we could avoid the `format!` string creation here by applying 
https://doc.rust-lang.org/stable/std/primitive.str.html#method.split_once with 
`=` on path and compare `pn` with the first component.

##########
File path: datafusion/src/datasource/listing/helpers.rs
##########
@@ -0,0 +1,682 @@
+// 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.
+
+//! Helper functions for the table implementation
+
+use std::sync::Arc;
+
+use arrow::{
+    array::{
+        Array, ArrayBuilder, ArrayRef, Date64Array, Date64Builder, StringArray,
+        StringBuilder, UInt64Array, UInt64Builder,
+    },
+    datatypes::{DataType, Field, Schema},
+    record_batch::RecordBatch,
+};
+use chrono::{TimeZone, Utc};
+use futures::{
+    stream::{self},
+    StreamExt, TryStreamExt,
+};
+use log::debug;
+
+use crate::{
+    error::Result,
+    execution::context::ExecutionContext,
+    logical_plan::{self, Expr},
+    physical_plan::functions::Volatility,
+    scalar::ScalarValue,
+};
+
+use crate::datasource::{
+    object_store::{FileMeta, ObjectStore, SizedFile},
+    MemTable, PartitionedFile, PartitionedFileStream,
+};
+
+const FILE_SIZE_COLUMN_NAME: &str = "_df_part_file_size_";
+const FILE_PATH_COLUMN_NAME: &str = "_df_part_file_path_";
+const FILE_MODIFIED_COLUMN_NAME: &str = "_df_part_file_modified_";
+
+/// Partition the list of files into `n` groups
+pub fn split_files(
+    partitioned_files: Vec<PartitionedFile>,
+    n: usize,
+) -> Vec<Vec<PartitionedFile>> {
+    if partitioned_files.is_empty() {
+        return vec![];
+    }
+    let mut chunk_size = partitioned_files.len() / n;
+    if partitioned_files.len() % n > 0 {
+        chunk_size += 1;
+    }
+    partitioned_files
+        .chunks(chunk_size)
+        .map(|c| c.to_vec())
+        .collect()
+}
+
+/// Discover the partitions on the given path and prune out files
+/// relative to irrelevant partitions using `filters` expressions
+/// TODO for tables with many files (10k+), it will usually more efficient
+/// to first list the folders relative to the first partition dimension,
+/// prune those, then list only the contain of the remaining folders.
+pub async fn pruned_partition_list(
+    store: &dyn ObjectStore,
+    table_path: &str,
+    filters: &[Expr],
+    file_extension: &str,
+    table_partition_dims: &[String],
+) -> Result<PartitionedFileStream> {
+    if table_partition_dims.is_empty() {
+        Ok(Box::pin(
+            store
+                .list_file_with_suffix(table_path, file_extension)
+                .await?
+                .map(|f| {
+                    Ok(PartitionedFile {
+                        file_meta: f?,
+                        partition_values: vec![],
+                    })
+                }),
+        ))
+    } else {
+        let applicable_filters = filters
+            .iter()
+            .filter(|f| expr_applicable_for_cols(table_partition_dims, f));

Review comment:
       to avoid the complexity of `expr_applicable_for_cols`, perhaps we could 
just throw a runtime error to the user if invalid filters are provided? this 
also makes sure that if there is a typo in the filter, we won't silent the 
error.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to