stayrascal commented on code in PR #324:
URL: 
https://github.com/apache/arrow-rs-object-store/pull/324#discussion_r2041117686


##########
src/memory.rs:
##########
@@ -311,52 +311,70 @@ impl ObjectStore for InMemory {
         Ok(())
     }
 
-    fn list(&self, prefix: Option<&Path>) -> BoxStream<'static, 
Result<ObjectMeta>> {
-        let root = Path::default();
-        let prefix = prefix.unwrap_or(&root);
+    fn list_opts(
+        &self,
+        prefix: Option<&Path>,
+        options: ListOpts,
+    ) -> BoxStream<'static, Result<ListResult>> {
+        if options.delimiter {
+            self.list_with_delimiter(prefix, options.offset.as_ref(), 
options.max_keys)
+        } else {
+            self.list_without_delimiter(prefix, options.offset.as_ref(), 
options.max_keys)
+        }
+    }
 
-        let storage = self.storage.read();
-        let values: Vec<_> = storage
-            .map
-            .range((prefix)..)
-            .take_while(|(key, _)| key.as_ref().starts_with(prefix.as_ref()))
-            .filter(|(key, _)| {
-                // Don't return for exact prefix match
-                key.prefix_match(prefix)
-                    .map(|mut x| x.next().is_some())
-                    .unwrap_or(false)
-            })
-            .map(|(key, value)| {
-                Ok(ObjectMeta {
-                    location: key.clone(),
-                    last_modified: value.last_modified,
-                    size: value.data.len() as u64,
-                    e_tag: Some(value.e_tag.to_string()),
-                    version: None,
-                })
-            })
-            .collect();
+    async fn copy(&self, from: &Path, to: &Path) -> Result<()> {
+        let entry = self.entry(from)?;
+        self.storage
+            .write()
+            .insert(to, entry.data, entry.attributes);
+        Ok(())
+    }
 
-        futures::stream::iter(values).boxed()
+    async fn copy_if_not_exists(&self, from: &Path, to: &Path) -> Result<()> {
+        let entry = self.entry(from)?;
+        let mut storage = self.storage.write();
+        if storage.map.contains_key(to) {
+            return Err(Error::AlreadyExists {
+                path: to.to_string(),
+            }
+            .into());
+        }
+        storage.insert(to, entry.data, entry.attributes);
+        Ok(())
     }
+}
 
-    /// The memory implementation returns all results, as opposed to the cloud
-    /// versions which limit their results to 1k or more because of API
-    /// limitations.
-    async fn list_with_delimiter(&self, prefix: Option<&Path>) -> 
Result<ListResult> {
+impl InMemory {
+    fn list_with_delimiter(
+        &self,
+        prefix: Option<&Path>,
+        offset: Option<&Path>,
+        max_keys: Option<usize>,
+    ) -> BoxStream<'static, Result<ListResult>> {
         let root = Path::default();
         let prefix = prefix.unwrap_or(&root);
+        let offset = offset.unwrap_or(&root);
 
         let mut common_prefixes = BTreeSet::new();
-
         // Only objects in this base level should be returned in the
         // response. Otherwise, we just collect the common prefixes.
         let mut objects = vec![];
         for (k, v) in self.storage.read().map.range((prefix)..) {
+            if let Some(may_keys) = max_keys {
+                if common_prefixes.len() + objects.len() >= may_keys {
+                    break;
+                }
+            }
+
             if !k.as_ref().starts_with(prefix.as_ref()) {
                 break;
             }
 
+            if k <= offset {

Review Comment:
   sure, maybe start listing from the bigger one between `offset` and `prefix`, 
because technically `offset` could be less than `prefix`.



-- 
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: github-unsubscr...@arrow.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to