|
@@ -34,6 +34,7 @@ import org.elasticsearch.common.blobstore.DeleteResult;
|
|
|
import org.elasticsearch.common.blobstore.fs.FsBlobContainer;
|
|
|
import org.elasticsearch.common.blobstore.support.AbstractBlobContainer;
|
|
|
import org.elasticsearch.common.blobstore.support.PlainBlobMetadata;
|
|
|
+import org.elasticsearch.common.bytes.BytesReference;
|
|
|
import org.elasticsearch.common.io.Streams;
|
|
|
import org.elasticsearch.repositories.hdfs.HdfsBlobStore.Operation;
|
|
|
|
|
@@ -150,12 +151,28 @@ final class HdfsBlobContainer extends AbstractBlobContainer {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void writeBlobAtomic(String blobName, InputStream inputStream, long blobSize, boolean failIfAlreadyExists) throws IOException {
|
|
|
+ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlreadyExists) throws IOException {
|
|
|
+ Path blob = new Path(path, blobName);
|
|
|
+ // we pass CREATE, which means it fails if a blob already exists.
|
|
|
+ final EnumSet<CreateFlag> flags = failIfAlreadyExists ? EnumSet.of(CreateFlag.CREATE, CreateFlag.SYNC_BLOCK)
|
|
|
+ : EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE, CreateFlag.SYNC_BLOCK);
|
|
|
+ store.execute((Operation<Void>) fileContext -> {
|
|
|
+ try {
|
|
|
+ writeToPath(bytes, blob, fileContext, flags);
|
|
|
+ } catch (org.apache.hadoop.fs.FileAlreadyExistsException faee) {
|
|
|
+ throw new FileAlreadyExistsException(blob.toString(), null, faee.getMessage());
|
|
|
+ }
|
|
|
+ return null;
|
|
|
+ });
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void writeBlobAtomic(String blobName, BytesReference bytes, boolean failIfAlreadyExists) throws IOException {
|
|
|
final String tempBlob = FsBlobContainer.tempBlobName(blobName);
|
|
|
final Path tempBlobPath = new Path(path, tempBlob);
|
|
|
final Path blob = new Path(path, blobName);
|
|
|
store.execute((Operation<Void>) fileContext -> {
|
|
|
- writeToPath(inputStream, blobSize, fileContext, tempBlobPath, EnumSet.of(CreateFlag.CREATE, CreateFlag.SYNC_BLOCK));
|
|
|
+ writeToPath(bytes, tempBlobPath, fileContext, EnumSet.of(CreateFlag.CREATE, CreateFlag.SYNC_BLOCK));
|
|
|
try {
|
|
|
fileContext.rename(tempBlobPath, blob, failIfAlreadyExists ? Options.Rename.NONE : Options.Rename.OVERWRITE);
|
|
|
} catch (org.apache.hadoop.fs.FileAlreadyExistsException faee) {
|
|
@@ -165,6 +182,13 @@ final class HdfsBlobContainer extends AbstractBlobContainer {
|
|
|
});
|
|
|
}
|
|
|
|
|
|
+ private void writeToPath(BytesReference bytes, Path blobPath, FileContext fileContext,
|
|
|
+ EnumSet<CreateFlag> createFlags) throws IOException {
|
|
|
+ try (FSDataOutputStream stream = fileContext.create(blobPath, createFlags)) {
|
|
|
+ bytes.writeTo(stream);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private void writeToPath(InputStream inputStream, long blobSize, FileContext fileContext, Path blobPath,
|
|
|
EnumSet<CreateFlag> createFlags) throws IOException {
|
|
|
final byte[] buffer = new byte[blobSize < bufferSize ? Math.toIntExact(blobSize) : bufferSize];
|