Hey Flink Users, We've been facing an issue with GCS that I'm keen to hear the community's thoughts or insights on.
We're using the GCS FileSystem on a FileSink to write parquets in our Flink app. We're finding sporadic instances of `com.google.cloud.storage.StorageException: Read timed out` that cause our job to restart. While we have tolerance in place for failed checkpoints, this causes many more failures/restarts as compared to other FileSystems like AWS or Azure we use. We've tried tuning the size of the files we write but found no improvement; our parquets are already "tiny" - _many_ parquets on the order of 1-10KB. Following multiple stack traces, we see the exception raised from multiple parts of the sink lifecycle: FileWriter::prepareCommit, FileWriter::write, and FileCommitter::commit. Our hypothesis is sporadic failures from GCS HTTP APIs that aren't getting retried correctly or need a longer timeout than the default (20 seconds for Read timeouts, 50 seconds for Retries overall). This problem is infrequent enough that it's hard to reproduce/test; it comes and goes on how noisy it is. I noticed we can't tune any google-cloud-storage parameters via flink-config; there's FLINK-32877[1] which proposed adding Read/Connection Timeout parameters for the HTTPTransportOptions[2] but it's still open. I also noticed there's more we can change like what gets retried in the StorageRetryStrategy[3] and the RetrySettings[4]. Ultimately I'm thinking of creating an alternate FileSystemFactory in our deployment (under a different scheme/plugin) to test how tweaking these options in the StorageOptions.Builder[5] call works out. Have other GCS FileSink users hit these exceptions? What did you do? Anything else we might need to consider? -Dylan [1]: https://issues.apache.org/jira/browse/FLINK-32877 [2]: https://cloud.google.com/java/docs/reference/google-cloud-core/latest/com.google.cloud.http.HttpTransportOptions [3]: https://cloud.google.com/java/docs/reference/google-cloud-storage/latest/com.google.cloud.storage.StorageRetryStrategy [4]: https://github.com/googleapis/sdk-platform-java/blob/a94c2f0e8a99f0ddf17106cbc8117cefe6b0e127/java-core/google-cloud-core/src/main/java/com/google/cloud/ServiceOptions.java#L787 [5]: https://github.com/apache/flink/blob/163b9cca6d2ccac0ff89dd985e3232667ddfb14f/flink-filesystems/flink-gs-fs-hadoop/src/main/java/org/apache/flink/fs/gs/GSFileSystemFactory.java#L94