Skip to content

Commit b3bae09

Browse files
committed
Drop explicit check for file existence while creating GCS blobs
Defer the file existence check to GCS API when the blob gets actually created.
1 parent 82fcef8 commit b3bae09

File tree

3 files changed

+9
-8
lines changed

3 files changed

+9
-8
lines changed

lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsOutputFile.java

Lines changed: 0 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,6 @@
2828
import java.nio.file.FileAlreadyExistsException;
2929

3030
import static com.google.common.base.Preconditions.checkArgument;
31-
import static io.trino.filesystem.gcs.GcsUtils.getBlob;
3231
import static io.trino.filesystem.gcs.GcsUtils.handleGcsException;
3332
import static java.net.HttpURLConnection.HTTP_PRECON_FAILED;
3433
import static java.util.Objects.requireNonNull;
@@ -65,9 +64,6 @@ public void createExclusive(byte[] data)
6564
throws IOException
6665
{
6766
try {
68-
if (getBlob(storage, location).isPresent()) {
69-
throw new FileAlreadyExistsException("File %s already exists".formatted(location));
70-
}
7167
storage.create(blobInfo(), data, BlobTargetOption.doesNotExist());
7268
}
7369
catch (RuntimeException e) {
@@ -81,9 +77,6 @@ public OutputStream create(AggregatedMemoryContext memoryContext)
8177
throws IOException
8278
{
8379
try {
84-
if (getBlob(storage, location).isPresent()) {
85-
throw new FileAlreadyExistsException("File %s already exists".formatted(location));
86-
}
8780
WriteChannel writeChannel = storage.writer(blobInfo(), BlobWriteOption.doesNotExist());
8881
return new GcsOutputStream(location, writeChannel, memoryContext, writeBlockSizeBytes);
8982
}

lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsOutputStream.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -14,16 +14,19 @@
1414
package io.trino.filesystem.gcs;
1515

1616
import com.google.cloud.WriteChannel;
17+
import com.google.cloud.storage.StorageException;
1718
import com.google.common.primitives.Ints;
1819
import io.trino.memory.context.AggregatedMemoryContext;
1920
import io.trino.memory.context.LocalMemoryContext;
2021

2122
import java.io.IOException;
2223
import java.io.OutputStream;
2324
import java.nio.ByteBuffer;
25+
import java.nio.file.FileAlreadyExistsException;
2426

2527
import static com.google.common.base.Preconditions.checkArgument;
2628
import static java.lang.Math.min;
29+
import static java.net.HttpURLConnection.HTTP_PRECON_FAILED;
2730
import static java.util.Objects.requireNonNull;
2831

2932
public class GcsOutputStream
@@ -134,6 +137,11 @@ public void close()
134137
try {
135138
writeChannel.close();
136139
}
140+
catch (StorageException e) {
141+
if (e.getCode() == HTTP_PRECON_FAILED) {
142+
throw new FileAlreadyExistsException(location.toString());
143+
}
144+
}
137145
catch (IOException e) {
138146
throw new IOException("Error closing file: " + location, e);
139147
}

lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -510,7 +510,7 @@ void testOutputFile()
510510

511511
if (isCreateExclusive()) {
512512
// re-create without overwrite is an error
513-
assertThatThrownBy(outputFile::create)
513+
assertThatThrownBy(() -> outputFile.create().close())
514514
.isInstanceOf(FileAlreadyExistsException.class)
515515
.hasMessageContaining(tempBlob.location().toString());
516516

0 commit comments

Comments
 (0)