From 318baccef23b451312077fba8282511506778182 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 16 Jun 2021 12:24:42 +0200 Subject: [PATCH 01/24] works in core --- .../blobstore/url/URLBlobContainer.java | 6 ++ .../common/blobstore/BlobContainer.java | 3 + .../common/blobstore/fs/FsBlobContainer.java | 33 +++++++++ .../support/FilterBlobContainer.java | 6 ++ .../blobstore/BlobStoreRepository.java | 38 ++++------- .../blobstore/ChecksumBlobStoreFormat.java | 67 ++++++++----------- .../snapshots/BlobStoreFormatTests.java | 11 ++- .../SnapshotInfoBlobSerializationTests.java | 19 ++---- .../AbstractSnapshotIntegTestCase.java | 2 +- .../snapshots/mockstore/MockRepository.java | 30 ++++++++- 10 files changed, 126 insertions(+), 89 deletions(-) diff --git a/modules/repository-url/src/main/java/org/elasticsearch/common/blobstore/url/URLBlobContainer.java b/modules/repository-url/src/main/java/org/elasticsearch/common/blobstore/url/URLBlobContainer.java index 0d8e6806a43e3..0d46c33095774 100644 --- a/modules/repository-url/src/main/java/org/elasticsearch/common/blobstore/url/URLBlobContainer.java +++ b/modules/repository-url/src/main/java/org/elasticsearch/common/blobstore/url/URLBlobContainer.java @@ -20,6 +20,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.net.URL; import java.nio.file.NoSuchFileException; import java.security.AccessController; @@ -121,6 +122,11 @@ public void writeBlob(String blobName, InputStream inputStream, long blobSize, b throw new UnsupportedOperationException("URL repository doesn't support this operation"); } + @Override + public OutputStream writeBlob(String blobName, boolean failIfAlreadyExists) throws IOException { + throw new UnsupportedOperationException("URL repository doesn't support this operation"); + } + @Override public void writeBlobAtomic(String blobName, BytesReference bytes, boolean failIfAlreadyExists) throws IOException { throw new UnsupportedOperationException("URL repository doesn't support this operation"); diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java b/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java index 266a1536274b2..5b2ea2174a92f 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java @@ -12,6 +12,7 @@ import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.nio.file.FileAlreadyExistsException; import java.nio.file.NoSuchFileException; import java.util.Iterator; @@ -116,6 +117,8 @@ default void writeBlob(String blobName, BytesReference bytes, boolean failIfAlre writeBlob(blobName, bytes.streamInput(), bytes.length(), failIfAlreadyExists); } + OutputStream writeBlob(String blobName, boolean failIfAlreadyExists) throws IOException; + /** * Reads blob content from a {@link BytesReference} and writes it to the container in a new blob with the given name, * using an atomic write operation if the implementation supports it. diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java b/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java index 2d1d75618ad45..4709a0d386b69 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java @@ -21,6 +21,7 @@ import org.elasticsearch.core.internal.io.IOUtils; import java.io.FileNotFoundException; +import java.io.FilterOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -243,6 +244,20 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea IOUtils.fsync(path, true); } + @Override + public OutputStream writeBlob(String blobName, boolean failIfAlreadyExists) throws IOException { + final Path file = path.resolve(blobName); + try { + return new BlobOutputStream(file); + } catch (FileAlreadyExistsException faee) { + if (failIfAlreadyExists) { + throw faee; + } + deleteBlobsIgnoringIfNotExists(Iterators.single(blobName)); + return new BlobOutputStream(file); + } + } + @Override public void writeBlobAtomic(final String blobName, BytesReference bytes, boolean failIfAlreadyExists) throws IOException { final String tempBlob = tempBlobName(blobName); @@ -306,4 +321,22 @@ public static String tempBlobName(final String blobName) { public static boolean isTempBlobName(final String blobName) { return blobName.startsWith(TEMP_FILE_PREFIX); } + + private class BlobOutputStream extends FilterOutputStream { + + BlobOutputStream(Path file) throws IOException { + super(Files.newOutputStream(file, StandardOpenOption.CREATE_NEW)); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + out.write(b, off, len); + } + + @Override + public void close() throws IOException { + super.close(); + IOUtils.fsync(path, true); + } + } } diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/support/FilterBlobContainer.java b/server/src/main/java/org/elasticsearch/common/blobstore/support/FilterBlobContainer.java index cb287531c8f7b..5c5dcf91eee9a 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/support/FilterBlobContainer.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/support/FilterBlobContainer.java @@ -16,6 +16,7 @@ import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.util.Iterator; import java.util.Map; import java.util.Objects; @@ -61,6 +62,11 @@ public void writeBlob(String blobName, InputStream inputStream, long blobSize, b delegate.writeBlob(blobName, inputStream, blobSize, failIfAlreadyExists); } + @Override + public OutputStream writeBlob(String blobName, boolean failIfAlreadyExists) throws IOException { + return delegate.writeBlob(blobName, failIfAlreadyExists); + } + @Override public void writeBlobAtomic(String blobName, BytesReference bytes, boolean failIfAlreadyExists) throws IOException { delegate.writeBlobAtomic(blobName, bytes, failIfAlreadyExists); diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index fb00b86df3568..4daac1b85d28c 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -553,15 +553,13 @@ public void cloneShardSnapshot( sourceMeta.asClone(target.getName(), startTime, threadPool.absoluteTimeInMillis() - startTime), shardContainer, target.getUUID(), - compress, - bigArrays + compress ); INDEX_SHARD_SNAPSHOTS_FORMAT.write( existingSnapshots.withClone(source.getName(), target.getName()), shardContainer, newGen, - compress, - bigArrays + compress ); return new ShardSnapshotResult( newGen, @@ -1393,7 +1391,7 @@ public void finalizeSnapshot( executor.execute( ActionRunnable.run( allMetaListener, - () -> GLOBAL_METADATA_FORMAT.write(clusterMetadata, blobContainer(), snapshotId.getUUID(), compress, bigArrays) + () -> GLOBAL_METADATA_FORMAT.write(clusterMetadata, blobContainer(), snapshotId.getUUID(), compress) ) ); @@ -1407,7 +1405,7 @@ public void finalizeSnapshot( if (metaUUID == null) { // We don't yet have this version of the metadata so we write it metaUUID = UUIDs.base64UUID(); - INDEX_METADATA_FORMAT.write(indexMetaData, indexContainer(index), metaUUID, compress, bigArrays); + INDEX_METADATA_FORMAT.write(indexMetaData, indexContainer(index), metaUUID, compress); indexMetaIdentifiers.put(identifiers, metaUUID); } indexMetas.put(index, identifiers); @@ -1416,8 +1414,7 @@ public void finalizeSnapshot( clusterMetadata.index(index.getName()), indexContainer(index), snapshotId.getUUID(), - compress, - bigArrays + compress ); } })); @@ -1425,7 +1422,7 @@ public void finalizeSnapshot( executor.execute( ActionRunnable.run( allMetaListener, - () -> SNAPSHOT_FORMAT.write(snapshotInfo, blobContainer(), snapshotId.getUUID(), compress, bigArrays) + () -> SNAPSHOT_FORMAT.write(snapshotInfo, blobContainer(), snapshotId.getUUID(), compress) ) ); }, onUpdateFailure); @@ -2682,13 +2679,7 @@ public void snapshotShard(SnapshotShardContext context) { // reference a generation that has not had all its files fully upload. indexGeneration = UUIDs.randomBase64UUID(); try { - INDEX_SHARD_SNAPSHOTS_FORMAT.write( - updatedBlobStoreIndexShardSnapshots, - shardContainer, - indexGeneration, - compress, - bigArrays - ); + INDEX_SHARD_SNAPSHOTS_FORMAT.write(updatedBlobStoreIndexShardSnapshots, shardContainer, indexGeneration, compress); } catch (IOException e) { throw new IndexShardSnapshotFailedException( shardId, @@ -2765,7 +2756,7 @@ public void snapshotShard(SnapshotShardContext context) { ); try { final String snapshotUUID = snapshotId.getUUID(); - INDEX_SHARD_SNAPSHOT_FORMAT.write(blobStoreIndexShardSnapshot, shardContainer, snapshotUUID, compress, bigArrays); + INDEX_SHARD_SNAPSHOT_FORMAT.write(blobStoreIndexShardSnapshot, shardContainer, snapshotUUID, compress); } catch (IOException e) { throw new IndexShardSnapshotFailedException(shardId, "Failed to write commit point", e); } @@ -3138,7 +3129,7 @@ private ShardSnapshotMetaDeleteResult deleteFromShardSnapshotMeta( final BlobStoreIndexShardSnapshots updatedSnapshots = new BlobStoreIndexShardSnapshots(newSnapshotsList); if (indexGeneration < 0L) { writtenGeneration = UUIDs.randomBase64UUID(); - INDEX_SHARD_SNAPSHOTS_FORMAT.write(updatedSnapshots, shardContainer, writtenGeneration, compress, bigArrays); + INDEX_SHARD_SNAPSHOTS_FORMAT.write(updatedSnapshots, shardContainer, writtenGeneration, compress); } else { writtenGeneration = String.valueOf(indexGeneration); writeShardIndexBlobAtomic(shardContainer, indexGeneration, updatedSnapshots); @@ -3178,13 +3169,10 @@ private void writeShardIndexBlobAtomic( () -> new ParameterizedMessage("[{}] Writing shard index [{}] to [{}]", metadata.name(), indexGeneration, shardContainer.path()) ); final String blobName = INDEX_SHARD_SNAPSHOTS_FORMAT.blobName(String.valueOf(indexGeneration)); - INDEX_SHARD_SNAPSHOTS_FORMAT.serialize( - updatedSnapshots, - blobName, - compress, - bigArrays, - bytes -> writeAtomic(shardContainer, blobName, bytes, true) - ); + try (ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(bigArrays)) { + INDEX_SHARD_SNAPSHOTS_FORMAT.serialize(updatedSnapshots, blobName, compress, out); + writeAtomic(shardContainer, blobName, out.bytes(), true); + } } // Unused blobs are all previous index-, data- and meta-blobs and that are not referenced by the new index- as well as all diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java index b969cb188ab0e..80f004f21d866 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java @@ -18,12 +18,9 @@ import org.elasticsearch.common.Numbers; import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.compress.CompressorFactory; import org.elasticsearch.common.io.Streams; -import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput; import org.elasticsearch.common.lucene.store.IndexOutputOutputStream; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.ToXContent; @@ -31,7 +28,6 @@ import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.CheckedFunction; import org.elasticsearch.gateway.CorruptStateException; import org.elasticsearch.snapshots.SnapshotInfo; @@ -279,47 +275,40 @@ private int getAvailable() throws IOException { * @param name blob name * @param compress whether to use compression */ - public void write(T obj, BlobContainer blobContainer, String name, boolean compress, BigArrays bigArrays) throws IOException { + public void write(T obj, BlobContainer blobContainer, String name, boolean compress) throws IOException { final String blobName = blobName(name); - serialize(obj, blobName, compress, bigArrays, bytes -> blobContainer.writeBlob(blobName, bytes, false)); + try (OutputStream out = blobContainer.writeBlob(blobName, false)) { + serialize(obj, blobName, compress, out); + } } - public void serialize( - final T obj, - final String blobName, - final boolean compress, - BigArrays bigArrays, - CheckedConsumer consumer - ) throws IOException { - try (ReleasableBytesStreamOutput outputStream = new ReleasableBytesStreamOutput(bigArrays)) { - try ( - OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput( - "ChecksumBlobStoreFormat.writeBlob(blob=\"" + blobName + "\")", - blobName, - org.elasticsearch.common.io.Streams.noCloseStream(outputStream), - BUFFER_SIZE + public void serialize(final T obj, final String blobName, final boolean compress, OutputStream outputStream) throws IOException { + try ( + OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput( + "ChecksumBlobStoreFormat.writeBlob(blob=\"" + blobName + "\")", + blobName, + org.elasticsearch.common.io.Streams.noCloseStream(outputStream), + BUFFER_SIZE + ) + ) { + CodecUtil.writeHeader(indexOutput, codec, VERSION); + try (OutputStream indexOutputOutputStream = new IndexOutputOutputStream(indexOutput) { + @Override + public void close() { + // this is important since some of the XContentBuilders write bytes on close. + // in order to write the footer we need to prevent closing the actual index input. + } + }; + XContentBuilder builder = XContentFactory.contentBuilder( + XContentType.SMILE, + compress ? CompressorFactory.COMPRESSOR.threadLocalOutputStream(indexOutputOutputStream) : indexOutputOutputStream ) ) { - CodecUtil.writeHeader(indexOutput, codec, VERSION); - try (OutputStream indexOutputOutputStream = new IndexOutputOutputStream(indexOutput) { - @Override - public void close() { - // this is important since some of the XContentBuilders write bytes on close. - // in order to write the footer we need to prevent closing the actual index input. - } - }; - XContentBuilder builder = XContentFactory.contentBuilder( - XContentType.SMILE, - compress ? CompressorFactory.COMPRESSOR.threadLocalOutputStream(indexOutputOutputStream) : indexOutputOutputStream - ) - ) { - builder.startObject(); - obj.toXContent(builder, SNAPSHOT_ONLY_FORMAT_PARAMS); - builder.endObject(); - } - CodecUtil.writeFooter(indexOutput); + builder.startObject(); + obj.toXContent(builder, SNAPSHOT_ONLY_FORMAT_PARAMS); + builder.endObject(); } - consumer.accept(outputStream.bytes()); + CodecUtil.writeFooter(indexOutput); } } } diff --git a/server/src/test/java/org/elasticsearch/snapshots/BlobStoreFormatTests.java b/server/src/test/java/org/elasticsearch/snapshots/BlobStoreFormatTests.java index e0d6203e20628..f6c3572444cd5 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/BlobStoreFormatTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/BlobStoreFormatTests.java @@ -17,7 +17,6 @@ import org.elasticsearch.common.blobstore.fs.FsBlobStore; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.io.Streams; -import org.elasticsearch.common.util.MockBigArrays; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -83,9 +82,9 @@ public void testBlobStoreOperations() throws IOException { // Write blobs in different formats final String randomText = randomAlphaOfLengthBetween(0, 1024 * 8 * 3); final String normalText = "checksum smile: " + randomText; - checksumSMILE.write(new BlobObj(normalText), blobContainer, "check-smile", false, MockBigArrays.NON_RECYCLING_INSTANCE); + checksumSMILE.write(new BlobObj(normalText), blobContainer, "check-smile", false); final String compressedText = "checksum smile compressed: " + randomText; - checksumSMILE.write(new BlobObj(compressedText), blobContainer, "check-smile-comp", true, MockBigArrays.NON_RECYCLING_INSTANCE); + checksumSMILE.write(new BlobObj(compressedText), blobContainer, "check-smile-comp", true); // Assert that all checksum blobs can be read assertEquals(normalText, checksumSMILE.read(blobContainer, "check-smile", xContentRegistry()).getText()); @@ -101,8 +100,8 @@ public void testCompressionIsApplied() throws IOException { } ChecksumBlobStoreFormat checksumFormat = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj::fromXContent); BlobObj blobObj = new BlobObj(veryRedundantText.toString()); - checksumFormat.write(blobObj, blobContainer, "blob-comp", true, MockBigArrays.NON_RECYCLING_INSTANCE); - checksumFormat.write(blobObj, blobContainer, "blob-not-comp", false, MockBigArrays.NON_RECYCLING_INSTANCE); + checksumFormat.write(blobObj, blobContainer, "blob-comp", true); + checksumFormat.write(blobObj, blobContainer, "blob-not-comp", false); Map blobs = blobContainer.listBlobsByPrefix("blob-"); assertEquals(blobs.size(), 2); assertThat(blobs.get("blob-not-comp").length(), greaterThan(blobs.get("blob-comp").length())); @@ -114,7 +113,7 @@ public void testBlobCorruption() throws IOException { String testString = randomAlphaOfLength(randomInt(10000)); BlobObj blobObj = new BlobObj(testString); ChecksumBlobStoreFormat checksumFormat = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj::fromXContent); - checksumFormat.write(blobObj, blobContainer, "test-path", randomBoolean(), MockBigArrays.NON_RECYCLING_INSTANCE); + checksumFormat.write(blobObj, blobContainer, "test-path", randomBoolean()); assertEquals(checksumFormat.read(blobContainer, "test-path", xContentRegistry()).getText(), testString); randomCorruption(blobContainer, "test-path"); try { diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotInfoBlobSerializationTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotInfoBlobSerializationTests.java index a0e489228a053..2c1e06bf98ab7 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotInfoBlobSerializationTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotInfoBlobSerializationTests.java @@ -9,9 +9,7 @@ package org.elasticsearch.snapshots; import org.elasticsearch.Version; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.support.PlainActionFuture; -import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.repositories.blobstore.BlobStoreRepository; import org.elasticsearch.test.AbstractWireTestCase; @@ -32,18 +30,9 @@ protected SnapshotInfo mutateInstance(SnapshotInfo instance) throws IOException @Override protected SnapshotInfo copyInstance(SnapshotInfo instance, Version version) throws IOException { - final PlainActionFuture future = new PlainActionFuture<>(); - BlobStoreRepository.SNAPSHOT_FORMAT.serialize( - instance, - "test", - randomBoolean(), - BigArrays.NON_RECYCLING_INSTANCE, - bytes -> ActionListener.completeWith( - future, - () -> BlobStoreRepository.SNAPSHOT_FORMAT.deserialize(NamedXContentRegistry.EMPTY, bytes.streamInput()) - ) - ); - return future.actionGet(); + final BytesStreamOutput out = new BytesStreamOutput(); + BlobStoreRepository.SNAPSHOT_FORMAT.serialize(instance, "test", randomBoolean(), out); + return BlobStoreRepository.SNAPSHOT_FORMAT.deserialize(NamedXContentRegistry.EMPTY, out.bytes().streamInput()); } } diff --git a/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java index e73e5625cef52..89a717d5dd836 100644 --- a/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java @@ -373,7 +373,7 @@ protected String initWithSnapshotVersion(String repoName, Path repoPath, Version PlainActionFuture.get(f -> blobStoreRepository.threadPool().generic().execute(ActionRunnable.run(f, () -> BlobStoreRepository.SNAPSHOT_FORMAT.write(downgradedSnapshotInfo, blobStoreRepository.blobStore().blobContainer(blobStoreRepository.basePath()), snapshotInfo.snapshotId().getUUID(), - randomBoolean(), internalCluster().getCurrentMasterNodeInstance(BigArrays.class))))); + randomBoolean())))); return oldVersionSnapshot; } diff --git a/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java b/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java index 639a9e7b08d19..85499c7148f01 100644 --- a/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java +++ b/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java @@ -38,9 +38,7 @@ import org.elasticsearch.repositories.blobstore.BlobStoreRepository; import org.elasticsearch.repositories.fs.FsRepository; -import java.io.IOException; -import java.io.InputStream; -import java.io.UnsupportedEncodingException; +import java.io.*; import java.nio.file.Path; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; @@ -501,6 +499,32 @@ && path().equals(basePath()) == false) { } } + @Override + public OutputStream writeBlob(String blobName, boolean failIfAlreadyExists) throws IOException { + maybeIOExceptionOrBlock(blobName); + if (blockOnWriteShardLevelMeta && blobName.startsWith(BlobStoreRepository.SNAPSHOT_PREFIX) + && path().equals(basePath()) == false) { + blockExecutionAndMaybeWait(blobName); + } + return new FilterOutputStream(super.writeBlob(blobName, failIfAlreadyExists)) { + + @Override + public void write(byte[] b, int off, int len) throws IOException { + super.write(b, off, len); + } + + @Override + public void close() throws IOException { + super.close(); + if (RandomizedContext.current().getRandom().nextBoolean()) { + // for network based repositories, the blob may have been written but we may still + // get an error with the client connection, so an IOException here simulates this + maybeIOExceptionOrBlock(blobName); + } + } + }; + } + @Override public void writeBlobAtomic(final String blobName, final BytesReference bytes, final boolean failIfAlreadyExists) throws IOException { From 998c3693a7ea490ddd19115fa2071e413e143ad0 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 16 Jun 2021 14:03:42 +0200 Subject: [PATCH 02/24] gcs support --- .../blobstore/url/URLBlobContainer.java | 5 +- .../gcs/GoogleCloudStorageBlobContainer.java | 9 +++ .../gcs/GoogleCloudStorageBlobStore.java | 69 +++++++++++++++++++ ...CloudStorageBlobContainerRetriesTests.java | 8 ++- .../common/blobstore/BlobContainer.java | 3 +- .../common/blobstore/fs/FsBlobContainer.java | 13 +++- .../support/FilterBlobContainer.java | 6 +- .../blobstore/ChecksumBlobStoreFormat.java | 4 +- .../gcs/GoogleCloudStorageHttpHandler.java | 6 +- .../snapshots/mockstore/MockRepository.java | 33 ++++----- 10 files changed, 123 insertions(+), 33 deletions(-) diff --git a/modules/repository-url/src/main/java/org/elasticsearch/common/blobstore/url/URLBlobContainer.java b/modules/repository-url/src/main/java/org/elasticsearch/common/blobstore/url/URLBlobContainer.java index 0d46c33095774..0e808ac300db1 100644 --- a/modules/repository-url/src/main/java/org/elasticsearch/common/blobstore/url/URLBlobContainer.java +++ b/modules/repository-url/src/main/java/org/elasticsearch/common/blobstore/url/URLBlobContainer.java @@ -8,6 +8,7 @@ package org.elasticsearch.common.blobstore.url; +import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.BlobMetadata; @@ -123,7 +124,9 @@ public void writeBlob(String blobName, InputStream inputStream, long blobSize, b } @Override - public OutputStream writeBlob(String blobName, boolean failIfAlreadyExists) throws IOException { + public void writeBlob(String blobName, + boolean failIfAlreadyExists, + CheckedConsumer writer) throws IOException { throw new UnsupportedOperationException("URL repository doesn't support this operation"); } diff --git a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainer.java b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainer.java index 74c697a167189..07b6354b7007d 100644 --- a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainer.java +++ b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainer.java @@ -15,9 +15,11 @@ import org.elasticsearch.common.blobstore.DeleteResult; import org.elasticsearch.common.blobstore.support.AbstractBlobContainer; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.core.CheckedConsumer; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.util.Iterator; import java.util.Map; @@ -76,6 +78,13 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea blobStore.writeBlob(buildKey(blobName), bytes, failIfAlreadyExists); } + @Override + public void writeBlob(String blobName, + boolean failIfAlreadyExists, + CheckedConsumer writer) throws IOException { + blobStore.writeBlob(buildKey(blobName), failIfAlreadyExists, writer); + } + @Override public void writeBlobAtomic(String blobName, BytesReference bytes, boolean failIfAlreadyExists) throws IOException { writeBlob(blobName, bytes, failIfAlreadyExists); diff --git a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java index a49a17699ffe4..adb531738058e 100644 --- a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java +++ b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java @@ -22,6 +22,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.BlobMetadata; @@ -37,8 +38,10 @@ import org.elasticsearch.common.unit.ByteSizeValue; import java.io.ByteArrayInputStream; +import java.io.FilterOutputStream; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.nio.ByteBuffer; import java.nio.channels.Channels; import java.nio.channels.WritableByteChannel; @@ -265,6 +268,72 @@ long getLargeBlobThresholdInBytes() { {Storage.BlobWriteOption.doesNotExist(), Storage.BlobWriteOption.md5Match()}; private static final Storage.BlobWriteOption[] OVERWRITE_CHECK_MD5 = {Storage.BlobWriteOption.md5Match()}; + void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) throws IOException { + final BlobInfo blobInfo = BlobInfo.newBuilder(bucketName, blobName).build(); + final Storage.BlobWriteOption[] writeOptions = failIfAlreadyExists ? NO_OVERWRITE_NO_MD5 : OVERWRITE_NO_MD5; + + StorageException storageException = null; + + for (int retry = 0; retry < 3; ++retry) { + try { + final WriteChannel writeChannel = SocketAccess.doPrivilegedIOException(() -> client().writer(blobInfo, writeOptions)); + try (OutputStream out = new FilterOutputStream( + Channels.newOutputStream( + new WritableByteChannel() { + + @SuppressForbidden(reason = "channel is based on a socket") + @Override + public int write(final ByteBuffer src) throws IOException { + return SocketAccess.doPrivilegedIOException(() -> writeChannel.write(src)); + } + + @Override + public boolean isOpen() { + return writeChannel.isOpen(); + } + + @Override + public void close() throws IOException { + SocketAccess.doPrivilegedVoidIOException(writeChannel::close); + } + } + ) + ) { + @Override + public void write(byte[] b, int off, int len) throws IOException { + int written = 0; + while (written < len) { + // at most write the default chunk size in one go to prevent allocating huge buffers in the SDK + // see com.google.cloud.BaseWriteChannel#DEFAULT_CHUNK_SIZE + final int toWrite = Math.min(len - written, 60 * 256 * 1024); + out.write(b, off + written, toWrite); + written += toWrite; + } + } + }) { + writer.accept(out); + } + stats.trackPutOperation(); + return; + } catch (final StorageException se) { + final int errorCode = se.getCode(); + if (errorCode == HTTP_GONE) { + logger.warn(() -> new ParameterizedMessage("Retrying broken resumable upload session for blob {}", blobInfo), se); + storageException = ExceptionsHelper.useOrSuppress(storageException, se); + continue; + } else if (failIfAlreadyExists && errorCode == HTTP_PRECON_FAILED) { + throw new FileAlreadyExistsException(blobInfo.getBlobId().getName(), null, se.getMessage()); + } + if (storageException != null) { + se.addSuppressed(storageException); + } + throw se; + } + } + assert storageException != null; + throw storageException; + } + /** * Uploads a blob using the "resumable upload" method (multiple requests, which * can be independently retried in case of failure, see diff --git a/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainerRetriesTests.java b/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainerRetriesTests.java index 1848e9abb1189..315095a531d2d 100644 --- a/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainerRetriesTests.java +++ b/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainerRetriesTests.java @@ -343,8 +343,12 @@ public void testWriteLargeBlob() throws IOException { final TimeValue readTimeout = allowReadTimeout.get() ? TimeValue.timeValueSeconds(3) : null; final BlobContainer blobContainer = createBlobContainer(nbErrors + 1, readTimeout, null, null); - try (InputStream stream = new InputStreamIndexInput(new ByteArrayIndexInput("desc", data), data.length)) { - blobContainer.writeBlob("write_large_blob", stream, data.length, false); + if (randomBoolean()) { + try (InputStream stream = new InputStreamIndexInput(new ByteArrayIndexInput("desc", data), data.length)) { + blobContainer.writeBlob("write_large_blob", stream, data.length, false); + } + } else { + blobContainer.writeBlob("write_large_blob", false, out -> out.write(data)); } assertThat(countInits.get(), equalTo(0)); diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java b/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java index 5b2ea2174a92f..ecff9058d10d3 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java @@ -9,6 +9,7 @@ package org.elasticsearch.common.blobstore; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.core.CheckedConsumer; import java.io.IOException; import java.io.InputStream; @@ -117,7 +118,7 @@ default void writeBlob(String blobName, BytesReference bytes, boolean failIfAlre writeBlob(blobName, bytes.streamInput(), bytes.length(), failIfAlreadyExists); } - OutputStream writeBlob(String blobName, boolean failIfAlreadyExists) throws IOException; + void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) throws IOException; /** * Reads blob content from a {@link BytesReference} and writes it to the container in a new blob with the given name, diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java b/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java index 4709a0d386b69..81438e54034e4 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java @@ -18,6 +18,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.io.Streams; +import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.internal.io.IOUtils; import java.io.FileNotFoundException; @@ -245,16 +246,22 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea } @Override - public OutputStream writeBlob(String blobName, boolean failIfAlreadyExists) throws IOException { + public void writeBlob(String blobName, + boolean failIfAlreadyExists, + CheckedConsumer writer) throws IOException { final Path file = path.resolve(blobName); try { - return new BlobOutputStream(file); + try (OutputStream out = new BlobOutputStream(file)) { + writer.accept(out); + } } catch (FileAlreadyExistsException faee) { if (failIfAlreadyExists) { throw faee; } deleteBlobsIgnoringIfNotExists(Iterators.single(blobName)); - return new BlobOutputStream(file); + try (OutputStream out = new BlobOutputStream(file)) { + writer.accept(out); + } } } diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/support/FilterBlobContainer.java b/server/src/main/java/org/elasticsearch/common/blobstore/support/FilterBlobContainer.java index 5c5dcf91eee9a..2bd28c2910803 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/support/FilterBlobContainer.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/support/FilterBlobContainer.java @@ -13,6 +13,7 @@ import org.elasticsearch.common.blobstore.BlobPath; import org.elasticsearch.common.blobstore.DeleteResult; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.core.CheckedConsumer; import java.io.IOException; import java.io.InputStream; @@ -63,8 +64,9 @@ public void writeBlob(String blobName, InputStream inputStream, long blobSize, b } @Override - public OutputStream writeBlob(String blobName, boolean failIfAlreadyExists) throws IOException { - return delegate.writeBlob(blobName, failIfAlreadyExists); + public void writeBlob(String blobName, boolean failIfAlreadyExists, + CheckedConsumer writer) throws IOException { + delegate.writeBlob(blobName, failIfAlreadyExists, writer); } @Override diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java index 80f004f21d866..e7bb24b4c3068 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java @@ -277,9 +277,7 @@ private int getAvailable() throws IOException { */ public void write(T obj, BlobContainer blobContainer, String name, boolean compress) throws IOException { final String blobName = blobName(name); - try (OutputStream out = blobContainer.writeBlob(blobName, false)) { - serialize(obj, blobName, compress, out); - } + blobContainer.writeBlob(blobName, false, out -> serialize(obj, blobName, compress, out)); } public void serialize(final T obj, final String blobName, final boolean compress, OutputStream outputStream) throws IOException { diff --git a/test/fixtures/gcs-fixture/src/main/java/fixture/gcs/GoogleCloudStorageHttpHandler.java b/test/fixtures/gcs-fixture/src/main/java/fixture/gcs/GoogleCloudStorageHttpHandler.java index 2258dbd4f4420..a2db70ebbf8e8 100644 --- a/test/fixtures/gcs-fixture/src/main/java/fixture/gcs/GoogleCloudStorageHttpHandler.java +++ b/test/fixtures/gcs-fixture/src/main/java/fixture/gcs/GoogleCloudStorageHttpHandler.java @@ -211,10 +211,12 @@ public void handle(final HttpExchange exchange) throws IOException { blobs.put(blobName, BytesArray.EMPTY); byte[] response = requestBody.utf8ToString().getBytes(UTF_8); - if (Paths.get(blobName).getFileName().toString().startsWith(BlobStoreRepository.UPLOADED_DATA_BLOB_PREFIX) == false) { + final String name = Paths.get(blobName).getFileName().toString(); + if (name.matches(BlobStoreRepository.INDEX_FILE_PREFIX + "\\d+") + || name.equals(BlobStoreRepository.INDEX_LATEST_BLOB)) { final Map parsedBody = XContentHelper.convertToMap(requestBody, false, XContentType.JSON).v2(); assert parsedBody.get("md5Hash") != null - : "file [" + blobName + "] is not a data blob but did not come with a md5 checksum"; + : "file [" + blobName + "] must be written atomically but did not come with a md5 checksum"; } exchange.getResponseHeaders().add("Content-Type", "application/json"); exchange.getResponseHeaders() diff --git a/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java b/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java index 85499c7148f01..b700aced84078 100644 --- a/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java +++ b/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java @@ -24,6 +24,7 @@ import org.elasticsearch.common.blobstore.support.FilterBlobContainer; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Iterators; +import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.PathUtils; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; @@ -38,7 +39,10 @@ import org.elasticsearch.repositories.blobstore.BlobStoreRepository; import org.elasticsearch.repositories.fs.FsRepository; -import java.io.*; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.UnsupportedEncodingException; import java.nio.file.Path; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; @@ -500,29 +504,20 @@ && path().equals(basePath()) == false) { } @Override - public OutputStream writeBlob(String blobName, boolean failIfAlreadyExists) throws IOException { + public void writeBlob(String blobName, + boolean failIfAlreadyExists, + CheckedConsumer writer) throws IOException { maybeIOExceptionOrBlock(blobName); if (blockOnWriteShardLevelMeta && blobName.startsWith(BlobStoreRepository.SNAPSHOT_PREFIX) && path().equals(basePath()) == false) { blockExecutionAndMaybeWait(blobName); } - return new FilterOutputStream(super.writeBlob(blobName, failIfAlreadyExists)) { - - @Override - public void write(byte[] b, int off, int len) throws IOException { - super.write(b, off, len); - } - - @Override - public void close() throws IOException { - super.close(); - if (RandomizedContext.current().getRandom().nextBoolean()) { - // for network based repositories, the blob may have been written but we may still - // get an error with the client connection, so an IOException here simulates this - maybeIOExceptionOrBlock(blobName); - } - } - }; + super.writeBlob(blobName, failIfAlreadyExists, writer); + if (RandomizedContext.current().getRandom().nextBoolean()) { + // for network based repositories, the blob may have been written but we may still + // get an error with the client connection, so an IOException here simulates this + maybeIOExceptionOrBlock(blobName); + } } @Override From 67ad357c053a8d35ad827c12537a5367519b14ba Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 16 Jun 2021 14:07:57 +0200 Subject: [PATCH 03/24] hdfs support --- .../repositories/hdfs/HdfsBlobContainer.java | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) 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 52e951e258c86..a1104189059c8 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 @@ -15,6 +15,7 @@ import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.Options.CreateOpts; import org.apache.hadoop.fs.Path; +import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Nullable; import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.BlobMetadata; @@ -31,6 +32,7 @@ import java.io.FilterInputStream; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.nio.file.FileAlreadyExistsException; import java.nio.file.NoSuchFileException; import java.util.Collections; @@ -156,6 +158,24 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea }); } + @Override + public void writeBlob(String blobName, + boolean failIfAlreadyExists, + CheckedConsumer writer) throws IOException { + Path blob = new Path(path, blobName); + // we pass CREATE, which means it fails if a blob already exists. + final EnumSet flags = failIfAlreadyExists ? EnumSet.of(CreateFlag.CREATE, CreateFlag.SYNC_BLOCK) + : EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE, CreateFlag.SYNC_BLOCK); + store.execute((Operation) fileContext -> { + try (FSDataOutputStream stream = fileContext.create(blob, flags)) { + writer.accept(stream); + } 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); From d7c0223e1f94150619ec349acb36dd79be8cd05a Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 16 Jun 2021 14:59:08 +0200 Subject: [PATCH 04/24] s3 --- .../repositories/s3/S3BlobContainer.java | 114 ++++++++++++++++-- 1 file changed, 103 insertions(+), 11 deletions(-) diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java index 6c096c835d132..3ef2ab3a65172 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java @@ -27,6 +27,9 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.util.SetOnce; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.blobstore.BlobContainer; @@ -45,6 +48,7 @@ import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; @@ -129,6 +133,103 @@ public void writeBlob(String blobName, InputStream inputStream, long blobSize, b }); } + private static final long MIN_MULTIPART_BYTES = MIN_PART_SIZE_USING_MULTIPART.getBytes(); + + @Override + public void writeBlob(String blobName, + boolean failIfAlreadyExists, + CheckedConsumer writer) throws IOException { + final BigArrays bigArrays = BigArrays.NON_RECYCLING_INSTANCE; + try (AmazonS3Reference clientReference = blobStore.clientReference(); + OutputStream out = new OutputStream() { + + private final List parts = new ArrayList<>(); + + private final SetOnce uploadId = new SetOnce<>(); + + long written = 0L; + + private ReleasableBytesStreamOutput buffer = new ReleasableBytesStreamOutput(bigArrays); + + @Override + public void write(int b) throws IOException { + buffer.write(b); + maybeFlushBuffer(); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + buffer.write(b, off, len); + maybeFlushBuffer(); + } + + private void maybeFlushBuffer() throws IOException { + if (buffer.size() >= MIN_MULTIPART_BYTES) { + if (written == 0L) { + uploadId.set(SocketAccess.doPrivileged(() -> + clientReference.client().initiateMultipartUpload(initiateMultiPartUpload(blobName)).getUploadId())); + if (Strings.isEmpty(uploadId.get())) { + throw new IOException("Failed to initialize multipart upload " + blobName); + } + } + flushBuffer(false); + } + } + + private void flushBuffer(boolean lastPart) throws IOException { + final UploadPartRequest uploadRequest = new UploadPartRequest(); + uploadRequest.setBucketName(blobStore.bucket()); + uploadRequest.setKey(blobName); + uploadRequest.setUploadId(uploadId.get()); + uploadRequest.setPartNumber(parts.size() + 1); + uploadRequest.setInputStream(buffer.bytes().streamInput()); + uploadRequest.setRequestMetricCollector(blobStore.multiPartUploadMetricCollector); + + uploadRequest.setPartSize(buffer.size()); + uploadRequest.setLastPart(lastPart); + + final UploadPartResult uploadResponse = + SocketAccess.doPrivileged(() -> clientReference.client().uploadPart(uploadRequest)); + written += uploadRequest.getPartSize(); + parts.add(uploadResponse.getPartETag()); + buffer.close(); + buffer = new ReleasableBytesStreamOutput(bigArrays); + } + + @Override + public void close() throws IOException { + try { + if (written == 0L) { + writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); + } else { + flushBuffer(true); + final CompleteMultipartUploadRequest complRequest = + new CompleteMultipartUploadRequest(blobStore.bucket(), blobName, uploadId.get(), parts); + complRequest.setRequestMetricCollector(blobStore.multiPartUploadMetricCollector); + SocketAccess.doPrivilegedVoid(() -> clientReference.client().completeMultipartUpload(complRequest)); + } + } finally { + buffer.close(); + } + } + }) { + writer.accept(out); + } + } + + private InitiateMultipartUploadRequest initiateMultiPartUpload(String blobName) { + final InitiateMultipartUploadRequest initRequest = new InitiateMultipartUploadRequest(blobStore.bucket(), blobName); + initRequest.setStorageClass(blobStore.getStorageClass()); + initRequest.setCannedACL(blobStore.getCannedACL()); + initRequest.setRequestMetricCollector(blobStore.multiPartUploadMetricCollector); + if (blobStore.serverSideEncryption()) { + final ObjectMetadata md = new ObjectMetadata(); + md.setSSEAlgorithm(ObjectMetadata.AES_256_SERVER_SIDE_ENCRYPTION); + initRequest.setObjectMetadata(md); + } + return initRequest; + } + // package private for testing long getLargeBlobThresholdInBytes() { return blobStore.bufferSizeInBytes(); @@ -389,19 +490,10 @@ void executeMultipartUpload(final S3BlobStore blobStore, final SetOnce uploadId = new SetOnce<>(); final String bucketName = blobStore.bucket(); boolean success = false; - - final InitiateMultipartUploadRequest initRequest = new InitiateMultipartUploadRequest(bucketName, blobName); - initRequest.setStorageClass(blobStore.getStorageClass()); - initRequest.setCannedACL(blobStore.getCannedACL()); - initRequest.setRequestMetricCollector(blobStore.multiPartUploadMetricCollector); - if (blobStore.serverSideEncryption()) { - final ObjectMetadata md = new ObjectMetadata(); - md.setSSEAlgorithm(ObjectMetadata.AES_256_SERVER_SIDE_ENCRYPTION); - initRequest.setObjectMetadata(md); - } try (AmazonS3Reference clientReference = blobStore.clientReference()) { - uploadId.set(SocketAccess.doPrivileged(() -> clientReference.client().initiateMultipartUpload(initRequest).getUploadId())); + uploadId.set(SocketAccess.doPrivileged(() -> + clientReference.client().initiateMultipartUpload(initiateMultiPartUpload(blobName)).getUploadId())); if (Strings.isEmpty(uploadId.get())) { throw new IOException("Failed to initialize multipart upload " + blobName); } From 75fa6967330eeb289cbde727c16f8f42efc95016 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 16 Jun 2021 15:23:56 +0200 Subject: [PATCH 05/24] mostly works --- .../repositories/azure/AzureBlobContainer.java | 12 ++++++++++++ .../fixture/gcs/GoogleCloudStorageHttpHandler.java | 3 +-- .../snapshots/AbstractSnapshotIntegTestCase.java | 1 - .../repositories/encrypted/EncryptedRepository.java | 11 +++++++++++ .../searchablesnapshots/cache/common/TestUtils.java | 7 +++++++ .../testkit/RepositoryAnalysisFailureIT.java | 11 +++++++++++ .../testkit/RepositoryAnalysisSuccessIT.java | 11 +++++++++++ 7 files changed, 53 insertions(+), 3 deletions(-) diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java index 6365b248470b2..48f5b96a2293e 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java @@ -12,6 +12,8 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.core.util.Throwables; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Nullable; import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.BlobMetadata; @@ -22,6 +24,7 @@ import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.nio.file.NoSuchFileException; import java.util.Iterator; import java.util.Map; @@ -99,6 +102,15 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea blobStore.writeBlob(buildKey(blobName), bytes, failIfAlreadyExists); } + @Override + public void writeBlob(String blobName, + boolean failIfAlreadyExists, + CheckedConsumer writer) throws IOException { + final BytesStreamOutput out = new BytesStreamOutput(); + writer.accept(out); + writeBlob(blobName, out.bytes(), failIfAlreadyExists); + } + @Override public DeleteResult delete() throws IOException { return blobStore.deleteBlobDirectory(keyPath); diff --git a/test/fixtures/gcs-fixture/src/main/java/fixture/gcs/GoogleCloudStorageHttpHandler.java b/test/fixtures/gcs-fixture/src/main/java/fixture/gcs/GoogleCloudStorageHttpHandler.java index a2db70ebbf8e8..0d770807a7812 100644 --- a/test/fixtures/gcs-fixture/src/main/java/fixture/gcs/GoogleCloudStorageHttpHandler.java +++ b/test/fixtures/gcs-fixture/src/main/java/fixture/gcs/GoogleCloudStorageHttpHandler.java @@ -212,8 +212,7 @@ public void handle(final HttpExchange exchange) throws IOException { byte[] response = requestBody.utf8ToString().getBytes(UTF_8); final String name = Paths.get(blobName).getFileName().toString(); - if (name.matches(BlobStoreRepository.INDEX_FILE_PREFIX + "\\d+") - || name.equals(BlobStoreRepository.INDEX_LATEST_BLOB)) { + if (name.matches(BlobStoreRepository.INDEX_FILE_PREFIX + "\\d+") || name.equals(BlobStoreRepository.INDEX_LATEST_BLOB)) { final Map parsedBody = XContentHelper.convertToMap(requestBody, false, XContentType.JSON).v2(); assert parsedBody.get("md5Hash") != null : "file [" + blobName + "] must be written atomically but did not come with a md5 checksum"; diff --git a/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java index 89a717d5dd836..67ccc1daf5384 100644 --- a/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java @@ -35,7 +35,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.core.TimeValue; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.xcontent.DeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentBuilder; diff --git a/x-pack/plugin/repository-encrypted/src/main/java/org/elasticsearch/repositories/encrypted/EncryptedRepository.java b/x-pack/plugin/repository-encrypted/src/main/java/org/elasticsearch/repositories/encrypted/EncryptedRepository.java index c70e1cf783c91..91446c8f5948a 100644 --- a/x-pack/plugin/repository-encrypted/src/main/java/org/elasticsearch/repositories/encrypted/EncryptedRepository.java +++ b/x-pack/plugin/repository-encrypted/src/main/java/org/elasticsearch/repositories/encrypted/EncryptedRepository.java @@ -29,10 +29,12 @@ import org.elasticsearch.common.cache.Cache; import org.elasticsearch.common.cache.CacheBuilder; import org.elasticsearch.common.io.Streams; +import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput; import org.elasticsearch.common.settings.SecureString; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.CheckedFunction; import org.elasticsearch.core.Tuple; import org.elasticsearch.indices.recovery.RecoverySettings; @@ -48,6 +50,7 @@ import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.nio.charset.StandardCharsets; import java.nio.file.NoSuchFileException; import java.security.GeneralSecurityException; @@ -622,6 +625,14 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea } } + @Override + public void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) + throws IOException { + final BytesStreamOutput out = new BytesStreamOutput(); + writer.accept(out); + writeBlob(blobName, out.bytes(), failIfAlreadyExists); + } + private ChainingInputStream encryptedInput(InputStream inputStream, SingleUseKey singleUseNonceAndDEK, BytesReference dekIdBytes) throws IOException { return ChainingInputStream.chain( diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/cache/common/TestUtils.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/cache/common/TestUtils.java index 537cc0903b558..8ffdea046d79f 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/cache/common/TestUtils.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/cache/common/TestUtils.java @@ -22,6 +22,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.PathUtilsForTesting; import org.elasticsearch.xpack.searchablesnapshots.cache.blob.BlobStoreCacheService; @@ -32,6 +33,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.nio.channels.FileChannel; import java.nio.file.FileSystem; import java.nio.file.OpenOption; @@ -269,6 +271,11 @@ public void writeBlob(String blobName, InputStream inputStream, long blobSize, b throw unsupportedException(); } + @Override + public void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) { + throw unsupportedException(); + } + @Override public void writeBlobAtomic(String blobName, BytesReference bytes, boolean failIfAlreadyExists) { throw unsupportedException(); diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisFailureIT.java b/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisFailureIT.java index 2837c2f31540f..d6254a72acc2d 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisFailureIT.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisFailureIT.java @@ -17,12 +17,14 @@ import org.elasticsearch.common.blobstore.DeleteResult; import org.elasticsearch.common.blobstore.support.PlainBlobMetadata; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Nullable; import org.elasticsearch.env.Environment; import org.elasticsearch.indices.recovery.RecoverySettings; @@ -42,6 +44,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.nio.file.FileAlreadyExistsException; import java.util.Arrays; import java.util.Collection; @@ -453,6 +456,14 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea writeBlob(blobName, bytes.streamInput(), bytes.length(), failIfAlreadyExists); } + @Override + public void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) + throws IOException { + final BytesStreamOutput out = new BytesStreamOutput(); + writer.accept(out); + writeBlob(blobName, out.bytes(), failIfAlreadyExists); + } + @Override public void writeBlobAtomic(String blobName, BytesReference bytes, boolean failIfAlreadyExists) throws IOException { final StreamInput inputStream; diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisSuccessIT.java b/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisSuccessIT.java index 1134e815f11a2..75ad6247b9e79 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisSuccessIT.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisSuccessIT.java @@ -16,11 +16,13 @@ import org.elasticsearch.common.blobstore.DeleteResult; import org.elasticsearch.common.blobstore.support.PlainBlobMetadata; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.TimeValue; import org.elasticsearch.env.Environment; @@ -40,6 +42,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.util.Collection; import java.util.Iterator; import java.util.List; @@ -326,6 +329,14 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea writeBlob(blobName, bytes.streamInput(), bytes.length(), failIfAlreadyExists); } + @Override + public void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) + throws IOException { + final BytesStreamOutput out = new BytesStreamOutput(); + writer.accept(out); + writeBlob(blobName, out.bytes(), failIfAlreadyExists); + } + @Override public void writeBlobAtomic(String blobName, BytesReference bytes, boolean failIfAlreadyExists) throws IOException { writeBlobAtomic(blobName, bytes.streamInput(), bytes.length(), failIfAlreadyExists); From cf4fa9ee183ccaba4d83f96a80ede427b2bb7bac Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 16 Jun 2021 16:46:52 +0200 Subject: [PATCH 06/24] works --- .../azure/AzureBlobContainer.java | 5 +- .../repositories/azure/AzureBlobStore.java | 79 ++++++++++++++++++- .../repositories/azure/AzureRepository.java | 2 +- .../azure/AzureBlobContainerRetriesTests.java | 3 +- .../repositories/s3/S3BlobContainer.java | 9 ++- .../repositories/s3/S3BlobStore.java | 10 ++- .../repositories/s3/S3Repository.java | 2 +- .../s3/S3BlobContainerRetriesTests.java | 3 +- .../encrypted/EncryptedRepository.java | 9 ++- 9 files changed, 105 insertions(+), 17 deletions(-) diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java index 48f5b96a2293e..2da53eece5d03 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java @@ -12,7 +12,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.core.util.Throwables; -import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Nullable; import org.elasticsearch.common.blobstore.BlobContainer; @@ -106,9 +105,7 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea public void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) throws IOException { - final BytesStreamOutput out = new BytesStreamOutput(); - writer.accept(out); - writeBlob(blobName, out.bytes(), failIfAlreadyExists); + blobStore.writeBlob(buildKey(blobName), failIfAlreadyExists, writer); } @Override diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java index e7683fb959e25..4d91616db0a8d 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java @@ -33,6 +33,9 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.cluster.metadata.RepositoryMetadata; +import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Nullable; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.blobstore.BlobContainer; @@ -53,6 +56,7 @@ import java.io.FilterInputStream; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.net.HttpURLConnection; import java.net.URI; import java.net.URISyntaxException; @@ -82,6 +86,8 @@ public class AzureBlobStore implements BlobStore { private final AzureStorageService service; + private final BigArrays bigArrays; + private final String clientName; private final String container; private final LocationMode locationMode; @@ -90,10 +96,11 @@ public class AzureBlobStore implements BlobStore { private final Stats stats = new Stats(); private final BiConsumer statsConsumer; - public AzureBlobStore(RepositoryMetadata metadata, AzureStorageService service) { + public AzureBlobStore(RepositoryMetadata metadata, AzureStorageService service, BigArrays bigArrays) { this.container = Repository.CONTAINER_SETTING.get(metadata.settings()); this.clientName = Repository.CLIENT_NAME.get(metadata.settings()); this.service = service; + this.bigArrays = bigArrays; // locationMode is set per repository, not per client this.locationMode = Repository.LOCATION_MODE_SETTING.get(metadata.settings()); this.maxSinglePartUploadSize = Repository.MAX_SINGLE_PART_UPLOAD_SIZE_SETTING.get(metadata.settings()); @@ -383,6 +390,76 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea executeSingleUpload(blobName, byteBufferFlux, bytes.length(), failIfAlreadyExists); } + private static final long FLUSH_BUFFER_BYTES = new ByteSizeValue(8, ByteSizeUnit.MB).getBytes(); + + public void writeBlob(String blobName, + boolean failIfAlreadyExists, + CheckedConsumer writer) throws IOException { + SocketAccess.doPrivilegedVoidException(() -> { + final BlobServiceAsyncClient asyncClient = asyncClient(); + final BlobAsyncClient blobAsyncClient = asyncClient.getBlobContainerAsyncClient(container) + .getBlobAsyncClient(blobName); + final BlockBlobAsyncClient blockBlobAsyncClient = blobAsyncClient.getBlockBlobAsyncClient(); + try (OutputStream out = new OutputStream() { + + private final List parts = new ArrayList<>(); + + long written = 0L; + + private ReleasableBytesStreamOutput buffer = new ReleasableBytesStreamOutput(bigArrays); + + @Override + public void write(int b) throws IOException { + buffer.write(b); + maybeFlushBuffer(); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + buffer.write(b, off, len); + maybeFlushBuffer(); + } + + private void maybeFlushBuffer() { + if (buffer.size() >= FLUSH_BUFFER_BYTES) { + flushBuffer(); + } + } + + private void flushBuffer() { + if (buffer.size() == 0) { + return; + } + final Base64.Encoder base64Encoder = Base64.getEncoder().withoutPadding(); + final Base64.Decoder base64UrlDecoder = Base64.getUrlDecoder(); + Flux byteBufferFlux = Flux.fromArray(BytesReference.toByteBuffers(buffer.bytes())); + final String blockId = base64Encoder.encodeToString(base64UrlDecoder.decode(UUIDs.base64UUID())); + blockBlobAsyncClient.stageBlock(blockId, byteBufferFlux, buffer.size()).block(); + written += buffer.size(); + parts.add(blockId); + buffer.close(); + buffer = new ReleasableBytesStreamOutput(bigArrays); + } + + @Override + public void close() { + try { + if (written == 0L) { + writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); + } else { + flushBuffer(); + blockBlobAsyncClient.commitBlockList(parts, failIfAlreadyExists == false).block(); + } + } finally { + buffer.close(); + } + } + }) { + writer.accept(out); + } + }); + } + public void writeBlob(String blobName, InputStream inputStream, long blobSize, boolean failIfAlreadyExists) throws IOException { assert inputStream.markSupported() : "Should not be used with non-mark supporting streams as their retry handling in the SDK is broken"; diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepository.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepository.java index d40f76b7bb138..777d0c14ac50a 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepository.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepository.java @@ -124,7 +124,7 @@ protected BlobStore getBlobStore() { @Override protected AzureBlobStore createBlobStore() { - final AzureBlobStore blobStore = new AzureBlobStore(metadata, storageService); + final AzureBlobStore blobStore = new AzureBlobStore(metadata, storageService, bigArrays); logger.debug(() -> new ParameterizedMessage( "using container [{}], chunk_size [{}], compress [{}], base_path [{}]", diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerRetriesTests.java b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerRetriesTests.java index 29dde60d590e7..b4434ad4f8363 100644 --- a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerRetriesTests.java +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerRetriesTests.java @@ -15,6 +15,7 @@ import fixture.azure.AzureHttpHandler; import org.elasticsearch.cluster.metadata.RepositoryMetadata; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.blobstore.BlobContainer; @@ -172,7 +173,7 @@ int getMaxReadRetries(String clientName) { .put(MAX_SINGLE_PART_UPLOAD_SIZE_SETTING.getKey(), new ByteSizeValue(1, ByteSizeUnit.MB)) .build()); - return new AzureBlobContainer(BlobPath.EMPTY, new AzureBlobStore(repositoryMetadata, service)); + return new AzureBlobContainer(BlobPath.EMPTY, new AzureBlobStore(repositoryMetadata, service, BigArrays.NON_RECYCLING_INSTANCE)); } public void testReadNonexistentBlobThrowsNoSuchFileException() { diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java index 3ef2ab3a65172..0559d0c144e69 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java @@ -133,13 +133,13 @@ public void writeBlob(String blobName, InputStream inputStream, long blobSize, b }); } - private static final long MIN_MULTIPART_BYTES = MIN_PART_SIZE_USING_MULTIPART.getBytes(); + private static final long FLUSH_BUFFER_BYTES = new ByteSizeValue(8, ByteSizeUnit.MB).getBytes(); @Override public void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) throws IOException { - final BigArrays bigArrays = BigArrays.NON_RECYCLING_INSTANCE; + final BigArrays bigArrays = blobStore.bigArrays(); try (AmazonS3Reference clientReference = blobStore.clientReference(); OutputStream out = new OutputStream() { @@ -164,7 +164,7 @@ public void write(byte[] b, int off, int len) throws IOException { } private void maybeFlushBuffer() throws IOException { - if (buffer.size() >= MIN_MULTIPART_BYTES) { + if (buffer.size() >= FLUSH_BUFFER_BYTES) { if (written == 0L) { uploadId.set(SocketAccess.doPrivileged(() -> clientReference.client().initiateMultipartUpload(initiateMultiPartUpload(blobName)).getUploadId())); @@ -177,6 +177,9 @@ private void maybeFlushBuffer() throws IOException { } private void flushBuffer(boolean lastPart) throws IOException { + if (buffer.size() == 0) { + return; + } final UploadPartRequest uploadRequest = new UploadPartRequest(); uploadRequest.setBucketName(blobStore.bucket()); uploadRequest.setKey(blobName); diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobStore.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobStore.java index a3279c1ef4976..90a86c4910ba5 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobStore.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobStore.java @@ -22,6 +22,7 @@ import org.elasticsearch.common.blobstore.BlobStore; import org.elasticsearch.common.blobstore.BlobStoreException; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; import java.io.IOException; import java.util.HashMap; @@ -35,6 +36,8 @@ class S3BlobStore implements BlobStore { private final S3Service service; + private final BigArrays bigArrays; + private final String bucket; private final ByteSizeValue bufferSize; @@ -56,8 +59,9 @@ class S3BlobStore implements BlobStore { S3BlobStore(S3Service service, String bucket, boolean serverSideEncryption, ByteSizeValue bufferSize, String cannedACL, String storageClass, - RepositoryMetadata repositoryMetadata) { + RepositoryMetadata repositoryMetadata, BigArrays bigArrays) { this.service = service; + this.bigArrays = bigArrays; this.bucket = bucket; this.serverSideEncryption = serverSideEncryption; this.bufferSize = bufferSize; @@ -136,6 +140,10 @@ public String bucket() { return bucket; } + public BigArrays bigArrays() { + return bigArrays; + } + public boolean serverSideEncryption() { return serverSideEncryption; } diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java index c5324f8572648..5df64cc38ca0f 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java @@ -308,7 +308,7 @@ private static BlobPath buildBasePath(RepositoryMetadata metadata) { @Override protected S3BlobStore createBlobStore() { - return new S3BlobStore(service, bucket, serverSideEncryption, bufferSize, cannedACL, storageClass, metadata); + return new S3BlobStore(service, bucket, serverSideEncryption, bufferSize, cannedACL, storageClass, metadata, bigArrays); } // only use for testing diff --git a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java index 5505cd592a8e6..3adc093b5a384 100644 --- a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java +++ b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java @@ -12,6 +12,7 @@ import com.amazonaws.util.Base16; import org.apache.http.HttpStatus; import org.elasticsearch.cluster.metadata.RepositoryMetadata; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.common.blobstore.BlobContainer; @@ -126,7 +127,7 @@ protected BlobContainer createBlobContainer(final @Nullable Integer maxRetries, bufferSize == null ? S3Repository.BUFFER_SIZE_SETTING.getDefault(Settings.EMPTY) : bufferSize, S3Repository.CANNED_ACL_SETTING.getDefault(Settings.EMPTY), S3Repository.STORAGE_CLASS_SETTING.getDefault(Settings.EMPTY), - repositoryMetadata)) { + repositoryMetadata, BigArrays.NON_RECYCLING_INSTANCE)) { @Override public InputStream readBlob(String blobName) throws IOException { return new AssertingInputStream(super.readBlob(blobName), blobName); diff --git a/x-pack/plugin/repository-encrypted/src/main/java/org/elasticsearch/repositories/encrypted/EncryptedRepository.java b/x-pack/plugin/repository-encrypted/src/main/java/org/elasticsearch/repositories/encrypted/EncryptedRepository.java index 91446c8f5948a..f4e64d9a79c66 100644 --- a/x-pack/plugin/repository-encrypted/src/main/java/org/elasticsearch/repositories/encrypted/EncryptedRepository.java +++ b/x-pack/plugin/repository-encrypted/src/main/java/org/elasticsearch/repositories/encrypted/EncryptedRepository.java @@ -29,7 +29,6 @@ import org.elasticsearch.common.cache.Cache; import org.elasticsearch.common.cache.CacheBuilder; import org.elasticsearch.common.io.Streams; -import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput; import org.elasticsearch.common.settings.SecureString; import org.elasticsearch.common.util.BigArrays; @@ -628,9 +627,11 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea @Override public void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) throws IOException { - final BytesStreamOutput out = new BytesStreamOutput(); - writer.accept(out); - writeBlob(blobName, out.bytes(), failIfAlreadyExists); + // TODO: this is just a stop-gap solution for until we have an encrypted output stream wrapper + try (ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(bigArrays)) { + writer.accept(out); + writeBlob(blobName, out.bytes(), failIfAlreadyExists); + } } private ChainingInputStream encryptedInput(InputStream inputStream, SingleUseKey singleUseNonceAndDEK, BytesReference dekIdBytes) From 206ee12a42068ac51b9902c1b3234baddea00ad6 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 16 Jun 2021 17:44:56 +0200 Subject: [PATCH 07/24] bck --- .../repositories/azure/AzureBlobStore.java | 27 ++-------- .../repositories/s3/S3BlobContainer.java | 29 ++-------- .../blobstore/ChunkedBlobOutputStream.java | 53 +++++++++++++++++++ 3 files changed, 61 insertions(+), 48 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java index 4d91616db0a8d..9393b239a3625 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java @@ -49,6 +49,7 @@ import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.repositories.azure.AzureRepository.Repository; +import org.elasticsearch.repositories.blobstore.ChunkedBlobOutputStream; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; import reactor.core.scheduler.Schedulers; @@ -400,27 +401,10 @@ public void writeBlob(String blobName, final BlobAsyncClient blobAsyncClient = asyncClient.getBlobContainerAsyncClient(container) .getBlobAsyncClient(blobName); final BlockBlobAsyncClient blockBlobAsyncClient = blobAsyncClient.getBlockBlobAsyncClient(); - try (OutputStream out = new OutputStream() { - - private final List parts = new ArrayList<>(); - - long written = 0L; - - private ReleasableBytesStreamOutput buffer = new ReleasableBytesStreamOutput(bigArrays); - - @Override - public void write(int b) throws IOException { - buffer.write(b); - maybeFlushBuffer(); - } + try (OutputStream out = new ChunkedBlobOutputStream(bigArrays) { @Override - public void write(byte[] b, int off, int len) throws IOException { - buffer.write(b, off, len); - maybeFlushBuffer(); - } - - private void maybeFlushBuffer() { + protected void maybeFlushBuffer() { if (buffer.size() >= FLUSH_BUFFER_BYTES) { flushBuffer(); } @@ -435,10 +419,7 @@ private void flushBuffer() { Flux byteBufferFlux = Flux.fromArray(BytesReference.toByteBuffers(buffer.bytes())); final String blockId = base64Encoder.encodeToString(base64UrlDecoder.decode(UUIDs.base64UUID())); blockBlobAsyncClient.stageBlock(blockId, byteBufferFlux, buffer.size()).block(); - written += buffer.size(); - parts.add(blockId); - buffer.close(); - buffer = new ReleasableBytesStreamOutput(bigArrays); + finishPart(blockId); } @Override diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java index 0559d0c144e69..cf35ad6b117d0 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java @@ -28,7 +28,6 @@ import org.apache.lucene.util.SetOnce; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Nullable; import org.elasticsearch.common.Strings; @@ -44,6 +43,7 @@ import org.elasticsearch.core.Tuple; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.repositories.blobstore.ChunkedBlobOutputStream; import java.io.ByteArrayInputStream; import java.io.IOException; @@ -139,31 +139,13 @@ public void writeBlob(String blobName, InputStream inputStream, long blobSize, b public void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) throws IOException { - final BigArrays bigArrays = blobStore.bigArrays(); try (AmazonS3Reference clientReference = blobStore.clientReference(); - OutputStream out = new OutputStream() { - - private final List parts = new ArrayList<>(); + OutputStream out = new ChunkedBlobOutputStream(blobStore.bigArrays()) { private final SetOnce uploadId = new SetOnce<>(); - long written = 0L; - - private ReleasableBytesStreamOutput buffer = new ReleasableBytesStreamOutput(bigArrays); - - @Override - public void write(int b) throws IOException { - buffer.write(b); - maybeFlushBuffer(); - } - @Override - public void write(byte[] b, int off, int len) throws IOException { - buffer.write(b, off, len); - maybeFlushBuffer(); - } - - private void maybeFlushBuffer() throws IOException { + protected void maybeFlushBuffer() throws IOException { if (buffer.size() >= FLUSH_BUFFER_BYTES) { if (written == 0L) { uploadId.set(SocketAccess.doPrivileged(() -> @@ -193,10 +175,7 @@ private void flushBuffer(boolean lastPart) throws IOException { final UploadPartResult uploadResponse = SocketAccess.doPrivileged(() -> clientReference.client().uploadPart(uploadRequest)); - written += uploadRequest.getPartSize(); - parts.add(uploadResponse.getPartETag()); - buffer.close(); - buffer = new ReleasableBytesStreamOutput(bigArrays); + finishPart(uploadResponse.getPartETag()); } @Override diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java new file mode 100644 index 0000000000000..459ac43e62b2b --- /dev/null +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java @@ -0,0 +1,53 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ +package org.elasticsearch.repositories.blobstore; + +import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput; +import org.elasticsearch.common.util.BigArrays; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.List; + +public abstract class ChunkedBlobOutputStream extends OutputStream { + + protected final List parts = new ArrayList<>(); + + protected ReleasableBytesStreamOutput buffer; + + private final BigArrays bigArrays; + + protected long written = 0L; + + protected ChunkedBlobOutputStream(BigArrays bigArrays) { + this.bigArrays = bigArrays; + buffer = new ReleasableBytesStreamOutput(bigArrays); + } + + @Override + public void write(int b) throws IOException { + buffer.write(b); + maybeFlushBuffer(); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + buffer.write(b, off, len); + maybeFlushBuffer(); + } + + protected final void finishPart(T partId) { + written += buffer.size(); + parts.add(partId); + buffer.close(); + buffer = new ReleasableBytesStreamOutput(bigArrays); + } + + protected abstract void maybeFlushBuffer() throws IOException; +} From 1f46a8bb4a586e796c54f83d395bb3b804d46d50 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 18 Jun 2021 13:27:02 +0200 Subject: [PATCH 08/24] fix cs --- .../org/elasticsearch/repositories/azure/AzureBlobStore.java | 1 - .../java/org/elasticsearch/repositories/s3/S3BlobContainer.java | 1 - 2 files changed, 2 deletions(-) diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java index 9393b239a3625..b6af3766c4018 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java @@ -33,7 +33,6 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.cluster.metadata.RepositoryMetadata; -import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Nullable; diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java index cf35ad6b117d0..749bab4d04e63 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java @@ -27,7 +27,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.util.SetOnce; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput; import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Nullable; import org.elasticsearch.common.Strings; From 6d7a2df2ecadf817c231f73a63d7307276844d96 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 18 Jun 2021 13:50:02 +0200 Subject: [PATCH 09/24] drier --- .../repositories/azure/AzureBlobStore.java | 23 ++--- .../repositories/s3/S3BlobContainer.java | 88 ++++++++++--------- .../blobstore/ChunkedBlobOutputStream.java | 10 +++ 3 files changed, 70 insertions(+), 51 deletions(-) diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java index b6af3766c4018..313ff5213b0ca 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java @@ -390,8 +390,6 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea executeSingleUpload(blobName, byteBufferFlux, bytes.length(), failIfAlreadyExists); } - private static final long FLUSH_BUFFER_BYTES = new ByteSizeValue(8, ByteSizeUnit.MB).getBytes(); - public void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) throws IOException { @@ -400,7 +398,11 @@ public void writeBlob(String blobName, final BlobAsyncClient blobAsyncClient = asyncClient.getBlobContainerAsyncClient(container) .getBlobAsyncClient(blobName); final BlockBlobAsyncClient blockBlobAsyncClient = blobAsyncClient.getBlockBlobAsyncClient(); - try (OutputStream out = new ChunkedBlobOutputStream(bigArrays) { + try (ChunkedBlobOutputStream out = new ChunkedBlobOutputStream<>(bigArrays) { + + private final Base64.Encoder base64Encoder = Base64.getEncoder().withoutPadding(); + + private final Base64.Decoder base64UrlDecoder = Base64.getUrlDecoder(); @Override protected void maybeFlushBuffer() { @@ -413,8 +415,6 @@ private void flushBuffer() { if (buffer.size() == 0) { return; } - final Base64.Encoder base64Encoder = Base64.getEncoder().withoutPadding(); - final Base64.Decoder base64UrlDecoder = Base64.getUrlDecoder(); Flux byteBufferFlux = Flux.fromArray(BytesReference.toByteBuffers(buffer.bytes())); final String blockId = base64Encoder.encodeToString(base64UrlDecoder.decode(UUIDs.base64UUID())); blockBlobAsyncClient.stageBlock(blockId, byteBufferFlux, buffer.size()).block(); @@ -424,11 +424,13 @@ private void flushBuffer() { @Override public void close() { try { - if (written == 0L) { - writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); - } else { - flushBuffer(); - blockBlobAsyncClient.commitBlockList(parts, failIfAlreadyExists == false).block(); + if (successful) { + if (written == 0L) { + writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); + } else { + flushBuffer(); + blockBlobAsyncClient.commitBlockList(parts, failIfAlreadyExists == false).block(); + } } } finally { buffer.close(); @@ -436,6 +438,7 @@ public void close() { } }) { writer.accept(out); + out.markSuccess(); } }); } diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java index 749bab4d04e63..d45c1852842c8 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java @@ -132,14 +132,12 @@ public void writeBlob(String blobName, InputStream inputStream, long blobSize, b }); } - private static final long FLUSH_BUFFER_BYTES = new ByteSizeValue(8, ByteSizeUnit.MB).getBytes(); - @Override public void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) throws IOException { try (AmazonS3Reference clientReference = blobStore.clientReference(); - OutputStream out = new ChunkedBlobOutputStream(blobStore.bigArrays()) { + ChunkedBlobOutputStream out = new ChunkedBlobOutputStream<>(blobStore.bigArrays()) { private final SetOnce uploadId = new SetOnce<>(); @@ -161,17 +159,8 @@ private void flushBuffer(boolean lastPart) throws IOException { if (buffer.size() == 0) { return; } - final UploadPartRequest uploadRequest = new UploadPartRequest(); - uploadRequest.setBucketName(blobStore.bucket()); - uploadRequest.setKey(blobName); - uploadRequest.setUploadId(uploadId.get()); - uploadRequest.setPartNumber(parts.size() + 1); - uploadRequest.setInputStream(buffer.bytes().streamInput()); - uploadRequest.setRequestMetricCollector(blobStore.multiPartUploadMetricCollector); - - uploadRequest.setPartSize(buffer.size()); - uploadRequest.setLastPart(lastPart); - + final UploadPartRequest uploadRequest = createPartUploadRequest( + buffer.bytes().streamInput(), uploadId.get(), parts.size() + 1, blobName, buffer.size(), lastPart); final UploadPartResult uploadResponse = SocketAccess.doPrivileged(() -> clientReference.client().uploadPart(uploadRequest)); finishPart(uploadResponse.getPartETag()); @@ -180,14 +169,19 @@ private void flushBuffer(boolean lastPart) throws IOException { @Override public void close() throws IOException { try { - if (written == 0L) { - writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); - } else { - flushBuffer(true); - final CompleteMultipartUploadRequest complRequest = - new CompleteMultipartUploadRequest(blobStore.bucket(), blobName, uploadId.get(), parts); - complRequest.setRequestMetricCollector(blobStore.multiPartUploadMetricCollector); - SocketAccess.doPrivilegedVoid(() -> clientReference.client().completeMultipartUpload(complRequest)); + final String bucketName = blobStore.bucket(); + if (successful) { + if (written == 0L) { + writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); + } else { + flushBuffer(true); + final CompleteMultipartUploadRequest complRequest = + new CompleteMultipartUploadRequest(bucketName, blobName, uploadId.get(), parts); + complRequest.setRequestMetricCollector(blobStore.multiPartUploadMetricCollector); + SocketAccess.doPrivilegedVoid(() -> clientReference.client().completeMultipartUpload(complRequest)); + } + } else if (Strings.hasText(uploadId.get())) { + abortMultiPartUpload(uploadId.get(), blobName); } } finally { buffer.close(); @@ -195,6 +189,33 @@ public void close() throws IOException { } }) { writer.accept(out); + out.markSuccess(); + } + } + + private UploadPartRequest createPartUploadRequest(InputStream stream, + String uploadId, + int number, + String blobName, + long size, + boolean lastPart) { + final UploadPartRequest uploadRequest = new UploadPartRequest(); + uploadRequest.setBucketName(blobStore.bucket()); + uploadRequest.setKey(blobName); + uploadRequest.setUploadId(uploadId); + uploadRequest.setPartNumber(number); + uploadRequest.setInputStream(stream); + uploadRequest.setRequestMetricCollector(blobStore.multiPartUploadMetricCollector); + uploadRequest.setPartSize(size); + uploadRequest.setLastPart(lastPart); + return uploadRequest; + } + + private void abortMultiPartUpload(String uploadId, String blobName) { + final AbortMultipartUploadRequest abortRequest = + new AbortMultipartUploadRequest(blobStore.bucket(), blobName, uploadId); + try (AmazonS3Reference clientReference = blobStore.clientReference()) { + SocketAccess.doPrivilegedVoid(() -> clientReference.client().abortMultipartUpload(abortRequest)); } } @@ -483,21 +504,9 @@ void executeMultipartUpload(final S3BlobStore blobStore, long bytesCount = 0; for (int i = 1; i <= nbParts; i++) { - final UploadPartRequest uploadRequest = new UploadPartRequest(); - uploadRequest.setBucketName(bucketName); - uploadRequest.setKey(blobName); - uploadRequest.setUploadId(uploadId.get()); - uploadRequest.setPartNumber(i); - uploadRequest.setInputStream(input); - uploadRequest.setRequestMetricCollector(blobStore.multiPartUploadMetricCollector); - - if (i < nbParts) { - uploadRequest.setPartSize(partSize); - uploadRequest.setLastPart(false); - } else { - uploadRequest.setPartSize(lastPartSize); - uploadRequest.setLastPart(true); - } + final boolean lastPart = i == nbParts; + final UploadPartRequest uploadRequest = + createPartUploadRequest(input, uploadId.get(), i, blobName, lastPart ? lastPartSize : partSize, lastPart); bytesCount += uploadRequest.getPartSize(); final UploadPartResult uploadResponse = SocketAccess.doPrivileged(() -> clientReference.client().uploadPart(uploadRequest)); @@ -519,10 +528,7 @@ void executeMultipartUpload(final S3BlobStore blobStore, throw new IOException("Unable to upload object [" + blobName + "] using multipart upload", e); } finally { if ((success == false) && Strings.hasLength(uploadId.get())) { - final AbortMultipartUploadRequest abortRequest = new AbortMultipartUploadRequest(bucketName, blobName, uploadId.get()); - try (AmazonS3Reference clientReference = blobStore.clientReference()) { - SocketAccess.doPrivilegedVoid(() -> clientReference.client().abortMultipartUpload(abortRequest)); - } + abortMultiPartUpload(uploadId.get(), blobName); } } } diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java index 459ac43e62b2b..31aca12119b85 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java @@ -8,6 +8,8 @@ package org.elasticsearch.repositories.blobstore; import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import java.io.IOException; @@ -17,12 +19,16 @@ public abstract class ChunkedBlobOutputStream extends OutputStream { + public static final long FLUSH_BUFFER_BYTES = new ByteSizeValue(8, ByteSizeUnit.MB).getBytes(); + protected final List parts = new ArrayList<>(); protected ReleasableBytesStreamOutput buffer; private final BigArrays bigArrays; + protected boolean successful = false; + protected long written = 0L; protected ChunkedBlobOutputStream(BigArrays bigArrays) { @@ -42,6 +48,10 @@ public void write(byte[] b, int off, int len) throws IOException { maybeFlushBuffer(); } + public void markSuccess() { + this.successful = true; + } + protected final void finishPart(T partId) { written += buffer.size(); parts.add(partId); From 0d67021221e24d1575f24d7454c2465d99fb628c Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 18 Jun 2021 14:03:30 +0200 Subject: [PATCH 10/24] nicer --- .../repositories/azure/AzureBlobStore.java | 24 +++++++++++-------- .../repositories/s3/S3BlobContainer.java | 1 + .../blobstore/ChunkedBlobOutputStream.java | 5 +++- 3 files changed, 19 insertions(+), 11 deletions(-) diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java index 313ff5213b0ca..88f5dd732651d 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java @@ -400,10 +400,6 @@ public void writeBlob(String blobName, final BlockBlobAsyncClient blockBlobAsyncClient = blobAsyncClient.getBlockBlobAsyncClient(); try (ChunkedBlobOutputStream out = new ChunkedBlobOutputStream<>(bigArrays) { - private final Base64.Encoder base64Encoder = Base64.getEncoder().withoutPadding(); - - private final Base64.Decoder base64UrlDecoder = Base64.getUrlDecoder(); - @Override protected void maybeFlushBuffer() { if (buffer.size() >= FLUSH_BUFFER_BYTES) { @@ -415,9 +411,12 @@ private void flushBuffer() { if (buffer.size() == 0) { return; } - Flux byteBufferFlux = Flux.fromArray(BytesReference.toByteBuffers(buffer.bytes())); - final String blockId = base64Encoder.encodeToString(base64UrlDecoder.decode(UUIDs.base64UUID())); - blockBlobAsyncClient.stageBlock(blockId, byteBufferFlux, buffer.size()).block(); + final String blockId = makeMultipartBlockId(); + blockBlobAsyncClient.stageBlock( + blockId, + Flux.fromArray(BytesReference.toByteBuffers(buffer.bytes())), + buffer.size() + ).block(); finishPart(blockId); } @@ -499,13 +498,11 @@ private void executeMultipartUpload(String blobName, InputStream inputStream, lo assert blobSize == (((nbParts - 1) * partSize) + lastPartSize) : "blobSize does not match multipart sizes"; final List blockIds = new ArrayList<>(nbParts); - final Base64.Encoder base64Encoder = Base64.getEncoder().withoutPadding(); - final Base64.Decoder base64UrlDecoder = Base64.getUrlDecoder(); for (int i = 0; i < nbParts; i++) { final long length = i < nbParts - 1 ? partSize : lastPartSize; Flux byteBufferFlux = convertStreamToByteBuffer(inputStream, length, DEFAULT_UPLOAD_BUFFERS_SIZE); - final String blockId = base64Encoder.encodeToString(base64UrlDecoder.decode(UUIDs.base64UUID())); + final String blockId = makeMultipartBlockId(); blockBlobAsyncClient.stageBlock(blockId, byteBufferFlux, length).block(); blockIds.add(blockId); } @@ -514,6 +511,13 @@ private void executeMultipartUpload(String blobName, InputStream inputStream, lo }); } + private static final Base64.Encoder base64Encoder = Base64.getEncoder().withoutPadding(); + private static final Base64.Decoder base64UrlDecoder = Base64.getUrlDecoder(); + + private String makeMultipartBlockId() { + return base64Encoder.encodeToString(base64UrlDecoder.decode(UUIDs.base64UUID())); + } + /** * Converts the provided input stream into a Flux of ByteBuffer. To avoid having large amounts of outstanding * memory this Flux reads the InputStream into ByteBuffers of {@code chunkSize} size. diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java index d45c1852842c8..a9bd35778fc3b 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java @@ -159,6 +159,7 @@ private void flushBuffer(boolean lastPart) throws IOException { if (buffer.size() == 0) { return; } + assert lastPart == false || successful : "must only write last part if successful"; final UploadPartRequest uploadRequest = createPartUploadRequest( buffer.bytes().streamInput(), uploadId.get(), parts.size() + 1, blobName, buffer.size(), lastPart); final UploadPartResult uploadResponse = diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java index 31aca12119b85..553acaae8d260 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java @@ -56,7 +56,10 @@ protected final void finishPart(T partId) { written += buffer.size(); parts.add(partId); buffer.close(); - buffer = new ReleasableBytesStreamOutput(bigArrays); + if (successful = false) { + // only need a new buffer if we're not done yet + buffer = new ReleasableBytesStreamOutput(bigArrays); + } } protected abstract void maybeFlushBuffer() throws IOException; From 3a1cc7d9dec7555e7a1e265784f4ac3211370901 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 18 Jun 2021 14:18:04 +0200 Subject: [PATCH 11/24] cleanup --- .../repositories/azure/AzureBlobStore.java | 26 +++++------ .../repositories/s3/S3BlobContainer.java | 44 +++++++++---------- .../blobstore/ChunkedBlobOutputStream.java | 25 +++++++++-- 3 files changed, 52 insertions(+), 43 deletions(-) diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java index 88f5dd732651d..2a8cb2c903c53 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java @@ -394,10 +394,8 @@ public void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) throws IOException { SocketAccess.doPrivilegedVoidException(() -> { - final BlobServiceAsyncClient asyncClient = asyncClient(); - final BlobAsyncClient blobAsyncClient = asyncClient.getBlobContainerAsyncClient(container) - .getBlobAsyncClient(blobName); - final BlockBlobAsyncClient blockBlobAsyncClient = blobAsyncClient.getBlockBlobAsyncClient(); + final BlockBlobAsyncClient blockBlobAsyncClient = asyncClient().getBlobContainerAsyncClient(container) + .getBlobAsyncClient(blobName).getBlockBlobAsyncClient(); try (ChunkedBlobOutputStream out = new ChunkedBlobOutputStream<>(bigArrays) { @Override @@ -421,18 +419,16 @@ private void flushBuffer() { } @Override - public void close() { - try { - if (successful) { - if (written == 0L) { - writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); - } else { - flushBuffer(); - blockBlobAsyncClient.commitBlockList(parts, failIfAlreadyExists == false).block(); - } + protected void doClose() { + if (successful) { + if (written == 0L) { + writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); + } else { + flushBuffer(); + blockBlobAsyncClient.commitBlockList(parts, failIfAlreadyExists == false).block(); } - } finally { - buffer.close(); + } else { + // TODO: here and in multi-part upload, should we clean up uploaded blobs? } } }) { diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java index a9bd35778fc3b..f166d72b2d08d 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java @@ -144,13 +144,6 @@ public void writeBlob(String blobName, @Override protected void maybeFlushBuffer() throws IOException { if (buffer.size() >= FLUSH_BUFFER_BYTES) { - if (written == 0L) { - uploadId.set(SocketAccess.doPrivileged(() -> - clientReference.client().initiateMultipartUpload(initiateMultiPartUpload(blobName)).getUploadId())); - if (Strings.isEmpty(uploadId.get())) { - throw new IOException("Failed to initialize multipart upload " + blobName); - } - } flushBuffer(false); } } @@ -159,6 +152,14 @@ private void flushBuffer(boolean lastPart) throws IOException { if (buffer.size() == 0) { return; } + if (written == 0L) { + assert lastPart == false : "use single part upload if there's only a single part"; + uploadId.set(SocketAccess.doPrivileged(() -> + clientReference.client().initiateMultipartUpload(initiateMultiPartUpload(blobName)).getUploadId())); + if (Strings.isEmpty(uploadId.get())) { + throw new IOException("Failed to initialize multipart upload " + blobName); + } + } assert lastPart == false || successful : "must only write last part if successful"; final UploadPartRequest uploadRequest = createPartUploadRequest( buffer.bytes().streamInput(), uploadId.get(), parts.size() + 1, blobName, buffer.size(), lastPart); @@ -168,24 +169,19 @@ private void flushBuffer(boolean lastPart) throws IOException { } @Override - public void close() throws IOException { - try { - final String bucketName = blobStore.bucket(); - if (successful) { - if (written == 0L) { - writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); - } else { - flushBuffer(true); - final CompleteMultipartUploadRequest complRequest = - new CompleteMultipartUploadRequest(bucketName, blobName, uploadId.get(), parts); - complRequest.setRequestMetricCollector(blobStore.multiPartUploadMetricCollector); - SocketAccess.doPrivilegedVoid(() -> clientReference.client().completeMultipartUpload(complRequest)); - } - } else if (Strings.hasText(uploadId.get())) { - abortMultiPartUpload(uploadId.get(), blobName); + protected void doClose() throws IOException { + if (successful) { + if (written == 0L) { + writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); + } else { + flushBuffer(true); + final CompleteMultipartUploadRequest complRequest = + new CompleteMultipartUploadRequest(blobStore.bucket(), blobName, uploadId.get(), parts); + complRequest.setRequestMetricCollector(blobStore.multiPartUploadMetricCollector); + SocketAccess.doPrivilegedVoid(() -> clientReference.client().completeMultipartUpload(complRequest)); } - } finally { - buffer.close(); + } else if (Strings.hasText(uploadId.get())) { + abortMultiPartUpload(uploadId.get(), blobName); } } }) { diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java index 553acaae8d260..7daa8e6f49260 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java @@ -11,6 +11,7 @@ import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.core.Releasables; import java.io.IOException; import java.io.OutputStream; @@ -19,14 +20,17 @@ public abstract class ChunkedBlobOutputStream extends OutputStream { + /** + * We buffer 8MB before flushing to storage. + */ public static final long FLUSH_BUFFER_BYTES = new ByteSizeValue(8, ByteSizeUnit.MB).getBytes(); protected final List parts = new ArrayList<>(); - protected ReleasableBytesStreamOutput buffer; - private final BigArrays bigArrays; + protected ReleasableBytesStreamOutput buffer; + protected boolean successful = false; protected long written = 0L; @@ -48,6 +52,15 @@ public void write(byte[] b, int off, int len) throws IOException { maybeFlushBuffer(); } + @Override + public void close() throws IOException { + try { + doClose(); + } finally { + Releasables.close(buffer); + } + } + public void markSuccess() { this.successful = true; } @@ -56,11 +69,15 @@ protected final void finishPart(T partId) { written += buffer.size(); parts.add(partId); buffer.close(); - if (successful = false) { - // only need a new buffer if we're not done yet + // only need a new buffer if we're not done yet + if (successful) { + buffer = null; + } else { buffer = new ReleasableBytesStreamOutput(bigArrays); } } protected abstract void maybeFlushBuffer() throws IOException; + + protected abstract void doClose() throws IOException; } From 1c8084d494b8fbc21b1f68ffc8e1bb68d6edfd3c Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 18 Jun 2021 16:26:27 +0200 Subject: [PATCH 12/24] stop materializing full messages --- .../blobstore/url/URLBlobContainer.java | 1 + .../azure/AzureBlobContainer.java | 1 + .../repositories/azure/AzureBlobStore.java | 4 +- .../gcs/GoogleCloudStorageBlobContainer.java | 1 + ...CloudStorageBlobContainerRetriesTests.java | 2 +- .../repositories/hdfs/HdfsBlobContainer.java | 36 +++++-- .../repositories/s3/S3BlobContainer.java | 5 +- .../s3/S3BlobContainerRetriesTests.java | 100 ++++++++++++++++++ .../common/blobstore/BlobContainer.java | 3 +- .../common/blobstore/fs/FsBlobContainer.java | 23 ++++ .../support/FilterBlobContainer.java | 4 +- .../blobstore/BlobStoreRepository.java | 32 +++--- .../blobstore/ChecksumBlobStoreFormat.java | 2 +- .../blobstore/ChunkedBlobOutputStream.java | 10 +- .../snapshots/mockstore/MockRepository.java | 52 +++++---- .../encrypted/EncryptedRepository.java | 14 ++- .../cache/common/TestUtils.java | 7 +- .../testkit/RepositoryAnalysisFailureIT.java | 14 ++- .../testkit/RepositoryAnalysisSuccessIT.java | 14 ++- 19 files changed, 253 insertions(+), 72 deletions(-) diff --git a/modules/repository-url/src/main/java/org/elasticsearch/common/blobstore/url/URLBlobContainer.java b/modules/repository-url/src/main/java/org/elasticsearch/common/blobstore/url/URLBlobContainer.java index 0e808ac300db1..db0b92a894053 100644 --- a/modules/repository-url/src/main/java/org/elasticsearch/common/blobstore/url/URLBlobContainer.java +++ b/modules/repository-url/src/main/java/org/elasticsearch/common/blobstore/url/URLBlobContainer.java @@ -126,6 +126,7 @@ public void writeBlob(String blobName, InputStream inputStream, long blobSize, b @Override public void writeBlob(String blobName, boolean failIfAlreadyExists, + boolean atomic, CheckedConsumer writer) throws IOException { throw new UnsupportedOperationException("URL repository doesn't support this operation"); } diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java index 2da53eece5d03..005c1027c9310 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java @@ -104,6 +104,7 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea @Override public void writeBlob(String blobName, boolean failIfAlreadyExists, + boolean atomic, CheckedConsumer writer) throws IOException { blobStore.writeBlob(buildKey(blobName), failIfAlreadyExists, writer); } diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java index 2a8cb2c903c53..c4958d1db67c0 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java @@ -396,11 +396,11 @@ public void writeBlob(String blobName, SocketAccess.doPrivilegedVoidException(() -> { final BlockBlobAsyncClient blockBlobAsyncClient = asyncClient().getBlobContainerAsyncClient(container) .getBlobAsyncClient(blobName).getBlockBlobAsyncClient(); - try (ChunkedBlobOutputStream out = new ChunkedBlobOutputStream<>(bigArrays) { + try (ChunkedBlobOutputStream out = new ChunkedBlobOutputStream<>(bigArrays, getUploadBlockSize()) { @Override protected void maybeFlushBuffer() { - if (buffer.size() >= FLUSH_BUFFER_BYTES) { + if (buffer.size() >= maxBytesToBuffer) { flushBuffer(); } } diff --git a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainer.java b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainer.java index 07b6354b7007d..ce2eae99b59f3 100644 --- a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainer.java +++ b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainer.java @@ -81,6 +81,7 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea @Override public void writeBlob(String blobName, boolean failIfAlreadyExists, + boolean atomic, CheckedConsumer writer) throws IOException { blobStore.writeBlob(buildKey(blobName), failIfAlreadyExists, writer); } diff --git a/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainerRetriesTests.java b/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainerRetriesTests.java index 315095a531d2d..6e2c149793f32 100644 --- a/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainerRetriesTests.java +++ b/plugins/repository-gcs/src/test/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobContainerRetriesTests.java @@ -348,7 +348,7 @@ public void testWriteLargeBlob() throws IOException { blobContainer.writeBlob("write_large_blob", stream, data.length, false); } } else { - blobContainer.writeBlob("write_large_blob", false, out -> out.write(data)); + blobContainer.writeBlob("write_large_blob", false, randomBoolean(), out -> out.write(data)); } assertThat(countInits.get(), equalTo(0)); 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 a1104189059c8..b60b0a969429d 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 @@ -161,19 +161,33 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea @Override public void writeBlob(String blobName, boolean failIfAlreadyExists, + boolean atomic, CheckedConsumer writer) throws IOException { Path blob = new Path(path, blobName); - // we pass CREATE, which means it fails if a blob already exists. - final EnumSet flags = failIfAlreadyExists ? EnumSet.of(CreateFlag.CREATE, CreateFlag.SYNC_BLOCK) - : EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE, CreateFlag.SYNC_BLOCK); - store.execute((Operation) fileContext -> { - try (FSDataOutputStream stream = fileContext.create(blob, flags)) { - writer.accept(stream); - } catch (org.apache.hadoop.fs.FileAlreadyExistsException faee) { - throw new FileAlreadyExistsException(blob.toString(), null, faee.getMessage()); - } - return null; - }); + if (atomic) { + final Path tempBlobPath = new Path(path, FsBlobContainer.tempBlobName(blobName)); + store.execute((Operation) fileContext -> { + try (FSDataOutputStream stream = fileContext.create(tempBlobPath, EnumSet.of(CreateFlag.CREATE, CreateFlag.SYNC_BLOCK))) { + writer.accept(stream); + fileContext.rename(tempBlobPath, blob, failIfAlreadyExists ? Options.Rename.NONE : Options.Rename.OVERWRITE); + } catch (org.apache.hadoop.fs.FileAlreadyExistsException faee) { + throw new FileAlreadyExistsException(blob.toString(), null, faee.getMessage()); + } + return null; + }); + } else { + // we pass CREATE, which means it fails if a blob already exists. + final EnumSet flags = failIfAlreadyExists ? EnumSet.of(CreateFlag.CREATE, CreateFlag.SYNC_BLOCK) + : EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE, CreateFlag.SYNC_BLOCK); + store.execute((Operation) fileContext -> { + try (FSDataOutputStream stream = fileContext.create(blob, flags)) { + writer.accept(stream); + } catch (org.apache.hadoop.fs.FileAlreadyExistsException faee) { + throw new FileAlreadyExistsException(blob.toString(), null, faee.getMessage()); + } + return null; + }); + } } @Override diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java index f166d72b2d08d..523867c0905d0 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java @@ -135,15 +135,16 @@ public void writeBlob(String blobName, InputStream inputStream, long blobSize, b @Override public void writeBlob(String blobName, boolean failIfAlreadyExists, + boolean atomic, CheckedConsumer writer) throws IOException { try (AmazonS3Reference clientReference = blobStore.clientReference(); - ChunkedBlobOutputStream out = new ChunkedBlobOutputStream<>(blobStore.bigArrays()) { + ChunkedBlobOutputStream out = new ChunkedBlobOutputStream<>(blobStore.bigArrays(), blobStore.bufferSizeInBytes()) { private final SetOnce uploadId = new SetOnce<>(); @Override protected void maybeFlushBuffer() throws IOException { - if (buffer.size() >= FLUSH_BUFFER_BYTES) { + if (buffer.size() >= maxBytesToBuffer) { flushBuffer(false); } } diff --git a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java index 3adc093b5a384..73905371c6c78 100644 --- a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java +++ b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java @@ -43,6 +43,7 @@ import java.util.Locale; import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import static org.elasticsearch.repositories.s3.S3ClientSettings.DISABLE_CHUNKED_ENCODING; import static org.elasticsearch.repositories.s3.S3ClientSettings.ENDPOINT_SETTING; @@ -297,6 +298,105 @@ public void testWriteLargeBlob() throws Exception { assertThat(countDownComplete.isCountedDown(), is(true)); } + public void testWriteLargeBlobStreaming() throws Exception { + final boolean useTimeout = rarely(); + final TimeValue readTimeout = useTimeout ? TimeValue.timeValueMillis(randomIntBetween(100, 500)) : null; + final ByteSizeValue bufferSize = new ByteSizeValue(5, ByteSizeUnit.MB); + final BlobContainer blobContainer = createBlobContainer(null, readTimeout, true, bufferSize); + + final int parts = randomIntBetween(1, 5); + final long lastPartSize = randomLongBetween(10, 512); + final long blobSize = (parts * bufferSize.getBytes()) + lastPartSize; + + final int nbErrors = 2; // we want all requests to fail at least once + final CountDown countDownInitiate = new CountDown(nbErrors); + final AtomicInteger countDownUploads = new AtomicInteger(nbErrors * (parts + 1)); + final AtomicLong bytesReceived = new AtomicLong(0L); + final CountDown countDownComplete = new CountDown(nbErrors); + + httpServer.createContext("/bucket/write_large_blob", exchange -> { + final long contentLength = Long.parseLong(exchange.getRequestHeaders().getFirst("Content-Length")); + + if ("POST".equals(exchange.getRequestMethod()) + && exchange.getRequestURI().getQuery().equals("uploads")) { + // initiate multipart upload request + if (countDownInitiate.countDown()) { + byte[] response = ("\n" + + "\n" + + " bucket\n" + + " write_large_blob\n" + + " TEST\n" + + "").getBytes(StandardCharsets.UTF_8); + exchange.getResponseHeaders().add("Content-Type", "application/xml"); + exchange.sendResponseHeaders(HttpStatus.SC_OK, response.length); + exchange.getResponseBody().write(response); + exchange.close(); + return; + } + } else if ("PUT".equals(exchange.getRequestMethod()) + && exchange.getRequestURI().getQuery().contains("uploadId=TEST") + && exchange.getRequestURI().getQuery().contains("partNumber=")) { + // upload part request + MD5DigestCalculatingInputStream md5 = new MD5DigestCalculatingInputStream(exchange.getRequestBody()); + BytesReference bytes = Streams.readFully(md5); + + if (countDownUploads.decrementAndGet() % 2 == 0) { + bytesReceived.addAndGet(bytes.length()); + exchange.getResponseHeaders().add("ETag", Base16.encodeAsString(md5.getMd5Digest())); + exchange.sendResponseHeaders(HttpStatus.SC_OK, -1); + exchange.close(); + return; + } + + } else if ("POST".equals(exchange.getRequestMethod()) + && exchange.getRequestURI().getQuery().equals("uploadId=TEST")) { + // complete multipart upload request + if (countDownComplete.countDown()) { + Streams.readFully(exchange.getRequestBody()); + byte[] response = ("\n" + + "\n" + + " bucket\n" + + " write_large_blob\n" + + "").getBytes(StandardCharsets.UTF_8); + exchange.getResponseHeaders().add("Content-Type", "application/xml"); + exchange.sendResponseHeaders(HttpStatus.SC_OK, response.length); + exchange.getResponseBody().write(response); + exchange.close(); + return; + } + } + + // sends an error back or let the request time out + if (useTimeout == false) { + if (randomBoolean() && contentLength > 0) { + Streams.readFully(exchange.getRequestBody(), new byte[randomIntBetween(1, Math.toIntExact(contentLength - 1))]); + } else { + Streams.readFully(exchange.getRequestBody()); + exchange.sendResponseHeaders(randomFrom(HttpStatus.SC_INTERNAL_SERVER_ERROR, HttpStatus.SC_BAD_GATEWAY, + HttpStatus.SC_SERVICE_UNAVAILABLE, HttpStatus.SC_GATEWAY_TIMEOUT), -1); + } + exchange.close(); + } + }); + + blobContainer.writeBlob("write_large_blob", false, randomBoolean(), out -> { + final byte[] buffer = new byte[16 * 1024]; + long outstanding = blobSize; + while (outstanding > 0) { + if (randomBoolean()) { + int toWrite = Math.toIntExact(Math.min(randomIntBetween(64, buffer.length), outstanding)); + out.write(buffer, 0, toWrite); + outstanding -= toWrite; + } else { + out.write(0); + outstanding--; + } + } + }); + + assertEquals(blobSize, bytesReceived.get()); + } + /** * Asserts that an InputStream is fully consumed, or aborted, when it is closed */ diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java b/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java index ecff9058d10d3..170708e56fe03 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java @@ -118,7 +118,8 @@ default void writeBlob(String blobName, BytesReference bytes, boolean failIfAlre writeBlob(blobName, bytes.streamInput(), bytes.length(), failIfAlreadyExists); } - void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) throws IOException; + void writeBlob(String blobName, boolean failIfAlreadyExists, boolean atomic, + CheckedConsumer writer) throws IOException; /** * Reads blob content from a {@link BytesReference} and writes it to the container in a new blob with the given name, diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java b/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java index 81438e54034e4..96d0a6c020dd3 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java @@ -248,7 +248,30 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea @Override public void writeBlob(String blobName, boolean failIfAlreadyExists, + boolean atomic, CheckedConsumer writer) throws IOException { + if (atomic) { + final String tempBlob = tempBlobName(blobName); + try { + writeToPath(tempBlob, failIfAlreadyExists, writer); + moveBlobAtomic(tempBlob, blobName, failIfAlreadyExists); + } catch (IOException ex) { + try { + deleteBlobsIgnoringIfNotExists(Iterators.single(tempBlob)); + } catch (IOException e) { + ex.addSuppressed(e); + } + throw ex; + } finally { + IOUtils.fsync(path, true); + } + } else { + writeToPath(blobName, failIfAlreadyExists, writer); + } + } + + private void writeToPath(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) + throws IOException { final Path file = path.resolve(blobName); try { try (OutputStream out = new BlobOutputStream(file)) { diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/support/FilterBlobContainer.java b/server/src/main/java/org/elasticsearch/common/blobstore/support/FilterBlobContainer.java index 2bd28c2910803..d2746fcb8d1f8 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/support/FilterBlobContainer.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/support/FilterBlobContainer.java @@ -64,9 +64,9 @@ public void writeBlob(String blobName, InputStream inputStream, long blobSize, b } @Override - public void writeBlob(String blobName, boolean failIfAlreadyExists, + public void writeBlob(String blobName, boolean failIfAlreadyExists, boolean atomic, CheckedConsumer writer) throws IOException { - delegate.writeBlob(blobName, failIfAlreadyExists, writer); + delegate.writeBlob(blobName, failIfAlreadyExists, atomic, writer); } @Override diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index 4daac1b85d28c..1a7ef5c061d17 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -53,11 +53,9 @@ import org.elasticsearch.common.blobstore.DeleteResult; import org.elasticsearch.common.blobstore.fs.FsBlobContainer; import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.compress.NotXContentException; import org.elasticsearch.common.io.Streams; -import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.store.InputStreamIndexInput; import org.elasticsearch.common.metrics.CounterMetric; @@ -74,6 +72,7 @@ import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Tuple; @@ -118,6 +117,7 @@ import java.io.FilterInputStream; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.nio.file.NoSuchFileException; import java.util.ArrayList; import java.util.Collection; @@ -2245,13 +2245,11 @@ public void onFailure(Exception e) { } final String indexBlob = INDEX_FILE_PREFIX + Long.toString(newGen); logger.debug("Repository [{}] writing new index generational blob [{}]", metadata.name(), indexBlob); - try (ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(bigArrays)) { + writeAtomic(blobContainer(), indexBlob, out -> { try (XContentBuilder xContentBuilder = XContentFactory.jsonBuilder(Streams.noCloseStream(out))) { newRepositoryData.snapshotsToXContent(xContentBuilder, version); } - final BytesReference serializedRepoData = out.bytes(); - writeAtomic(blobContainer(), indexBlob, serializedRepoData, true); - } + }, true); maybeWriteIndexLatest(newGen); // Step 3: Update CS to reflect new repository generation. @@ -2344,7 +2342,7 @@ private void maybeWriteIndexLatest(long newGen) { if (supportURLRepo) { logger.debug("Repository [{}] updating index.latest with generation [{}]", metadata.name(), newGen); try { - writeAtomic(blobContainer(), INDEX_LATEST_BLOB, new BytesArray(Numbers.longToBytes(newGen)), false); + writeAtomic(blobContainer(), INDEX_LATEST_BLOB, out -> out.write(Numbers.longToBytes(newGen)), false); } catch (Exception e) { logger.warn( () -> new ParameterizedMessage( @@ -2525,10 +2523,14 @@ private long latestGeneration(Collection rootBlobs) { return latest; } - private void writeAtomic(BlobContainer container, final String blobName, final BytesReference bytesRef, boolean failIfAlreadyExists) - throws IOException { + private void writeAtomic( + BlobContainer container, + final String blobName, + CheckedConsumer writer, + boolean failIfAlreadyExists + ) throws IOException { logger.trace(() -> new ParameterizedMessage("[{}] Writing [{}] to {} atomically", metadata.name(), blobName, container.path())); - container.writeBlobAtomic(blobName, bytesRef, failIfAlreadyExists); + container.writeBlob(blobName, failIfAlreadyExists, true, writer); } @Override @@ -3169,10 +3171,12 @@ private void writeShardIndexBlobAtomic( () -> new ParameterizedMessage("[{}] Writing shard index [{}] to [{}]", metadata.name(), indexGeneration, shardContainer.path()) ); final String blobName = INDEX_SHARD_SNAPSHOTS_FORMAT.blobName(String.valueOf(indexGeneration)); - try (ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(bigArrays)) { - INDEX_SHARD_SNAPSHOTS_FORMAT.serialize(updatedSnapshots, blobName, compress, out); - writeAtomic(shardContainer, blobName, out.bytes(), true); - } + writeAtomic( + shardContainer, + blobName, + out -> INDEX_SHARD_SNAPSHOTS_FORMAT.serialize(updatedSnapshots, blobName, compress, out), + true + ); } // Unused blobs are all previous index-, data- and meta-blobs and that are not referenced by the new index- as well as all diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java index abe28b510efc1..9f95faad6bbca 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java @@ -269,7 +269,7 @@ private int getAvailable() throws IOException { */ public void write(T obj, BlobContainer blobContainer, String name, boolean compress) throws IOException { final String blobName = blobName(name); - blobContainer.writeBlob(blobName, false, out -> serialize(obj, blobName, compress, out)); + blobContainer.writeBlob(blobName, false, false, out -> serialize(obj, blobName, compress, out)); } public void serialize(final T obj, final String blobName, final boolean compress, OutputStream outputStream) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java index 7daa8e6f49260..bf0a0bfb6ebfa 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java @@ -8,8 +8,6 @@ package org.elasticsearch.repositories.blobstore; import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput; -import org.elasticsearch.common.unit.ByteSizeUnit; -import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.core.Releasables; @@ -20,10 +18,7 @@ public abstract class ChunkedBlobOutputStream extends OutputStream { - /** - * We buffer 8MB before flushing to storage. - */ - public static final long FLUSH_BUFFER_BYTES = new ByteSizeValue(8, ByteSizeUnit.MB).getBytes(); + protected final long maxBytesToBuffer; protected final List parts = new ArrayList<>(); @@ -35,8 +30,9 @@ public abstract class ChunkedBlobOutputStream extends OutputStream { protected long written = 0L; - protected ChunkedBlobOutputStream(BigArrays bigArrays) { + protected ChunkedBlobOutputStream(BigArrays bigArrays, long maxBytesToBuffer) { this.bigArrays = bigArrays; + this.maxBytesToBuffer = maxBytesToBuffer; buffer = new ReleasableBytesStreamOutput(bigArrays); } diff --git a/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java b/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java index b700aced84078..0aa6e8d13ca8a 100644 --- a/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java +++ b/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java @@ -490,11 +490,7 @@ public Map listBlobsByPrefix(String blobNamePrefix) throws @Override public void writeBlob(String blobName, InputStream inputStream, long blobSize, boolean failIfAlreadyExists) throws IOException { - maybeIOExceptionOrBlock(blobName); - if (blockOnWriteShardLevelMeta && blobName.startsWith(BlobStoreRepository.SNAPSHOT_PREFIX) - && path().equals(basePath()) == false) { - blockExecutionAndMaybeWait(blobName); - } + beforeWrite(blobName); super.writeBlob(blobName, inputStream, blobSize, failIfAlreadyExists); if (RandomizedContext.current().getRandom().nextBoolean()) { // for network based repositories, the blob may have been written but we may still @@ -506,13 +502,14 @@ && path().equals(basePath()) == false) { @Override public void writeBlob(String blobName, boolean failIfAlreadyExists, + boolean atomic, CheckedConsumer writer) throws IOException { - maybeIOExceptionOrBlock(blobName); - if (blockOnWriteShardLevelMeta && blobName.startsWith(BlobStoreRepository.SNAPSHOT_PREFIX) - && path().equals(basePath()) == false) { - blockExecutionAndMaybeWait(blobName); + if (atomic) { + beforeAtomicWrite(blobName); + } else { + beforeWrite(blobName); } - super.writeBlob(blobName, failIfAlreadyExists, writer); + super.writeBlob(blobName, failIfAlreadyExists, atomic, writer); if (RandomizedContext.current().getRandom().nextBoolean()) { // for network based repositories, the blob may have been written but we may still // get an error with the client connection, so an IOException here simulates this @@ -520,20 +517,18 @@ && path().equals(basePath()) == false) { } } + private void beforeWrite(String blobName) throws IOException { + maybeIOExceptionOrBlock(blobName); + if (blockOnWriteShardLevelMeta && blobName.startsWith(BlobStoreRepository.SNAPSHOT_PREFIX) + && path().equals(basePath()) == false) { + blockExecutionAndMaybeWait(blobName); + } + } + @Override public void writeBlobAtomic(final String blobName, final BytesReference bytes, final boolean failIfAlreadyExists) throws IOException { - final Random random = RandomizedContext.current().getRandom(); - if (failOnIndexLatest && BlobStoreRepository.INDEX_LATEST_BLOB.equals(blobName)) { - throw new IOException("Random IOException"); - } - if (blobName.startsWith(BlobStoreRepository.INDEX_FILE_PREFIX)) { - if (blockAndFailOnWriteIndexFile) { - blockExecutionAndFail(blobName); - } else if (blockOnWriteIndexFile) { - blockExecutionAndMaybeWait(blobName); - } - } + final Random random = beforeAtomicWrite(blobName); if ((delegate() instanceof FsBlobContainer) && (random.nextBoolean())) { // Simulate a failure between the write and move operation in FsBlobContainer final String tempBlobName = FsBlobContainer.tempBlobName(blobName); @@ -548,6 +543,21 @@ public void writeBlobAtomic(final String blobName, final BytesReference bytes, super.writeBlobAtomic(blobName, bytes, failIfAlreadyExists); } } + + private Random beforeAtomicWrite(String blobName) throws IOException { + final Random random = RandomizedContext.current().getRandom(); + if (failOnIndexLatest && BlobStoreRepository.INDEX_LATEST_BLOB.equals(blobName)) { + throw new IOException("Random IOException"); + } + if (blobName.startsWith(BlobStoreRepository.INDEX_FILE_PREFIX)) { + if (blockAndFailOnWriteIndexFile) { + blockExecutionAndFail(blobName); + } else if (blockOnWriteIndexFile) { + blockExecutionAndMaybeWait(blobName); + } + } + return random; + } } } } diff --git a/x-pack/plugin/repository-encrypted/src/main/java/org/elasticsearch/repositories/encrypted/EncryptedRepository.java b/x-pack/plugin/repository-encrypted/src/main/java/org/elasticsearch/repositories/encrypted/EncryptedRepository.java index f4e64d9a79c66..63ab370d8a820 100644 --- a/x-pack/plugin/repository-encrypted/src/main/java/org/elasticsearch/repositories/encrypted/EncryptedRepository.java +++ b/x-pack/plugin/repository-encrypted/src/main/java/org/elasticsearch/repositories/encrypted/EncryptedRepository.java @@ -625,12 +625,20 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea } @Override - public void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) - throws IOException { + public void writeBlob( + String blobName, + boolean failIfAlreadyExists, + boolean atomic, + CheckedConsumer writer + ) throws IOException { // TODO: this is just a stop-gap solution for until we have an encrypted output stream wrapper try (ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(bigArrays)) { writer.accept(out); - writeBlob(blobName, out.bytes(), failIfAlreadyExists); + if (atomic) { + writeBlobAtomic(blobName, out.bytes(), failIfAlreadyExists); + } else { + writeBlob(blobName, out.bytes(), failIfAlreadyExists); + } } } diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/cache/common/TestUtils.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/cache/common/TestUtils.java index 8ffdea046d79f..398ef75189e07 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/cache/common/TestUtils.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/cache/common/TestUtils.java @@ -272,7 +272,12 @@ public void writeBlob(String blobName, InputStream inputStream, long blobSize, b } @Override - public void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) { + public void writeBlob( + String blobName, + boolean failIfAlreadyExists, + boolean atomic, + CheckedConsumer writer + ) { throw unsupportedException(); } diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisFailureIT.java b/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisFailureIT.java index d6254a72acc2d..44e730c89ad5f 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisFailureIT.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisFailureIT.java @@ -457,11 +457,19 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea } @Override - public void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) - throws IOException { + public void writeBlob( + String blobName, + boolean failIfAlreadyExists, + boolean atomic, + CheckedConsumer writer + ) throws IOException { final BytesStreamOutput out = new BytesStreamOutput(); writer.accept(out); - writeBlob(blobName, out.bytes(), failIfAlreadyExists); + if (atomic) { + writeBlobAtomic(blobName, out.bytes(), failIfAlreadyExists); + } else { + writeBlob(blobName, out.bytes(), failIfAlreadyExists); + } } @Override diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisSuccessIT.java b/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisSuccessIT.java index 75ad6247b9e79..2f883759b7fe0 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisSuccessIT.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisSuccessIT.java @@ -330,11 +330,19 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea } @Override - public void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) - throws IOException { + public void writeBlob( + String blobName, + boolean failIfAlreadyExists, + boolean atomic, + CheckedConsumer writer + ) throws IOException { final BytesStreamOutput out = new BytesStreamOutput(); writer.accept(out); - writeBlob(blobName, out.bytes(), failIfAlreadyExists); + if (atomic) { + writeBlobAtomic(blobName, out.bytes(), failIfAlreadyExists); + } else { + writeBlob(blobName, out.bytes(), failIfAlreadyExists); + } } @Override From 682c8ca324ff59673d2723bba7ffea55f53662b1 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 18 Jun 2021 17:25:41 +0200 Subject: [PATCH 13/24] docs --- .../common/blobstore/fs/FsBlobContainer.java | 14 +++------- .../blobstore/ChunkedBlobOutputStream.java | 26 +++++++++++++++++++ 2 files changed, 30 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java b/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java index 96d0a6c020dd3..e7f7fd0889136 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java @@ -253,7 +253,7 @@ public void writeBlob(String blobName, if (atomic) { final String tempBlob = tempBlobName(blobName); try { - writeToPath(tempBlob, failIfAlreadyExists, writer); + writeToPath(tempBlob, true, writer); moveBlobAtomic(tempBlob, blobName, failIfAlreadyExists); } catch (IOException ex) { try { @@ -262,12 +262,11 @@ public void writeBlob(String blobName, ex.addSuppressed(e); } throw ex; - } finally { - IOUtils.fsync(path, true); } } else { writeToPath(blobName, failIfAlreadyExists, writer); } + IOUtils.fsync(path, true); } private void writeToPath(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) @@ -286,6 +285,7 @@ private void writeToPath(String blobName, boolean failIfAlreadyExists, CheckedCo writer.accept(out); } } + IOUtils.fsync(file, false); } @Override @@ -352,7 +352,7 @@ public static boolean isTempBlobName(final String blobName) { return blobName.startsWith(TEMP_FILE_PREFIX); } - private class BlobOutputStream extends FilterOutputStream { + private static class BlobOutputStream extends FilterOutputStream { BlobOutputStream(Path file) throws IOException { super(Files.newOutputStream(file, StandardOpenOption.CREATE_NEW)); @@ -362,11 +362,5 @@ private class BlobOutputStream extends FilterOutputStream { public void write(byte[] b, int off, int len) throws IOException { out.write(b, off, len); } - - @Override - public void close() throws IOException { - super.close(); - IOUtils.fsync(path, true); - } } } diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java index bf0a0bfb6ebfa..f7504ec3a8a92 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java @@ -16,18 +16,44 @@ import java.util.ArrayList; import java.util.List; +/** + * Base class for doing chunked writes to a blob store. Some blob stores require either up-front knowledge of the size of the blob that + * will be written or writing it in chunks that are then joined into the final blob at the end of the write. This class provides a basis + * on which to implement an output stream that encapsulates such a chunked write. + * + * @param type of chunk identifier + */ public abstract class ChunkedBlobOutputStream extends OutputStream { + /** + * Size of the write buffer above which it must be flushed to storage. + */ protected final long maxBytesToBuffer; + /** + * List of identifiers of already written chunks. + */ protected final List parts = new ArrayList<>(); + /** + * Big arrays to be able to allocate buffers from pooled bytes. + */ private final BigArrays bigArrays; + /** + * Current write buffer. + */ protected ReleasableBytesStreamOutput buffer; + /** + * Set to true once no more calls to {@link #write} are expected and the blob has been received by {@link #write} in full so that + * {@link #close()} knows whether to clean up existing chunks or finish a chunked write. + */ protected boolean successful = false; + /** + * Number of bytes written to storage writeBlso far. + */ protected long written = 0L; protected ChunkedBlobOutputStream(BigArrays bigArrays, long maxBytesToBuffer) { From 1d6ec23b6c40bcd0ebcffa7626d8f6c1caed89d0 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 18 Jun 2021 17:37:08 +0200 Subject: [PATCH 14/24] fix gcs --- .../repositories/gcs/GoogleCloudStorageBlobStore.java | 5 +++-- .../fixture/gcs/GoogleCloudStorageHttpHandler.java | 10 ---------- 2 files changed, 3 insertions(+), 12 deletions(-) diff --git a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java index adb531738058e..0c2748bfd6646 100644 --- a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java +++ b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java @@ -293,8 +293,8 @@ public boolean isOpen() { } @Override - public void close() throws IOException { - SocketAccess.doPrivilegedVoidIOException(writeChannel::close); + public void close() { + // noop } } ) @@ -312,6 +312,7 @@ public void write(byte[] b, int off, int len) throws IOException { } }) { writer.accept(out); + SocketAccess.doPrivilegedVoidIOException(writeChannel::close); } stats.trackPutOperation(); return; diff --git a/test/fixtures/gcs-fixture/src/main/java/fixture/gcs/GoogleCloudStorageHttpHandler.java b/test/fixtures/gcs-fixture/src/main/java/fixture/gcs/GoogleCloudStorageHttpHandler.java index 0d770807a7812..aee7e0a760478 100644 --- a/test/fixtures/gcs-fixture/src/main/java/fixture/gcs/GoogleCloudStorageHttpHandler.java +++ b/test/fixtures/gcs-fixture/src/main/java/fixture/gcs/GoogleCloudStorageHttpHandler.java @@ -20,11 +20,8 @@ import org.elasticsearch.common.bytes.CompositeBytesReference; import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.regex.Regex; -import org.elasticsearch.common.xcontent.XContentHelper; -import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.core.Tuple; -import org.elasticsearch.repositories.blobstore.BlobStoreRepository; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestUtils; @@ -33,7 +30,6 @@ import java.io.InputStream; import java.io.InputStreamReader; import java.net.URLDecoder; -import java.nio.file.Paths; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -211,12 +207,6 @@ public void handle(final HttpExchange exchange) throws IOException { blobs.put(blobName, BytesArray.EMPTY); byte[] response = requestBody.utf8ToString().getBytes(UTF_8); - final String name = Paths.get(blobName).getFileName().toString(); - if (name.matches(BlobStoreRepository.INDEX_FILE_PREFIX + "\\d+") || name.equals(BlobStoreRepository.INDEX_LATEST_BLOB)) { - final Map parsedBody = XContentHelper.convertToMap(requestBody, false, XContentType.JSON).v2(); - assert parsedBody.get("md5Hash") != null - : "file [" + blobName + "] must be written atomically but did not come with a md5 checksum"; - } exchange.getResponseHeaders().add("Content-Type", "application/json"); exchange.getResponseHeaders() .add( From f98d10dfd26b22461191c7de43642ce785ea1f48 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sat, 19 Jun 2021 17:39:26 +0200 Subject: [PATCH 15/24] more docs --- .../repositories/blobstore/ChunkedBlobOutputStream.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java index f7504ec3a8a92..d492eba7eff39 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java @@ -83,10 +83,19 @@ public void close() throws IOException { } } + /** + * Mark all blob bytes as properly received by {@link #write}, indicating that {@link #close} may finalize the blob. + */ public void markSuccess() { this.successful = true; } + /** + * Finish writing the current buffer contents to storage and track them by the given {@code partId}. Depending on whether all contents + * have already been written either prepare the write buffer for additional writes or release the buffer. + * + * @param partId part identifier to track for use when closing + */ protected final void finishPart(T partId) { written += buffer.size(); parts.add(partId); From 98b61192854d401e9cf4b4e7918ced3cf6b1d1fb Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sat, 19 Jun 2021 19:06:11 +0200 Subject: [PATCH 16/24] nicer --- .../repositories/azure/AzureBlobStore.java | 8 +------- .../repositories/s3/S3BlobContainer.java | 6 ++---- .../blobstore/ChunkedBlobOutputStream.java | 18 ++++++++++++------ 3 files changed, 15 insertions(+), 17 deletions(-) diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java index c4958d1db67c0..ba3d2bbc9dd3e 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java @@ -399,13 +399,7 @@ public void writeBlob(String blobName, try (ChunkedBlobOutputStream out = new ChunkedBlobOutputStream<>(bigArrays, getUploadBlockSize()) { @Override - protected void maybeFlushBuffer() { - if (buffer.size() >= maxBytesToBuffer) { - flushBuffer(); - } - } - - private void flushBuffer() { + protected void flushBuffer() { if (buffer.size() == 0) { return; } diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java index 523867c0905d0..600d2a76521d1 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java @@ -143,10 +143,8 @@ public void writeBlob(String blobName, private final SetOnce uploadId = new SetOnce<>(); @Override - protected void maybeFlushBuffer() throws IOException { - if (buffer.size() >= maxBytesToBuffer) { - flushBuffer(false); - } + protected void flushBuffer() throws IOException { + flushBuffer(false); } private void flushBuffer(boolean lastPart) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java index d492eba7eff39..76b3b76a53475 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java @@ -26,14 +26,14 @@ public abstract class ChunkedBlobOutputStream extends OutputStream { /** - * Size of the write buffer above which it must be flushed to storage. + * List of identifiers of already written chunks. */ - protected final long maxBytesToBuffer; + protected final List parts = new ArrayList<>(); /** - * List of identifiers of already written chunks. + * Size of the write buffer above which it must be flushed to storage. */ - protected final List parts = new ArrayList<>(); + private final long maxBytesToBuffer; /** * Big arrays to be able to allocate buffers from pooled bytes. @@ -108,7 +108,13 @@ protected final void finishPart(T partId) { } } - protected abstract void maybeFlushBuffer() throws IOException; - protected abstract void doClose() throws IOException; + + protected abstract void flushBuffer() throws IOException; + + private void maybeFlushBuffer() throws IOException { + if (buffer.size() >= maxBytesToBuffer) { + flushBuffer(); + } + } } From 6de3bc5e8233ca5a7799439a346166facf644bc1 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sat, 19 Jun 2021 19:21:24 +0200 Subject: [PATCH 17/24] docs and drier --- .../repositories/azure/AzureBlobStore.java | 19 +++++------ .../repositories/s3/S3BlobContainer.java | 27 +++++++++------- .../blobstore/ChunkedBlobOutputStream.java | 32 +++++++++++++++---- 3 files changed, 50 insertions(+), 28 deletions(-) diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java index ba3d2bbc9dd3e..b5bbb6c766f5c 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java @@ -413,18 +413,19 @@ protected void flushBuffer() { } @Override - protected void doClose() { - if (successful) { - if (written == 0L) { - writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); - } else { - flushBuffer(); - blockBlobAsyncClient.commitBlockList(parts, failIfAlreadyExists == false).block(); - } + protected void onAllPartsReady() { + if (written == 0L) { + writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); } else { - // TODO: here and in multi-part upload, should we clean up uploaded blobs? + flushBuffer(); + blockBlobAsyncClient.commitBlockList(parts, failIfAlreadyExists == false).block(); } } + + @Override + protected void onFailure() { + // TODO: here and in multi-part upload, should we clean up uploaded blobs? + } }) { writer.accept(out); out.markSuccess(); diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java index 600d2a76521d1..c7a8db53445ce 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java @@ -168,18 +168,21 @@ private void flushBuffer(boolean lastPart) throws IOException { } @Override - protected void doClose() throws IOException { - if (successful) { - if (written == 0L) { - writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); - } else { - flushBuffer(true); - final CompleteMultipartUploadRequest complRequest = - new CompleteMultipartUploadRequest(blobStore.bucket(), blobName, uploadId.get(), parts); - complRequest.setRequestMetricCollector(blobStore.multiPartUploadMetricCollector); - SocketAccess.doPrivilegedVoid(() -> clientReference.client().completeMultipartUpload(complRequest)); - } - } else if (Strings.hasText(uploadId.get())) { + protected void onAllPartsReady() throws IOException { + if (written == 0L) { + writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); + } else { + flushBuffer(true); + final CompleteMultipartUploadRequest complRequest = + new CompleteMultipartUploadRequest(blobStore.bucket(), blobName, uploadId.get(), parts); + complRequest.setRequestMetricCollector(blobStore.multiPartUploadMetricCollector); + SocketAccess.doPrivilegedVoid(() -> clientReference.client().completeMultipartUpload(complRequest)); + } + } + + @Override + protected void onFailure() { + if (Strings.hasText(uploadId.get())) { abortMultiPartUpload(uploadId.get(), blobName); } } diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java index 76b3b76a53475..25cad3dd0dd66 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java @@ -63,21 +63,25 @@ protected ChunkedBlobOutputStream(BigArrays bigArrays, long maxBytesToBuffer) { } @Override - public void write(int b) throws IOException { + public final void write(int b) throws IOException { buffer.write(b); maybeFlushBuffer(); } @Override - public void write(byte[] b, int off, int len) throws IOException { + public final void write(byte[] b, int off, int len) throws IOException { buffer.write(b, off, len); maybeFlushBuffer(); } @Override - public void close() throws IOException { + public final void close() throws IOException { try { - doClose(); + if (successful) { + onAllPartsReady(); + } else { + onFailure(); + } } finally { Releasables.close(buffer); } @@ -86,7 +90,7 @@ public void close() throws IOException { /** * Mark all blob bytes as properly received by {@link #write}, indicating that {@link #close} may finalize the blob. */ - public void markSuccess() { + public final void markSuccess() { this.successful = true; } @@ -108,10 +112,24 @@ protected final void finishPart(T partId) { } } - protected abstract void doClose() throws IOException; - + /** + * Write the contents of {@link #buffer} to storage. Implementations should call {@link #finishPart} at the end to track the the chunk + * of data just written and ready {@link #buffer} for the next write. + */ protected abstract void flushBuffer() throws IOException; + /** + * Invoked once all write chunks/parts are ready to be combined into the final blob. Implementations must invoke the necessary logic + * for combining the uploaded chunks into the final blob in this method. + */ + protected abstract void onAllPartsReady() throws IOException; + + /** + * Invoked in case writing all chunks of data to storage failed. Implementations should run any cleanup required for the already + * written data in this method. + */ + protected abstract void onFailure(); + private void maybeFlushBuffer() throws IOException { if (buffer.size() >= maxBytesToBuffer) { flushBuffer(); From ced642a6e1643257388480524eb28519bb6940dc Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sat, 19 Jun 2021 19:43:59 +0200 Subject: [PATCH 18/24] tests and fixes Azure --- .../repositories/azure/AzureBlobStore.java | 65 ++++++++-------- .../azure/AzureBlobContainerRetriesTests.java | 78 +++++++++++++++++++ 2 files changed, 110 insertions(+), 33 deletions(-) diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java index b5bbb6c766f5c..8657e11d56acb 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java @@ -393,44 +393,43 @@ public void writeBlob(String blobName, BytesReference bytes, boolean failIfAlrea public void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer writer) throws IOException { - SocketAccess.doPrivilegedVoidException(() -> { - final BlockBlobAsyncClient blockBlobAsyncClient = asyncClient().getBlobContainerAsyncClient(container) - .getBlobAsyncClient(blobName).getBlockBlobAsyncClient(); - try (ChunkedBlobOutputStream out = new ChunkedBlobOutputStream<>(bigArrays, getUploadBlockSize()) { - - @Override - protected void flushBuffer() { - if (buffer.size() == 0) { - return; - } - final String blockId = makeMultipartBlockId(); - blockBlobAsyncClient.stageBlock( - blockId, - Flux.fromArray(BytesReference.toByteBuffers(buffer.bytes())), - buffer.size() - ).block(); - finishPart(blockId); - } + final BlockBlobAsyncClient blockBlobAsyncClient = asyncClient().getBlobContainerAsyncClient(container) + .getBlobAsyncClient(blobName).getBlockBlobAsyncClient(); + try (ChunkedBlobOutputStream out = new ChunkedBlobOutputStream<>(bigArrays, getUploadBlockSize()) { - @Override - protected void onAllPartsReady() { - if (written == 0L) { - writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); - } else { - flushBuffer(); - blockBlobAsyncClient.commitBlockList(parts, failIfAlreadyExists == false).block(); - } + @Override + protected void flushBuffer() { + if (buffer.size() == 0) { + return; } + final String blockId = makeMultipartBlockId(); + SocketAccess.doPrivilegedVoidException(() -> blockBlobAsyncClient.stageBlock( + blockId, + Flux.fromArray(BytesReference.toByteBuffers(buffer.bytes())), + buffer.size() + ).block()); + finishPart(blockId); + } - @Override - protected void onFailure() { - // TODO: here and in multi-part upload, should we clean up uploaded blobs? + @Override + protected void onAllPartsReady() { + if (written == 0L) { + writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); + } else { + flushBuffer(); + SocketAccess.doPrivilegedVoidException( + () -> blockBlobAsyncClient.commitBlockList(parts, failIfAlreadyExists == false).block()); } - }) { - writer.accept(out); - out.markSuccess(); } - }); + + @Override + protected void onFailure() { + // TODO: here and in multi-part upload, should we clean up uploaded blobs? + } + }) { + writer.accept(out); + out.markSuccess(); + } } public void writeBlob(String blobName, InputStream inputStream, long blobSize, boolean failIfAlreadyExists) throws IOException { diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerRetriesTests.java b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerRetriesTests.java index b4434ad4f8363..1029c627384d0 100644 --- a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerRetriesTests.java +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerRetriesTests.java @@ -59,6 +59,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -392,6 +393,83 @@ public void testWriteLargeBlob() throws Exception { assertThat(blocks.isEmpty(), is(true)); } + public void testWriteLargeBlobStreaming() throws Exception { + final int maxRetries = randomIntBetween(2, 5); + + final int blobSize = (int) ByteSizeUnit.MB.toBytes(10); + final byte[] data = randomBytes(blobSize); + int nbBlocks = (int) Math.ceil((double) data.length / (double) ByteSizeUnit.MB.toBytes(1)); + + final int nbErrors = 2; // we want all requests to fail at least once + final AtomicInteger countDownUploads = new AtomicInteger(nbErrors * nbBlocks); + final AtomicLong bytesReceived = new AtomicLong(0L); + final CountDown countDownComplete = new CountDown(nbErrors); + + final Map blocks = new ConcurrentHashMap<>(); + httpServer.createContext("/account/container/write_large_blob", exchange -> { + + if ("PUT".equals(exchange.getRequestMethod())) { + final Map params = new HashMap<>(); + RestUtils.decodeQueryString(exchange.getRequestURI().getRawQuery(), 0, params); + + final String blockId = params.get("blockid"); + assert Strings.hasText(blockId) == false || AzureFixtureHelper.assertValidBlockId(blockId); + + if (Strings.hasText(blockId) && (countDownUploads.decrementAndGet() % 2 == 0)) { + final BytesReference blockData = Streams.readFully(exchange.getRequestBody()); + blocks.put(blockId, blockData); + bytesReceived.addAndGet(blockData.length()); + exchange.sendResponseHeaders(RestStatus.CREATED.getStatus(), -1); + exchange.close(); + return; + } + + final String complete = params.get("comp"); + if ("blocklist".equals(complete) && (countDownComplete.countDown())) { + final String blockList = Streams.copyToString(new InputStreamReader(exchange.getRequestBody(), UTF_8)); + final List blockUids = Arrays.stream(blockList.split("")) + .filter(line -> line.contains("")) + .map(line -> line.substring(0, line.indexOf(""))) + .collect(Collectors.toList()); + + final ByteArrayOutputStream blob = new ByteArrayOutputStream(); + for (String blockUid : blockUids) { + BytesReference block = blocks.remove(blockUid); + assert block != null; + block.writeTo(blob); + } + assertArrayEquals(data, blob.toByteArray()); + exchange.getResponseHeaders().add("x-ms-request-server-encrypted", "false"); + exchange.sendResponseHeaders(RestStatus.CREATED.getStatus(), -1); + exchange.close(); + return; + } + } + + if (randomBoolean()) { + Streams.readFully(exchange.getRequestBody()); + AzureHttpHandler.sendError(exchange, randomFrom(RestStatus.INTERNAL_SERVER_ERROR, RestStatus.SERVICE_UNAVAILABLE)); + } + exchange.close(); + }); + + final BlobContainer blobContainer = createBlobContainer(maxRetries); + blobContainer.writeBlob("write_large_blob", false, randomBoolean(), out -> { + int outstanding = data.length; + while (outstanding > 0) { + if (randomBoolean()) { + int toWrite = Math.toIntExact(Math.min(randomIntBetween(64, data.length), outstanding)); + out.write(data, data.length - outstanding, toWrite); + outstanding -= toWrite; + } else { + out.write(data[data.length - outstanding]); + outstanding--; + } + } + }); + assertEquals(blobSize, bytesReceived.get()); + } + public void testRetryUntilFail() throws Exception { final int maxRetries = randomIntBetween(2, 5); final AtomicInteger requestsReceived = new AtomicInteger(0); From 3d060599b6c467b46d0c2216a6c8bc675cc1833b Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sat, 19 Jun 2021 20:41:10 +0200 Subject: [PATCH 19/24] drier --- .../gcs/GoogleCloudStorageBlobStore.java | 69 ++++++++----------- 1 file changed, 29 insertions(+), 40 deletions(-) diff --git a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java index 0c2748bfd6646..3a546cdd0f13e 100644 --- a/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java +++ b/plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStore.java @@ -277,28 +277,7 @@ void writeBlob(String blobName, boolean failIfAlreadyExists, CheckedConsumer client().writer(blobInfo, writeOptions)); - try (OutputStream out = new FilterOutputStream( - Channels.newOutputStream( - new WritableByteChannel() { - - @SuppressForbidden(reason = "channel is based on a socket") - @Override - public int write(final ByteBuffer src) throws IOException { - return SocketAccess.doPrivilegedIOException(() -> writeChannel.write(src)); - } - - @Override - public boolean isOpen() { - return writeChannel.isOpen(); - } - - @Override - public void close() { - // noop - } - } - ) - ) { + try (OutputStream out = new FilterOutputStream(Channels.newOutputStream(new WritableBlobChannel(writeChannel))) { @Override public void write(byte[] b, int off, int len) throws IOException { int written = 0; @@ -367,24 +346,9 @@ private void writeBlobResumable(BlobInfo blobInfo, InputStream inputStream, long * It is not enough to wrap the call to Streams#copy, we have to wrap the privileged calls too; this is because Streams#copy * is in the stacktrace and is not granted the permissions needed to close and write the channel. */ - org.elasticsearch.core.internal.io.Streams.copy(inputStream, Channels.newOutputStream(new WritableByteChannel() { - - @SuppressForbidden(reason = "channel is based on a socket") - @Override - public int write(final ByteBuffer src) throws IOException { - return SocketAccess.doPrivilegedIOException(() -> writeChannel.write(src)); - } - - @Override - public boolean isOpen() { - return writeChannel.isOpen(); - } - - @Override - public void close() throws IOException { - SocketAccess.doPrivilegedVoidIOException(writeChannel::close); - } - }), buffer); + org.elasticsearch.core.internal.io.Streams.copy( + inputStream, Channels.newOutputStream(new WritableBlobChannel(writeChannel)), buffer); + SocketAccess.doPrivilegedVoidIOException(writeChannel::close); // We don't track this operation on the http layer as // we do with the GET/LIST operations since this operations // can trigger multiple underlying http requests but only one @@ -548,4 +512,29 @@ private static String buildKey(String keyPath, String s) { public Map stats() { return stats.toMap(); } + + private static final class WritableBlobChannel implements WritableByteChannel { + + private final WriteChannel channel; + + WritableBlobChannel(WriteChannel writeChannel) { + this.channel = writeChannel; + } + + @SuppressForbidden(reason = "channel is based on a socket") + @Override + public int write(final ByteBuffer src) throws IOException { + return SocketAccess.doPrivilegedIOException(() -> channel.write(src)); + } + + @Override + public boolean isOpen() { + return channel.isOpen(); + } + + @Override + public void close() { + // we manually close the channel later to have control over whether or not we want to finalize a blob + } + } } From 3af17d7769846cb05b64ae180b4e806773e8ace9 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sun, 20 Jun 2021 14:42:03 +0200 Subject: [PATCH 20/24] docs --- .../elasticsearch/common/blobstore/BlobContainer.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java b/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java index 170708e56fe03..90da41f654528 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java @@ -118,6 +118,15 @@ default void writeBlob(String blobName, BytesReference bytes, boolean failIfAlre writeBlob(blobName, bytes.streamInput(), bytes.length(), failIfAlreadyExists); } + /** + * Write a blob by providing a consumer that will write its contents to an output stream. This method allows serializing a blob's + * contents directly to the blob store without having to materialize the serialized version in full before writing. + * + * @param blobName the name of the blob to write + * @param failIfAlreadyExists whether to throw a FileAlreadyExistsException if the given blob already exists + * @param atomic whether the write should be atomic in case the implementation supports it + * @param writer consumer for an output stream that will write the blob contents to the stream + */ void writeBlob(String blobName, boolean failIfAlreadyExists, boolean atomic, CheckedConsumer writer) throws IOException; From 7ebffa96290f22f464881b127db3ca3a4ae3a410 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sun, 20 Jun 2021 17:23:32 +0200 Subject: [PATCH 21/24] readability --- .../repositories/azure/AzureBlobContainerRetriesTests.java | 5 ++--- .../repositories/s3/S3BlobContainerRetriesTests.java | 4 ++-- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerRetriesTests.java b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerRetriesTests.java index 1029c627384d0..275f972bfcbbc 100644 --- a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerRetriesTests.java +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerRetriesTests.java @@ -398,10 +398,9 @@ public void testWriteLargeBlobStreaming() throws Exception { final int blobSize = (int) ByteSizeUnit.MB.toBytes(10); final byte[] data = randomBytes(blobSize); - int nbBlocks = (int) Math.ceil((double) data.length / (double) ByteSizeUnit.MB.toBytes(1)); final int nbErrors = 2; // we want all requests to fail at least once - final AtomicInteger countDownUploads = new AtomicInteger(nbErrors * nbBlocks); + final AtomicInteger counterUploads = new AtomicInteger(0); final AtomicLong bytesReceived = new AtomicLong(0L); final CountDown countDownComplete = new CountDown(nbErrors); @@ -415,7 +414,7 @@ public void testWriteLargeBlobStreaming() throws Exception { final String blockId = params.get("blockid"); assert Strings.hasText(blockId) == false || AzureFixtureHelper.assertValidBlockId(blockId); - if (Strings.hasText(blockId) && (countDownUploads.decrementAndGet() % 2 == 0)) { + if (Strings.hasText(blockId) && (counterUploads.incrementAndGet() % 2 == 0)) { final BytesReference blockData = Streams.readFully(exchange.getRequestBody()); blocks.put(blockId, blockData); bytesReceived.addAndGet(blockData.length()); diff --git a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java index 73905371c6c78..852cc578d8d3e 100644 --- a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java +++ b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java @@ -310,7 +310,7 @@ public void testWriteLargeBlobStreaming() throws Exception { final int nbErrors = 2; // we want all requests to fail at least once final CountDown countDownInitiate = new CountDown(nbErrors); - final AtomicInteger countDownUploads = new AtomicInteger(nbErrors * (parts + 1)); + final AtomicInteger counterUploads = new AtomicInteger(0); final AtomicLong bytesReceived = new AtomicLong(0L); final CountDown countDownComplete = new CountDown(nbErrors); @@ -340,7 +340,7 @@ public void testWriteLargeBlobStreaming() throws Exception { MD5DigestCalculatingInputStream md5 = new MD5DigestCalculatingInputStream(exchange.getRequestBody()); BytesReference bytes = Streams.readFully(md5); - if (countDownUploads.decrementAndGet() % 2 == 0) { + if (counterUploads.incrementAndGet() % 2 == 0) { bytesReceived.addAndGet(bytes.length()); exchange.getResponseHeaders().add("ETag", Base16.encodeAsString(md5.getMd5Digest())); exchange.sendResponseHeaders(HttpStatus.SC_OK, -1); From 039db6a6d6768a1e990cf572588a270a547fba6e Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 24 Jun 2021 12:34:22 +0200 Subject: [PATCH 22/24] PR comments --- .../elasticsearch/repositories/azure/AzureBlobStore.java | 5 +++-- .../org/elasticsearch/repositories/s3/S3BlobContainer.java | 4 ++-- .../repositories/blobstore/ChunkedBlobOutputStream.java | 6 +++--- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java index 8657e11d56acb..611e1c47e839f 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java @@ -413,7 +413,7 @@ protected void flushBuffer() { @Override protected void onAllPartsReady() { - if (written == 0L) { + if (flushedBytes == 0L) { writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); } else { flushBuffer(); @@ -424,7 +424,8 @@ protected void onAllPartsReady() { @Override protected void onFailure() { - // TODO: here and in multi-part upload, should we clean up uploaded blobs? + // Nothing to do here, already uploaded blocks will be GCed by Azure after a week. + // see https://docs.microsoft.com/en-us/rest/api/storageservices/put-block#remarks } }) { writer.accept(out); diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java index c7a8db53445ce..9fa5a198a450f 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java @@ -151,7 +151,7 @@ private void flushBuffer(boolean lastPart) throws IOException { if (buffer.size() == 0) { return; } - if (written == 0L) { + if (flushedBytes == 0L) { assert lastPart == false : "use single part upload if there's only a single part"; uploadId.set(SocketAccess.doPrivileged(() -> clientReference.client().initiateMultipartUpload(initiateMultiPartUpload(blobName)).getUploadId())); @@ -169,7 +169,7 @@ private void flushBuffer(boolean lastPart) throws IOException { @Override protected void onAllPartsReady() throws IOException { - if (written == 0L) { + if (flushedBytes == 0L) { writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); } else { flushBuffer(true); diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java index 25cad3dd0dd66..72567fb5b3ac1 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java @@ -52,9 +52,9 @@ public abstract class ChunkedBlobOutputStream extends OutputStream { protected boolean successful = false; /** - * Number of bytes written to storage writeBlso far. + * Number of bytes flushed to blob storage so far. */ - protected long written = 0L; + protected long flushedBytes = 0L; protected ChunkedBlobOutputStream(BigArrays bigArrays, long maxBytesToBuffer) { this.bigArrays = bigArrays; @@ -101,7 +101,7 @@ public final void markSuccess() { * @param partId part identifier to track for use when closing */ protected final void finishPart(T partId) { - written += buffer.size(); + flushedBytes += buffer.size(); parts.add(partId); buffer.close(); // only need a new buffer if we're not done yet From e37d207f3e3455e6bf2c3893ea1c26006b7ff257 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 24 Jun 2021 16:54:11 +0200 Subject: [PATCH 23/24] CR: comments --- .../repositories/azure/AzureBlobStore.java | 2 +- .../azure/AzureBlobContainerRetriesTests.java | 4 ++-- .../repositories/s3/S3BlobContainer.java | 2 +- .../s3/S3BlobContainerRetriesTests.java | 6 +++--- .../blobstore/ChunkedBlobOutputStream.java | 15 +++++++++++++-- 5 files changed, 20 insertions(+), 9 deletions(-) diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java index 611e1c47e839f..35482bbe93fa0 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java @@ -412,7 +412,7 @@ protected void flushBuffer() { } @Override - protected void onAllPartsReady() { + protected void onCompletion() { if (flushedBytes == 0L) { writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); } else { diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerRetriesTests.java b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerRetriesTests.java index 275f972bfcbbc..0909fc7a5e237 100644 --- a/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerRetriesTests.java +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerRetriesTests.java @@ -405,7 +405,7 @@ public void testWriteLargeBlobStreaming() throws Exception { final CountDown countDownComplete = new CountDown(nbErrors); final Map blocks = new ConcurrentHashMap<>(); - httpServer.createContext("/account/container/write_large_blob", exchange -> { + httpServer.createContext("/account/container/write_large_blob_streaming", exchange -> { if ("PUT".equals(exchange.getRequestMethod())) { final Map params = new HashMap<>(); @@ -453,7 +453,7 @@ public void testWriteLargeBlobStreaming() throws Exception { }); final BlobContainer blobContainer = createBlobContainer(maxRetries); - blobContainer.writeBlob("write_large_blob", false, randomBoolean(), out -> { + blobContainer.writeBlob("write_large_blob_streaming", false, randomBoolean(), out -> { int outstanding = data.length; while (outstanding > 0) { if (randomBoolean()) { diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java index 9fa5a198a450f..c5d9cc65531a9 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java @@ -168,7 +168,7 @@ private void flushBuffer(boolean lastPart) throws IOException { } @Override - protected void onAllPartsReady() throws IOException { + protected void onCompletion() throws IOException { if (flushedBytes == 0L) { writeBlob(blobName, buffer.bytes(), failIfAlreadyExists); } else { diff --git a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java index 852cc578d8d3e..86874bc9e2c40 100644 --- a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java +++ b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java @@ -324,7 +324,7 @@ public void testWriteLargeBlobStreaming() throws Exception { byte[] response = ("\n" + "\n" + " bucket\n" + - " write_large_blob\n" + + " write_large_blob_streaming\n" + " TEST\n" + "").getBytes(StandardCharsets.UTF_8); exchange.getResponseHeaders().add("Content-Type", "application/xml"); @@ -356,7 +356,7 @@ public void testWriteLargeBlobStreaming() throws Exception { byte[] response = ("\n" + "\n" + " bucket\n" + - " write_large_blob\n" + + " write_large_blob_streaming\n" + "").getBytes(StandardCharsets.UTF_8); exchange.getResponseHeaders().add("Content-Type", "application/xml"); exchange.sendResponseHeaders(HttpStatus.SC_OK, response.length); @@ -379,7 +379,7 @@ public void testWriteLargeBlobStreaming() throws Exception { } }); - blobContainer.writeBlob("write_large_blob", false, randomBoolean(), out -> { + blobContainer.writeBlob("write_large_blob_streaming", false, randomBoolean(), out -> { final byte[] buffer = new byte[16 * 1024]; long outstanding = blobSize; while (outstanding > 0) { diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java index 72567fb5b3ac1..226da2f76c104 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChunkedBlobOutputStream.java @@ -7,6 +7,7 @@ */ package org.elasticsearch.repositories.blobstore; +import org.apache.lucene.store.AlreadyClosedException; import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.core.Releasables; @@ -51,6 +52,11 @@ public abstract class ChunkedBlobOutputStream extends OutputStream { */ protected boolean successful = false; + /** + * Is set to {@code true} once this stream has been closed. + */ + private boolean closed = false; + /** * Number of bytes flushed to blob storage so far. */ @@ -76,9 +82,14 @@ public final void write(byte[] b, int off, int len) throws IOException { @Override public final void close() throws IOException { + if (closed) { + assert false : "this output stream should only be closed once"; + throw new AlreadyClosedException("already closed"); + } + closed = true; try { if (successful) { - onAllPartsReady(); + onCompletion(); } else { onFailure(); } @@ -122,7 +133,7 @@ protected final void finishPart(T partId) { * Invoked once all write chunks/parts are ready to be combined into the final blob. Implementations must invoke the necessary logic * for combining the uploaded chunks into the final blob in this method. */ - protected abstract void onAllPartsReady() throws IOException; + protected abstract void onCompletion() throws IOException; /** * Invoked in case writing all chunks of data to storage failed. Implementations should run any cleanup required for the already From 4340b39a990d97ac772320697c88e497d8242e3d Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 29 Jun 2021 10:34:52 +0200 Subject: [PATCH 24/24] CR: comments --- .../repositories/s3/S3BlobContainerRetriesTests.java | 2 +- .../repositories/blobstore/ChecksumBlobStoreFormat.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java index 86874bc9e2c40..fb26d06a8a3ce 100644 --- a/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java +++ b/plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3BlobContainerRetriesTests.java @@ -314,7 +314,7 @@ public void testWriteLargeBlobStreaming() throws Exception { final AtomicLong bytesReceived = new AtomicLong(0L); final CountDown countDownComplete = new CountDown(nbErrors); - httpServer.createContext("/bucket/write_large_blob", exchange -> { + httpServer.createContext("/bucket/write_large_blob_streaming", exchange -> { final long contentLength = Long.parseLong(exchange.getRequestHeaders().getFirst("Content-Length")); if ("POST".equals(exchange.getRequestMethod()) diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java index cc629251557a9..0425213c9a650 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java @@ -276,7 +276,7 @@ public void write(T obj, BlobContainer blobContainer, String name, boolean compr public void serialize(final T obj, final String blobName, final boolean compress, OutputStream outputStream) throws IOException { try ( OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput( - "ChecksumBlobStoreFormat.writeBlob(blob=\"" + blobName + "\")", + "ChecksumBlobStoreFormat.serialize(blob=\"" + blobName + "\")", blobName, org.elasticsearch.common.io.Streams.noCloseStream(outputStream), BUFFER_SIZE