GoogleCloudDataproc / hadoop-connectors

Libraries and tools for interoperability between Hadoop-related open-source software and Google Cloud Platform.
Apache License 2.0
280 stars 238 forks source link

GCS connector should not create bucket in mkdirs. #727

Open dondelicaat opened 2 years ago

dondelicaat commented 2 years ago

Contrary to the claim in #475 the function mkdirsInternal actually creates a bucket when making a new directory, see here. This should not be happening but rather it should try to write to the bucket and throw an exception when the bucket does not exist. It should not be the responsibility of this component to create an actual bucket as this is a way to privileged action.

simonvanderveldt commented 2 years ago

This came as a bit of a surprise to us. We were happy to see a fix for https://github.com/GoogleCloudDataproc/hadoop-connectors/issues/356 which was fixed by #468 but when testing any release >=2.2.0 we ended up with a failure because the GCS connector now tries to create a bucket. None of our Spark applications have storage.buckets.create permissions since all our infrastructure including buckets is managed and runtime components don't have any permission to manage GCP resources. It seems like the PR that introduced this change (#475) was supposed to work this way as well given the wording in the PR description

Do not create bucket if it does not exist - this will require that buckets to be created explicitly.

As well as in the changelog https://github.com/GoogleCloudDataproc/hadoop-connectors/pull/475/files#diff-42d4cbc889ca61ca3f64298dcc90bf0a99ed5690722f90abb3c56190ac3eda6eR135

But the actual implementation in that PR does something totally different, namely hardcoding a bucket create call. This looks like a mistake?

medb commented 2 years ago

475 PR addressed the issue that if gs://buc/a/b/c/ is created, then only gs://buc/a/b/c/ directory object will be created and it will fail if gs://buc/ does not exist, but if gs://buc/ created explicitly then it will be created, because application explicitly requested to create this directory, i.e. bucket.

In what Spark application flow bucket creation is attempted, that you would like will not happen? May you share full stacktrace of failed bucket creation without storage.buckets.create permission?

dondelicaat commented 2 years ago

Hi, it happens when we initialize the sparkContext. It tries to create a log file which in turns calls the mkdirsInternal method in gcsio/src/main/java/com/google/cloud/hadoop/gcsio/GoogleCloudStorageFileSystem.java. However, instead of assuming the bucket exists and throwing an error when it doesn't it instead tries to create the bucket and throws an error if it already exists. The stack trace:

