rdettai commented on a change in pull request #950:
URL: https://github.com/apache/arrow-datafusion/pull/950#discussion_r697994301



##########
File path: datafusion/src/datasource/object_store/mod.rs
##########
@@ -0,0 +1,130 @@
+// 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.
+
+//! Object Store abstracts access to an underlying file/object storage.
+
+pub mod local;
+
+use std::collections::HashMap;
+use std::fmt::Debug;
+use std::pin::Pin;
+use std::sync::{Arc, RwLock};
+
+use async_trait::async_trait;
+use futures::{AsyncRead, Stream};
+
+use local::LocalFileSystem;
+
+use crate::error::{DataFusionError, Result};
+use chrono::Utc;
+
+/// Object Reader for one file in a object store
+#[async_trait]
+pub trait ObjectReader {
+    /// Get reader for a part [start, start + length] in the file 
asynchronously
+    async fn chunk_reader(&self, start: u64, length: usize)
+        -> Result<Arc<dyn AsyncRead>>;
+
+    /// Get length for the file
+    fn length(&self) -> u64;
+}
+
+/// File meta we got from object store
+#[derive(Debug)]
+pub struct FileMeta {
+    /// Path of the file
+    pub path: String,
+    /// Last time the file was modified in UTC
+    pub last_modified: Option<chrono::DateTime<Utc>>,
+    /// File size in total
+    pub size: u64,
+}
+
+/// Stream of files get listed from object store
+pub type FileMetaStream =
+    Pin<Box<dyn Stream<Item = Result<FileMeta>> + Send + Sync + 'static>>;
+
+/// A ObjectStore abstracts access to an underlying file/object storage.
+/// It maps strings (e.g. URLs, filesystem paths, etc) to sources of bytes
+#[async_trait]
+pub trait ObjectStore: Sync + Send + Debug {
+    /// Returns all the files in path `prefix` asynchronously.
+    async fn list(&self, prefix: &str) -> Result<FileMetaStream>;
+
+    /// Get object reader for one file
+    fn file_reader(&self, file: FileMeta) -> Result<Arc<dyn ObjectReader>>;

Review comment:
       👍 for the method renames also 😉 
   
   > Maybe we should keep ObjectStore::get_reader as async? Otherwise, it looks 
like every call to ObjectReader::get_reader would need to open a new file 
descriptor.
   
   - `ObjectStore` can cache the file descriptor if it wants to, this just 
forces it to get it lazily
   - Re-using the file descriptor means that the various AsyncRead objects 
generated might interfer with each other if used in parallel, no? (calling 
`read` on one might move the cursor thus moving the cursor of the other one`). 
Is this ok?




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