You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/04/18 13:51:23 UTC

[GitHub] [arrow] pitrou commented on a diff in pull request #12763: ARROW-14892: [Python][C++] GCS Bindings

pitrou commented on code in PR #12763:
URL: https://github.com/apache/arrow/pull/12763#discussion_r852086836


##########
ci/scripts/python_test.sh:
##########
@@ -47,9 +48,15 @@ export PYARROW_TEST_CUDA
 export PYARROW_TEST_DATASET
 export PYARROW_TEST_FLIGHT
 export PYARROW_TEST_GANDIVA
+export PYARROW_TEST_GCS
 export PYARROW_TEST_HDFS
 export PYARROW_TEST_ORC
 export PYARROW_TEST_PARQUET
 export PYARROW_TEST_S3
 
+# DO NOT SUBMIT

Review Comment:
   Why is this there? Is this PR actually ready? Did you perhaps forget to push some followup changes?



##########
.github/workflows/cpp.yml:
##########
@@ -128,6 +128,8 @@ jobs:
       ARROW_GANDIVA: ON
       ARROW_HDFS: ON
       ARROW_HOME: /usr/local
+      # TODO(ARROW-16102): Enable this once we can figure out builds.
+      ARROW_GCS: OFF

Review Comment:
   @emkornfield Did you mean to add this? ARROW-16102 is fixed already.



##########
cpp/src/arrow/filesystem/gcsfs.cc:
##########
@@ -318,19 +326,43 @@ class GcsFileSystem::Impl {
       return GetFileInfoBucket(path, std::move(meta).status());
     }
     auto meta = client_.GetObjectMetadata(path.bucket, path.object);
-    return GetFileInfoObject(path, meta);
+    Result<FileInfo> info = GetFileInfoObject(path, meta);
+    if (!info.ok() || info->type() != FileType::NotFound) {
+      return info;
+    }
+    // Not found case.  It could be this was written to GCS with a different
+    // "Directory" convention.  So it if there are is at least one objec that
+    // matches the prefix we assume it is a directory.

Review Comment:
   Can you check this comment? It's unreadable to me, there may be grammar or spelling errors.



