diff --git a/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsOutputFile.java b/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsOutputFile.java index 302286f3f98c..aa461c347d5d 100644 --- a/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsOutputFile.java +++ b/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsOutputFile.java @@ -28,7 +28,6 @@ import java.nio.file.FileAlreadyExistsException; import static com.google.common.base.Preconditions.checkArgument; -import static io.trino.filesystem.gcs.GcsUtils.getBlob; import static io.trino.filesystem.gcs.GcsUtils.handleGcsException; import static java.net.HttpURLConnection.HTTP_PRECON_FAILED; import static java.util.Objects.requireNonNull; @@ -65,9 +64,6 @@ public void createExclusive(byte[] data) throws IOException { try { - if (getBlob(storage, location).isPresent()) { - throw new FileAlreadyExistsException("File %s already exists".formatted(location)); - } storage.create(blobInfo(), data, BlobTargetOption.doesNotExist()); } catch (RuntimeException e) { @@ -81,9 +77,6 @@ public OutputStream create(AggregatedMemoryContext memoryContext) throws IOException { try { - if (getBlob(storage, location).isPresent()) { - throw new FileAlreadyExistsException("File %s already exists".formatted(location)); - } WriteChannel writeChannel = storage.writer(blobInfo(), BlobWriteOption.doesNotExist()); return new GcsOutputStream(location, writeChannel, memoryContext, writeBlockSizeBytes); } diff --git a/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsOutputStream.java b/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsOutputStream.java index d00a2b8ad483..a931e24cf849 100644 --- a/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsOutputStream.java +++ b/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsOutputStream.java @@ -14,6 +14,7 @@ package io.trino.filesystem.gcs; import com.google.cloud.WriteChannel; +import com.google.cloud.storage.StorageException; import com.google.common.primitives.Ints; import io.trino.memory.context.AggregatedMemoryContext; import io.trino.memory.context.LocalMemoryContext; @@ -21,9 +22,11 @@ import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; +import java.nio.file.FileAlreadyExistsException; import static com.google.common.base.Preconditions.checkArgument; import static java.lang.Math.min; +import static java.net.HttpURLConnection.HTTP_PRECON_FAILED; import static java.util.Objects.requireNonNull; public class GcsOutputStream @@ -134,6 +137,11 @@ public void close() try { writeChannel.close(); } + catch (StorageException e) { + if (e.getCode() == HTTP_PRECON_FAILED) { + throw new FileAlreadyExistsException(location.toString()); + } + } catch (IOException e) { throw new IOException("Error closing file: " + location, e); } diff --git a/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java b/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java index dce101db24d5..e3ca87854b62 100644 --- a/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java +++ b/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java @@ -510,7 +510,7 @@ void testOutputFile() if (isCreateExclusive()) { // re-create without overwrite is an error - assertThatThrownBy(outputFile::create) + assertThatThrownBy(() -> outputFile.create().close()) .isInstanceOf(FileAlreadyExistsException.class) .hasMessageContaining(tempBlob.location().toString());