From fc870fdb4c098b6ccfabecc91ecdf19549b3432b Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 11 May 2018 09:50:37 +0200 Subject: [PATCH] Use simpler write-once semantics for HDFS repository (#30439) There's no need for an extra `blobExists()` call when writing a blob to the HDFS service. The writeBlob implementation for the HDFS repository already uses the `CreateFlag.CREATE` option on the file creation, which ensures that the blob that's uploaded does not already exist. This saves one network roundtrip. --- .../elasticsearch/repositories/hdfs/HdfsBlobContainer.java | 7 ++----- .../repositories/ESBlobStoreContainerTestCase.java | 3 ++- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsBlobContainer.java b/plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsBlobContainer.java index fa9cda06589c6..926cf0b2ad4af 100644 --- a/plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsBlobContainer.java +++ b/plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsBlobContainer.java @@ -100,14 +100,9 @@ public InputStream readBlob(String blobName) throws IOException { @Override public void writeBlob(String blobName, InputStream inputStream, long blobSize) throws IOException { - if (blobExists(blobName)) { - throw new FileAlreadyExistsException("blob [" + blobName + "] already exists, cannot overwrite"); - } store.execute((Operation) fileContext -> { Path blob = new Path(path, blobName); // we pass CREATE, which means it fails if a blob already exists. - // NOTE: this behavior differs from FSBlobContainer, which passes TRUNCATE_EXISTING - // that should be fixed there, no need to bring truncation into this, give the user an error. EnumSet flags = EnumSet.of(CreateFlag.CREATE, CreateFlag.SYNC_BLOCK); CreateOpts[] opts = {CreateOpts.bufferSize(bufferSize)}; try (FSDataOutputStream stream = fileContext.create(blob, flags, opts)) { @@ -121,6 +116,8 @@ public void writeBlob(String blobName, InputStream inputStream, long blobSize) t // if true synchronous behavior is required" stream.hsync(); } + } catch (org.apache.hadoop.fs.FileAlreadyExistsException faee) { + throw new FileAlreadyExistsException(blob.toString(), null, faee.getMessage()); } return null; }); diff --git a/test/framework/src/main/java/org/elasticsearch/repositories/ESBlobStoreContainerTestCase.java b/test/framework/src/main/java/org/elasticsearch/repositories/ESBlobStoreContainerTestCase.java index 8aff12edc8a53..743be6d1bcb01 100644 --- a/test/framework/src/main/java/org/elasticsearch/repositories/ESBlobStoreContainerTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/repositories/ESBlobStoreContainerTestCase.java @@ -29,6 +29,7 @@ import java.io.IOException; import java.io.InputStream; +import java.nio.file.FileAlreadyExistsException; import java.nio.file.NoSuchFileException; import java.util.Arrays; import java.util.HashMap; @@ -149,7 +150,7 @@ public void testVerifyOverwriteFails() throws IOException { final BytesArray bytesArray = new BytesArray(data); writeBlob(container, blobName, bytesArray); // should not be able to overwrite existing blob - expectThrows(IOException.class, () -> writeBlob(container, blobName, bytesArray)); + expectThrows(FileAlreadyExistsException.class, () -> writeBlob(container, blobName, bytesArray)); container.deleteBlob(blobName); writeBlob(container, blobName, bytesArray); // after deleting the previous blob, we should be able to write to it again }