##########
cpp/src/arrow/filesystem/gcsfs.h:
##########
@@ -27,11 +27,37 @@
 namespace arrow {
 namespace fs {
 
-struct GcsCredentials;
+// Opaque wrapper for GCS's library credentials to avoid exposing in Arrow headers.
+struct GcsCredentialsHolder;
+class GcsFileSystem;
+
+/// \brief Container for GCS Credentials an information necessary to recreate
+///  them.

Review Comment:
   ```suggestion
   /// \brief Container for GCS Credentials and information necessary to recreate them.
   ```



##########
cpp/src/arrow/filesystem/gcsfs.cc:
##########
@@ -318,19 +326,43 @@ class GcsFileSystem::Impl {
       return GetFileInfoBucket(path, std::move(meta).status());
     }
     auto meta = client_.GetObjectMetadata(path.bucket, path.object);
-    return GetFileInfoObject(path, meta);
+    Result<FileInfo> info = GetFileInfoObject(path, meta);
+    if (!info.ok() || info->type() != FileType::NotFound) {
+      return info;
+    }
+    // Not found case.  It could be this was written to GCS with a different
+    // "Directory" convention.  So it if there are is at least one objec that
+    // matches the prefix we assume it is a directory.
+    std::string canonical = internal::EnsureTrailingSlash(path.object);
+    std::string end = canonical;
+    end.back() += 1;

Review Comment:
   Hmm, this looks weird. Since `canonical` has a trailing slash, this turns it into a `0`?



##########
cpp/src/arrow/filesystem/gcsfs.cc:
##########
@@ -596,9 +639,15 @@ class GcsFileSystem::Impl {
   }
 
   static FileInfo ToFileInfo(const std::string& full_path,
-                             const gcs::ObjectMetadata& meta) {
-    if (IsDirectory(meta)) {
-      return FileInfo(full_path, FileType::Directory);
+                             const gcs::ObjectMetadata& meta,
+                             bool normalize_directories = false) {

Review Comment:
   Why would we ever want `normalize_directories` to be false?



##########
cpp/src/arrow/filesystem/gcsfs.cc:
##########
@@ -340,11 +372,13 @@ class GcsFileSystem::Impl {
       }
       // Skip the directory itself from the results, and any result that is "too deep"
       // into the recursion.
-      if (o->name() == p.object || internal::Depth(o->name()) > max_depth) {
+      bool has_trailing_slash = !o->name().empty() && o->name().back() == '/';
+      if (o->name() == canonical || o->name() == p.object ||

Review Comment:
   It's not obvious why you compare both to `canonical` and `p.object`?



##########
cpp/src/arrow/filesystem/gcsfs.cc:
##########
@@ -690,11 +745,17 @@ Result<GcsOptions> GcsOptions::FromUri(const arrow::internal::Uri& uri,
     options_map.emplace(kv.first, kv.second);
   }
 
-  if (!uri.password().empty() || !uri.username().empty()) {
-    return Status::Invalid("GCS does not accept username or password.");
+  const std::string& username = uri.username();
+  bool anonymous = username == "anonymous";
+  if (!uri.password().empty() || (!username.empty() && !anonymous)) {
+    return Status::Invalid(
+        "GCS does not accept username except \"anonymous\" or password.");
   }
-
   auto options = GcsOptions::Defaults();
+  if (anonymous) {
+    options = GcsOptions::Anonymous();
+  }

Review Comment:
   I'm not sure how the Google cloud SDK works under the hood, but is it expensive to call `GcsOptions::Defaults`? If so, perhaps:
   ```suggestion
     auto options = anonymous ? GcsOptions::Anonymous() : GcsOptions::Defaults();
   ```



##########
cpp/src/arrow/filesystem/gcsfs.cc:
##########
@@ -690,11 +745,17 @@ Result<GcsOptions> GcsOptions::FromUri(const arrow::internal::Uri& uri,
     options_map.emplace(kv.first, kv.second);
   }
 
-  if (!uri.password().empty() || !uri.username().empty()) {
-    return Status::Invalid("GCS does not accept username or password.");
+  const std::string& username = uri.username();
+  bool anonymous = username == "anonymous";
+  if (!uri.password().empty() || (!username.empty() && !anonymous)) {
+    return Status::Invalid(
+        "GCS does not accept username except \"anonymous\" or password.");
   }

Review Comment:
   The error message is cryptic. Do you mean something like:
   ```suggestion
     if (!username.empty() && !anonymous) {
       return Status::Invalid(
           "GCS URIs do not accept username except \"anonymous\".");
     }
     if (!uri.password().empty()) {
       return Status::Invalid(
           "GCS URIs do not accept password.");
     }
   ```



##########
cpp/src/arrow/filesystem/gcsfs.h:
##########
@@ -27,11 +27,37 @@
 namespace arrow {
 namespace fs {
 
-struct GcsCredentials;
+// Opaque wrapper for GCS's library credentials to avoid exposing in Arrow headers.
+struct GcsCredentialsHolder;
+class GcsFileSystem;
+
+/// \brief Container for GCS Credentials an information necessary to recreate
+///  them.
+class GcsCredentials {
+ public:
+  bool Equals(const GcsCredentials& other) const;
+  bool anonymous() const { return anonymous_; }
+  const std::string& access_token() { return access_token_; }
+  TimePoint expiration() const { return expiration_; }
+  const std::string& target_service_account() { return target_service_account_; }
+  const std::string& json_credentials() { return json_credentials_; }
+  const std::shared_ptr<GcsCredentialsHolder>& holder() const { return holder_; }
+
+ private:
+  GcsCredentials() = default;
+  bool anonymous_ = false;
+  std::string access_token_;
+  TimePoint expiration_;
+  std::string target_service_account_;
+  std::string json_credentials_;
+  std::shared_ptr<GcsCredentialsHolder> holder_;
+  friend class GcsFileSystem;
+  friend struct GcsOptions;
+};
 
 /// Options for the GcsFileSystem implementation.
 struct ARROW_EXPORT GcsOptions {
-  std::shared_ptr<GcsCredentials> credentials;
+  GcsCredentials credentials;

Review Comment:
   `GcsOptions` would be a more user-friendly API if it was possible to default-construct it, IMHO.



##########
dev/archery/archery/cli.py:
##########
@@ -183,6 +183,8 @@ def _apply_options(cmd, options):
 @click.option("--with-r", default=None, type=BOOL,
               help="Build the Arrow R extensions. This is not a CMake option, "
               "it will toggle required options")
+@click.option("--with-gcs", default=None, type=BOOL,

Review Comment:
   Can you keep this alphabetically ordered?



##########
dev/archery/archery/lang/cpp.py:
##########
@@ -56,7 +56,7 @@ def __init__(self,
                  with_ipc=True, with_json=None, with_jni=None,
                  with_mimalloc=None, with_jemalloc=None,
                  with_parquet=None, with_plasma=None, with_python=True,
-                 with_r=None, with_s3=None,
+                 with_r=None, with_gcs=None, with_s3=None,

Review Comment:
   Please, let's keep this "components" list alphabetically ordered.
   Also, while we're at it, can you make all those constructor arguments keyword-only?
   ```python
   
   class CppConfiguration:
       def __init__(self, *,
                    # toolchain
                    cc=None, cxx=None, cxx_flags=None,
   ```
   



##########
python/pyarrow/_gcsfs.pyx:
##########
@@ -0,0 +1,173 @@
+# 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.
+
+# cython: language_level = 3
+
+from pyarrow.lib cimport (check_status, pyarrow_wrap_metadata,
+                          pyarrow_unwrap_metadata)
+from pyarrow.lib import frombytes, tobytes, KeyValueMetadata
+from pyarrow.includes.common cimport *
+from pyarrow.includes.libarrow cimport *
+from pyarrow.includes.libarrow_fs cimport *
+from pyarrow._fs cimport FileSystem
+from cython.operator cimport dereference as deref
+
+from datetime import datetime
+
+
+cdef class GcsFileSystem(FileSystem):
+    """
+    GCS-backed FileSystem implementation
+
+    By default uses the process described in https://google.aip.dev/auth/4110
+    to resolve credentials. If not running on GCP this generally requires the
+    environment variable GOOGLE_APPLICATION_CREDENTIALS to point to a JSON
+    file containing credentials.
+
+    Note: GCS buckets are special and the operations available on them may be
+    limited or more expensive than expected compared to local file systems.
+
+    Note: When pickling a GcsFileSystem that uses default credential resolution
+    credentials are not stored in the serialized data. Therefore, when unpickling
+    it is assumed that the necessary credentials are in place for the target
+    process.
+
+    Parameters
+    ----------
+    anonymous : boolean, default False
+        Whether to connect anonymously.
+        If true, will not attempt to look up credentials using standard GCP
+        configuration methods.
+    access_token : str, default None
+        GCP access token.  If provided temporary credentials will be fetched by
+        assuming this role. If specified an credential_token_expiration must be
+        specified with the token.

Review Comment:
   ```suggestion
           GCP access token.  If provided, temporary credentials will be fetched by
           assuming this role; also, a `credential_token_expiration` must be
           specified as well.
   ```



##########
cpp/src/arrow/filesystem/gcsfs_test.cc:
##########
@@ -568,7 +574,46 @@ TEST_F(GcsIntegrationTest, GetFileInfoBucket) {
   ASSERT_RAISES(Invalid, fs->GetFileInfo("gs://" + PreexistingBucketName()));
 }
 
-TEST_F(GcsIntegrationTest, GetFileInfoObject) {
+TEST_F(GcsIntegrationTest, GetFileInfoObjectWithNestedStructure) {

Review Comment:
   Please add a comment explaining what this test does (are the generic filesystem tests not sufficient?).



##########
python/pyarrow/_gcsfs.pyx:
##########
@@ -0,0 +1,173 @@
+# 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.
+
+# cython: language_level = 3
+
+from pyarrow.lib cimport (check_status, pyarrow_wrap_metadata,
+                          pyarrow_unwrap_metadata)
+from pyarrow.lib import frombytes, tobytes, KeyValueMetadata
+from pyarrow.includes.common cimport *
+from pyarrow.includes.libarrow cimport *
+from pyarrow.includes.libarrow_fs cimport *
+from pyarrow._fs cimport FileSystem
+from cython.operator cimport dereference as deref
+
+from datetime import datetime
+
+
+cdef class GcsFileSystem(FileSystem):
+    """
+    GCS-backed FileSystem implementation
+
+    By default uses the process described in https://google.aip.dev/auth/4110
+    to resolve credentials. If not running on GCP this generally requires the
+    environment variable GOOGLE_APPLICATION_CREDENTIALS to point to a JSON
+    file containing credentials.
+
+    Note: GCS buckets are special and the operations available on them may be
+    limited or more expensive than expected compared to local file systems.
+
+    Note: When pickling a GcsFileSystem that uses default credential resolution
+    credentials are not stored in the serialized data. Therefore, when unpickling
+    it is assumed that the necessary credentials are in place for the target
+    process.
+
+    Parameters
+    ----------
+    anonymous : boolean, default False
+        Whether to connect anonymously.
+        If true, will not attempt to look up credentials using standard GCP
+        configuration methods.
+    access_token : str, default None
+        GCP access token.  If provided temporary credentials will be fetched by
+        assuming this role. If specified an credential_token_expiration must be
+        specified with the token.
+    target_service_account : str, default None
+        An optional service account to try to impersonate when accessing GCS. This
+        requires the specified credential user/service_account has the necessary
+        permissions.
+    credential_token_expiration : datetime, default None
+        Expiration for credential generated with an access token. Must be specified
+        if token is specified.

Review Comment:
   ```suggestion
           Expiration for credential generated with an access token. Must be specified
           if `access_token` is specified.
   ```



##########
python/pyarrow/_gcsfs.pyx:
##########
@@ -0,0 +1,173 @@
+# 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.
+
+# cython: language_level = 3
+
+from pyarrow.lib cimport (check_status, pyarrow_wrap_metadata,
+                          pyarrow_unwrap_metadata)
+from pyarrow.lib import frombytes, tobytes, KeyValueMetadata
+from pyarrow.includes.common cimport *
+from pyarrow.includes.libarrow cimport *
+from pyarrow.includes.libarrow_fs cimport *
+from pyarrow._fs cimport FileSystem
+from cython.operator cimport dereference as deref
+
+from datetime import datetime
+
+
+cdef class GcsFileSystem(FileSystem):
+    """
+    GCS-backed FileSystem implementation
+
+    By default uses the process described in https://google.aip.dev/auth/4110
+    to resolve credentials. If not running on GCP this generally requires the
+    environment variable GOOGLE_APPLICATION_CREDENTIALS to point to a JSON
+    file containing credentials.
+
+    Note: GCS buckets are special and the operations available on them may be
+    limited or more expensive than expected compared to local file systems.
+
+    Note: When pickling a GcsFileSystem that uses default credential resolution
+    credentials are not stored in the serialized data. Therefore, when unpickling
+    it is assumed that the necessary credentials are in place for the target
+    process.
+
+    Parameters
+    ----------
+    anonymous : boolean, default False
+        Whether to connect anonymously.
+        If true, will not attempt to look up credentials using standard GCP
+        configuration methods.
+    access_token : str, default None
+        GCP access token.  If provided temporary credentials will be fetched by
+        assuming this role. If specified an credential_token_expiration must be
+        specified with the token.
+    target_service_account : str, default None
+        An optional service account to try to impersonate when accessing GCS. This
+        requires the specified credential user/service_account has the necessary
+        permissions.
+    credential_token_expiration : datetime, default None
+        Expiration for credential generated with an access token. Must be specified
+        if token is specified.
+    default_bucket_location : str, default 'US-CENTRAL1'
+        GCP region to create buckets in.
+    scheme : str, default 'https'
+        GCS connection transport scheme.
+    endpoint_override : str, default None
+        Override endpoint with a connect string such as "localhost:9000"
+    default_metadata : mapping or pyarrow.KeyValueMetadata, default None
+        Default metadata for open_output_stream.  This will be ignored if
+        non-empty metadata is passed to open_output_stream.
+    """
+
+    cdef:
+        CGcsFileSystem* gcsfs
+
+    def __init__(self, *, bint anonymous=False, access_token=None,
+                 target_service_account=None, credential_token_expiration=None,
+                 default_bucket_location='US-CENTRAL1',

Review Comment:
   Is "US-CENTRAL1" the standard default in GCS, or is it just something you chose?



##########
python/pyarrow/_gcsfs.pyx:
##########
@@ -0,0 +1,173 @@
+# 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.
+
+# cython: language_level = 3
+
+from pyarrow.lib cimport (check_status, pyarrow_wrap_metadata,
+                          pyarrow_unwrap_metadata)
+from pyarrow.lib import frombytes, tobytes, KeyValueMetadata
+from pyarrow.includes.common cimport *
+from pyarrow.includes.libarrow cimport *
+from pyarrow.includes.libarrow_fs cimport *
+from pyarrow._fs cimport FileSystem
+from cython.operator cimport dereference as deref
+
+from datetime import datetime
+
+
+cdef class GcsFileSystem(FileSystem):
+    """
+    GCS-backed FileSystem implementation
+
+    By default uses the process described in https://google.aip.dev/auth/4110
+    to resolve credentials. If not running on GCP this generally requires the
+    environment variable GOOGLE_APPLICATION_CREDENTIALS to point to a JSON
+    file containing credentials.
+
+    Note: GCS buckets are special and the operations available on them may be
+    limited or more expensive than expected compared to local file systems.
+
+    Note: When pickling a GcsFileSystem that uses default credential resolution
+    credentials are not stored in the serialized data. Therefore, when unpickling
+    it is assumed that the necessary credentials are in place for the target
+    process.
+
+    Parameters
+    ----------
+    anonymous : boolean, default False
+        Whether to connect anonymously.
+        If true, will not attempt to look up credentials using standard GCP
+        configuration methods.
+    access_token : str, default None
+        GCP access token.  If provided temporary credentials will be fetched by
+        assuming this role. If specified an credential_token_expiration must be
+        specified with the token.
+    target_service_account : str, default None
+        An optional service account to try to impersonate when accessing GCS. This
+        requires the specified credential user/service_account has the necessary
+        permissions.
+    credential_token_expiration : datetime, default None
+        Expiration for credential generated with an access token. Must be specified
+        if token is specified.
+    default_bucket_location : str, default 'US-CENTRAL1'
+        GCP region to create buckets in.
+    scheme : str, default 'https'
+        GCS connection transport scheme.
+    endpoint_override : str, default None
+        Override endpoint with a connect string such as "localhost:9000"
+    default_metadata : mapping or pyarrow.KeyValueMetadata, default None
+        Default metadata for open_output_stream.  This will be ignored if
+        non-empty metadata is passed to open_output_stream.
+    """
+
+    cdef:
+        CGcsFileSystem* gcsfs
+
+    def __init__(self, *, bint anonymous=False, access_token=None,
+                 target_service_account=None, credential_token_expiration=None,
+                 default_bucket_location='US-CENTRAL1',
+                 scheme=None,
+                 endpoint_override=None,
+                 default_metadata=None):
+        cdef:
+            CGcsOptions options
+            shared_ptr[CGcsFileSystem] wrapped
+
+        # Intentional use of truthiness because empty strings aren't valid and
+        # for reconstruction from pickling will give empty strings.
+        if anonymous and (target_service_account or access_token):
+            raise ValueError(
+                'anonymous option is not compatible with target_service_account and '
+                'access_token please only specify only one.'
+            )

Review Comment:
   ```suggestion
               raise ValueError(
                   'anonymous option is not compatible with target_service_account and '
                   'access_token'
               )
   ```



##########
python/pyarrow/_gcsfs.pyx:
##########
@@ -0,0 +1,173 @@
+# 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.
+
+# cython: language_level = 3
+
+from pyarrow.lib cimport (check_status, pyarrow_wrap_metadata,
+                          pyarrow_unwrap_metadata)
+from pyarrow.lib import frombytes, tobytes, KeyValueMetadata
+from pyarrow.includes.common cimport *
+from pyarrow.includes.libarrow cimport *
+from pyarrow.includes.libarrow_fs cimport *
+from pyarrow._fs cimport FileSystem
+from cython.operator cimport dereference as deref
+
+from datetime import datetime
+
+
+cdef class GcsFileSystem(FileSystem):
+    """
+    GCS-backed FileSystem implementation
+
+    By default uses the process described in https://google.aip.dev/auth/4110
+    to resolve credentials. If not running on GCP this generally requires the
+    environment variable GOOGLE_APPLICATION_CREDENTIALS to point to a JSON
+    file containing credentials.
+
+    Note: GCS buckets are special and the operations available on them may be
+    limited or more expensive than expected compared to local file systems.
+
+    Note: When pickling a GcsFileSystem that uses default credential resolution
+    credentials are not stored in the serialized data. Therefore, when unpickling
+    it is assumed that the necessary credentials are in place for the target
+    process.
+
+    Parameters
+    ----------
+    anonymous : boolean, default False
+        Whether to connect anonymously.
+        If true, will not attempt to look up credentials using standard GCP
+        configuration methods.
+    access_token : str, default None
+        GCP access token.  If provided temporary credentials will be fetched by
+        assuming this role. If specified an credential_token_expiration must be
+        specified with the token.
+    target_service_account : str, default None
+        An optional service account to try to impersonate when accessing GCS. This
+        requires the specified credential user/service_account has the necessary
+        permissions.
+    credential_token_expiration : datetime, default None
+        Expiration for credential generated with an access token. Must be specified
+        if token is specified.
+    default_bucket_location : str, default 'US-CENTRAL1'
+        GCP region to create buckets in.
+    scheme : str, default 'https'
+        GCS connection transport scheme.
+    endpoint_override : str, default None
+        Override endpoint with a connect string such as "localhost:9000"
+    default_metadata : mapping or pyarrow.KeyValueMetadata, default None
+        Default metadata for open_output_stream.  This will be ignored if
+        non-empty metadata is passed to open_output_stream.
+    """
+
+    cdef:
+        CGcsFileSystem* gcsfs
+
+    def __init__(self, *, bint anonymous=False, access_token=None,
+                 target_service_account=None, credential_token_expiration=None,
+                 default_bucket_location='US-CENTRAL1',
+                 scheme=None,
+                 endpoint_override=None,
+                 default_metadata=None):
+        cdef:
+            CGcsOptions options
+            shared_ptr[CGcsFileSystem] wrapped
+
+        # Intentional use of truthiness because empty strings aren't valid and
+        # for reconstruction from pickling will give empty strings.
+        if anonymous and (target_service_account or access_token):
+            raise ValueError(
+                'anonymous option is not compatible with target_service_account and '
+                'access_token please only specify only one.'
+            )
+        elif ((access_token and credential_token_expiration is None) or
+              (not access_token and
+                  credential_token_expiration is not None)):

Review Comment:
   Perhaps simpler
   ```suggestion
           elif bool(access_token) != bool(credential_token_expiration):
   ```



##########
cpp/src/arrow/filesystem/gcsfs.cc:
##########
@@ -33,13 +33,23 @@
 
 namespace arrow {
 namespace fs {
-struct GcsCredentials {
-  explicit GcsCredentials(std::shared_ptr<google::cloud::Credentials> c)
+struct GcsCredentialsHolder {
+  explicit GcsCredentialsHolder(std::shared_ptr<google::cloud::Credentials> c)

Review Comment:
   I don't think this constructor is necessary, just let C++ define it implicitly for you?



##########
cpp/src/arrow/filesystem/gcsfs.cc:
##########
@@ -297,8 +304,9 @@ google::cloud::Options AsGoogleCloudOptions(const GcsOptions& o) {
   if (!o.endpoint_override.empty()) {
     options.set<gcs::RestEndpointOption>(scheme + "://" + o.endpoint_override);
   }
-  if (o.credentials && o.credentials->credentials) {
-    options.set<google::cloud::UnifiedCredentialsOption>(o.credentials->credentials);
+  if (o.credentials.holder() && (o.credentials.holder())->credentials) {

Review Comment:
   Nit :-)
   ```suggestion
     if (o.credentials.holder() && o.credentials.holder()->credentials) {
   ```



##########
python/pyarrow/_gcsfs.pyx:
##########
@@ -0,0 +1,173 @@
+# 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.
+
+# cython: language_level = 3
+
+from pyarrow.lib cimport (check_status, pyarrow_wrap_metadata,
+                          pyarrow_unwrap_metadata)
+from pyarrow.lib import frombytes, tobytes, KeyValueMetadata
+from pyarrow.includes.common cimport *
+from pyarrow.includes.libarrow cimport *
+from pyarrow.includes.libarrow_fs cimport *
+from pyarrow._fs cimport FileSystem
+from cython.operator cimport dereference as deref
+
+from datetime import datetime
+
+
+cdef class GcsFileSystem(FileSystem):
+    """
+    GCS-backed FileSystem implementation
+
+    By default uses the process described in https://google.aip.dev/auth/4110
+    to resolve credentials. If not running on GCP this generally requires the
+    environment variable GOOGLE_APPLICATION_CREDENTIALS to point to a JSON
+    file containing credentials.
+
+    Note: GCS buckets are special and the operations available on them may be
+    limited or more expensive than expected compared to local file systems.
+
+    Note: When pickling a GcsFileSystem that uses default credential resolution
+    credentials are not stored in the serialized data. Therefore, when unpickling
+    it is assumed that the necessary credentials are in place for the target
+    process.
+
+    Parameters
+    ----------
+    anonymous : boolean, default False
+        Whether to connect anonymously.
+        If true, will not attempt to look up credentials using standard GCP
+        configuration methods.
+    access_token : str, default None
+        GCP access token.  If provided temporary credentials will be fetched by
+        assuming this role. If specified an credential_token_expiration must be
+        specified with the token.
+    target_service_account : str, default None
+        An optional service account to try to impersonate when accessing GCS. This
+        requires the specified credential user/service_account has the necessary
+        permissions.
+    credential_token_expiration : datetime, default None
+        Expiration for credential generated with an access token. Must be specified
+        if token is specified.
+    default_bucket_location : str, default 'US-CENTRAL1'
+        GCP region to create buckets in.
+    scheme : str, default 'https'
+        GCS connection transport scheme.
+    endpoint_override : str, default None
+        Override endpoint with a connect string such as "localhost:9000"
+    default_metadata : mapping or pyarrow.KeyValueMetadata, default None
+        Default metadata for open_output_stream.  This will be ignored if
+        non-empty metadata is passed to open_output_stream.
+    """
+
+    cdef:
+        CGcsFileSystem* gcsfs
+
+    def __init__(self, *, bint anonymous=False, access_token=None,
+                 target_service_account=None, credential_token_expiration=None,
+                 default_bucket_location='US-CENTRAL1',
+                 scheme=None,
+                 endpoint_override=None,
+                 default_metadata=None):
+        cdef:
+            CGcsOptions options
+            shared_ptr[CGcsFileSystem] wrapped
+
+        # Intentional use of truthiness because empty strings aren't valid and
+        # for reconstruction from pickling will give empty strings.
+        if anonymous and (target_service_account or access_token):
+            raise ValueError(
+                'anonymous option is not compatible with target_service_account and '
+                'access_token please only specify only one.'
+            )
+        elif ((access_token and credential_token_expiration is None) or
+              (not access_token and
+                  credential_token_expiration is not None)):
+            raise ValueError(
+                'access_token and credential_token_expiration must be '
+                'specified together'
+            )
+
+        elif anonymous:
+            options = CGcsOptions.Anonymous()
+        elif access_token:
+            options = CGcsOptions.FromAccessToken(
+                tobytes(access_token),
+                PyDateTime_to_TimePoint(<PyDateTime_DateTime*>credential_token_expiration))
+        else:
+            options = CGcsOptions.Defaults()
+
+        if target_service_account:
+            options = CGcsOptions.FromImpersonatedServiceAccount(
+                options.credentials, tobytes(target_service_account))
+
+        options.default_bucket_location = tobytes(default_bucket_location)
+
+        if scheme is not None:
+            options.scheme = tobytes(scheme)
+        if endpoint_override is not None:
+            options.endpoint_override = tobytes(endpoint_override)
+        if default_metadata is not None:
+            if not isinstance(default_metadata, KeyValueMetadata):
+                default_metadata = KeyValueMetadata(default_metadata)
+            options.default_metadata = pyarrow_unwrap_metadata(
+                default_metadata)
+
+        with nogil:
+            wrapped = GetResultValue(CGcsFileSystem.Make(options))
+
+        self.init(<shared_ptr[CFileSystem]> wrapped)
+
+    cdef init(self, const shared_ptr[CFileSystem]& wrapped):
+        FileSystem.init(self, wrapped)
+        self.gcsfs = <CGcsFileSystem*> wrapped.get()
+
+    @classmethod
+    def _reconstruct(cls, kwargs):
+        return cls(**kwargs)
+
+    def _expiration_datetime_from_options(self):
+        cdef CGcsOptions opts = self.gcsfs.options()
+        expiration_ns = TimePoint_to_ns(opts.credentials.expiration())

Review Comment:
   Nit, but this will make a copy of the options scrupt, is it actually necessary?
   ```suggestion
           expiration_ns = TimePoint_to_ns(
               self.gcsfs.options().credentials.expiration())
   ```



##########
python/pyarrow/_gcsfs.pyx:
##########
@@ -0,0 +1,173 @@
+# 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.
+
+# cython: language_level = 3
+
+from pyarrow.lib cimport (check_status, pyarrow_wrap_metadata,
+                          pyarrow_unwrap_metadata)
+from pyarrow.lib import frombytes, tobytes, KeyValueMetadata
+from pyarrow.includes.common cimport *
+from pyarrow.includes.libarrow cimport *
+from pyarrow.includes.libarrow_fs cimport *
+from pyarrow._fs cimport FileSystem
+from cython.operator cimport dereference as deref
+
+from datetime import datetime
+
+
+cdef class GcsFileSystem(FileSystem):
+    """
+    GCS-backed FileSystem implementation
+
+    By default uses the process described in https://google.aip.dev/auth/4110
+    to resolve credentials. If not running on GCP this generally requires the
+    environment variable GOOGLE_APPLICATION_CREDENTIALS to point to a JSON
+    file containing credentials.
+
+    Note: GCS buckets are special and the operations available on them may be
+    limited or more expensive than expected compared to local file systems.
+
+    Note: When pickling a GcsFileSystem that uses default credential resolution
+    credentials are not stored in the serialized data. Therefore, when unpickling
+    it is assumed that the necessary credentials are in place for the target
+    process.
+
+    Parameters
+    ----------
+    anonymous : boolean, default False
+        Whether to connect anonymously.
+        If true, will not attempt to look up credentials using standard GCP
+        configuration methods.
+    access_token : str, default None
+        GCP access token.  If provided temporary credentials will be fetched by
+        assuming this role. If specified an credential_token_expiration must be
+        specified with the token.
+    target_service_account : str, default None
+        An optional service account to try to impersonate when accessing GCS. This
+        requires the specified credential user/service_account has the necessary
+        permissions.
+    credential_token_expiration : datetime, default None
+        Expiration for credential generated with an access token. Must be specified
+        if token is specified.
+    default_bucket_location : str, default 'US-CENTRAL1'
+        GCP region to create buckets in.
+    scheme : str, default 'https'
+        GCS connection transport scheme.
+    endpoint_override : str, default None
+        Override endpoint with a connect string such as "localhost:9000"
+    default_metadata : mapping or pyarrow.KeyValueMetadata, default None
+        Default metadata for open_output_stream.  This will be ignored if
+        non-empty metadata is passed to open_output_stream.
+    """
+
+    cdef:
+        CGcsFileSystem* gcsfs
+
+    def __init__(self, *, bint anonymous=False, access_token=None,
+                 target_service_account=None, credential_token_expiration=None,
+                 default_bucket_location='US-CENTRAL1',
+                 scheme=None,
+                 endpoint_override=None,
+                 default_metadata=None):
+        cdef:
+            CGcsOptions options
+            shared_ptr[CGcsFileSystem] wrapped
+
+        # Intentional use of truthiness because empty strings aren't valid and
+        # for reconstruction from pickling will give empty strings.
+        if anonymous and (target_service_account or access_token):
+            raise ValueError(
+                'anonymous option is not compatible with target_service_account and '
+                'access_token please only specify only one.'
+            )
+        elif ((access_token and credential_token_expiration is None) or
+              (not access_token and
+                  credential_token_expiration is not None)):
+            raise ValueError(
+                'access_token and credential_token_expiration must be '
+                'specified together'
+            )
+
+        elif anonymous:
+            options = CGcsOptions.Anonymous()
+        elif access_token:
+            options = CGcsOptions.FromAccessToken(
+                tobytes(access_token),
+                PyDateTime_to_TimePoint(<PyDateTime_DateTime*>credential_token_expiration))
+        else:
+            options = CGcsOptions.Defaults()
+
+        if target_service_account:
+            options = CGcsOptions.FromImpersonatedServiceAccount(
+                options.credentials, tobytes(target_service_account))
+
+        options.default_bucket_location = tobytes(default_bucket_location)
+
+        if scheme is not None:
+            options.scheme = tobytes(scheme)
+        if endpoint_override is not None:
+            options.endpoint_override = tobytes(endpoint_override)
+        if default_metadata is not None:
+            if not isinstance(default_metadata, KeyValueMetadata):
+                default_metadata = KeyValueMetadata(default_metadata)
+            options.default_metadata = pyarrow_unwrap_metadata(
+                default_metadata)
+
+        with nogil:
+            wrapped = GetResultValue(CGcsFileSystem.Make(options))
+
+        self.init(<shared_ptr[CFileSystem]> wrapped)
+
+    cdef init(self, const shared_ptr[CFileSystem]& wrapped):
+        FileSystem.init(self, wrapped)
+        self.gcsfs = <CGcsFileSystem*> wrapped.get()
+
+    @classmethod
+    def _reconstruct(cls, kwargs):
+        return cls(**kwargs)
+
+    def _expiration_datetime_from_options(self):
+        cdef CGcsOptions opts = self.gcsfs.options()
+        expiration_ns = TimePoint_to_ns(opts.credentials.expiration())
+        if expiration_ns == 0:
+            return None
+        ns_per_sec = 1000000000.0
+        return datetime.fromtimestamp(expiration_ns / ns_per_sec)
+
+    def __reduce__(self):
+        cdef CGcsOptions opts = self.gcsfs.options()
+        service_account = frombytes(opts.credentials.target_service_account())
+        expiration_dt = self._expiration_datetime_from_options()
+        return (
+            GcsFileSystem._reconstruct, (dict(
+                access_token=frombytes(opts.credentials.access_token()),
+                anonymous=opts.credentials.anonymous(),
+                credential_token_expiration=expiration_dt,
+                target_service_account=service_account,
+                scheme=frombytes(opts.scheme),
+                endpoint_override=frombytes(opts.endpoint_override),
+                default_bucket_location=frombytes(
+                    opts.default_bucket_location),
+                default_metadata=pyarrow_wrap_metadata(opts.default_metadata),

Review Comment:
   It seems we don't handle `json_credentials`, is there a reason for that?



##########
cpp/src/arrow/filesystem/gcsfs.cc:
##########
@@ -318,19 +326,43 @@ class GcsFileSystem::Impl {
       return GetFileInfoBucket(path, std::move(meta).status());
     }
     auto meta = client_.GetObjectMetadata(path.bucket, path.object);
-    return GetFileInfoObject(path, meta);
+    Result<FileInfo> info = GetFileInfoObject(path, meta);
+    if (!info.ok() || info->type() != FileType::NotFound) {
+      return info;
+    }
+    // Not found case.  It could be this was written to GCS with a different
+    // "Directory" convention.  So it if there are is at least one objec that
+    // matches the prefix we assume it is a directory.
+    std::string canonical = internal::EnsureTrailingSlash(path.object);
+    std::string end = canonical;
+    end.back() += 1;
+    auto list_result =
+        client_.ListObjects(path.bucket, gcs::Prefix(canonical), gcs::EndOffset(end));
+    if (list_result.begin() != list_result.end()) {
+      // If there is at least one result it indicates this is a directory (at
+      // least one object exists that starts with "path/"
+      return FileInfo(path.full_path, FileType::Directory);
+    }
+    // Return the original not-found info if there was no match.
+    return info;
   }
 
   Result<FileInfoVector> GetFileInfo(const FileSelector& select) {
     ARROW_ASSIGN_OR_RAISE(auto p, GcsPath::FromString(select.base_dir));
-    // Adding the trailing '/' avoids problems with files named 'a', 'ab', 'ac'  where GCS
-    // would return all of them if the prefix is 'a'.
+    // Adding the trailing '/' avoids problems with files named 'a', 'ab', 'ac'  where
+    // GCS would return all of them if the prefix is 'a'.
     const auto canonical = internal::EnsureTrailingSlash(p.object);
-    const auto max_depth = internal::Depth(canonical) + select.max_recursion;
+    // Need to add one level when the object is not empty because all
+    // directories have an extra slash.
+    const auto max_depth =
+        internal::Depth(p.object) + select.max_recursion + !p.object.empty();
     auto prefix = p.object.empty() ? gcs::Prefix() : gcs::Prefix(canonical);
     auto delimiter = select.recursive ? gcs::Delimiter() : gcs::Delimiter("/");
+    auto include_trailing = select.recursive ? gcs::IncludeTrailingDelimiter(false)
+                                             : gcs::IncludeTrailingDelimiter(true);

Review Comment:
   I'm curious why this needs to be special-cased for non-recursive lookups.



##########
cpp/src/arrow/filesystem/gcsfs.cc:
##########
@@ -318,19 +326,43 @@ class GcsFileSystem::Impl {
       return GetFileInfoBucket(path, std::move(meta).status());
     }
     auto meta = client_.GetObjectMetadata(path.bucket, path.object);
-    return GetFileInfoObject(path, meta);
+    Result<FileInfo> info = GetFileInfoObject(path, meta);
+    if (!info.ok() || info->type() != FileType::NotFound) {
+      return info;
+    }
+    // Not found case.  It could be this was written to GCS with a different
+    // "Directory" convention.  So it if there are is at least one objec that
+    // matches the prefix we assume it is a directory.
+    std::string canonical = internal::EnsureTrailingSlash(path.object);
+    std::string end = canonical;
+    end.back() += 1;
+    auto list_result =
+        client_.ListObjects(path.bucket, gcs::Prefix(canonical), gcs::EndOffset(end));
+    if (list_result.begin() != list_result.end()) {
+      // If there is at least one result it indicates this is a directory (at
+      // least one object exists that starts with "path/"

Review Comment:
   ```suggestion
         // If there is at least one result it indicates this is a directory (at
         // least one object exists that starts with "path/")
   ```



##########
cpp/src/arrow/filesystem/gcsfs.cc:
##########
@@ -536,8 +546,7 @@ class GcsFileSystem::Impl {
                                                            gcs::ReadFromOffset offset) {
     auto stream = client_.ReadObject(path.bucket, path.object, generation, offset);
     ARROW_GCS_RETURN_NOT_OK(stream.status());
-    return std::make_shared<GcsInputStream>(std::move(stream), path, gcs::Generation(),
-                                            offset, client_);
+    return std::make_shared<GcsInputStream>(std::move(stream), path, generation, client_);

Review Comment:
   @emkornfield I'm curious this wasn't caught by the C++ tests. Is it possible to enhance the generic filesystem tests to cover this?



##########
cpp/src/arrow/filesystem/gcsfs.cc:
##########
@@ -318,19 +326,43 @@ class GcsFileSystem::Impl {
       return GetFileInfoBucket(path, std::move(meta).status());
     }
     auto meta = client_.GetObjectMetadata(path.bucket, path.object);
-    return GetFileInfoObject(path, meta);
+    Result<FileInfo> info = GetFileInfoObject(path, meta);
+    if (!info.ok() || info->type() != FileType::NotFound) {
+      return info;
+    }
+    // Not found case.  It could be this was written to GCS with a different
+    // "Directory" convention.  So it if there are is at least one objec that
+    // matches the prefix we assume it is a directory.
+    std::string canonical = internal::EnsureTrailingSlash(path.object);
+    std::string end = canonical;
+    end.back() += 1;
+    auto list_result =
+        client_.ListObjects(path.bucket, gcs::Prefix(canonical), gcs::EndOffset(end));

Review Comment:
   Since you're using `Prefix`, is `EndOffset` useful at all? @coryan what do you think?



##########
python/pyarrow/_gcsfs.pyx:
##########
@@ -0,0 +1,173 @@
+# 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.
+
+# cython: language_level = 3
+
+from pyarrow.lib cimport (check_status, pyarrow_wrap_metadata,
+                          pyarrow_unwrap_metadata)
+from pyarrow.lib import frombytes, tobytes, KeyValueMetadata
+from pyarrow.includes.common cimport *
+from pyarrow.includes.libarrow cimport *
+from pyarrow.includes.libarrow_fs cimport *
+from pyarrow._fs cimport FileSystem
+from cython.operator cimport dereference as deref
+
+from datetime import datetime
+
+
+cdef class GcsFileSystem(FileSystem):
+    """
+    GCS-backed FileSystem implementation
+
+    By default uses the process described in https://google.aip.dev/auth/4110
+    to resolve credentials. If not running on GCP this generally requires the
+    environment variable GOOGLE_APPLICATION_CREDENTIALS to point to a JSON
+    file containing credentials.
+
+    Note: GCS buckets are special and the operations available on them may be
+    limited or more expensive than expected compared to local file systems.
+
+    Note: When pickling a GcsFileSystem that uses default credential resolution
+    credentials are not stored in the serialized data. Therefore, when unpickling
+    it is assumed that the necessary credentials are in place for the target
+    process.

Review Comment:
   Also, is "resolution credentials" standard terminology in GCS? @coryan 



##########
cpp/src/arrow/filesystem/gcsfs.cc:
##########
@@ -340,11 +372,13 @@ class GcsFileSystem::Impl {
       }
       // Skip the directory itself from the results, and any result that is "too deep"
       // into the recursion.
-      if (o->name() == p.object || internal::Depth(o->name()) > max_depth) {
+      bool has_trailing_slash = !o->name().empty() && o->name().back() == '/';
+      if (o->name() == canonical || o->name() == p.object ||
+          internal::Depth(o->name()) > (max_depth + has_trailing_slash)) {

Review Comment:
   This logic looks weird and difficult to follow. Can you try to simplify both this and the `include_trailing` oddity above?



##########
cpp/src/arrow/filesystem/gcsfs.cc:
##########
@@ -417,7 +458,7 @@ class GcsFileSystem::Impl {
     // Note that the list of parents are sorted from deepest to most shallow, this is
     // convenient because as soon as we find a directory we can stop the iteration.
     for (auto const& d : missing_parents) {
-      auto o = CreateDirMarker(bucket, d);
+      auto o = CreateDirMarker(bucket, internal::EnsureTrailingSlash(d));

Review Comment:
   `CreateDirMarker` already seems to call `EnsureTrailingSlash`, why are you calling it here too?



##########
cpp/src/arrow/filesystem/gcsfs.h:
##########
@@ -27,11 +27,37 @@
 namespace arrow {
 namespace fs {
 
-struct GcsCredentials;
+// Opaque wrapper for GCS's library credentials to avoid exposing in Arrow headers.
+struct GcsCredentialsHolder;
+class GcsFileSystem;
+
+/// \brief Container for GCS Credentials an information necessary to recreate
+///  them.
+class GcsCredentials {
+ public:
+  bool Equals(const GcsCredentials& other) const;
+  bool anonymous() const { return anonymous_; }
+  const std::string& access_token() { return access_token_; }
+  TimePoint expiration() const { return expiration_; }
+  const std::string& target_service_account() { return target_service_account_; }
+  const std::string& json_credentials() { return json_credentials_; }
+  const std::shared_ptr<GcsCredentialsHolder>& holder() const { return holder_; }

Review Comment:
   Please make sure all accessors are this-const.



##########
cpp/src/arrow/filesystem/gcsfs_test.cc:
##########
@@ -291,15 +292,15 @@ TEST(GcsFileSystem, OptionsCompare) {
 
 TEST(GcsFileSystem, OptionsAnonymous) {
   GcsOptions a = GcsOptions::Anonymous();
-  EXPECT_THAT(a.credentials, NotNull());
+  EXPECT_THAT(a.credentials.holder(), NotNull());
+  EXPECT_THAT(a.credentials.anonymous(), true);

Review Comment:
   Uh. Why not `ASSERT_TRUE` or `EXPECT_TRUE`?



##########
cpp/src/arrow/filesystem/gcsfs_test.cc:
##########
@@ -568,7 +574,46 @@ TEST_F(GcsIntegrationTest, GetFileInfoBucket) {
   ASSERT_RAISES(Invalid, fs->GetFileInfo("gs://" + PreexistingBucketName()));
 }
 
-TEST_F(GcsIntegrationTest, GetFileInfoObject) {
+TEST_F(GcsIntegrationTest, GetFileInfoObjectWithNestedStructure) {
+  auto fs = GcsFileSystem::Make(TestGcsOptions());
+  constexpr auto kObjectName = "test-object-dir/some_other_dir/another_dir/foo";
+  ASSERT_OK_AND_ASSIGN(
+      auto output,
+      fs->OpenOutputStream(PreexistingBucketPath() + kObjectName, /*metadata=*/{}));
+  const auto data = std::string(kLoremIpsum);
+  ASSERT_OK(output->Write(data.data(), data.size()));
+  ASSERT_OK(output->Close());
+
+  ASSERT_OK_AND_ASSIGN(output, fs->OpenOutputStream(PreexistingBucketPath() +
+                                                        "test-object-dir/some_other_dir0",

Review Comment:
   Why "some_other_dir0"? Is it specifically to check the recursive lookup logic? Please add a comment.



##########
cpp/src/arrow/filesystem/gcsfs.cc:
##########
@@ -596,9 +639,15 @@ class GcsFileSystem::Impl {
   }
 
   static FileInfo ToFileInfo(const std::string& full_path,
-                             const gcs::ObjectMetadata& meta) {
-    if (IsDirectory(meta)) {
-      return FileInfo(full_path, FileType::Directory);
+                             const gcs::ObjectMetadata& meta,
+                             bool normalize_directories = false) {
+    if (IsDirectory(meta) || (!full_path.empty() && full_path.back() == '/')) {
+      if (normalize_directories) {
+        auto normalized = std::string(internal::RemoveTrailingSlash(full_path));
+        return FileInfo(normalized, FileType::Directory);

Review Comment:
   ```suggestion
           return FileInfo(std::move(normalized), FileType::Directory);
   ```



##########
dev/archery/archery/lang/cpp.py:
##########
@@ -106,6 +106,7 @@ def __init__(self,
         self.with_plasma = with_plasma
         self.with_python = with_python
         self.with_r = with_r
+        self.with_gcs = with_gcs

Review Comment:
   Same here (alphebetical ordering).



##########
dev/archery/archery/cli.py:
##########
@@ -183,6 +183,8 @@ def _apply_options(cmd, options):
 @click.option("--with-r", default=None, type=BOOL,
               help="Build the Arrow R extensions. This is not a CMake option, "
               "it will toggle required options")
+@click.option("--with-gcs", default=None, type=BOOL,
+              help="Build Arrow with Google Cloud Storage (gcs) support.")

Review Comment:
   ```suggestion
                 help="Build Arrow with Google Cloud Storage (GCS) support.")
   ```



##########
python/pyarrow/_gcsfs.pyx:
##########
@@ -0,0 +1,173 @@
+# 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.
+
+# cython: language_level = 3
+
+from pyarrow.lib cimport (check_status, pyarrow_wrap_metadata,
+                          pyarrow_unwrap_metadata)
+from pyarrow.lib import frombytes, tobytes, KeyValueMetadata
+from pyarrow.includes.common cimport *
+from pyarrow.includes.libarrow cimport *
+from pyarrow.includes.libarrow_fs cimport *
+from pyarrow._fs cimport FileSystem
+from cython.operator cimport dereference as deref
+
+from datetime import datetime
+
+
+cdef class GcsFileSystem(FileSystem):
+    """
+    GCS-backed FileSystem implementation
+
+    By default uses the process described in https://google.aip.dev/auth/4110
+    to resolve credentials. If not running on GCP this generally requires the
+    environment variable GOOGLE_APPLICATION_CREDENTIALS to point to a JSON
+    file containing credentials.
+
+    Note: GCS buckets are special and the operations available on them may be
+    limited or more expensive than expected compared to local file systems.
+
+    Note: When pickling a GcsFileSystem that uses default credential resolution
+    credentials are not stored in the serialized data. Therefore, when unpickling
+    it is assumed that the necessary credentials are in place for the target
+    process.

Review Comment:
   ```suggestion
       Note: When pickling a GcsFileSystem that uses default credentials, resolution
       credentials are not stored in the serialized data. Therefore, when unpickling
       it is assumed that the necessary credentials are in place for the target
       process.
   ```



##########
python/pyarrow/_gcsfs.pyx:
##########
@@ -0,0 +1,173 @@
+# 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.
+
+# cython: language_level = 3
+
+from pyarrow.lib cimport (check_status, pyarrow_wrap_metadata,
+                          pyarrow_unwrap_metadata)
+from pyarrow.lib import frombytes, tobytes, KeyValueMetadata
+from pyarrow.includes.common cimport *
+from pyarrow.includes.libarrow cimport *
+from pyarrow.includes.libarrow_fs cimport *
+from pyarrow._fs cimport FileSystem
+from cython.operator cimport dereference as deref
+
+from datetime import datetime
+
+
+cdef class GcsFileSystem(FileSystem):
+    """
+    GCS-backed FileSystem implementation
+
+    By default uses the process described in https://google.aip.dev/auth/4110
+    to resolve credentials. If not running on GCP this generally requires the
+    environment variable GOOGLE_APPLICATION_CREDENTIALS to point to a JSON
+    file containing credentials.
+
+    Note: GCS buckets are special and the operations available on them may be
+    limited or more expensive than expected compared to local file systems.
+
+    Note: When pickling a GcsFileSystem that uses default credential resolution
+    credentials are not stored in the serialized data. Therefore, when unpickling
+    it is assumed that the necessary credentials are in place for the target
+    process.
+
+    Parameters
+    ----------
+    anonymous : boolean, default False
+        Whether to connect anonymously.
+        If true, will not attempt to look up credentials using standard GCP
+        configuration methods.
+    access_token : str, default None
+        GCP access token.  If provided temporary credentials will be fetched by
+        assuming this role. If specified an credential_token_expiration must be
+        specified with the token.
+    target_service_account : str, default None
+        An optional service account to try to impersonate when accessing GCS. This
+        requires the specified credential user/service_account has the necessary
+        permissions.

Review Comment:
   ```suggestion
           An optional service account to try to impersonate when accessing GCS. This
           requires the specified credential user or service account to have the necessary
           permissions.
   ```



##########
python/pyarrow/_gcsfs.pyx:
##########
@@ -0,0 +1,173 @@
+# 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.
+
+# cython: language_level = 3
+
+from pyarrow.lib cimport (check_status, pyarrow_wrap_metadata,
+                          pyarrow_unwrap_metadata)
+from pyarrow.lib import frombytes, tobytes, KeyValueMetadata
+from pyarrow.includes.common cimport *
+from pyarrow.includes.libarrow cimport *
+from pyarrow.includes.libarrow_fs cimport *
+from pyarrow._fs cimport FileSystem
+from cython.operator cimport dereference as deref
+
+from datetime import datetime
+
+
+cdef class GcsFileSystem(FileSystem):
+    """
+    GCS-backed FileSystem implementation
+
+    By default uses the process described in https://google.aip.dev/auth/4110
+    to resolve credentials. If not running on GCP this generally requires the
+    environment variable GOOGLE_APPLICATION_CREDENTIALS to point to a JSON
+    file containing credentials.
+
+    Note: GCS buckets are special and the operations available on them may be
+    limited or more expensive than expected compared to local file systems.
+
+    Note: When pickling a GcsFileSystem that uses default credential resolution
+    credentials are not stored in the serialized data. Therefore, when unpickling
+    it is assumed that the necessary credentials are in place for the target
+    process.
+
+    Parameters
+    ----------
+    anonymous : boolean, default False
+        Whether to connect anonymously.
+        If true, will not attempt to look up credentials using standard GCP
+        configuration methods.
+    access_token : str, default None
+        GCP access token.  If provided temporary credentials will be fetched by
+        assuming this role. If specified an credential_token_expiration must be
+        specified with the token.
+    target_service_account : str, default None
+        An optional service account to try to impersonate when accessing GCS. This
+        requires the specified credential user/service_account has the necessary
+        permissions.
+    credential_token_expiration : datetime, default None
+        Expiration for credential generated with an access token. Must be specified
+        if token is specified.
+    default_bucket_location : str, default 'US-CENTRAL1'
+        GCP region to create buckets in.
+    scheme : str, default 'https'
+        GCS connection transport scheme.
+    endpoint_override : str, default None
+        Override endpoint with a connect string such as "localhost:9000"
+    default_metadata : mapping or pyarrow.KeyValueMetadata, default None
+        Default metadata for open_output_stream.  This will be ignored if
+        non-empty metadata is passed to open_output_stream.

Review Comment:
   ```suggestion
           Default metadata for `open_output_stream`.  This will be ignored if
           non-empty metadata is passed to `open_output_stream`.
   ```



##########
python/pyarrow/_gcsfs.pyx:
##########
@@ -0,0 +1,173 @@
+# 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.
+
+# cython: language_level = 3
+
+from pyarrow.lib cimport (check_status, pyarrow_wrap_metadata,
+                          pyarrow_unwrap_metadata)
+from pyarrow.lib import frombytes, tobytes, KeyValueMetadata
+from pyarrow.includes.common cimport *
+from pyarrow.includes.libarrow cimport *
+from pyarrow.includes.libarrow_fs cimport *
+from pyarrow._fs cimport FileSystem
+from cython.operator cimport dereference as deref
+
+from datetime import datetime
+
+
+cdef class GcsFileSystem(FileSystem):
+    """
+    GCS-backed FileSystem implementation
+
+    By default uses the process described in https://google.aip.dev/auth/4110
+    to resolve credentials. If not running on GCP this generally requires the
+    environment variable GOOGLE_APPLICATION_CREDENTIALS to point to a JSON
+    file containing credentials.
+
+    Note: GCS buckets are special and the operations available on them may be
+    limited or more expensive than expected compared to local file systems.
+
+    Note: When pickling a GcsFileSystem that uses default credential resolution
+    credentials are not stored in the serialized data. Therefore, when unpickling
+    it is assumed that the necessary credentials are in place for the target
+    process.
+
+    Parameters
+    ----------
+    anonymous : boolean, default False
+        Whether to connect anonymously.
+        If true, will not attempt to look up credentials using standard GCP
+        configuration methods.
+    access_token : str, default None
+        GCP access token.  If provided temporary credentials will be fetched by
+        assuming this role. If specified an credential_token_expiration must be
+        specified with the token.
+    target_service_account : str, default None
+        An optional service account to try to impersonate when accessing GCS. This
+        requires the specified credential user/service_account has the necessary
+        permissions.
+    credential_token_expiration : datetime, default None
+        Expiration for credential generated with an access token. Must be specified
+        if token is specified.
+    default_bucket_location : str, default 'US-CENTRAL1'
+        GCP region to create buckets in.
+    scheme : str, default 'https'
+        GCS connection transport scheme.
+    endpoint_override : str, default None
+        Override endpoint with a connect string such as "localhost:9000"
+    default_metadata : mapping or pyarrow.KeyValueMetadata, default None
+        Default metadata for open_output_stream.  This will be ignored if
+        non-empty metadata is passed to open_output_stream.
+    """
+
+    cdef:
+        CGcsFileSystem* gcsfs
+
+    def __init__(self, *, bint anonymous=False, access_token=None,
+                 target_service_account=None, credential_token_expiration=None,

Review Comment:
   Need to make sure the expiration has the right type here:
   ```suggestion
                    target_service_account=None, datetime credential_token_expiration=None,
   ```



##########
python/pyarrow/_gcsfs.pyx:
##########
@@ -0,0 +1,173 @@
+# 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.
+
+# cython: language_level = 3
+
+from pyarrow.lib cimport (check_status, pyarrow_wrap_metadata,
+                          pyarrow_unwrap_metadata)
+from pyarrow.lib import frombytes, tobytes, KeyValueMetadata
+from pyarrow.includes.common cimport *
+from pyarrow.includes.libarrow cimport *
+from pyarrow.includes.libarrow_fs cimport *
+from pyarrow._fs cimport FileSystem
+from cython.operator cimport dereference as deref
+
+from datetime import datetime
+
+
+cdef class GcsFileSystem(FileSystem):
+    """
+    GCS-backed FileSystem implementation
+
+    By default uses the process described in https://google.aip.dev/auth/4110
+    to resolve credentials. If not running on GCP this generally requires the
+    environment variable GOOGLE_APPLICATION_CREDENTIALS to point to a JSON
+    file containing credentials.
+
+    Note: GCS buckets are special and the operations available on them may be
+    limited or more expensive than expected compared to local file systems.
+
+    Note: When pickling a GcsFileSystem that uses default credential resolution
+    credentials are not stored in the serialized data. Therefore, when unpickling
+    it is assumed that the necessary credentials are in place for the target
+    process.
+
+    Parameters
+    ----------
+    anonymous : boolean, default False
+        Whether to connect anonymously.
+        If true, will not attempt to look up credentials using standard GCP
+        configuration methods.
+    access_token : str, default None
+        GCP access token.  If provided temporary credentials will be fetched by
+        assuming this role. If specified an credential_token_expiration must be
+        specified with the token.
+    target_service_account : str, default None
+        An optional service account to try to impersonate when accessing GCS. This
+        requires the specified credential user/service_account has the necessary
+        permissions.
+    credential_token_expiration : datetime, default None
+        Expiration for credential generated with an access token. Must be specified
+        if token is specified.
+    default_bucket_location : str, default 'US-CENTRAL1'
+        GCP region to create buckets in.
+    scheme : str, default 'https'
+        GCS connection transport scheme.
+    endpoint_override : str, default None
+        Override endpoint with a connect string such as "localhost:9000"
+    default_metadata : mapping or pyarrow.KeyValueMetadata, default None
+        Default metadata for open_output_stream.  This will be ignored if
+        non-empty metadata is passed to open_output_stream.
+    """
+
+    cdef:
+        CGcsFileSystem* gcsfs
+
+    def __init__(self, *, bint anonymous=False, access_token=None,
+                 target_service_account=None, credential_token_expiration=None,
+                 default_bucket_location='US-CENTRAL1',
+                 scheme=None,
+                 endpoint_override=None,
+                 default_metadata=None):
+        cdef:
+            CGcsOptions options
+            shared_ptr[CGcsFileSystem] wrapped
+
+        # Intentional use of truthiness because empty strings aren't valid and
+        # for reconstruction from pickling will give empty strings.
+        if anonymous and (target_service_account or access_token):
+            raise ValueError(
+                'anonymous option is not compatible with target_service_account and '
+                'access_token please only specify only one.'
+            )
+        elif ((access_token and credential_token_expiration is None) or
+              (not access_token and
+                  credential_token_expiration is not None)):
+            raise ValueError(
+                'access_token and credential_token_expiration must be '
+                'specified together'
+            )
+
+        elif anonymous:
+            options = CGcsOptions.Anonymous()
+        elif access_token:
+            options = CGcsOptions.FromAccessToken(
+                tobytes(access_token),
+                PyDateTime_to_TimePoint(<PyDateTime_DateTime*>credential_token_expiration))
+        else:
+            options = CGcsOptions.Defaults()
+
+        if target_service_account:
+            options = CGcsOptions.FromImpersonatedServiceAccount(
+                options.credentials, tobytes(target_service_account))
+
+        options.default_bucket_location = tobytes(default_bucket_location)
+
+        if scheme is not None:
+            options.scheme = tobytes(scheme)
+        if endpoint_override is not None:
+            options.endpoint_override = tobytes(endpoint_override)
+        if default_metadata is not None:
+            if not isinstance(default_metadata, KeyValueMetadata):
+                default_metadata = KeyValueMetadata(default_metadata)
+            options.default_metadata = pyarrow_unwrap_metadata(
+                default_metadata)
+
+        with nogil:
+            wrapped = GetResultValue(CGcsFileSystem.Make(options))
+
+        self.init(<shared_ptr[CFileSystem]> wrapped)
+
+    cdef init(self, const shared_ptr[CFileSystem]& wrapped):
+        FileSystem.init(self, wrapped)
+        self.gcsfs = <CGcsFileSystem*> wrapped.get()
+
+    @classmethod
+    def _reconstruct(cls, kwargs):
+        return cls(**kwargs)
+
+    def _expiration_datetime_from_options(self):
+        cdef CGcsOptions opts = self.gcsfs.options()
+        expiration_ns = TimePoint_to_ns(opts.credentials.expiration())
+        if expiration_ns == 0:
+            return None
+        ns_per_sec = 1000000000.0
+        return datetime.fromtimestamp(expiration_ns / ns_per_sec)

Review Comment:
   ```suggestion
           return datetime.fromtimestamp(expiration_ns / 1e9)
   ```



##########
python/pyarrow/_gcsfs.pyx:
##########
@@ -0,0 +1,173 @@
+# 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.
+
+# cython: language_level = 3
+
+from pyarrow.lib cimport (check_status, pyarrow_wrap_metadata,
+                          pyarrow_unwrap_metadata)
+from pyarrow.lib import frombytes, tobytes, KeyValueMetadata
+from pyarrow.includes.common cimport *
+from pyarrow.includes.libarrow cimport *
+from pyarrow.includes.libarrow_fs cimport *
+from pyarrow._fs cimport FileSystem
+from cython.operator cimport dereference as deref
+
+from datetime import datetime
+
+
+cdef class GcsFileSystem(FileSystem):
+    """
+    GCS-backed FileSystem implementation
+
+    By default uses the process described in https://google.aip.dev/auth/4110
+    to resolve credentials. If not running on GCP this generally requires the
+    environment variable GOOGLE_APPLICATION_CREDENTIALS to point to a JSON
+    file containing credentials.
+
+    Note: GCS buckets are special and the operations available on them may be
+    limited or more expensive than expected compared to local file systems.
+
+    Note: When pickling a GcsFileSystem that uses default credential resolution
+    credentials are not stored in the serialized data. Therefore, when unpickling
+    it is assumed that the necessary credentials are in place for the target
+    process.
+
+    Parameters
+    ----------
+    anonymous : boolean, default False
+        Whether to connect anonymously.
+        If true, will not attempt to look up credentials using standard GCP
+        configuration methods.
+    access_token : str, default None
+        GCP access token.  If provided temporary credentials will be fetched by
+        assuming this role. If specified an credential_token_expiration must be
+        specified with the token.
+    target_service_account : str, default None
+        An optional service account to try to impersonate when accessing GCS. This
+        requires the specified credential user/service_account has the necessary
+        permissions.
+    credential_token_expiration : datetime, default None
+        Expiration for credential generated with an access token. Must be specified
+        if token is specified.
+    default_bucket_location : str, default 'US-CENTRAL1'
+        GCP region to create buckets in.
+    scheme : str, default 'https'
+        GCS connection transport scheme.
+    endpoint_override : str, default None
+        Override endpoint with a connect string such as "localhost:9000"
+    default_metadata : mapping or pyarrow.KeyValueMetadata, default None
+        Default metadata for open_output_stream.  This will be ignored if
+        non-empty metadata is passed to open_output_stream.
+    """
+
+    cdef:
+        CGcsFileSystem* gcsfs
+
+    def __init__(self, *, bint anonymous=False, access_token=None,
+                 target_service_account=None, credential_token_expiration=None,
+                 default_bucket_location='US-CENTRAL1',
+                 scheme=None,
+                 endpoint_override=None,
+                 default_metadata=None):
+        cdef:
+            CGcsOptions options
+            shared_ptr[CGcsFileSystem] wrapped
+
+        # Intentional use of truthiness because empty strings aren't valid and
+        # for reconstruction from pickling will give empty strings.
+        if anonymous and (target_service_account or access_token):
+            raise ValueError(
+                'anonymous option is not compatible with target_service_account and '
+                'access_token please only specify only one.'
+            )
+        elif ((access_token and credential_token_expiration is None) or
+              (not access_token and
+                  credential_token_expiration is not None)):
+            raise ValueError(
+                'access_token and credential_token_expiration must be '
+                'specified together'
+            )
+
+        elif anonymous:
+            options = CGcsOptions.Anonymous()
+        elif access_token:
+            options = CGcsOptions.FromAccessToken(
+                tobytes(access_token),
+                PyDateTime_to_TimePoint(<PyDateTime_DateTime*>credential_token_expiration))
+        else:
+            options = CGcsOptions.Defaults()
+
+        if target_service_account:

Review Comment:
   Why not under the `elif` chain above?



##########
python/pyarrow/tests/conftest.py:
##########
@@ -311,3 +320,23 @@ def s3_server(s3_connection):
         finally:
             if proc is not None:
                 proc.kill()
+
+
+@pytest.fixture(scope='session')
+def gcs_server():
+    port = find_free_port()
+    env = os.environ.copy()
+    args = [sys.executable, '-m', 'testbench', '--port', str(port)]
+    proc = None
+    try:
+        proc = subprocess.Popen(args, env=env)
+    except OSError:
+        pytest.skip('`gcs test bench` command cannot be located')

Review Comment:
   Why this message? Is "gcs test bench" an actual command?



##########
cpp/src/arrow/filesystem/gcsfs.cc:
##########
@@ -33,13 +33,23 @@
 
 namespace arrow {
 namespace fs {
-struct GcsCredentials {
-  explicit GcsCredentials(std::shared_ptr<google::cloud::Credentials> c)
+struct GcsCredentialsHolder {
+  explicit GcsCredentialsHolder(std::shared_ptr<google::cloud::Credentials> c)
       : credentials(std::move(c)) {}
 
   std::shared_ptr<google::cloud::Credentials> credentials;
 };
 
+bool GcsCredentials::Equals(const GcsCredentials& other) const {
+  if (holder_->credentials == other.holder_->credentials) {
+    return true;
+  }
+  return anonymous_ == other.anonymous_ && access_token_ == other.access_token_ &&

Review Comment:
   Right, the Python side requires the ability to recreate an existing credentials instance in e.g. a different process. Besides, testing for equality is generally useful :-)



##########
python/pyarrow/_gcsfs.pyx:
##########
@@ -0,0 +1,173 @@
+# 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.
+
+# cython: language_level = 3
+
+from pyarrow.lib cimport (check_status, pyarrow_wrap_metadata,
+                          pyarrow_unwrap_metadata)
+from pyarrow.lib import frombytes, tobytes, KeyValueMetadata
+from pyarrow.includes.common cimport *
+from pyarrow.includes.libarrow cimport *
+from pyarrow.includes.libarrow_fs cimport *
+from pyarrow._fs cimport FileSystem
+from cython.operator cimport dereference as deref
+
+from datetime import datetime
+
+
+cdef class GcsFileSystem(FileSystem):
+    """
+    GCS-backed FileSystem implementation
+
+    By default uses the process described in https://google.aip.dev/auth/4110
+    to resolve credentials. If not running on GCP this generally requires the
+    environment variable GOOGLE_APPLICATION_CREDENTIALS to point to a JSON
+    file containing credentials.
+
+    Note: GCS buckets are special and the operations available on them may be
+    limited or more expensive than expected compared to local file systems.
+
+    Note: When pickling a GcsFileSystem that uses default credential resolution
+    credentials are not stored in the serialized data. Therefore, when unpickling
+    it is assumed that the necessary credentials are in place for the target
+    process.
+
+    Parameters
+    ----------
+    anonymous : boolean, default False
+        Whether to connect anonymously.
+        If true, will not attempt to look up credentials using standard GCP
+        configuration methods.
+    access_token : str, default None
+        GCP access token.  If provided temporary credentials will be fetched by
+        assuming this role. If specified an credential_token_expiration must be
+        specified with the token.
+    target_service_account : str, default None
+        An optional service account to try to impersonate when accessing GCS. This
+        requires the specified credential user/service_account has the necessary
+        permissions.
+    credential_token_expiration : datetime, default None
+        Expiration for credential generated with an access token. Must be specified
+        if token is specified.
+    default_bucket_location : str, default 'US-CENTRAL1'
+        GCP region to create buckets in.
+    scheme : str, default 'https'
+        GCS connection transport scheme.
+    endpoint_override : str, default None
+        Override endpoint with a connect string such as "localhost:9000"
+    default_metadata : mapping or pyarrow.KeyValueMetadata, default None
+        Default metadata for open_output_stream.  This will be ignored if
+        non-empty metadata is passed to open_output_stream.
+    """
+
+    cdef:
+        CGcsFileSystem* gcsfs
+
+    def __init__(self, *, bint anonymous=False, access_token=None,
+                 target_service_account=None, credential_token_expiration=None,
+                 default_bucket_location='US-CENTRAL1',
+                 scheme=None,
+                 endpoint_override=None,
+                 default_metadata=None):
+        cdef:
+            CGcsOptions options
+            shared_ptr[CGcsFileSystem] wrapped
+
+        # Intentional use of truthiness because empty strings aren't valid and
+        # for reconstruction from pickling will give empty strings.
+        if anonymous and (target_service_account or access_token):
+            raise ValueError(
+                'anonymous option is not compatible with target_service_account and '
+                'access_token please only specify only one.'
+            )
+        elif ((access_token and credential_token_expiration is None) or
+              (not access_token and
+                  credential_token_expiration is not None)):
+            raise ValueError(
+                'access_token and credential_token_expiration must be '
+                'specified together'
+            )
+
+        elif anonymous:
+            options = CGcsOptions.Anonymous()
+        elif access_token:
+            options = CGcsOptions.FromAccessToken(
+                tobytes(access_token),
+                PyDateTime_to_TimePoint(<PyDateTime_DateTime*>credential_token_expiration))
+        else:
+            options = CGcsOptions.Defaults()
+
+        if target_service_account:
+            options = CGcsOptions.FromImpersonatedServiceAccount(
+                options.credentials, tobytes(target_service_account))
+
+        options.default_bucket_location = tobytes(default_bucket_location)
+
+        if scheme is not None:
+            options.scheme = tobytes(scheme)
+        if endpoint_override is not None:
+            options.endpoint_override = tobytes(endpoint_override)
+        if default_metadata is not None:
+            if not isinstance(default_metadata, KeyValueMetadata):
+                default_metadata = KeyValueMetadata(default_metadata)
+            options.default_metadata = pyarrow_unwrap_metadata(
+                default_metadata)

Review Comment:
   It should be possible to write this:
   ```suggestion
               options.default_metadata = pyarrow_unwrap_metadata(
                   ensure_metadata(default_metadata))
   ```
   
   Perhaps you'll also need to declare `ensure_metadata` in `lib.pxd`:
   ```diff
   diff --git a/python/pyarrow/lib.pxd b/python/pyarrow/lib.pxd
   index 5b9d3001b..b2aa69afa 100644
   --- a/python/pyarrow/lib.pxd
   +++ b/python/pyarrow/lib.pxd
   @@ -531,6 +531,7 @@ cdef shared_ptr[CInputStream] native_transcoding_input_stream(
    
    # Default is allow_none=False
    cpdef DataType ensure_type(object type, bint allow_none=*)
   +cdef KeyValueMetadata ensure_metadata(object meta, c_bool allow_none=*)
    
    cdef timeunit_to_string(TimeUnit unit)
    cdef TimeUnit string_to_timeunit(unit) except *
   ```



-- 
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-unsubscribe@arrow.apache.org

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