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



##########
File path: datafusion/src/datasource/protocol_registry.rs
##########
@@ -0,0 +1,81 @@
+// 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 std::collections::HashMap;
+use std::sync::{Arc, RwLock};
+
+use super::datasource2::DataSource2;
+use crate::error::{DataFusionError, Result};
+use crate::parquet::file::reader::ChunkReader;
+use std::any::Any;
+use std::fs::File;
+
+pub trait ProtocolHandler: Sync + Send {
+    /// Returns the protocol handler as [`Any`](std::any::Any)
+    /// so that it can be downcast to a specific implementation.
+    fn as_any(&self) -> &dyn Any;
+
+    fn list_all_files(&self, root_path: &str, ext: &str) -> 
Result<Vec<String>>;
+
+    fn get_reader(&self, file_path: &str) -> Result<dyn ChunkReader>;
+}
+
+pub struct LocalFSHandler;
+
+impl ProtocolHander for LocalFSHander {
+    fn as_any(&self) -> &dyn Any {
+        return self;
+    }
+
+    fn list_all_files(&self, root_path: &str, ext: &str) -> 
Result<Vec<String>> {
+        let mut filenames: Vec<String> = Vec::new();
+        crate::datasource::local::list_all_files(root_path, &mut filenames, 
ext);
+        Ok(filenames)
+    }
+
+    fn get_reader(&self, file_path: &str) -> Result<R> {
+        Ok(File::open(file_path)?)
+    }
+}
+
+pub struct ProtocolRegistry {
+    pub protocol_handlers: RwLock<HashMap<String, Arc<dyn ProtocolHandler>>>,
+}
+
+impl ProtocolRegistry {
+    pub fn new() -> Self {
+        Self {
+            protocol_handlers: RwLock::new(HashMap::new()),
+        }
+    }
+
+    /// Adds a new handler to this registry.
+    /// If a handler of the same prefix existed before, it is replaced in the 
registry and returned.
+    pub fn register_handler(
+        &self,
+        prefix: &str,

Review comment:
       nitpick, i feel like scheme would have been a better name instead of 
prefix here.

##########
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:
       it seems a bit odd to have a abstract data source trait coupled with a 
trait from the parquet crate.

##########
File path: datafusion/src/execution/context.rs
##########
@@ -840,6 +859,8 @@ pub struct ExecutionContextState {
     pub config: ExecutionConfig,
     /// Execution properties
     pub execution_props: ExecutionProps,
+    /// Protocol handlers
+    pub protocol_registry: ProtocolRegistry,

Review comment:
       protocol_registry is not being used at the moment right?

##########
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>;
+
+    fn statistics(&self) -> &Statistics;
+}
+
+pub trait SourceDescBuilder {
+    fn get_source_desc(root_path: &str) -> Result<SourceDescriptor> {
+        let filenames = Self::get_all_files(root_path)?;
+        if filenames.is_empty() {
+            return Err(DataFusionError::Plan(format!(
+                "No Parquet files (with .parquet extension) found at path {}",
+                root_path
+            )));
+        }
+
+        // build a list of Parquet partitions with statistics and gather all 
unique schemas
+        // used in this data set
+        let mut schemas: Vec<Schema> = vec![];
+
+        let partitioned_files = filenames
+            .iter()
+            .map(|file_path| {
+                let pf = Self::get_file_meta(file_path)?;
+                let schema = pf.schema.clone();
+                if schemas.is_empty() {
+                    schemas.push(schema);
+                } else if schema != schemas[0] {
+                    // we currently get the schema information from the first 
file rather than do
+                    // schema merging and this is a limitation.
+                    // See https://issues.apache.org/jira/browse/ARROW-11017
+                    return Err(DataFusionError::Plan(format!(
+                        "The file {} have different schema from the first file 
and DataFusion does \
+                        not yet support schema merging",
+                        file_path
+                    )));
+                }
+                Ok(pf)
+            }).collect::<Result<Vec<PartitionedFile>>>();
+
+        Ok(SourceDescriptor {
+            partition_files: partitioned_files?,
+            schema: Arc::new(schemas.pop().unwrap()),
+        })
+    }
+
+    fn get_all_files(root_path: &str) -> Result<Vec<String>>;
+
+    fn get_file_meta(file_path: &str) -> Result<PartitionedFile>;
+
+    fn reader_for_file_meta(file_path: &str) -> Result<dyn ChunkReader>;

Review comment:
       this method is supposed to return reader for the file content itself, 
not the file metadata right?




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