ERROR SparkContext: Error initializing SparkContext.
com.google.cloud.hadoop.repackaged.gcs.com.google.api.client.googleapis.json.GoogleJsonResponseException: 403 Forbidden
POST [https://storage.googleapis.com/storage/v1/b?project=](https://storage.googleapis.com/storage/v1/b?project=<OUR-PROJECT>)
{
  "code" : 403,
  "errors" : [ {
    "domain" : "global",
    "message" : "<OUR-SERVICE-ACCOUNT>@<OUR-PROJECT>.iam.gserviceaccount.com does not have storage.buckets.create access to the Google Cloud project.",
    "reason" : "forbidden"
  } ],
  "message" : "<OUR-SERVICE-ACCOUNT>@<OUR-PROJECT>.iam.gserviceaccount.com does not have storage.buckets.create access to the Google Cloud project."
}
   at com.google.cloud.hadoop.repackaged.gcs.com.google.api.client.googleapis.json.GoogleJsonResponseException.from(GoogleJsonResponseException.java:146)
   at com.google.cloud.hadoop.repackaged.gcs.com.google.api.client.googleapis.services.json.AbstractGoogleJsonClientRequest.newExceptionOnError(AbstractGoogleJsonClientRequest.java:118)
   at com.google.cloud.hadoop.repackaged.gcs.com.google.api.client.googleapis.services.json.AbstractGoogleJsonClientRequest.newExceptionOnError(AbstractGoogleJsonClientRequest.java:37)
   at com.google.cloud.hadoop.repackaged.gcs.com.google.api.client.googleapis.services.AbstractGoogleClientRequest$1.interceptResponse(AbstractGoogleClientRequest.java:428)
   at com.google.cloud.hadoop.repackaged.gcs.com.google.api.client.http.HttpRequest.execute(HttpRequest.java:1111)
   at com.google.cloud.hadoop.repackaged.gcs.com.google.api.client.googleapis.services.AbstractGoogleClientRequest.executeUnparsed(AbstractGoogleClientRequest.java:514)
   at com.google.cloud.hadoop.repackaged.gcs.com.google.api.client.googleapis.services.AbstractGoogleClientRequest.executeUnparsed(AbstractGoogleClientRequest.java:455)
   at com.google.cloud.hadoop.repackaged.gcs.com.google.api.client.googleapis.services.AbstractGoogleClientRequest.execute(AbstractGoogleClientRequest.java:565)
   at com.google.cloud.hadoop.repackaged.gcs.com.google.cloud.hadoop.util.ResilientOperation.retry(ResilientOperation.java:66)
   at com.google.cloud.hadoop.repackaged.gcs.com.google.cloud.hadoop.gcsio.GoogleCloudStorageImpl.createBucket(GoogleCloudStorageImpl.java:587)
   at com.google.cloud.hadoop.repackaged.gcs.com.google.cloud.hadoop.gcsio.GoogleCloudStorage.createBucket(GoogleCloudStorage.java:94)
   at com.google.cloud.hadoop.repackaged.gcs.com.google.cloud.hadoop.gcsio.GoogleCloudStorageFileSystem.mkdirsInternal(GoogleCloudStorageFileSystem.java:484)
   at com.google.cloud.hadoop.repackaged.gcs.com.google.cloud.hadoop.gcsio.GoogleCloudStorageFileSystem.mkdirs(GoogleCloudStorageFileSystem.java:472)
   at com.google.cloud.hadoop.fs.gcs.GoogleHadoopFileSystemBase.mkdirs(GoogleHadoopFileSystemBase.java:921)
   at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:2388)
   at org.apache.spark.deploy.SparkHadoopUtil$.createFile(SparkHadoopUtil.scala:531)
   at org.apache.spark.deploy.history.EventLogFileWriter.initLogFile(EventLogFileWriters.scala:98)
   at org.apache.spark.deploy.history.SingleEventLogFileWriter.start(EventLogFileWriters.scala:223)
   at org.apache.spark.scheduler.EventLoggingListener.start(EventLoggingListener.scala:83)
   at org.apache.spark.SparkContext.<init>(SparkContext.scala:610)
   at org.apache.spark.api.java.JavaSparkContext.<init>(JavaSparkContext.scala:58)
   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
   at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
   at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
   at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
   at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:247)
   at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
   at py4j.Gateway.invoke(Gateway.java:238)
   at py4j.commands.ConstructorCommand.invokeConstructor(ConstructorCommand.java:80)
   at py4j.commands.ConstructorCommand.execute(ConstructorCommand.java:69)
   at py4j.ClientServerConnection.waitForCommands(ClientServerConnection.java:182)
   at py4j.ClientServerConnection.run(ClientServerConnection.java:106)
   at java.lang.Thread.run(Thread.java:748)
simonvanderveldt commented 2 years ago

This bug has become more relevant to us since we're running into a policy size limitation on GCP which is partially caused by the additional conditions on parent prefixes we currently still need to add to each binding because of #356.

475 PR addressed the issue that if gs://buc/a/b/c/ is created, then only gs://buc/a/b/c/ directory object will be created and it will fail if gs://buc/ does not exist, but if gs://buc/ created explicitly then it will be created, because application explicitly requested to create this directory, i.e. bucket.

In what Spark application flow bucket creation is attempted, that you would like will not happen? May you share full stacktrace of failed bucket creation without storage.buckets.create permission?

@medb What do you mean with "because application explicitly requested to create this directory"? Our code is not requesting to create a directory, we're just using df.write.parquet(<some prefix>, mode="overwrite") in our Spark application. The creation of bucket happens before our code even runs.

Given these lines

   at org.apache.spark.deploy.history.EventLogFileWriter.initLogFile(EventLogFileWriters.scala:98)
   at org.apache.spark.deploy.history.SingleEventLogFileWriter.start(EventLogFileWriters.scala:223)
   at org.apache.spark.scheduler.EventLoggingListener.start(EventLoggingListener.scala:83)

it seems like this is caused by Spark itself given that it's about Spark trying to create a directory/bucket for the event logs. We use spark.eventLog.dir=gs://<bucket>, so something might be going wrong in that part of Spark? Spark seems to issue a createFile here https://github.com/apache/spark/blob/e6839ad7340bc9eb5df03df2a62110bdda805e6b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala#L98 maybe that still somehow results in the GCS connector trying to create a bucket?

dondelicaat commented 2 years ago

@medb Could you give an update on this issue? It would be great to update and benefit from the other changes made! :)