Skip to content

Commit 710977b

Browse files
committed
revert type param on parquet
1 parent 77e1eb2 commit 710977b

File tree

7 files changed

+61
-52
lines changed

7 files changed

+61
-52
lines changed

datafusion/src/datasource/datasource2.rs

Lines changed: 7 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -48,17 +48,17 @@ pub struct SourceDescriptor {
4848
pub schema: SchemaRef,
4949
}
5050

51-
pub trait DataSource2<R: ChunkReader + 'static>: Send + Sync {
51+
pub trait DataSource2: Send + Sync {
5252
fn list_partitions(&self, max_concurrency: usize) -> Result<Arc<FilePartition>>;
5353

5454
fn schema(&self) -> Result<Arc<Schema>>;
5555

56-
fn get_read_for_file(&self, partitioned_file: PartitionedFile) -> Result<R>;
56+
fn get_read_for_file(&self, partitioned_file: PartitionedFile) -> Result<dyn ChunkReader>;
5757

5858
fn statistics(&self) -> &Statistics;
5959
}
6060

61-
pub trait SourceDescBuilder<R: ChunkReader + 'static> {
61+
pub trait SourceDescBuilder {
6262
fn get_source_desc(root_path: &str) -> Result<SourceDescriptor> {
6363
let filenames = Self::get_all_files(root_path)?;
6464
if filenames.is_empty() {
@@ -71,13 +71,12 @@ pub trait SourceDescBuilder<R: ChunkReader + 'static> {
7171
// build a list of Parquet partitions with statistics and gather all unique schemas
7272
// used in this data set
7373
let mut schemas: Vec<Schema> = vec![];
74-
let mut partitioned_files: Vec<PartitionedFile> = vec![];
7574

7675
let partitioned_files = filenames
7776
.iter()
7877
.map(|file_path| {
7978
let pf = Self::get_file_meta(file_path)?;
80-
let schema = pf.schema;
79+
let schema = pf.schema.clone();
8180
if schemas.is_empty() {
8281
schemas.push(schema);
8382
} else if schema != schemas[0] {
@@ -103,15 +102,15 @@ pub trait SourceDescBuilder<R: ChunkReader + 'static> {
103102

104103
fn get_file_meta(file_path: &str) -> Result<PartitionedFile>;
105104

106-
fn reader_for_file_meta(file_path: &str) -> Result<R>;
105+
fn reader_for_file_meta(file_path: &str) -> Result<dyn ChunkReader>;
107106
}
108107

109108
pub trait ParquetSourceDescBuilder: SourceDescBuilder {
110109

111-
fn get_file_meta(file_path: &str) {
110+
fn get_file_meta(file_path: &str) -> Result<PartitionedFile> {
112111
let chunk_reader = Self::reader_for_file_meta(file_path)?;
113112
let file_reader = Arc::new(SerializedFileReader::new(chunk_reader)?);
114-
let arrow_reader = ParquetFileArrowReader::new(file_reader);
113+
let mut arrow_reader = ParquetFileArrowReader::new(file_reader);
115114
let file_path = file_path.to_string();
116115
let schema = arrow_reader.get_schema()?;
117116
let num_fields = schema.fields().len();

datafusion/src/datasource/local/mod.rs

Lines changed: 1 addition & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -23,14 +23,8 @@ use super::datasource2::DataSource2;
2323
use std::fs;
2424
use std::fs::metadata;
2525

26-
struct LocalFSHander {}
27-
28-
impl DataSource2 for LocalFSHander {
29-
30-
}
31-
3226
/// Recursively build a list of files in a directory with a given extension with an accumulator list
33-
fn list_all_files(dir: &str, filenames: &mut Vec<String>, ext: &str) -> Result<()> {
27+
pub fn list_all_files(dir: &str, filenames: &mut Vec<String>, ext: &str) -> Result<()> {
3428
let metadata = metadata(dir)?;
3529
if metadata.is_file() {
3630
if dir.ends_with(ext) {

datafusion/src/datasource/local/parquet_source.rs

Lines changed: 10 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,7 @@
1515
// specific language governing permissions and limitations
1616
// under the License.
1717

18-
use crate::datasource::datasource2::SourceDescBuilder;
18+
use crate::datasource::datasource2::{SourceDescBuilder, PartitionedFile};
1919
use crate::datasource::datasource2::SourceDescriptor;
2020
use crate::error::Result;
2121
use std::fs::File;
@@ -32,14 +32,18 @@ impl LocalParquetSource {
3232
}
3333
}
3434

35-
impl SourceDescBuilder<File> for LocalParquetSource {
36-
fn reader_for_file_meta(file_path: &str) -> Result<File> {
37-
Ok(File::open(file_path)?)
38-
}
39-
35+
impl SourceDescBuilder for LocalParquetSource {
4036
fn get_all_files(root_path: &str) -> Result<Vec<String>> {
4137
let mut filenames: Vec<String> = Vec::new();
4238
super::list_all_files(root_path, &mut filenames, ".parquet");
4339
Ok(filenames)
4440
}
41+
42+
fn get_file_meta(file_path: &str) -> Result<PartitionedFile> {
43+
todo!()
44+
}
45+
46+
fn reader_for_file_meta(file_path: &str) -> Result<File> {
47+
Ok(File::open(file_path)?)
48+
}
4549
}

datafusion/src/datasource/parquet.rs

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -34,20 +34,20 @@ use super::datasource::TableProviderFilterPushDown;
3434
use super::datasource2::DataSource2;
3535

3636
/// Table-based representation of a `ParquetFile`.
37-
pub struct ParquetTable<R: ChunkReader + 'static> {
38-
source: Arc<Box<dyn DataSource2<R>>>,
37+
pub struct ParquetTable {
38+
source: Arc<Box<dyn DataSource2>>,
3939
max_concurrency: usize,
4040
enable_pruning: bool,
4141
}
4242

43-
impl<R: ChunkReader + 'static> ParquetTable<R> {
43+
impl ParquetTable {
4444
/// Attempt to initialize a new `ParquetTable` from a file path.
4545
pub fn try_new(
46-
source: Arc<dyn DataSource2<R>>,
46+
source: Arc<dyn DataSource2>,
4747
max_concurrency: usize,
4848
) -> Result<Self> {
4949
Ok(Self {
50-
source: Box::new(source),
50+
source,
5151
max_concurrency,
5252
enable_pruning: true,
5353
})
@@ -65,7 +65,7 @@ impl<R: ChunkReader + 'static> ParquetTable<R> {
6565
}
6666
}
6767

68-
impl<R: ChunkReader + 'static> TableProvider for ParquetTable<R> {
68+
impl TableProvider for ParquetTable {
6969
fn as_any(&self) -> &dyn Any {
7070
self
7171
}

datafusion/src/datasource/protocol_registry.rs

Lines changed: 25 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -19,31 +19,41 @@ use std::collections::HashMap;
1919
use std::sync::{Arc, RwLock};
2020

2121
use super::datasource2::DataSource2;
22+
use crate::parquet::file::reader::ChunkReader;
23+
use crate::error::{DataFusionError, Result};
24+
use std::any::Any;
25+
use std::fs::File;
2226

23-
pub trait ProtocolHander<R: ChunkReader + 'static>: Sync + Send {
27+
pub trait ProtocolHandler: Sync + Send {
2428
/// Returns the protocol handler as [`Any`](std::any::Any)
2529
/// so that it can be downcast to a specific implementation.
2630
fn as_any(&self) -> &dyn Any;
2731

28-
fn source(&self,
29-
30-
) -> Result<Arc<dyn DataSource2>>;
32+
fn list_all_files(&self, root_path: &str, ext: &str) -> Result<Vec<String>>;
3133

32-
fn list_all_files(&self, root_path: &str) -> Result<Vec<String>>;
33-
34-
fn get_reader(&self, file_path: &str) -> Result<R>;
34+
fn get_reader(&self, file_path: &str) -> Result<dyn ChunkReader>;
3535
}
3636

37-
struct LocalFSHander {
38-
39-
}
37+
pub struct LocalFSHandler;
4038

4139
impl ProtocolHander for LocalFSHander {
40+
fn as_any(&self) -> &dyn Any {
41+
return self;
42+
}
4243

44+
fn list_all_files(&self, root_path: &str, ext: &str) -> Result<Vec<String>> {
45+
let mut filenames: Vec<String> = Vec::new();
46+
crate::datasource::local::list_all_files(root_path, &mut filenames, ext);
47+
Ok(filenames)
48+
}
49+
50+
fn get_reader(&self, file_path: &str) -> Result<R> {
51+
Ok(File::open(file_path)?)
52+
}
4353
}
4454

4555
pub struct ProtocolRegistry {
46-
pub protocol_handlers: RwLock<HashMap<String, Arc<dyn ProtocolHander>>>,
56+
pub protocol_handlers: RwLock<HashMap<String, Arc<dyn ProtocolHandler>>>,
4757
}
4858

4959
impl ProtocolRegistry {
@@ -60,12 +70,12 @@ impl ProtocolRegistry {
6070
prefix: &str,
6171
handler: Arc<dyn ProtocolHander>,
6272
) -> Option<Arc<dyn ProtocolHander>> {
63-
let mut handler = self.protocol_handlers.write().unwrap();
64-
handler.insert(prefix.to_string(), handler)
73+
let mut handlers = self.protocol_handlers.write().unwrap();
74+
handlers.insert(prefix.to_string(), handler)
6575
}
6676

6777
pub fn handler(&self, prefix: &str) -> Option<Arc<dyn ProtocolHander>> {
68-
let handler = self.protocol_handlers.read().unwrap();
69-
handler.get(prefix).cloned()
78+
let handlers = self.protocol_handlers.read().unwrap();
79+
handlers.get(prefix).cloned()
7080
}
7181
}

datafusion/src/execution/context.rs

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -50,6 +50,8 @@ use crate::catalog::{
5050
};
5151
use crate::datasource::csv::CsvFile;
5252
use crate::datasource::parquet::ParquetTable;
53+
use crate::datasource::protocol_registry::ProtocolRegistry;
54+
use crate::datasource::protocol_registry::LocalFSHandler;
5355
use crate::datasource::TableProvider;
5456
use crate::error::{DataFusionError, Result};
5557
use crate::execution::dataframe_impl::DataFrameImpl;
@@ -156,8 +158,9 @@ impl ExecutionContext {
156158
.register_catalog(config.default_catalog.clone(), default_catalog);
157159
}
158160

161+
// register local handler to enable read file from localFS
159162
let protocol_registry = ProtocolRegistry::new();
160-
protocol_registry.register_handler("file", )
163+
protocol_registry.register_handler("file", Arc::new(LocalFSHandler{}));
161164

162165
Self {
163166
state: Arc::new(Mutex::new(ExecutionContextState {
@@ -367,8 +370,6 @@ impl ExecutionContext {
367370
prefix: &str,
368371
handler: Arc<dyn ProtocolHander>,
369372
) -> Option<Arc<dyn ProtocolHander>> {
370-
let prefix = prefix.to_string();
371-
372373
self.state
373374
.lock()
374375
.unwrap()

datafusion/src/physical_plan/parquet.rs

Lines changed: 8 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -65,14 +65,15 @@ use async_trait::async_trait;
6565
use futures::stream::{Stream, StreamExt};
6666

6767
use super::SQLMetric;
68+
use crate::datasource::datasource2::PartitionedFile;
6869

6970
/// Execution plan for scanning one or more Parquet partitions
7071
#[derive(Debug, Clone)]
71-
pub struct ParquetExec<R: ChunkReader + 'static> {
72+
pub struct ParquetExec {
7273
/// Parquet partitions to read
7374
partitions: Vec<ParquetPartition>,
7475
/// Source used for get reader for partitions
75-
source: Box<dyn DataSource2<R>>,
76+
source: Box<dyn DataSource2>,
7677
/// Schema after projection is applied
7778
schema: SchemaRef,
7879
/// Projection for which columns to load
@@ -122,7 +123,7 @@ struct ParquetPartitionMetrics {
122123
pub row_groups_pruned: Arc<SQLMetric>,
123124
}
124125

125-
impl <R: ChunkReader + 'static> ParquetExec<R> {
126+
impl ParquetExec {
126127
/// Create a new Parquet reader execution plan based on the specified Parquet filename or
127128
/// directory containing Parquet files
128129
pub fn try_from_path(
@@ -158,7 +159,7 @@ impl <R: ChunkReader + 'static> ParquetExec<R> {
158159
}
159160

160161
pub fn try_new(
161-
source: Box<dyn DataSource2<R>>,
162+
source: Box<dyn DataSource2>,
162163
source_desc: SourceDescriptor,
163164
projection: Option<Vec<usize>>,
164165
predicate: Option<Expr>,
@@ -251,7 +252,7 @@ impl <R: ChunkReader + 'static> ParquetExec<R> {
251252
/// Create a new Parquet reader execution plan with provided partitions and schema
252253
pub fn new(
253254
partitions: Vec<ParquetPartition>,
254-
source: Box<dyn DataSource2<R>>,
255+
source: Box<dyn DataSource2>,
255256
schema: SchemaRef,
256257
projection: Option<Vec<usize>>,
257258
statistics: Statistics,
@@ -611,8 +612,8 @@ fn build_row_group_predicate(
611612
}
612613
}
613614

614-
fn read_files<R: ChunkReader + 'static>(
615-
source: Arc<dyn DataSource2<R>>,
615+
fn read_files(
616+
source: Arc<dyn DataSource2>,
616617
partition: ParquetPartition,
617618
metrics: ParquetPartitionMetrics,
618619
projection: &[usize],

0 commit comments

Comments
 (0)