Dandandan commented on a change in pull request #1141: URL: https://github.com/apache/arrow-datafusion/pull/1141#discussion_r735326166
########## 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: Wouldn't a larger batch size also be fine (e.g. 1K or the now default batch size (8K))? ATM we don't gain anything from having it in multiple batches as we wait on the entire results to be returned? For a streaming `MemTable` we could revisit this and collect into smaller chunks? Maybe the page size of the backend could be retrieved from the store (e.g. S3 has 1000) or the store returns the files in chunks itself? -- 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]
