This is an automated email from the ASF dual-hosted git repository.

tustvold pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow-rs.git


The following commit(s) were added to refs/heads/master by this push:
     new e4bb1e9ec0 Support list_with_offset for GCS (#4993)
e4bb1e9ec0 is described below

commit e4bb1e9ec0b6d957da1358bed954b7ca19a76337
Author: Raphael Taylor-Davies <[email protected]>
AuthorDate: Mon Oct 30 11:09:33 2023 +0000

    Support list_with_offset for GCS (#4993)
---
 object_store/src/gcp/client.rs | 6 ++++--
 object_store/src/gcp/mod.rs    | 8 ++++++++
 2 files changed, 12 insertions(+), 2 deletions(-)

diff --git a/object_store/src/gcp/client.rs b/object_store/src/gcp/client.rs
index 78964077e2..e4b0f9af7d 100644
--- a/object_store/src/gcp/client.rs
+++ b/object_store/src/gcp/client.rs
@@ -472,8 +472,6 @@ impl ListClient for GoogleCloudStorageClient {
         page_token: Option<&str>,
         offset: Option<&str>,
     ) -> Result<(ListResult, Option<String>)> {
-        assert!(offset.is_none()); // Not yet supported
-
         let credential = self.get_credential().await?;
         let url = format!("{}/{}", self.config.base_url, 
self.bucket_name_encoded);
 
@@ -495,6 +493,10 @@ impl ListClient for GoogleCloudStorageClient {
             query.push(("max-keys", max_results))
         }
 
+        if let Some(offset) = offset {
+            query.push(("start-after", offset))
+        }
+
         let response = self
             .client
             .request(Method::GET, url)
diff --git a/object_store/src/gcp/mod.rs b/object_store/src/gcp/mod.rs
index 7721b1278a..11fa68310a 100644
--- a/object_store/src/gcp/mod.rs
+++ b/object_store/src/gcp/mod.rs
@@ -147,6 +147,14 @@ impl ObjectStore for GoogleCloudStorage {
         self.client.list(prefix)
     }
 
+    fn list_with_offset(
+        &self,
+        prefix: Option<&Path>,
+        offset: &Path,
+    ) -> BoxStream<'_, Result<ObjectMeta>> {
+        self.client.list_with_offset(prefix, offset)
+    }
+
     async fn list_with_delimiter(&self, prefix: Option<&Path>) -> 
Result<ListResult> {
         self.client.list_with_delimiter(prefix).await
     }

Reply via email to