yjshen commented on a change in pull request #811:
URL: https://github.com/apache/arrow-datafusion/pull/811#discussion_r682247394



##########
File path: datafusion/src/datasource/datasource2.rs
##########
@@ -0,0 +1,163 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::datasource::datasource::{ColumnStatistics, Statistics};
+use crate::error::{DataFusionError, Result};
+use crate::scalar::ScalarValue;
+use arrow::datatypes::{Schema, SchemaRef};
+
+use parquet::arrow::ArrowReader;
+use parquet::arrow::ParquetFileArrowReader;
+use parquet::file::reader::ChunkReader;
+use parquet::file::serialized_reader::SerializedFileReader;
+use std::sync::Arc;
+
+#[derive(Debug, Clone)]
+pub struct PartitionedFile {
+    pub file_path: String,
+    pub schema: Schema,
+    pub statistics: Statistics,
+    pub partition_value: Option<ScalarValue>,
+    pub partition_schema: Option<Schema>,
+    // We may include row group range here for a more fine-grained parallel 
execution
+}
+
+#[derive(Debug, Clone)]
+pub struct FilePartition {
+    pub index: usize,
+    pub files: Vec<PartitionedFile>,
+}
+
+#[derive(Debug, Clone)]
+pub struct SourceDescriptor {
+    pub partition_files: Vec<PartitionedFile>,
+    pub schema: SchemaRef,
+}
+
+pub trait DataSource2: Send + Sync {
+    fn list_partitions(&self, max_concurrency: usize) -> 
Result<Arc<FilePartition>>;
+
+    fn schema(&self) -> Result<Arc<Schema>>;
+
+    fn get_read_for_file(
+        &self,
+        partitioned_file: PartitionedFile,
+    ) -> Result<dyn ChunkReader>;

Review comment:
       Yes, I find it hard to get a suitable abstraction over readable objects 
that is both storages agnostic and generalized enough for data reading, for 
parquet / JSON / cvs all. Therefore I use `ChunkReader` here from parquet first 
and find it storage agnostic. Do you have a suggestion on how to achieve this? 
   
   At the same time, I get stuck with ChunkReader right now since it needs 
associate type defined, it may propagate type parameter to its users and 
finally made the ProtocolHandler not able to find a suitable def that contains 
all kinds of type params.




-- 
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