Date: Wed, 3 Sep 2025 14:53:18 +0100
Subject: [PATCH 09/25] Moved the limit logic out of the streams submodule and
into BlobStoreRepository
---
.../common/io/stream/BytesStreamOutput.java | 16 +-
.../stream/ReleasableBytesStreamOutput.java | 4 -
.../common/io/stream/StreamInput.java | 2 +-
.../blobstore/BlobStoreRepository.java | 14 +-
.../io/stream/BytesStreamOutputTests.java | 145 ------------------
.../common/io/stream/BytesStreamsTests.java | 33 ++--
.../blobstore/BlobStoreRepositoryTests.java | 133 ++++++++++++++--
7 files changed, 144 insertions(+), 203 deletions(-)
delete mode 100644 server/src/test/java/org/elasticsearch/common/io/stream/BytesStreamOutputTests.java
diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java b/server/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java
index 2914327f5dd79..18394a26a4192 100644
--- a/server/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java
+++ b/server/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java
@@ -33,7 +33,6 @@ public class BytesStreamOutput extends BytesStream {
@Nullable
protected ByteArray bytes;
protected int count;
- protected int maximumSize;
/**
* Create a non recycling {@link BytesStreamOutput} with an initial capacity of 0.
@@ -55,15 +54,10 @@ public BytesStreamOutput(int expectedSize) {
}
protected BytesStreamOutput(int expectedSize, BigArrays bigArrays) {
- this(expectedSize, bigArrays, Integer.MAX_VALUE);
- }
-
- protected BytesStreamOutput(int expectedSize, BigArrays bigArrays, int maximumSize) {
this.bigArrays = bigArrays;
if (expectedSize != 0) {
this.bytes = bigArrays.newByteArray(expectedSize, false);
}
- this.maximumSize = maximumSize;
}
@Override
@@ -177,8 +171,8 @@ private static void copyToArray(BytesReference bytesReference, byte[] arr) {
}
protected void ensureCapacity(long offset) {
- if (offset > this.maximumSize) {
- throw new IllegalArgumentException(getClass().getSimpleName() + " has exceeded it's max size of " + this.maximumSize);
+ if (offset > Integer.MAX_VALUE) {
+ throw new IllegalArgumentException(getClass().getSimpleName() + " cannot hold more than 2GB of data");
}
if (bytes == null) {
this.bytes = bigArrays.newByteArray(BigArrays.overSize(offset, PageCacheRecycler.PAGE_SIZE_IN_BYTES, 1), false);
@@ -187,10 +181,4 @@ protected void ensureCapacity(long offset) {
}
}
- public boolean hasCapacity(int length) {
- if (length < 0) {
- throw new IllegalArgumentException("Negative length");
- }
- return count + length <= maximumSize;
- }
}
diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/ReleasableBytesStreamOutput.java b/server/src/main/java/org/elasticsearch/common/io/stream/ReleasableBytesStreamOutput.java
index aa585a8171cd0..f9f66c9b3eb9f 100644
--- a/server/src/main/java/org/elasticsearch/common/io/stream/ReleasableBytesStreamOutput.java
+++ b/server/src/main/java/org/elasticsearch/common/io/stream/ReleasableBytesStreamOutput.java
@@ -30,10 +30,6 @@ public ReleasableBytesStreamOutput(BigArrays bigarrays) {
this(PageCacheRecycler.PAGE_SIZE_IN_BYTES, bigarrays);
}
- public ReleasableBytesStreamOutput(BigArrays bigArrays, int maximumSize) {
- super(PageCacheRecycler.PAGE_SIZE_IN_BYTES, bigArrays, maximumSize);
- }
-
public ReleasableBytesStreamOutput(int expectedSize, BigArrays bigArrays) {
super(expectedSize, bigArrays);
}
diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java b/server/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java
index 47f61c21099cb..7dffc253c6219 100644
--- a/server/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java
+++ b/server/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java
@@ -72,7 +72,7 @@
* it means that the "barrier to entry" for adding new methods to this class is relatively low even though it is a shared class with code
* everywhere. That being said, this class deals primarily with {@code List}s rather than Arrays. For the most part calls should adapt to
* lists, either by storing {@code List}s internally or just converting to and from a {@code List} when calling. This comment is repeated
- * on {@link StreamInput}.
+ * on {@link StreamOutput}.
*/
public abstract class StreamInput extends InputStream {
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 e5469216d657c..ecd95a60134a9 100644
--- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
+++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
@@ -1674,6 +1674,7 @@ void writeTo(StreamOutput out) throws IOException {
}
}
+ private final int shardDeleteResultsMaxSize;
/**
*
* Shard-level results, i.e. a sequence of {@link ShardSnapshotMetaDeleteResult} objects, except serialized, concatenated, and
@@ -1705,9 +1706,9 @@ void writeTo(StreamOutput out) throws IOException {
);
ShardBlobsToDelete(Settings settings) {
- int maxSizeOfShardDeleteResults = calculateMaximumShardDeleteResultsSize(settings);
- if (maxSizeOfShardDeleteResults > 0) {
- this.shardDeleteResults = new ReleasableBytesStreamOutput(bigArrays, maxSizeOfShardDeleteResults);
+ this.shardDeleteResultsMaxSize = calculateMaximumShardDeleteResultsSize(settings);
+ if (this.shardDeleteResultsMaxSize > 0) {
+ this.shardDeleteResults = new ReleasableBytesStreamOutput(bigArrays);
this.compressed = new OutputStreamStreamOutput(
new BufferedOutputStream(
new DeflaterOutputStream(Streams.flushOnCloseStream(shardDeleteResults)),
@@ -1724,7 +1725,7 @@ void writeTo(StreamOutput out) throws IOException {
}
/**
- * Calculates the maximum size of the shardDeleteResults BytesStreamOutput
+ * Calculates the maximum size of the shardDeleteResults BytesStreamOutput.
* The size should at most be 2GB, but no more than 25% of the total remaining heap space
* @return The maximum number of bytes the shardDeleteResults BytesStreamOutput can consume in the heap
*/
@@ -1759,14 +1760,15 @@ synchronized void addShardDeleteResult(
try {
shardGenerationsBuilder.put(indexId, shardId, newGeneration);
- // Calculate how much space we'd need to write
+ // Calculate how many bytes we want to write
int bytesToWriteIndexId = StreamOutput.bytesInString(indexId.getId());
int bytesToWriteShardId = StreamOutput.bytesInVInt(shardId);
int bytesToWriteBlobsToDelete = StreamOutput.bytesInStringCollection(blobsToDelete);
int totalBytesRequired = bytesToWriteIndexId + bytesToWriteShardId + bytesToWriteBlobsToDelete;
+ int i = shardDeleteResults.size();
// Only perform the write if there is capacity left
- if (shardDeleteResults.hasCapacity(totalBytesRequired)) {
+ if (shardDeleteResults.size() + totalBytesRequired <= shardDeleteResultsMaxSize) {
new ShardSnapshotMetaDeleteResult(Objects.requireNonNull(indexId.getId()), shardId, blobsToDelete).writeTo(compressed);
resultCount += 1;
} else {
diff --git a/server/src/test/java/org/elasticsearch/common/io/stream/BytesStreamOutputTests.java b/server/src/test/java/org/elasticsearch/common/io/stream/BytesStreamOutputTests.java
deleted file mode 100644
index 9a52f63dd9be7..0000000000000
--- a/server/src/test/java/org/elasticsearch/common/io/stream/BytesStreamOutputTests.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public
- * License v3.0 only", or the "Server Side Public License, v 1".
- */
-
-package org.elasticsearch.common.io.stream;
-
-import org.elasticsearch.common.bytes.BytesArray;
-import org.elasticsearch.common.bytes.BytesReference;
-import org.elasticsearch.common.util.BigArrays;
-import org.elasticsearch.test.ESTestCase;
-import org.junit.Before;
-
-public class BytesStreamOutputTests extends ESTestCase {
-
- private BytesStreamOutput stream;
-
- @Before
- public void setUp() throws Exception {
- super.setUp();
- stream = new BytesStreamOutput();
- }
-
- public void testDefaultConstructor() {
- assertEquals(0, stream.size());
- assertEquals(0, stream.position());
- assertEquals(BytesArray.EMPTY, stream.bytes());
- }
-
- public void testConstructorWithExpectedSize() {
- BytesStreamOutput s = new BytesStreamOutput(randomNonNegativeInt());
- assertEquals(0, s.size());
- assertEquals(0, s.position());
- }
-
- public void testConstructorWithMaximumSize() {
- int maximumSize = randomIntBetween(0, Integer.MAX_VALUE);
- BytesStreamOutput s = new BytesStreamOutput(randomNonNegativeInt(), BigArrays.NON_RECYCLING_INSTANCE, maximumSize);
- assertEquals(0, s.size());
- assertEquals(0, s.position());
- assertEquals(maximumSize, s.maximumSize);
- }
-
- public void testWriteByte() {
- byte i1 = randomByte();
- byte i2 = randomByte();
- stream.writeByte(i1);
- stream.writeByte(i2);
- assertEquals(2, stream.size());
- assertEquals(2, stream.position());
- assertEquals(i1, stream.bytes().get(0));
- assertEquals(i2, stream.bytes().get(1));
- }
-
- public void testWriteBytes() {
- int maxSize = randomIntBetween(10, 100);
- byte[] data = randomByteArrayOfLength(maxSize);
- stream.writeBytes(data, 0, data.length);
- assertEquals(data.length, stream.size());
- for (int i = 0; i < data.length; i++) {
- assertEquals(data[i], stream.bytes().get(i));
- }
- }
-
- public void testWriteBytesWithOffset() {
- int maxSize = randomIntBetween(10, 100);
- byte[] data = randomByteArrayOfLength(maxSize);
- int offset = randomIntBetween(0, maxSize / 2);
- int length = randomIntBetween(0, maxSize / 2);
- stream.writeBytes(data, offset, length); // should write 30, 40
- assertEquals(length, stream.size());
- for (int i = 0; i < stream.size(); i++) {
- assertEquals(stream.bytes().get(i), data[i + offset]);
- }
- }
-
- public void testHasCapacityWithPositiveLength() {
- int i1 = randomIntBetween(1, 100);
- int i2 = randomIntBetween(1, 100);
- BytesStreamOutput s = new BytesStreamOutput(0, BigArrays.NON_RECYCLING_INSTANCE, i1 + i2);
- assertTrue(s.hasCapacity(i1));
- s.writeBytes(randomByteArrayOfLength(i1), 0, i1);
- assertTrue(s.hasCapacity(i2));
- s.writeBytes(randomByteArrayOfLength(i2), 0, i2);
- assertFalse(s.hasCapacity(1));
- }
-
- public void testHasCapacityWithNegativeLength() {
- assertThrows(IllegalArgumentException.class, () -> stream.hasCapacity(randomNegativeInt()));
- }
-
- public void testCapacityExceededThrows() {
- int maximumSize = randomIntBetween(0, 100);
- BytesStreamOutput s = new BytesStreamOutput(0, BigArrays.NON_RECYCLING_INSTANCE, maximumSize);
- assertThrows(IllegalArgumentException.class, () -> s.writeBytes(randomByteArrayOfLength(maximumSize + 1), 0, maximumSize + 1));
- }
-
- public void testResetShrinksAndResetsCount() {
- int byteArrayLength = randomIntBetween(0, 100);
- stream.writeBytes(randomByteArrayOfLength(byteArrayLength), 0, byteArrayLength);
- stream.reset();
- assertEquals(0, stream.size());
- // After reset, should be able to write again
- byte b = randomByte();
- stream.writeByte(b);
- assertEquals(1, stream.size());
- assertEquals(b, stream.bytes().get(0));
- }
-
- public void testSeekAndSkip() {
- int byteArrayLength = randomIntBetween(0, 100);
- byte[] byteArray = randomByteArrayOfLength(byteArrayLength);
- stream.writeBytes(byteArray, 0, byteArrayLength);
-
- int seekPosition = randomIntBetween(byteArrayLength, byteArrayLength + 100);
- stream.seek(seekPosition);
- assertEquals(seekPosition, stream.size());
-
- int skipLength = randomIntBetween(0, 100);
- stream.skip(skipLength);
- assertEquals(seekPosition + skipLength, stream.size());
- }
-
- public void testCopyBytes() {
- int byteArrayLength = randomIntBetween(0, 100);
- byte[] byteArray = randomByteArrayOfLength(byteArrayLength);
- stream.writeBytes(byteArray, 0, byteArrayLength);
- BytesReference copy = stream.copyBytes();
- assertEquals(byteArrayLength, copy.length());
- for (int i = 0; i < byteArrayLength; i++) {
- assertEquals(byteArray[i], copy.get(i));
- }
- }
-
- public void testFlushAndCloseNoop() {
- stream.writeByte(randomByte());
- stream.flush(); // should do nothing
- stream.close(); // should do nothing
- assertEquals(1, stream.size());
- }
-}
diff --git a/server/src/test/java/org/elasticsearch/common/io/stream/BytesStreamsTests.java b/server/src/test/java/org/elasticsearch/common/io/stream/BytesStreamsTests.java
index f7d4fe77dd1de..811f9b692399a 100644
--- a/server/src/test/java/org/elasticsearch/common/io/stream/BytesStreamsTests.java
+++ b/server/src/test/java/org/elasticsearch/common/io/stream/BytesStreamsTests.java
@@ -16,7 +16,6 @@
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.lucene.BytesRefs;
-import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.Maps;
import org.elasticsearch.common.util.PageCacheRecycler;
import org.elasticsearch.core.TimeValue;
@@ -64,7 +63,7 @@ public void testEmpty() throws Exception {
out.close();
}
- public void testSingleByte() {
+ public void testSingleByte() throws Exception {
TestStreamOutput out = new TestStreamOutput();
assertEquals(0, out.size());
@@ -79,7 +78,7 @@ public void testSingleByte() {
out.close();
}
- public void testSingleShortPage() {
+ public void testSingleShortPage() throws Exception {
TestStreamOutput out = new TestStreamOutput();
int expectedSize = 10;
@@ -96,7 +95,7 @@ public void testSingleShortPage() {
out.close();
}
- public void testIllegalBulkWrite() {
+ public void testIllegalBulkWrite() throws Exception {
TestStreamOutput out = new TestStreamOutput();
// bulk-write with wrong args
@@ -139,7 +138,7 @@ public void testSingleFullPageBulkWrite() throws Exception {
out.close();
}
- public void testSingleFullPageBulkWriteWithOffset() {
+ public void testSingleFullPageBulkWriteWithOffset() throws Exception {
TestStreamOutput out = new TestStreamOutput();
int initialOffset = 10;
@@ -158,7 +157,7 @@ public void testSingleFullPageBulkWriteWithOffset() {
out.close();
}
- public void testSingleFullPageBulkWriteWithOffsetCrossover() {
+ public void testSingleFullPageBulkWriteWithOffsetCrossover() throws Exception {
TestStreamOutput out = new TestStreamOutput();
int initialOffset = 10;
@@ -177,7 +176,7 @@ public void testSingleFullPageBulkWriteWithOffsetCrossover() {
out.close();
}
- public void testSingleFullPage() {
+ public void testSingleFullPage() throws Exception {
TestStreamOutput out = new TestStreamOutput();
int expectedSize = PageCacheRecycler.BYTE_PAGE_SIZE;
@@ -194,7 +193,7 @@ public void testSingleFullPage() {
out.close();
}
- public void testOneFullOneShortPage() {
+ public void testOneFullOneShortPage() throws Exception {
TestStreamOutput out = new TestStreamOutput();
int expectedSize = PageCacheRecycler.BYTE_PAGE_SIZE + 10;
@@ -211,7 +210,7 @@ public void testOneFullOneShortPage() {
out.close();
}
- public void testTwoFullOneShortPage() {
+ public void testTwoFullOneShortPage() throws Exception {
TestStreamOutput out = new TestStreamOutput();
int expectedSize = (PageCacheRecycler.BYTE_PAGE_SIZE * 2) + 1;
@@ -228,9 +227,8 @@ public void testTwoFullOneShortPage() {
out.close();
}
- public void testSeek() {
- int maximumSize = randomIntBetween(0, Integer.MAX_VALUE);
- BytesStreamOutput out = new BytesStreamOutput(0, BigArrays.NON_RECYCLING_INSTANCE, maximumSize);
+ public void testSeek() throws Exception {
+ BytesStreamOutput out = new BytesStreamOutput();
int position = 0;
assertEquals(position, out.position());
@@ -243,14 +241,13 @@ public void testSeek() {
assertEquals(position, BytesReference.toBytes(out.bytes()).length);
IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> out.seek(Integer.MAX_VALUE + 1L));
- assertEquals("BytesStreamOutput has exceeded it's max size of " + maximumSize, iae.getMessage());
+ assertEquals("BytesStreamOutput cannot hold more than 2GB of data", iae.getMessage());
out.close();
}
- public void testSkip() {
- int maximumSize = randomIntBetween(0, Integer.MAX_VALUE);
- BytesStreamOutput out = new BytesStreamOutput(0, BigArrays.NON_RECYCLING_INSTANCE, maximumSize);
+ public void testSkip() throws Exception {
+ BytesStreamOutput out = new BytesStreamOutput();
int position = 0;
assertEquals(position, out.position());
@@ -259,8 +256,8 @@ public void testSkip() {
out.skip(forward);
assertEquals(position + forward, out.position());
- IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> out.skip(maximumSize - 50));
- assertEquals("BytesStreamOutput has exceeded it's max size of " + maximumSize, iae.getMessage());
+ IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> out.skip(Integer.MAX_VALUE - 50));
+ assertEquals("BytesStreamOutput cannot hold more than 2GB of data", iae.getMessage());
out.close();
}
diff --git a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
index 89eeba80f64bf..5164f29aa8a37 100644
--- a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
+++ b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
@@ -42,6 +42,7 @@
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.blobstore.OperationPurpose;
import org.elasticsearch.common.bytes.BytesArray;
+import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
@@ -758,10 +759,12 @@ public void testUuidCreationLogging() {
// =============== Shard Blobs to Delete Tests =================
+ /*
+ There is sufficient heap space to perform all operations on shardBlobsToDelete
+ */
@TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
public void testShardBlobsToDeleteWithSufficientHeapSpace() {
// When the heap memory is above 2GB (Integer.MAX_VALUE) then we expect it to be limited to 2GB
- // inside ShardBlobsToDelete.calculateMaximumShardDeleteResultsSize
long heapMemory = randomLongBetween(Integer.MAX_VALUE / 2, Integer.MAX_VALUE * 2L);
Settings.Builder settings = Settings.builder().put("repositories.blobstore.max_shard_delete_results_size", heapMemory + "b");
final var repo = setupRepo(settings);
@@ -783,13 +786,8 @@ public void testShardBlobsToDeleteWithSufficientHeapSpace() {
final var shardId = shard;
final var shardGeneration = new ShardGeneration(randomUUID());
expectedShardGenerations.put(indexId, shard, shardGeneration);
- final var blobsToDelete = randomList(
- 100,
- () -> randomFrom(METADATA_PREFIX, INDEX_FILE_PREFIX, SNAPSHOT_PREFIX) + randomUUID() + randomFrom(
- "",
- METADATA_BLOB_NAME_SUFFIX
- )
- );
+ final var blobsToDelete = generateRandomBlobsToDelete();
+
blobCount += blobsToDelete.size();
final var indexPath = repo.basePath()
.add("indices")
@@ -821,6 +819,101 @@ public void testShardBlobsToDeleteWithSufficientHeapSpace() {
}
}
+ /*
+ There is limited heap space for N-1 writes, but will run out for the Nth write
+ */
+ @TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
+ public void testShardBlobsToDeleteWithLimitedHeapSpace() {
+ final IndexId indexId = new IndexId(randomIdentifier(), randomUUID());
+ final int shardId = between(1, 30);
+ final List blobsToDelete = generateRandomBlobsToDelete();
+
+ final ShardGeneration shardGeneration = new ShardGeneration(randomUUID());
+ final var expectedShardGenerations = ShardGenerations.builder().put(indexId, shardId, shardGeneration).build();
+
+ int bytesToWriteIndexId = StreamOutput.bytesInString(indexId.getId());
+ int bytesToWriteShardId = StreamOutput.bytesInVInt(shardId);
+ int bytesToWriteBlobsToDelete = StreamOutput.bytesInStringCollection(blobsToDelete);
+ int totalBytesRequired = bytesToWriteIndexId + bytesToWriteShardId + bytesToWriteBlobsToDelete;
+
+ Settings.Builder settings = Settings.builder().put("repositories.blobstore.max_shard_delete_results_size", totalBytesRequired + "b");
+ final var repo = setupRepo(settings);
+ try (var shardBlobsToDelete = repo.new ShardBlobsToDelete(settings.build())) {
+
+ // The first time we expect the write to succeed
+ try (var mockLog = MockLog.capture(BlobStoreRepository.class)) {
+ addWarnLogCountExpectation(mockLog, 0);
+
+ final var expectedBlobsToDelete = new HashSet();
+ final var indexPath = repo.basePath()
+ .add("indices")
+ .add(indexId.getId())
+ .add(Integer.toString(shardId))
+ .buildAsString();
+ for (final var blobToDelete : blobsToDelete) {
+ expectedBlobsToDelete.add(indexPath + blobToDelete);
+ }
+
+ final var countDownLatch = new CountDownLatch(1);
+ try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
+ repo.threadPool()
+ .generic()
+ .execute(
+ ActionRunnable.run(
+ refs.acquireListener(),
+ () -> shardBlobsToDelete.addShardDeleteResult(indexId, shardId, shardGeneration, blobsToDelete)
+ )
+ );
+ }
+ safeAwait(countDownLatch);
+ assertEquals(expectedShardGenerations, shardBlobsToDelete.getUpdatedShardGenerations());
+ shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
+ assertThat(expectedBlobsToDelete, empty());
+ mockLog.assertAllExpectationsMatched();
+ }
+
+ // The second time we expect the write to fail
+ try (var mockLog = MockLog.capture(BlobStoreRepository.class)) {
+ addWarnLogCountExpectation(mockLog, 1);
+
+ final var expectedBlobsToDelete = new HashSet();
+ final var indexPath = repo.basePath()
+ .add("indices")
+ .add(indexId.getId())
+ .add(Integer.toString(shardId))
+ .buildAsString();
+ for (final var blobToDelete : blobsToDelete) {
+ expectedBlobsToDelete.add(indexPath + blobToDelete);
+ }
+
+ final var countDownLatch = new CountDownLatch(1);
+ try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
+ repo.threadPool()
+ .generic()
+ .execute(
+ ActionRunnable.run(
+ refs.acquireListener(),
+ () -> shardBlobsToDelete.addShardDeleteResult(indexId, shardId, shardGeneration, blobsToDelete)
+ )
+ );
+ }
+ safeAwait(countDownLatch);
+ assertEquals(expectedShardGenerations, shardBlobsToDelete.getUpdatedShardGenerations());
+
+ int expectedBlobsToDeleteSizeBeforeRemoving = expectedBlobsToDelete.size();
+ var y = shardBlobsToDelete.getBlobPaths();
+ shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
+ assertEquals(expectedBlobsToDelete.size(), expectedBlobsToDeleteSizeBeforeRemoving);
+
+ mockLog.assertAllExpectationsMatched();
+ }
+ }
+ }
+
+ /*
+ There is insufficient / no heap space, so no writes to shardDeletesResults will succeed
+ */
+ @TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
public void testShardBlobsToDeleteWithOutHeapSpace() {
boolean noHeap = randomBoolean();
Settings.Builder settings;
@@ -864,13 +957,7 @@ public void testShardBlobsToDeleteWithOutHeapSpace() {
final var shardId = shard;
final var shardGeneration = new ShardGeneration(randomUUID());
expectedShardGenerations.put(indexId, shard, shardGeneration);
- final var blobsToDelete = randomList(
- 100,
- () -> randomFrom(METADATA_PREFIX, INDEX_FILE_PREFIX, SNAPSHOT_PREFIX) + randomUUID() + randomFrom(
- "",
- METADATA_BLOB_NAME_SUFFIX
- )
- );
+ final var blobsToDelete = generateRandomBlobsToDelete();
final var indexPath = repo.basePath()
.add("indices")
@@ -896,14 +983,30 @@ public void testShardBlobsToDeleteWithOutHeapSpace() {
safeAwait(countDownLatch);
assertEquals(expectedShardGenerations.build(), shardBlobsToDelete.getUpdatedShardGenerations());
+ /*
+ If there is no heap space, then getBlobPaths() returns an empty iterator
+ If there is heap space, but it was too small for all write operations to succeed,
+ only the blob paths of the successful writes will be return
+ */
int expectedBlobsToDeleteSizeBeforeRemoving = expectedBlobsToDelete.size();
shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
assertEquals(expectedBlobsToDelete.size(), expectedBlobsToDeleteSizeBeforeRemoving);
+
mockLog.assertAllExpectationsMatched();
}
}
}
+ private List generateRandomBlobsToDelete() {
+ return randomList(
+ 100,
+ () -> randomFrom(METADATA_PREFIX, INDEX_FILE_PREFIX, SNAPSHOT_PREFIX) + randomUUID() + randomFrom(
+ "",
+ METADATA_BLOB_NAME_SUFFIX
+ )
+ );
+ }
+
private void addWarnLogCountExpectation(MockLog mockLog, int expectedWarnLogsThrown) {
mockLog.addExpectation(new MockLog.LoggingExpectation() {
int count = 0;
From 0355c2aabf224c9d74d1814e39b7a2eb70224ac8 Mon Sep 17 00:00:00 2001
From: Joshua Adams
Date: Thu, 4 Sep 2025 11:50:36 +0100
Subject: [PATCH 10/25] Add tests
---
.../blobstore/BlobStoreRepository.java | 1 -
.../blobstore/BlobStoreRepositoryTests.java | 288 +++++++++---------
2 files changed, 145 insertions(+), 144 deletions(-)
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 d2ac8047e91f8..6b00fdf1e0a1d 100644
--- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
+++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
@@ -1766,7 +1766,6 @@ synchronized void addShardDeleteResult(
int bytesToWriteBlobsToDelete = StreamOutput.bytesInStringCollection(blobsToDelete);
int totalBytesRequired = bytesToWriteIndexId + bytesToWriteShardId + bytesToWriteBlobsToDelete;
- int i = shardDeleteResults.size();
// Only perform the write if there is capacity left
if (shardDeleteResults.size() + totalBytesRequired <= shardDeleteResultsMaxSize) {
new ShardSnapshotMetaDeleteResult(Objects.requireNonNull(indexId.getId()), shardId, blobsToDelete).writeTo(compressed);
diff --git a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
index 5164f29aa8a37..2a898ad1debfc 100644
--- a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
+++ b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
@@ -57,6 +57,7 @@
import org.elasticsearch.repositories.FinalizeSnapshotContext.UpdatedShardGenerations;
import org.elasticsearch.repositories.IndexId;
import org.elasticsearch.repositories.RepositoriesService;
+import org.elasticsearch.repositories.Repository;
import org.elasticsearch.repositories.RepositoryData;
import org.elasticsearch.repositories.RepositoryException;
import org.elasticsearch.repositories.RepositoryMissingException;
@@ -772,7 +773,7 @@ public void testShardBlobsToDeleteWithSufficientHeapSpace() {
// Ensure the logging is as expected
try (var mockLog = MockLog.capture(BlobStoreRepository.class)) {
// We expect every write to succeed
- addWarnLogCountExpectation(mockLog, 0);
+ addFailureToCleanUpDanglingBlobsUnseenExpectation(mockLog);
final var expectedShardGenerations = ShardGenerations.builder();
final var expectedBlobsToDelete = new HashSet();
@@ -820,186 +821,193 @@ public void testShardBlobsToDeleteWithSufficientHeapSpace() {
}
/*
- There is limited heap space for N-1 writes, but will run out for the Nth write
+ There is sufficient heap space for the first write, but not the second
*/
@TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
public void testShardBlobsToDeleteWithLimitedHeapSpace() {
- final IndexId indexId = new IndexId(randomIdentifier(), randomUUID());
- final int shardId = between(1, 30);
- final List blobsToDelete = generateRandomBlobsToDelete();
+ TestShardSnapshotMetaDeleteResult blob1Result = generateTestShardSnapshotMetaDeleteResult(0, 20);
+ // blob1Result can be compressed up to 4x when written to shardsDeleteResults so make the second blob
+ // at least 4x larger than the first to guarantee this blob cannot also be written
+ TestShardSnapshotMetaDeleteResult blob2Result = generateTestShardSnapshotMetaDeleteResult(80, 100);
- final ShardGeneration shardGeneration = new ShardGeneration(randomUUID());
- final var expectedShardGenerations = ShardGenerations.builder().put(indexId, shardId, shardGeneration).build();
-
- int bytesToWriteIndexId = StreamOutput.bytesInString(indexId.getId());
- int bytesToWriteShardId = StreamOutput.bytesInVInt(shardId);
- int bytesToWriteBlobsToDelete = StreamOutput.bytesInStringCollection(blobsToDelete);
- int totalBytesRequired = bytesToWriteIndexId + bytesToWriteShardId + bytesToWriteBlobsToDelete;
+ final var expectedShardGenerations = ShardGenerations.builder()
+ .put(blob1Result.indexId, blob1Result.shardId, blob1Result.shardGeneration)
+ .put(blob2Result.indexId, blob2Result.shardId, blob2Result.shardGeneration)
+ .build();
+ int totalBytesRequired = calculateBytesRequiredToWriteShardSnapshotMetaDeleteResult(blob1Result);
Settings.Builder settings = Settings.builder().put("repositories.blobstore.max_shard_delete_results_size", totalBytesRequired + "b");
final var repo = setupRepo(settings);
try (var shardBlobsToDelete = repo.new ShardBlobsToDelete(settings.build())) {
-
- // The first time we expect the write to succeed
try (var mockLog = MockLog.capture(BlobStoreRepository.class)) {
- addWarnLogCountExpectation(mockLog, 0);
+ addFailureToCleanUpDanglingBlobsSeenExpectation(mockLog);
final var expectedBlobsToDelete = new HashSet();
- final var indexPath = repo.basePath()
- .add("indices")
- .add(indexId.getId())
- .add(Integer.toString(shardId))
- .buildAsString();
- for (final var blobToDelete : blobsToDelete) {
- expectedBlobsToDelete.add(indexPath + blobToDelete);
+ var countDownLatch = new CountDownLatch(1);
+ try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
+ final var indexPath1 = generateRepoPath(repo, blob1Result);
+ for (final var blobToDelete : blob1Result.blobsToDelete) {
+ expectedBlobsToDelete.add(indexPath1 + blobToDelete);
+ }
+ repo.threadPool()
+ .generic()
+ .execute(
+ ActionRunnable.run(
+ refs.acquireListener(),
+ () -> shardBlobsToDelete.addShardDeleteResult(blob1Result.indexId, blob1Result.shardId, blob1Result.shardGeneration, blob1Result.blobsToDelete)
+ )
+ );
}
+ safeAwait(countDownLatch);
- final var countDownLatch = new CountDownLatch(1);
+ countDownLatch = new CountDownLatch(1);
try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
- repo.threadPool()
- .generic()
- .execute(
- ActionRunnable.run(
- refs.acquireListener(),
- () -> shardBlobsToDelete.addShardDeleteResult(indexId, shardId, shardGeneration, blobsToDelete)
- )
- );
+ final var indexPath2 = generateRepoPath(repo, blob2Result);
+ for (final var blobToDelete : blob2Result.blobsToDelete) {
+ expectedBlobsToDelete.add(indexPath2 + blobToDelete);
+ }
+ repo.threadPool()
+ .generic()
+ .execute(
+ ActionRunnable.run(
+ refs.acquireListener(),
+ () -> shardBlobsToDelete.addShardDeleteResult(blob2Result.indexId, blob2Result.shardId, blob2Result.shardGeneration, blob2Result.blobsToDelete)
+ )
+ );
}
+
safeAwait(countDownLatch);
assertEquals(expectedShardGenerations, shardBlobsToDelete.getUpdatedShardGenerations());
shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
- assertThat(expectedBlobsToDelete, empty());
+ // None of the second writes succeeded and are therefore not returned by blobsToDelete()
+ assertEquals(expectedBlobsToDelete.size(), blob2Result.blobsToDelete.size());
mockLog.assertAllExpectationsMatched();
}
+ }
+ }
- // The second time we expect the write to fail
+ /*
+ There is heap space, but it's insufficient for any write requests
+ */
+ @TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
+ public void testShardBlobsToDeleteWithSmallHeapSpace() {
+ TestShardSnapshotMetaDeleteResult result = generateTestShardSnapshotMetaDeleteResult(0, 20);
+ final var expectedShardGenerations = ShardGenerations.builder()
+ .put(result.indexId, result.shardId, result.shardGeneration)
+ .build();
+
+ int totalBytesRequired = calculateBytesRequiredToWriteShardSnapshotMetaDeleteResult(result);
+ int heapSpace = randomIntBetween(1, totalBytesRequired - 1);
+ Settings.Builder settings = Settings.builder().put("repositories.blobstore.max_shard_delete_results_size", heapSpace + "b");
+ final var repo = setupRepo(settings);
+ try (var shardBlobsToDelete = repo.new ShardBlobsToDelete(settings.build())) {
try (var mockLog = MockLog.capture(BlobStoreRepository.class)) {
- addWarnLogCountExpectation(mockLog, 1);
+ addFailureToCleanUpDanglingBlobsSeenExpectation(mockLog);
final var expectedBlobsToDelete = new HashSet();
- final var indexPath = repo.basePath()
- .add("indices")
- .add(indexId.getId())
- .add(Integer.toString(shardId))
- .buildAsString();
- for (final var blobToDelete : blobsToDelete) {
- expectedBlobsToDelete.add(indexPath + blobToDelete);
- }
-
- final var countDownLatch = new CountDownLatch(1);
+ var countDownLatch = new CountDownLatch(1);
try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
+ final var indexPath = generateRepoPath(repo, result);
+ for (final var blobToDelete : result.blobsToDelete) {
+ expectedBlobsToDelete.add(indexPath + blobToDelete);
+ }
repo.threadPool()
.generic()
.execute(
ActionRunnable.run(
refs.acquireListener(),
- () -> shardBlobsToDelete.addShardDeleteResult(indexId, shardId, shardGeneration, blobsToDelete)
+ () -> shardBlobsToDelete.addShardDeleteResult(result.indexId, result.shardId, result.shardGeneration, result.blobsToDelete)
)
);
+
+
}
safeAwait(countDownLatch);
assertEquals(expectedShardGenerations, shardBlobsToDelete.getUpdatedShardGenerations());
-
- int expectedBlobsToDeleteSizeBeforeRemoving = expectedBlobsToDelete.size();
- var y = shardBlobsToDelete.getBlobPaths();
shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
- assertEquals(expectedBlobsToDelete.size(), expectedBlobsToDeleteSizeBeforeRemoving);
-
+ assertEquals(result.blobsToDelete.size(), expectedBlobsToDelete.size());
mockLog.assertAllExpectationsMatched();
}
}
}
/*
- There is insufficient / no heap space, so no writes to shardDeletesResults will succeed
+ There's no heap space, so we don't even attempt to write
*/
@TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
public void testShardBlobsToDeleteWithOutHeapSpace() {
- boolean noHeap = randomBoolean();
- Settings.Builder settings;
- if (noHeap) {
- // We have no heap for some reason
- settings = Settings.builder().put("repositories.blobstore.max_shard_delete_results_size", "0b");
- } else {
- // Set the heap stupidly low so that it fails
- settings = Settings.builder().put("repositories.blobstore.max_shard_delete_results_size", "1b");
- }
+ TestShardSnapshotMetaDeleteResult result = generateTestShardSnapshotMetaDeleteResult(0, 20);
+ final var expectedShardGenerations = ShardGenerations.builder()
+ .put(result.indexId, result.shardId, result.shardGeneration)
+ .build();
+ Settings.Builder settings = Settings.builder().put("repositories.blobstore.max_shard_delete_results_size", "0b");
final var repo = setupRepo(settings);
try (var shardBlobsToDelete = repo.new ShardBlobsToDelete(settings.build())) {
- // Ensure the logging is as expected
try (var mockLog = MockLog.capture(BlobStoreRepository.class)) {
- final var expectedShardGenerations = ShardGenerations.builder();
- final var expectedBlobsToDelete = new HashSet();
- final var countDownLatch = new CountDownLatch(1);
-
- int indexCount = between(0, 1000);
- List shardCounts = new ArrayList<>();
- int count = 0;
- for (int i = 0; i < indexCount; i++) {
- int shardCount = between(1, 30);
- shardCounts.add(shardCount);
- count += shardCount;
- }
-
- if (noHeap) {
- // If there is no heap we don't even attempt to write
- addWarnLogCountExpectation(mockLog, 0);
- } else {
- // We expect every write to fail
- addWarnLogCountExpectation(mockLog, count);
- }
+ addFailureToCleanUpDanglingBlobsUnseenExpectation(mockLog);
+ final var expectedBlobsToDelete = new HashSet();
+ var countDownLatch = new CountDownLatch(1);
try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
- for (int index = 0; index < indexCount; index++) {
- final var indexId = new IndexId(randomIdentifier(), randomUUID());
- for (int shard = 0; shard < shardCounts.get(index); shard++) {
- final var shardId = shard;
- final var shardGeneration = new ShardGeneration(randomUUID());
- expectedShardGenerations.put(indexId, shard, shardGeneration);
- final var blobsToDelete = generateRandomBlobsToDelete();
+ final var indexPath = generateRepoPath(repo, result);
+ for (final var blobToDelete : result.blobsToDelete) {
+ expectedBlobsToDelete.add(indexPath + blobToDelete);
+ }
+ repo.threadPool()
+ .generic()
+ .execute(
+ ActionRunnable.run(
+ refs.acquireListener(),
+ () -> shardBlobsToDelete.addShardDeleteResult(result.indexId, result.shardId, result.shardGeneration, result.blobsToDelete)
+ )
+ );
- final var indexPath = repo.basePath()
- .add("indices")
- .add(indexId.getId())
- .add(Integer.toString(shard))
- .buildAsString();
- for (final var blobToDelete : blobsToDelete) {
- expectedBlobsToDelete.add(indexPath + blobToDelete);
- }
- repo.threadPool()
- .generic()
- .execute(
- ActionRunnable.run(
- refs.acquireListener(),
- () -> shardBlobsToDelete.addShardDeleteResult(indexId, shardId, shardGeneration, blobsToDelete)
- )
- );
- }
- }
}
-
safeAwait(countDownLatch);
- assertEquals(expectedShardGenerations.build(), shardBlobsToDelete.getUpdatedShardGenerations());
-
- /*
- If there is no heap space, then getBlobPaths() returns an empty iterator
- If there is heap space, but it was too small for all write operations to succeed,
- only the blob paths of the successful writes will be return
- */
- int expectedBlobsToDeleteSizeBeforeRemoving = expectedBlobsToDelete.size();
+ assertEquals(expectedShardGenerations, shardBlobsToDelete.getUpdatedShardGenerations());
shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
- assertEquals(expectedBlobsToDelete.size(), expectedBlobsToDeleteSizeBeforeRemoving);
-
+ assertEquals(result.blobsToDelete.size(), expectedBlobsToDelete.size());
mockLog.assertAllExpectationsMatched();
}
}
}
+ private record TestShardSnapshotMetaDeleteResult(IndexId indexId, int shardId, Collection blobsToDelete, ShardGeneration shardGeneration) {}
+
+ private TestShardSnapshotMetaDeleteResult generateTestShardSnapshotMetaDeleteResult(int minBlobSize, int maxBlobSize) {
+ return new TestShardSnapshotMetaDeleteResult(
+ new IndexId(randomIdentifier(), randomUUID()),
+ between(1, 30),
+ generateRandomBlobsToDelete(minBlobSize, maxBlobSize),
+ new ShardGeneration(randomUUID())
+ );
+ }
+
+ private int calculateBytesRequiredToWriteShardSnapshotMetaDeleteResult(TestShardSnapshotMetaDeleteResult result) {
+ int bytesToWriteIndexId = StreamOutput.bytesInString(result.indexId.getId());
+ int bytesToWriteShardId = StreamOutput.bytesInVInt(result.shardId);
+ int bytesToWriteBlobsToDelete = StreamOutput.bytesInStringCollection(result.blobsToDelete);
+ return bytesToWriteIndexId + bytesToWriteShardId + bytesToWriteBlobsToDelete;
+ }
+
+ private String generateRepoPath(BlobStoreRepository repo, TestShardSnapshotMetaDeleteResult result) {
+ return repo.basePath()
+ .add("indices")
+ .add(result.indexId.getId())
+ .add(Integer.toString(result.shardId))
+ .buildAsString();
+ }
+
private List generateRandomBlobsToDelete() {
+ return generateRandomBlobsToDelete(0, 100);
+ }
+
+ private List generateRandomBlobsToDelete(int minSize, int maxSize) {
return randomList(
- 100,
+ minSize,
+ maxSize,
() -> randomFrom(METADATA_PREFIX, INDEX_FILE_PREFIX, SNAPSHOT_PREFIX) + randomUUID() + randomFrom(
"",
METADATA_BLOB_NAME_SUFFIX
@@ -1007,31 +1015,25 @@ private List generateRandomBlobsToDelete() {
);
}
- private void addWarnLogCountExpectation(MockLog mockLog, int expectedWarnLogsThrown) {
- mockLog.addExpectation(new MockLog.LoggingExpectation() {
- int count = 0;
-
- @Override
- public void match(LogEvent event) {
- if (event.getLevel() != Level.WARN) {
- return;
- }
- if (event.getLoggerName().equals(BlobStoreRepository.class.getCanonicalName()) == false) {
- return;
- }
-
- Pattern pattern = Pattern.compile("Failure to clean up the following dangling blobs");
- Matcher matcher = pattern.matcher(event.getMessage().getFormattedMessage());
-
- if (matcher.find()) {
- count++;
- }
- }
+ private void addFailureToCleanUpDanglingBlobsUnseenExpectation(MockLog mockLog) {
+ mockLog.addExpectation(
+ new MockLog.UnseenEventExpectation(
+ "failure to clean up dangling blobs warn logs",
+ "org.elasticsearch.repositories.blobstore.BlobStoreRepository",
+ Level.WARN,
+ "Failure to clean up the following dangling blobs"
+ )
+ );
+ }
- @Override
- public void assertMatched() {
- assertEquals(count, expectedWarnLogsThrown);
- }
- });
+ private void addFailureToCleanUpDanglingBlobsSeenExpectation(MockLog mockLog) {
+ mockLog.addExpectation(
+ new MockLog.SeenEventExpectation(
+ "failure to clean up dangling blobs warn logs",
+ "org.elasticsearch.repositories.blobstore.BlobStoreRepository",
+ Level.WARN,
+ "Failure to clean up the following dangling blobs"
+ )
+ );
}
}
From f072128db427d437ed013839310443167e4f2aa9 Mon Sep 17 00:00:00 2001
From: Joshua Adams
Date: Thu, 4 Sep 2025 14:01:56 +0100
Subject: [PATCH 11/25] Run ./gradlew spotlessApply precommit
---
.../blobstore/BlobStoreRepositoryTests.java | 63 +++++++++++--------
1 file changed, 36 insertions(+), 27 deletions(-)
diff --git a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
index 2a898ad1debfc..a8c3916755200 100644
--- a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
+++ b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
@@ -10,7 +10,6 @@
package org.elasticsearch.repositories.blobstore;
import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.core.LogEvent;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRunnable;
import org.elasticsearch.action.admin.cluster.repositories.delete.DeleteRepositoryRequest;
@@ -57,7 +56,6 @@
import org.elasticsearch.repositories.FinalizeSnapshotContext.UpdatedShardGenerations;
import org.elasticsearch.repositories.IndexId;
import org.elasticsearch.repositories.RepositoriesService;
-import org.elasticsearch.repositories.Repository;
import org.elasticsearch.repositories.RepositoryData;
import org.elasticsearch.repositories.RepositoryException;
import org.elasticsearch.repositories.RepositoryMissingException;
@@ -92,8 +90,6 @@
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
import java.util.function.Function;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
import java.util.stream.Collectors;
import static org.elasticsearch.repositories.RepositoryDataTests.generateRandomRepoData;
@@ -836,7 +832,8 @@ public void testShardBlobsToDeleteWithLimitedHeapSpace() {
.build();
int totalBytesRequired = calculateBytesRequiredToWriteShardSnapshotMetaDeleteResult(blob1Result);
- Settings.Builder settings = Settings.builder().put("repositories.blobstore.max_shard_delete_results_size", totalBytesRequired + "b");
+ Settings.Builder settings = Settings.builder()
+ .put("repositories.blobstore.max_shard_delete_results_size", totalBytesRequired + "b");
final var repo = setupRepo(settings);
try (var shardBlobsToDelete = repo.new ShardBlobsToDelete(settings.build())) {
try (var mockLog = MockLog.capture(BlobStoreRepository.class)) {
@@ -854,7 +851,12 @@ public void testShardBlobsToDeleteWithLimitedHeapSpace() {
.execute(
ActionRunnable.run(
refs.acquireListener(),
- () -> shardBlobsToDelete.addShardDeleteResult(blob1Result.indexId, blob1Result.shardId, blob1Result.shardGeneration, blob1Result.blobsToDelete)
+ () -> shardBlobsToDelete.addShardDeleteResult(
+ blob1Result.indexId,
+ blob1Result.shardId,
+ blob1Result.shardGeneration,
+ blob1Result.blobsToDelete
+ )
)
);
}
@@ -871,7 +873,12 @@ public void testShardBlobsToDeleteWithLimitedHeapSpace() {
.execute(
ActionRunnable.run(
refs.acquireListener(),
- () -> shardBlobsToDelete.addShardDeleteResult(blob2Result.indexId, blob2Result.shardId, blob2Result.shardGeneration, blob2Result.blobsToDelete)
+ () -> shardBlobsToDelete.addShardDeleteResult(
+ blob2Result.indexId,
+ blob2Result.shardId,
+ blob2Result.shardGeneration,
+ blob2Result.blobsToDelete
+ )
)
);
}
@@ -892,9 +899,7 @@ public void testShardBlobsToDeleteWithLimitedHeapSpace() {
@TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
public void testShardBlobsToDeleteWithSmallHeapSpace() {
TestShardSnapshotMetaDeleteResult result = generateTestShardSnapshotMetaDeleteResult(0, 20);
- final var expectedShardGenerations = ShardGenerations.builder()
- .put(result.indexId, result.shardId, result.shardGeneration)
- .build();
+ final var expectedShardGenerations = ShardGenerations.builder().put(result.indexId, result.shardId, result.shardGeneration).build();
int totalBytesRequired = calculateBytesRequiredToWriteShardSnapshotMetaDeleteResult(result);
int heapSpace = randomIntBetween(1, totalBytesRequired - 1);
@@ -916,11 +921,15 @@ public void testShardBlobsToDeleteWithSmallHeapSpace() {
.execute(
ActionRunnable.run(
refs.acquireListener(),
- () -> shardBlobsToDelete.addShardDeleteResult(result.indexId, result.shardId, result.shardGeneration, result.blobsToDelete)
+ () -> shardBlobsToDelete.addShardDeleteResult(
+ result.indexId,
+ result.shardId,
+ result.shardGeneration,
+ result.blobsToDelete
+ )
)
);
-
}
safeAwait(countDownLatch);
assertEquals(expectedShardGenerations, shardBlobsToDelete.getUpdatedShardGenerations());
@@ -937,9 +946,7 @@ public void testShardBlobsToDeleteWithSmallHeapSpace() {
@TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
public void testShardBlobsToDeleteWithOutHeapSpace() {
TestShardSnapshotMetaDeleteResult result = generateTestShardSnapshotMetaDeleteResult(0, 20);
- final var expectedShardGenerations = ShardGenerations.builder()
- .put(result.indexId, result.shardId, result.shardGeneration)
- .build();
+ final var expectedShardGenerations = ShardGenerations.builder().put(result.indexId, result.shardId, result.shardGeneration).build();
Settings.Builder settings = Settings.builder().put("repositories.blobstore.max_shard_delete_results_size", "0b");
final var repo = setupRepo(settings);
@@ -959,11 +966,15 @@ public void testShardBlobsToDeleteWithOutHeapSpace() {
.execute(
ActionRunnable.run(
refs.acquireListener(),
- () -> shardBlobsToDelete.addShardDeleteResult(result.indexId, result.shardId, result.shardGeneration, result.blobsToDelete)
+ () -> shardBlobsToDelete.addShardDeleteResult(
+ result.indexId,
+ result.shardId,
+ result.shardGeneration,
+ result.blobsToDelete
+ )
)
);
-
}
safeAwait(countDownLatch);
assertEquals(expectedShardGenerations, shardBlobsToDelete.getUpdatedShardGenerations());
@@ -974,7 +985,12 @@ public void testShardBlobsToDeleteWithOutHeapSpace() {
}
}
- private record TestShardSnapshotMetaDeleteResult(IndexId indexId, int shardId, Collection blobsToDelete, ShardGeneration shardGeneration) {}
+ private record TestShardSnapshotMetaDeleteResult(
+ IndexId indexId,
+ int shardId,
+ Collection blobsToDelete,
+ ShardGeneration shardGeneration
+ ) {}
private TestShardSnapshotMetaDeleteResult generateTestShardSnapshotMetaDeleteResult(int minBlobSize, int maxBlobSize) {
return new TestShardSnapshotMetaDeleteResult(
@@ -993,11 +1009,7 @@ private int calculateBytesRequiredToWriteShardSnapshotMetaDeleteResult(TestShard
}
private String generateRepoPath(BlobStoreRepository repo, TestShardSnapshotMetaDeleteResult result) {
- return repo.basePath()
- .add("indices")
- .add(result.indexId.getId())
- .add(Integer.toString(result.shardId))
- .buildAsString();
+ return repo.basePath().add("indices").add(result.indexId.getId()).add(Integer.toString(result.shardId)).buildAsString();
}
private List generateRandomBlobsToDelete() {
@@ -1008,10 +1020,7 @@ private List generateRandomBlobsToDelete(int minSize, int maxSize) {
return randomList(
minSize,
maxSize,
- () -> randomFrom(METADATA_PREFIX, INDEX_FILE_PREFIX, SNAPSHOT_PREFIX) + randomUUID() + randomFrom(
- "",
- METADATA_BLOB_NAME_SUFFIX
- )
+ () -> randomFrom(METADATA_PREFIX, INDEX_FILE_PREFIX, SNAPSHOT_PREFIX) + randomUUID() + randomFrom("", METADATA_BLOB_NAME_SUFFIX)
);
}
From 654ebf2743578fff1df5d30d7e23591c74b832d6 Mon Sep 17 00:00:00 2001
From: Joshua Adams
Date: Thu, 11 Sep 2025 14:13:18 +0100
Subject: [PATCH 12/25] Creates BoundedOutputStream
---
.../common/io/stream/BoundedOutputStream.java | 73 +++++++++++++++++++
...undedOutputStreamFailedWriteException.java | 21 ++++++
.../blobstore/BlobStoreRepository.java | 71 ++++++++----------
3 files changed, 123 insertions(+), 42 deletions(-)
create mode 100644 server/src/main/java/org/elasticsearch/common/io/stream/BoundedOutputStream.java
create mode 100644 server/src/main/java/org/elasticsearch/common/io/stream/BoundedOutputStreamFailedWriteException.java
diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/BoundedOutputStream.java b/server/src/main/java/org/elasticsearch/common/io/stream/BoundedOutputStream.java
new file mode 100644
index 0000000000000..a5ac2709ba057
--- /dev/null
+++ b/server/src/main/java/org/elasticsearch/common/io/stream/BoundedOutputStream.java
@@ -0,0 +1,73 @@
+/*
+ * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public
+ * License v3.0 only", or the "Server Side Public License, v 1".
+ */
+
+package org.elasticsearch.common.io.stream;
+
+import java.io.FilterOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * Prevents writes when the max size is breached
+ */
+public class BoundedOutputStream extends FilterOutputStream {
+ private final int maxSize;
+ private int size;
+
+ // As soon as a write request exceeds maxSize, permit no more writes, even if there is capacity for them
+ private boolean closed = false;
+
+ public BoundedOutputStream(OutputStream out, int maxSize) {
+ super(out);
+ this.maxSize = maxSize;
+ this.size = 0;
+ }
+
+ private boolean hasCapacity(int bytes) {
+ return size + bytes <= maxSize;
+ }
+
+ @Override
+ public void write(int b) throws IOException {
+ if (closed == false && hasCapacity(1)) {
+ super.write(b);
+
+ /*
+ We only need to increment size here as both super.write(byte[] b) and
+ super.write(byte[] b, int off, int len) write each byte individually via this
+ method, and we have already checked in each respective method whether we have
+ sufficient capacity for that entire write
+ */
+ size++;
+ } else {
+ closed = true;
+ throw new BoundedOutputStreamFailedWriteException();
+ }
+ }
+
+ @Override
+ public void write(byte[] b) throws IOException {
+ if (closed == false && hasCapacity(b.length)) {
+ super.write(b);
+ } else {
+ closed = true;
+ throw new BoundedOutputStreamFailedWriteException();
+ }
+ }
+
+ @Override
+ public void write(byte[] b, int off, int len) throws IOException {
+ if (closed == false && hasCapacity(len)) {
+ super.write(b, off, len);
+ } else {
+ closed = true;
+ throw new BoundedOutputStreamFailedWriteException();
+ }
+ }
+}
diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/BoundedOutputStreamFailedWriteException.java b/server/src/main/java/org/elasticsearch/common/io/stream/BoundedOutputStreamFailedWriteException.java
new file mode 100644
index 0000000000000..9132c0c69e419
--- /dev/null
+++ b/server/src/main/java/org/elasticsearch/common/io/stream/BoundedOutputStreamFailedWriteException.java
@@ -0,0 +1,21 @@
+/*
+ * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public
+ * License v3.0 only", or the "Server Side Public License, v 1".
+ */
+
+package org.elasticsearch.common.io.stream;
+
+import org.elasticsearch.ElasticsearchException;
+
+/**
+ * An exception indicating we have tried to write to the BoundedOutputStream and have exceeded capacity
+ */
+public class BoundedOutputStreamFailedWriteException extends ElasticsearchException {
+ public BoundedOutputStreamFailedWriteException() {
+ super("The write failed because there is no more capacity inside the BoundedOutputStream");
+ }
+}
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 6b00fdf1e0a1d..4df57ea7183f4 100644
--- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
+++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
@@ -65,6 +65,8 @@
import org.elasticsearch.common.compress.DeflateCompressor;
import org.elasticsearch.common.compress.NotXContentException;
import org.elasticsearch.common.io.Streams;
+import org.elasticsearch.common.io.stream.BoundedOutputStream;
+import org.elasticsearch.common.io.stream.BoundedOutputStreamFailedWriteException;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.InputStreamStreamInput;
import org.elasticsearch.common.io.stream.OutputStreamStreamOutput;
@@ -76,7 +78,6 @@
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
-import org.elasticsearch.common.unit.RelativeByteSizeValue;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
@@ -1697,11 +1698,9 @@ void writeTo(StreamOutput out) throws IOException {
private final ShardGenerations.Builder shardGenerationsBuilder = ShardGenerations.builder();
- private final String SHARD_DELETE_RESULTS_MAX_SIZE_SETTING_NAME = "repositories.blobstore.max_shard_delete_results_size";
- private final Setting MAX_SHARD_DELETE_RESULTS_SIZE_SETTING = new Setting<>(
- SHARD_DELETE_RESULTS_MAX_SIZE_SETTING_NAME,
+ public final Setting MAX_SHARD_DELETE_RESULTS_SIZE_SETTING = Setting.memorySizeSetting(
+ "repositories.blobstore.max_shard_delete_results_size",
"25%",
- s -> RelativeByteSizeValue.parseRelativeByteSizeValue(s, SHARD_DELETE_RESULTS_MAX_SIZE_SETTING_NAME),
Setting.Property.NodeScope
);
@@ -1710,9 +1709,12 @@ void writeTo(StreamOutput out) throws IOException {
if (this.shardDeleteResultsMaxSize > 0) {
this.shardDeleteResults = new ReleasableBytesStreamOutput(bigArrays);
this.compressed = new OutputStreamStreamOutput(
- new BufferedOutputStream(
- new DeflaterOutputStream(Streams.flushOnCloseStream(shardDeleteResults)),
- DeflateCompressor.BUFFER_SIZE
+ new BoundedOutputStream(
+ new BufferedOutputStream(
+ new DeflaterOutputStream(Streams.flushOnCloseStream(shardDeleteResults)),
+ DeflateCompressor.BUFFER_SIZE
+ ),
+ this.shardDeleteResultsMaxSize
)
);
resources.add(compressed);
@@ -1721,29 +1723,21 @@ void writeTo(StreamOutput out) throws IOException {
this.shardDeleteResults = null;
this.compressed = null;
}
-
}
/**
* Calculates the maximum size of the shardDeleteResults BytesStreamOutput.
- * The size should at most be 2GB, but no more than 25% of the total remaining heap space
+ * The size should at most be 2GB, but no more than 25% of the total remaining heap space.
+ * A buffer of 1MB is maintained, so that even if the stream is of max size, there is room to flush
* @return The maximum number of bytes the shardDeleteResults BytesStreamOutput can consume in the heap
*/
int calculateMaximumShardDeleteResultsSize(Settings settings) {
- RelativeByteSizeValue configuredValue = MAX_SHARD_DELETE_RESULTS_SIZE_SETTING.get(settings);
-
- long maxAllowedSizeInBytes;
- if (configuredValue.isAbsolute()) {
- maxAllowedSizeInBytes = configuredValue.getAbsolute().getBytes();
- } else {
- maxAllowedSizeInBytes = configuredValue.calculateValue(ByteSizeValue.ofBytes(Runtime.getRuntime().maxMemory()), null)
- .getBytes();
+ long maxSizeInBytes = MAX_SHARD_DELETE_RESULTS_SIZE_SETTING.get(settings).getBytes();
+ int oneMBBuffer = 1024 * 1024;
+ if (maxSizeInBytes > Integer.MAX_VALUE) {
+ return Integer.MAX_VALUE - oneMBBuffer;
}
-
- if (maxAllowedSizeInBytes > Integer.MAX_VALUE) {
- return Integer.MAX_VALUE;
- }
- return (int) maxAllowedSizeInBytes;
+ return (int) maxSizeInBytes;
}
synchronized void addShardDeleteResult(
@@ -1759,26 +1753,17 @@ synchronized void addShardDeleteResult(
}
try {
shardGenerationsBuilder.put(indexId, shardId, newGeneration);
+ new ShardSnapshotMetaDeleteResult(Objects.requireNonNull(indexId.getId()), shardId, blobsToDelete).writeTo(compressed);
- // Calculate how many bytes we want to write
- int bytesToWriteIndexId = StreamOutput.bytesInString(indexId.getId());
- int bytesToWriteShardId = StreamOutput.bytesInVInt(shardId);
- int bytesToWriteBlobsToDelete = StreamOutput.bytesInStringCollection(blobsToDelete);
- int totalBytesRequired = bytesToWriteIndexId + bytesToWriteShardId + bytesToWriteBlobsToDelete;
-
- // Only perform the write if there is capacity left
- if (shardDeleteResults.size() + totalBytesRequired <= shardDeleteResultsMaxSize) {
- new ShardSnapshotMetaDeleteResult(Objects.requireNonNull(indexId.getId()), shardId, blobsToDelete).writeTo(compressed);
- resultCount += 1;
- } else {
- logger.warn(
- "Failure to clean up the following dangling blobs, {}, for index {} and shard {}",
- blobsToDelete,
- indexId,
- shardId
- );
- compressed.close();
- }
+ // The resultCount is only incremented after a successful complete write
+ resultCount += 1;
+ } catch(BoundedOutputStreamFailedWriteException ex) {
+ logger.warn(
+ "Failure to clean up the following dangling blobs, {}, for index {} and shard {}",
+ blobsToDelete,
+ indexId,
+ shardId
+ );
} catch (IOException e) {
assert false : e; // no IO actually happens here
throw new UncheckedIOException(e);
@@ -1809,6 +1794,8 @@ public Iterator getBlobPaths() {
throw new UncheckedIOException(e);
}
+ // Iterates through complete ShardSnapshotMetaDeleteResults written to compressed
+ // Partially written ShardSnapshotMetaDeleteResults are dropped
return Iterators.flatMap(Iterators.forRange(0, resultCount, i -> {
try {
return new ShardSnapshotMetaDeleteResult(input);
From ba81bcfbe3e64717cac8ed78f1cd01536cbaa166 Mon Sep 17 00:00:00 2001
From: elasticsearchmachine
Date: Thu, 11 Sep 2025 13:25:07 +0000
Subject: [PATCH 13/25] [CI] Auto commit changes from spotless
---
.../repositories/blobstore/BlobStoreRepository.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
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 4df57ea7183f4..eb2c5a5218152 100644
--- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
+++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
@@ -1757,7 +1757,7 @@ synchronized void addShardDeleteResult(
// The resultCount is only incremented after a successful complete write
resultCount += 1;
- } catch(BoundedOutputStreamFailedWriteException ex) {
+ } catch (BoundedOutputStreamFailedWriteException ex) {
logger.warn(
"Failure to clean up the following dangling blobs, {}, for index {} and shard {}",
blobsToDelete,
From acd21820fe63791a97426ca14b723022a0fe1f3a Mon Sep 17 00:00:00 2001
From: Joshua Adams
Date: Thu, 11 Sep 2025 14:33:29 +0100
Subject: [PATCH 14/25] Revert StreamOutput and delete tests
---
.../common/io/stream/StreamOutput.java | 54 +----------
.../common/io/stream/StreamOutputTests.java | 94 -------------------
2 files changed, 1 insertion(+), 147 deletions(-)
delete mode 100644 server/src/test/java/org/elasticsearch/common/io/stream/StreamOutputTests.java
diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java b/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java
index 84d945a15bf22..d1fb6c700a9f8 100644
--- a/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java
+++ b/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java
@@ -220,26 +220,6 @@ public void writeIntLE(int i) throws IOException {
writeBytes(buffer, 0, 4);
}
- /**
- * Returns the number of bytes needed to encode the given int as a VInt.
- */
- public static int bytesInVInt(int i) {
- // Single byte shortcut: fits in 7 bits (i.e., values 0..127)
- if (Integer.numberOfLeadingZeros(i) >= 25) {
- return 1;
- }
- int byteCount = 1;
- i >>>= 7;
- while ((i & ~0x7F) != 0) {
- byteCount++;
- i >>>= 7;
- }
- if (i != 0) {
- byteCount++;
- }
- return byteCount;
- }
-
/**
* Writes an int in a variable-length format. Writes between one and
* five bytes. Smaller values take fewer bytes. Negative numbers
@@ -255,7 +235,7 @@ public void writeVInt(int i) throws IOException {
* In that case benchmarks of the method itself are faster but
* benchmarks of methods that use this method are slower.
* This is philosophically in line with vint in general - it biases
- * towards being simple and fast for smaller numbers.
+ * twoards being simple and fast for smaller numbers.
*/
if (Integer.numberOfLeadingZeros(i) >= 25) {
writeByte((byte) i);
@@ -450,27 +430,6 @@ public void writeString(String str) throws IOException {
writeString(str, scratch.get(), 0);
}
- /**
- * Returns the number of bytes needed to encode the given string in UTF-8.
- */
- public static int bytesInString(String str) {
- int byteCount = 0;
- final int charCount = str.length();
- for (int i = 0; i < charCount; i++) {
- final int c = str.charAt(i);
- if (c <= 0x007F) {
- byteCount += 1;
- } else if (c > 0x07FF) {
- byteCount += 3;
- } else {
- byteCount += 2;
- }
- }
- // Add bytes for the length prefix (VInt)
- byteCount += bytesInVInt(charCount);
- return byteCount;
- }
-
/**
* Write string as well as possibly the beginning of the given {@code buffer}. The given {@code buffer} will also be used when encoding
* the given string.
@@ -1178,17 +1137,6 @@ public void writeCollection(final Collection collection, final Writer
}
}
- /**
- * Returns the number of bytes needed to encode the given string collectiom
- */
- public static int bytesInStringCollection(Collection collection) {
- int byteCount = bytesInVInt(collection.size());
- for (String item : collection) {
- byteCount += bytesInString(item);
- }
- return byteCount;
- }
-
/**
* Writes a collection of strings which can then be read using {@link StreamInput#readStringCollectionAsList} or another {@code
* readStringCollectionAs*} method. Make sure to read the collection back into the same type as was originally written.
diff --git a/server/src/test/java/org/elasticsearch/common/io/stream/StreamOutputTests.java b/server/src/test/java/org/elasticsearch/common/io/stream/StreamOutputTests.java
deleted file mode 100644
index c243bcd2e23c4..0000000000000
--- a/server/src/test/java/org/elasticsearch/common/io/stream/StreamOutputTests.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public
- * License v3.0 only", or the "Server Side Public License, v 1".
- */
-
-package org.elasticsearch.common.io.stream;
-
-import org.elasticsearch.test.ESTestCase;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import static org.elasticsearch.common.io.stream.StreamOutput.bytesInString;
-import static org.elasticsearch.common.io.stream.StreamOutput.bytesInStringCollection;
-import static org.elasticsearch.common.io.stream.StreamOutput.bytesInVInt;
-
-public class StreamOutputTests extends ESTestCase {
-
- public void testBytesInVInt() {
- // 0..127 should use the single byte shortcut
- assertEquals(1, bytesInVInt(0));
- assertEquals(1, bytesInVInt(127));
- assertEquals(1, bytesInVInt(randomIntBetween(0, 127)));
-
- assertEquals(2, bytesInVInt(128));
- assertEquals(2, bytesInVInt(16383));
- assertEquals(2, bytesInVInt(randomIntBetween(128, 16383)));
-
- assertEquals(3, bytesInVInt(16384));
- assertEquals(3, bytesInVInt(2097151));
- assertEquals(3, bytesInVInt(randomIntBetween(16384, 2097151)));
-
- assertEquals(4, bytesInVInt(2097152));
- assertEquals(4, bytesInVInt(268435455));
- assertEquals(4, bytesInVInt(randomIntBetween(2097152, 268435455)));
-
- assertEquals(5, bytesInVInt(268435456));
- assertEquals(5, bytesInVInt(Integer.MAX_VALUE));
- assertEquals(5, bytesInVInt(randomIntBetween(268435456, Integer.MAX_VALUE)));
- }
-
- public void testBytesInString() {
- assertEquals(1, bytesInString(""));
-
- // Since the length of the string is stored as a VInt, this has additional bytes
- int randomSize = randomIntBetween(0, 256);
- int vintBytes = randomSize <= 127 ? 1 : 2;
- assertEquals(randomSize + vintBytes, bytesInString(randomAlphaOfLength(randomSize)));
-
- // This is U+00E9, and 2 bytes in UTF-8
- String s = "é";
- assertEquals(2 + 1, bytesInString(s));
-
- // A mixed string of 1, 2 and 3 bytes respectively
- s = "aéअ";
- assertEquals(6 + 1, bytesInString(s));
- }
-
- public void testBytesInStringCollection() {
- assertEquals(1, bytesInStringCollection(Collections.emptyList()));
-
- List listOfStrings = new ArrayList<>();
- int randomSize = randomIntBetween(0, 256);
- // Start with the number of bytes needed to store the size of the collection
- int totalStringBytes = randomSize <= 127 ? 1 : 2;
- for (int i = 0; i < randomSize; i++) {
- int lengthOfString = randomIntBetween(0, 256);
- // The size of the string is stored as a VInt
- int stringBytes = lengthOfString <= 127 ? 1 : 2;
- StringBuilder s = new StringBuilder();
- for (int j = 0; j < lengthOfString; j++) {
- int characterBytes = randomIntBetween(1, 3);
- if (characterBytes == 1) {
- s.append(randomAlphaOfLength(1));
- } else if (characterBytes == 2) {
- s.append("é");
- } else {
- s.append("अ");
- }
- stringBytes += characterBytes;
- }
-
- listOfStrings.add(s.toString());
- totalStringBytes += stringBytes;
- }
-
- assertEquals(totalStringBytes, bytesInStringCollection(listOfStrings));
- }
-}
From be05a1f0f3769b323a59a970c34d02297e79e26c Mon Sep 17 00:00:00 2001
From: Joshua Adams
Date: Thu, 11 Sep 2025 16:40:40 +0100
Subject: [PATCH 15/25] Adds BoundedOutputStreamTests
---
.../io/stream/BoundedOutputStreamTests.java | 116 ++++++++
.../blobstore/BlobStoreRepositoryTests.java | 247 +++++++++---------
2 files changed, 236 insertions(+), 127 deletions(-)
create mode 100644 server/src/test/java/org/elasticsearch/common/io/stream/BoundedOutputStreamTests.java
diff --git a/server/src/test/java/org/elasticsearch/common/io/stream/BoundedOutputStreamTests.java b/server/src/test/java/org/elasticsearch/common/io/stream/BoundedOutputStreamTests.java
new file mode 100644
index 0000000000000..c6ef13cc4b437
--- /dev/null
+++ b/server/src/test/java/org/elasticsearch/common/io/stream/BoundedOutputStreamTests.java
@@ -0,0 +1,116 @@
+/*
+ * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public
+ * License v3.0 only", or the "Server Side Public License, v 1".
+ */
+
+package org.elasticsearch.common.io.stream;
+import org.elasticsearch.test.ESTestCase;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+public class BoundedOutputStreamTests extends ESTestCase {
+
+ public void testWriteSingleBytes() throws IOException {
+ ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+ int maxSize = randomIntBetween(0, 100);
+ BoundedOutputStream boundedOutputStream = new BoundedOutputStream(byteArrayOutputStream, maxSize);
+
+ byte[] values = new byte[maxSize];
+ for (int i = 0; i < maxSize; i++) {
+ byte value = randomByte();
+ boundedOutputStream.write(value);
+ values[i] = value;
+ }
+ assertArrayEquals(values, byteArrayOutputStream.toByteArray());
+
+ // Subsequent writes all fail since the size is exceeded
+ for (int i = 0; i < randomIntBetween(5, 15); i++) {
+ assertThrows(
+ BoundedOutputStreamFailedWriteException.class,
+ () -> boundedOutputStream.write(randomByte())
+ );
+ }
+ }
+
+ public void testWriteByteArray() throws IOException {
+ ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+ int maxSize = randomIntBetween(0, 100);
+ BoundedOutputStream boundedOutputStream = new BoundedOutputStream(byteArrayOutputStream, maxSize);
+
+ byte[] values = randomByteArrayOfLength(maxSize);
+ boundedOutputStream.write(values);
+ assertArrayEquals(values, byteArrayOutputStream.toByteArray());
+ }
+
+ public void testWriteByteArrayWithArrayLengthGreaterThanMaxSize() {
+ ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+ int maxSize = randomIntBetween(0, 100);
+ BoundedOutputStream boundedOutputStream = new BoundedOutputStream(byteArrayOutputStream, maxSize);
+
+ assertThrows(
+ BoundedOutputStreamFailedWriteException.class,
+ () -> boundedOutputStream.write(randomByteArrayOfLength(maxSize + randomIntBetween(1, 100)))
+ );
+ }
+
+ public void testWriteByteArrayWithOffset() throws IOException {
+ ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+ int maxSize = randomIntBetween(50, 100);
+ BoundedOutputStream boundedOutputStream = new BoundedOutputStream(byteArrayOutputStream, maxSize);
+
+ byte[] values = new byte[maxSize];
+ for (int i = 0; i < maxSize; i++) {
+ values[i] = randomByte();
+ }
+
+ int offset = randomIntBetween(0, 50);
+ int length = randomIntBetween(offset, maxSize);
+ boundedOutputStream.write(values, offset, length);
+ byte[] expectedResult = new byte[length];
+ System.arraycopy(values, offset, expectedResult, 0, length);
+ assertArrayEquals(expectedResult, byteArrayOutputStream.toByteArray());
+ }
+
+
+ public void testWriteByteArrayWithOffsetWithLengthExceedingMaxSize() {
+ ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+// int maxSize = randomIntBetween(0, 100);
+ int maxSize = 1;
+ BoundedOutputStream boundedOutputStream = new BoundedOutputStream(byteArrayOutputStream, maxSize);
+
+// int randomOffset = randomIntBetween(0, maxSize);
+// int randomLength = randomIntBetween(maxSize, 200);
+ int randomOffset = 0;
+ int randomLength = 1;
+
+ logger.warn("maxsize: " + maxSize);
+ logger.warn("offset: " + randomOffset);
+ logger.warn("length: " + randomLength);
+
+ assertThrows(
+ BoundedOutputStreamFailedWriteException.class,
+ () -> boundedOutputStream.write(randomByteArrayOfLength(randomLength), randomOffset, randomLength)
+ );
+ }
+
+
+ public void testNoFurtherWritesAfterLimitExceeded() throws IOException {
+ ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+ int maxSize = randomIntBetween(0, 100);
+ BoundedOutputStream boundedOutputStream = new BoundedOutputStream(byteArrayOutputStream, maxSize);
+
+ // Write until maxSize is reached
+ for (int i = 0; i < maxSize; i++) {
+ boundedOutputStream.write(randomByte());
+ }
+
+ assertThrows(BoundedOutputStreamFailedWriteException.class, () -> boundedOutputStream.write(randomByte()));
+ assertThrows(BoundedOutputStreamFailedWriteException.class, () -> boundedOutputStream.write(randomByteArrayOfLength(randomIntBetween(0, 100))));
+ assertThrows(BoundedOutputStreamFailedWriteException.class, () -> boundedOutputStream.write(randomByteArrayOfLength(randomIntBetween(0, 100)), randomIntBetween(0, 100), randomIntBetween(0, 100)));
+ }
+}
diff --git a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
index a8c3916755200..aeef37139c9f1 100644
--- a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
+++ b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
@@ -819,126 +819,126 @@ public void testShardBlobsToDeleteWithSufficientHeapSpace() {
/*
There is sufficient heap space for the first write, but not the second
*/
- @TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
- public void testShardBlobsToDeleteWithLimitedHeapSpace() {
- TestShardSnapshotMetaDeleteResult blob1Result = generateTestShardSnapshotMetaDeleteResult(0, 20);
- // blob1Result can be compressed up to 4x when written to shardsDeleteResults so make the second blob
- // at least 4x larger than the first to guarantee this blob cannot also be written
- TestShardSnapshotMetaDeleteResult blob2Result = generateTestShardSnapshotMetaDeleteResult(80, 100);
-
- final var expectedShardGenerations = ShardGenerations.builder()
- .put(blob1Result.indexId, blob1Result.shardId, blob1Result.shardGeneration)
- .put(blob2Result.indexId, blob2Result.shardId, blob2Result.shardGeneration)
- .build();
-
- int totalBytesRequired = calculateBytesRequiredToWriteShardSnapshotMetaDeleteResult(blob1Result);
- Settings.Builder settings = Settings.builder()
- .put("repositories.blobstore.max_shard_delete_results_size", totalBytesRequired + "b");
- final var repo = setupRepo(settings);
- try (var shardBlobsToDelete = repo.new ShardBlobsToDelete(settings.build())) {
- try (var mockLog = MockLog.capture(BlobStoreRepository.class)) {
- addFailureToCleanUpDanglingBlobsSeenExpectation(mockLog);
-
- final var expectedBlobsToDelete = new HashSet();
- var countDownLatch = new CountDownLatch(1);
- try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
- final var indexPath1 = generateRepoPath(repo, blob1Result);
- for (final var blobToDelete : blob1Result.blobsToDelete) {
- expectedBlobsToDelete.add(indexPath1 + blobToDelete);
- }
- repo.threadPool()
- .generic()
- .execute(
- ActionRunnable.run(
- refs.acquireListener(),
- () -> shardBlobsToDelete.addShardDeleteResult(
- blob1Result.indexId,
- blob1Result.shardId,
- blob1Result.shardGeneration,
- blob1Result.blobsToDelete
- )
- )
- );
- }
- safeAwait(countDownLatch);
-
- countDownLatch = new CountDownLatch(1);
- try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
- final var indexPath2 = generateRepoPath(repo, blob2Result);
- for (final var blobToDelete : blob2Result.blobsToDelete) {
- expectedBlobsToDelete.add(indexPath2 + blobToDelete);
- }
- repo.threadPool()
- .generic()
- .execute(
- ActionRunnable.run(
- refs.acquireListener(),
- () -> shardBlobsToDelete.addShardDeleteResult(
- blob2Result.indexId,
- blob2Result.shardId,
- blob2Result.shardGeneration,
- blob2Result.blobsToDelete
- )
- )
- );
- }
-
- safeAwait(countDownLatch);
- assertEquals(expectedShardGenerations, shardBlobsToDelete.getUpdatedShardGenerations());
- shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
- // None of the second writes succeeded and are therefore not returned by blobsToDelete()
- assertEquals(expectedBlobsToDelete.size(), blob2Result.blobsToDelete.size());
- mockLog.assertAllExpectationsMatched();
- }
- }
- }
-
- /*
- There is heap space, but it's insufficient for any write requests
- */
- @TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
- public void testShardBlobsToDeleteWithSmallHeapSpace() {
- TestShardSnapshotMetaDeleteResult result = generateTestShardSnapshotMetaDeleteResult(0, 20);
- final var expectedShardGenerations = ShardGenerations.builder().put(result.indexId, result.shardId, result.shardGeneration).build();
-
- int totalBytesRequired = calculateBytesRequiredToWriteShardSnapshotMetaDeleteResult(result);
- int heapSpace = randomIntBetween(1, totalBytesRequired - 1);
- Settings.Builder settings = Settings.builder().put("repositories.blobstore.max_shard_delete_results_size", heapSpace + "b");
- final var repo = setupRepo(settings);
- try (var shardBlobsToDelete = repo.new ShardBlobsToDelete(settings.build())) {
- try (var mockLog = MockLog.capture(BlobStoreRepository.class)) {
- addFailureToCleanUpDanglingBlobsSeenExpectation(mockLog);
-
- final var expectedBlobsToDelete = new HashSet();
- var countDownLatch = new CountDownLatch(1);
- try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
- final var indexPath = generateRepoPath(repo, result);
- for (final var blobToDelete : result.blobsToDelete) {
- expectedBlobsToDelete.add(indexPath + blobToDelete);
- }
- repo.threadPool()
- .generic()
- .execute(
- ActionRunnable.run(
- refs.acquireListener(),
- () -> shardBlobsToDelete.addShardDeleteResult(
- result.indexId,
- result.shardId,
- result.shardGeneration,
- result.blobsToDelete
- )
- )
- );
-
- }
- safeAwait(countDownLatch);
- assertEquals(expectedShardGenerations, shardBlobsToDelete.getUpdatedShardGenerations());
- shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
- assertEquals(result.blobsToDelete.size(), expectedBlobsToDelete.size());
- mockLog.assertAllExpectationsMatched();
- }
- }
- }
+// @TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
+// public void testShardBlobsToDeleteWithLimitedHeapSpace() {
+// TestShardSnapshotMetaDeleteResult blob1Result = generateTestShardSnapshotMetaDeleteResult(0, 20);
+// // blob1Result can be compressed up to 4x when written to shardsDeleteResults so make the second blob
+// // at least 4x larger than the first to guarantee this blob cannot also be written
+// TestShardSnapshotMetaDeleteResult blob2Result = generateTestShardSnapshotMetaDeleteResult(80, 100);
+//
+// final var expectedShardGenerations = ShardGenerations.builder()
+// .put(blob1Result.indexId, blob1Result.shardId, blob1Result.shardGeneration)
+// .put(blob2Result.indexId, blob2Result.shardId, blob2Result.shardGeneration)
+// .build();
+//
+// int totalBytesRequired = calculateBytesRequiredToWriteShardSnapshotMetaDeleteResult(blob1Result);
+// Settings.Builder settings = Settings.builder()
+// .put("repositories.blobstore.max_shard_delete_results_size", totalBytesRequired + "b");
+// final var repo = setupRepo(settings);
+// try (var shardBlobsToDelete = repo.new ShardBlobsToDelete(settings.build())) {
+// try (var mockLog = MockLog.capture(BlobStoreRepository.class)) {
+// addFailureToCleanUpDanglingBlobsSeenExpectation(mockLog);
+//
+// final var expectedBlobsToDelete = new HashSet();
+// var countDownLatch = new CountDownLatch(1);
+// try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
+// final var indexPath1 = generateRepoPath(repo, blob1Result);
+// for (final var blobToDelete : blob1Result.blobsToDelete) {
+// expectedBlobsToDelete.add(indexPath1 + blobToDelete);
+// }
+// repo.threadPool()
+// .generic()
+// .execute(
+// ActionRunnable.run(
+// refs.acquireListener(),
+// () -> shardBlobsToDelete.addShardDeleteResult(
+// blob1Result.indexId,
+// blob1Result.shardId,
+// blob1Result.shardGeneration,
+// blob1Result.blobsToDelete
+// )
+// )
+// );
+// }
+// safeAwait(countDownLatch);
+//
+// countDownLatch = new CountDownLatch(1);
+// try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
+// final var indexPath2 = generateRepoPath(repo, blob2Result);
+// for (final var blobToDelete : blob2Result.blobsToDelete) {
+// expectedBlobsToDelete.add(indexPath2 + blobToDelete);
+// }
+// repo.threadPool()
+// .generic()
+// .execute(
+// ActionRunnable.run(
+// refs.acquireListener(),
+// () -> shardBlobsToDelete.addShardDeleteResult(
+// blob2Result.indexId,
+// blob2Result.shardId,
+// blob2Result.shardGeneration,
+// blob2Result.blobsToDelete
+// )
+// )
+// );
+// }
+//
+// safeAwait(countDownLatch);
+// assertEquals(expectedShardGenerations, shardBlobsToDelete.getUpdatedShardGenerations());
+// shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
+// // None of the second writes succeeded and are therefore not returned by blobsToDelete()
+// assertEquals(expectedBlobsToDelete.size(), blob2Result.blobsToDelete.size());
+// mockLog.assertAllExpectationsMatched();
+// }
+// }
+// }
+//
+// /*
+// There is heap space, but it's insufficient for any write requests
+// */
+// @TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
+// public void testShardBlobsToDeleteWithSmallHeapSpace() {
+// TestShardSnapshotMetaDeleteResult result = generateTestShardSnapshotMetaDeleteResult(0, 20);
+// final var expectedShardGenerations = ShardGenerations.builder().put(result.indexId, result.shardId, result.shardGeneration).build();
+//
+// int totalBytesRequired = calculateBytesRequiredToWriteShardSnapshotMetaDeleteResult(result);
+// int heapSpace = randomIntBetween(1, totalBytesRequired - 1);
+// Settings.Builder settings = Settings.builder().put("repositories.blobstore.max_shard_delete_results_size", heapSpace + "b");
+// final var repo = setupRepo(settings);
+// try (var shardBlobsToDelete = repo.new ShardBlobsToDelete(settings.build())) {
+// try (var mockLog = MockLog.capture(BlobStoreRepository.class)) {
+// addFailureToCleanUpDanglingBlobsSeenExpectation(mockLog);
+//
+// final var expectedBlobsToDelete = new HashSet();
+// var countDownLatch = new CountDownLatch(1);
+// try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
+// final var indexPath = generateRepoPath(repo, result);
+// for (final var blobToDelete : result.blobsToDelete) {
+// expectedBlobsToDelete.add(indexPath + blobToDelete);
+// }
+// repo.threadPool()
+// .generic()
+// .execute(
+// ActionRunnable.run(
+// refs.acquireListener(),
+// () -> shardBlobsToDelete.addShardDeleteResult(
+// result.indexId,
+// result.shardId,
+// result.shardGeneration,
+// result.blobsToDelete
+// )
+// )
+// );
+//
+// }
+// safeAwait(countDownLatch);
+// assertEquals(expectedShardGenerations, shardBlobsToDelete.getUpdatedShardGenerations());
+// shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
+// assertEquals(result.blobsToDelete.size(), expectedBlobsToDelete.size());
+// mockLog.assertAllExpectationsMatched();
+// }
+// }
+// }
/*
There's no heap space, so we don't even attempt to write
@@ -1001,13 +1001,6 @@ private TestShardSnapshotMetaDeleteResult generateTestShardSnapshotMetaDeleteRes
);
}
- private int calculateBytesRequiredToWriteShardSnapshotMetaDeleteResult(TestShardSnapshotMetaDeleteResult result) {
- int bytesToWriteIndexId = StreamOutput.bytesInString(result.indexId.getId());
- int bytesToWriteShardId = StreamOutput.bytesInVInt(result.shardId);
- int bytesToWriteBlobsToDelete = StreamOutput.bytesInStringCollection(result.blobsToDelete);
- return bytesToWriteIndexId + bytesToWriteShardId + bytesToWriteBlobsToDelete;
- }
-
private String generateRepoPath(BlobStoreRepository repo, TestShardSnapshotMetaDeleteResult result) {
return repo.basePath().add("indices").add(result.indexId.getId()).add(Integer.toString(result.shardId)).buildAsString();
}
From 8d66c1e583279ae83bd677c98cc91ef47c36c8e7 Mon Sep 17 00:00:00 2001
From: Joshua Adams
Date: Fri, 12 Sep 2025 12:25:21 +0100
Subject: [PATCH 16/25] Creates TruncatedOutputStream
---
.../common/io/stream/BoundedOutputStream.java | 73 -----
...undedOutputStreamFailedWriteException.java | 21 --
.../io/stream/TruncatedOutputStream.java | 60 ++++
.../blobstore/BlobStoreRepository.java | 92 +++---
.../io/stream/BoundedOutputStreamTests.java | 116 --------
.../io/stream/TruncatedOutputStreamTests.java | 122 ++++++++
.../blobstore/BlobStoreRepositoryTests.java | 266 ++++++------------
7 files changed, 315 insertions(+), 435 deletions(-)
delete mode 100644 server/src/main/java/org/elasticsearch/common/io/stream/BoundedOutputStream.java
delete mode 100644 server/src/main/java/org/elasticsearch/common/io/stream/BoundedOutputStreamFailedWriteException.java
create mode 100644 server/src/main/java/org/elasticsearch/common/io/stream/TruncatedOutputStream.java
delete mode 100644 server/src/test/java/org/elasticsearch/common/io/stream/BoundedOutputStreamTests.java
create mode 100644 server/src/test/java/org/elasticsearch/common/io/stream/TruncatedOutputStreamTests.java
diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/BoundedOutputStream.java b/server/src/main/java/org/elasticsearch/common/io/stream/BoundedOutputStream.java
deleted file mode 100644
index a5ac2709ba057..0000000000000
--- a/server/src/main/java/org/elasticsearch/common/io/stream/BoundedOutputStream.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public
- * License v3.0 only", or the "Server Side Public License, v 1".
- */
-
-package org.elasticsearch.common.io.stream;
-
-import java.io.FilterOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-
-/**
- * Prevents writes when the max size is breached
- */
-public class BoundedOutputStream extends FilterOutputStream {
- private final int maxSize;
- private int size;
-
- // As soon as a write request exceeds maxSize, permit no more writes, even if there is capacity for them
- private boolean closed = false;
-
- public BoundedOutputStream(OutputStream out, int maxSize) {
- super(out);
- this.maxSize = maxSize;
- this.size = 0;
- }
-
- private boolean hasCapacity(int bytes) {
- return size + bytes <= maxSize;
- }
-
- @Override
- public void write(int b) throws IOException {
- if (closed == false && hasCapacity(1)) {
- super.write(b);
-
- /*
- We only need to increment size here as both super.write(byte[] b) and
- super.write(byte[] b, int off, int len) write each byte individually via this
- method, and we have already checked in each respective method whether we have
- sufficient capacity for that entire write
- */
- size++;
- } else {
- closed = true;
- throw new BoundedOutputStreamFailedWriteException();
- }
- }
-
- @Override
- public void write(byte[] b) throws IOException {
- if (closed == false && hasCapacity(b.length)) {
- super.write(b);
- } else {
- closed = true;
- throw new BoundedOutputStreamFailedWriteException();
- }
- }
-
- @Override
- public void write(byte[] b, int off, int len) throws IOException {
- if (closed == false && hasCapacity(len)) {
- super.write(b, off, len);
- } else {
- closed = true;
- throw new BoundedOutputStreamFailedWriteException();
- }
- }
-}
diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/BoundedOutputStreamFailedWriteException.java b/server/src/main/java/org/elasticsearch/common/io/stream/BoundedOutputStreamFailedWriteException.java
deleted file mode 100644
index 9132c0c69e419..0000000000000
--- a/server/src/main/java/org/elasticsearch/common/io/stream/BoundedOutputStreamFailedWriteException.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public
- * License v3.0 only", or the "Server Side Public License, v 1".
- */
-
-package org.elasticsearch.common.io.stream;
-
-import org.elasticsearch.ElasticsearchException;
-
-/**
- * An exception indicating we have tried to write to the BoundedOutputStream and have exceeded capacity
- */
-public class BoundedOutputStreamFailedWriteException extends ElasticsearchException {
- public BoundedOutputStreamFailedWriteException() {
- super("The write failed because there is no more capacity inside the BoundedOutputStream");
- }
-}
diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/TruncatedOutputStream.java b/server/src/main/java/org/elasticsearch/common/io/stream/TruncatedOutputStream.java
new file mode 100644
index 0000000000000..756dd04be8130
--- /dev/null
+++ b/server/src/main/java/org/elasticsearch/common/io/stream/TruncatedOutputStream.java
@@ -0,0 +1,60 @@
+/*
+ * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public
+ * License v3.0 only", or the "Server Side Public License, v 1".
+ */
+
+package org.elasticsearch.common.io.stream;
+
+import java.io.FilterOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.function.IntSupplier;
+
+/**
+ * Truncates writes once the max size is exceeded.
+ * However, when writing byte arrays, the stream does not check whether there is capacity for the full
+ * array prior to writing, so there is overspill of up to b.length - 1.
+ */
+public class TruncatedOutputStream extends FilterOutputStream {
+ private final IntSupplier currentSizeSupplier;
+ private final int maxSize;
+ private boolean hasCapacity = true;
+
+ public TruncatedOutputStream(OutputStream out, IntSupplier currentSizeSupplier, int maxSize) {
+ super(out);
+ this.currentSizeSupplier = currentSizeSupplier;
+ this.maxSize = maxSize;
+ }
+
+ private boolean hasCapacity() {
+ if (hasCapacity) {
+ hasCapacity = currentSizeSupplier.getAsInt() < maxSize;
+ }
+ return hasCapacity;
+ }
+
+ @Override
+ public void write(int b) throws IOException {
+ if (hasCapacity()) {
+ out.write(b);
+ }
+ }
+
+ @Override
+ public void write(byte[] b) throws IOException {
+ if (hasCapacity()) {
+ out.write(b);
+ }
+ }
+
+ @Override
+ public void write(byte[] b, int off, int len) throws IOException {
+ if (hasCapacity()) {
+ out.write(b, off, len);
+ }
+ }
+}
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 4df57ea7183f4..39ba5d3770fac 100644
--- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
+++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
@@ -65,14 +65,13 @@
import org.elasticsearch.common.compress.DeflateCompressor;
import org.elasticsearch.common.compress.NotXContentException;
import org.elasticsearch.common.io.Streams;
-import org.elasticsearch.common.io.stream.BoundedOutputStream;
-import org.elasticsearch.common.io.stream.BoundedOutputStreamFailedWriteException;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.InputStreamStreamInput;
import org.elasticsearch.common.io.stream.OutputStreamStreamOutput;
import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.io.stream.TruncatedOutputStream;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.store.InputStreamIndexInput;
import org.elasticsearch.common.settings.Setting;
@@ -148,6 +147,7 @@
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
import java.io.Closeable;
+import java.io.EOFException;
import java.io.FilterInputStream;
import java.io.IOException;
import java.io.InputStream;
@@ -1078,7 +1078,6 @@ class SnapshotsDeletion {
* {@link RepositoryData} blob newer than the one identified by {@link #originalRepositoryDataGeneration}.
*/
private final RepositoryData originalRepositoryData;
- private final Settings settings;
/**
* Executor to use for all repository interactions.
@@ -1118,9 +1117,8 @@ class SnapshotsDeletion {
this.originalRootBlobs = originalRootBlobs;
this.originalIndexContainers = originalIndexContainers;
this.originalRepositoryData = originalRepositoryData;
- this.settings = settings;
- shardBlobsToDelete = new ShardBlobsToDelete(this.settings);
+ shardBlobsToDelete = new ShardBlobsToDelete(settings);
}
// ---------------------------------------------------------------------------------------------------------------------------------
@@ -1442,12 +1440,12 @@ private static List unusedBlobs(
.filter(
blob -> blob.startsWith(SNAPSHOT_INDEX_PREFIX)
|| (blob.startsWith(SNAPSHOT_PREFIX)
- && blob.endsWith(METADATA_BLOB_NAME_SUFFIX)
- && survivingSnapshotUUIDs.contains(
- blob.substring(SNAPSHOT_PREFIX.length(), blob.length() - METADATA_BLOB_NAME_SUFFIX.length())
- ) == false)
+ && blob.endsWith(METADATA_BLOB_NAME_SUFFIX)
+ && survivingSnapshotUUIDs.contains(
+ blob.substring(SNAPSHOT_PREFIX.length(), blob.length() - METADATA_BLOB_NAME_SUFFIX.length())
+ ) == false)
|| (blob.startsWith(UPLOADED_DATA_BLOB_PREFIX)
- && updatedSnapshots.findNameFile(canonicalName(blob)) == null)
+ && updatedSnapshots.findNameFile(canonicalName(blob)) == null)
|| FsBlobContainer.isTempBlobName(blob)
)
.toList();
@@ -1485,7 +1483,6 @@ private void cleanupUnlinkedShardLevelBlobs(ActionListener listener) {
listener.onResponse(null);
return;
}
-
snapshotExecutor.execute(ActionRunnable.wrap(listener, l -> {
try {
deleteFromContainer(OperationPurpose.SNAPSHOT_DATA, blobContainer(), filesToDelete);
@@ -1709,11 +1706,12 @@ void writeTo(StreamOutput out) throws IOException {
if (this.shardDeleteResultsMaxSize > 0) {
this.shardDeleteResults = new ReleasableBytesStreamOutput(bigArrays);
this.compressed = new OutputStreamStreamOutput(
- new BoundedOutputStream(
+ new TruncatedOutputStream(
new BufferedOutputStream(
new DeflaterOutputStream(Streams.flushOnCloseStream(shardDeleteResults)),
DeflateCompressor.BUFFER_SIZE
),
+ shardDeleteResults::size,
this.shardDeleteResultsMaxSize
)
);
@@ -1754,16 +1752,7 @@ synchronized void addShardDeleteResult(
try {
shardGenerationsBuilder.put(indexId, shardId, newGeneration);
new ShardSnapshotMetaDeleteResult(Objects.requireNonNull(indexId.getId()), shardId, blobsToDelete).writeTo(compressed);
-
- // The resultCount is only incremented after a successful complete write
resultCount += 1;
- } catch(BoundedOutputStreamFailedWriteException ex) {
- logger.warn(
- "Failure to clean up the following dangling blobs, {}, for index {} and shard {}",
- blobsToDelete,
- indexId,
- shardId
- );
} catch (IOException e) {
assert false : e; // no IO actually happens here
throw new UncheckedIOException(e);
@@ -1794,18 +1783,25 @@ public Iterator getBlobPaths() {
throw new UncheckedIOException(e);
}
- // Iterates through complete ShardSnapshotMetaDeleteResults written to compressed
- // Partially written ShardSnapshotMetaDeleteResults are dropped
- return Iterators.flatMap(Iterators.forRange(0, resultCount, i -> {
+ List blobPaths = new ArrayList<>();
+ for (int i = 0; i < resultCount; i++) {
+ ShardSnapshotMetaDeleteResult shardResult;
try {
- return new ShardSnapshotMetaDeleteResult(input);
+ shardResult = new ShardSnapshotMetaDeleteResult(input);
+ } catch (EOFException ex) {
+ // There was insufficient stream space to write this blob
+ // All subsequent blobs will likewise be dangling
+ logger.warn("Failure to clean up {} dangling blobs", resultCount - i);
+ break;
} catch (IOException e) {
throw new UncheckedIOException(e);
}
- }), shardResult -> {
- final var shardPath = shardPath(new IndexId("_na_", shardResult.indexId), shardResult.shardId).buildAsString();
- return Iterators.map(shardResult.blobsToDelete.iterator(), blob -> shardPath + blob);
- });
+ String shardPath = shardPath(new IndexId("_na_", shardResult.indexId), shardResult.shardId).buildAsString();
+ for (String blob : shardResult.blobsToDelete) {
+ blobPaths.add(shardPath + blob);
+ }
+ }
+ return blobPaths.iterator();
}
@Override
@@ -2627,10 +2623,10 @@ private void cacheRepositoryData(RepositoryData repositoryData, IndexVersion ver
toCache = repositoryData.withoutShardGenerations();
assert repositoryData.indexMetaDataGenerations().equals(IndexMetaDataGenerations.EMPTY)
: "repository data should not contain index generations at version ["
- + version.toReleaseVersion()
- + "] but saw ["
- + repositoryData.indexMetaDataGenerations()
- + "]";
+ + version.toReleaseVersion()
+ + "] but saw ["
+ + repositoryData.indexMetaDataGenerations()
+ + "]";
}
assert toCache.getGenId() >= 0 : "No need to cache abstract generations but attempted to cache [" + toCache.getGenId() + "]";
latestKnownRepositoryData.updateAndGet(known -> {
@@ -2654,10 +2650,10 @@ private static String previousWriterMessage(@Nullable Tuple previo
return previousWriterInfo == null
? ""
: " The last cluster to write to this repository was ["
- + previousWriterInfo.v2()
- + "] at generation ["
- + previousWriterInfo.v1()
- + "].";
+ + previousWriterInfo.v2()
+ + "] at generation ["
+ + previousWriterInfo.v1()
+ + "].";
}
/**
@@ -2867,10 +2863,10 @@ public ClusterState execute(ClusterState currentState) {
newGen = uninitializedMeta ? Math.max(expectedGen + 1, nextPendingGen) : nextPendingGen;
assert newGen > latestKnownRepoGen.get()
: "Attempted new generation ["
- + newGen
- + "] must be larger than latest known generation ["
- + latestKnownRepoGen.get()
- + "]";
+ + newGen
+ + "] must be larger than latest known generation ["
+ + latestKnownRepoGen.get()
+ + "]";
return ClusterState.builder(currentState)
.putProjectMetadata(
ProjectMetadata.builder(project)
@@ -3486,9 +3482,9 @@ private void doSnapshotShard(SnapshotShardContext context) {
.max()
.orElse(-1L) < Long.parseLong(indexGeneration.toString())
: "Tried to delete an index-N blob newer than the current generation ["
- + indexGeneration
- + "] when deleting index-N blobs "
- + blobsToDelete;
+ + indexGeneration
+ + "] when deleting index-N blobs "
+ + blobsToDelete;
final var finalFilesInShardMetadataCount = filesInShardMetadataCount;
final var finalFilesInShardMetadataSize = filesInShardMetadataSize;
@@ -4165,10 +4161,10 @@ private Tuple buildBlobStoreIndexShardSnapsh
.anyMatch(
b -> b.startsWith(SNAPSHOT_PREFIX) || b.startsWith(SNAPSHOT_INDEX_PREFIX) || b.startsWith(UPLOADED_DATA_BLOB_PREFIX)
)) {
- logger.warn(
- "Could not find a readable index-N file in a non-empty shard snapshot directory [" + shardContainer.path() + "]"
- );
- }
+ logger.warn(
+ "Could not find a readable index-N file in a non-empty shard snapshot directory [" + shardContainer.path() + "]"
+ );
+ }
return new Tuple<>(BlobStoreIndexShardSnapshots.EMPTY, latest);
}
diff --git a/server/src/test/java/org/elasticsearch/common/io/stream/BoundedOutputStreamTests.java b/server/src/test/java/org/elasticsearch/common/io/stream/BoundedOutputStreamTests.java
deleted file mode 100644
index c6ef13cc4b437..0000000000000
--- a/server/src/test/java/org/elasticsearch/common/io/stream/BoundedOutputStreamTests.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public
- * License v3.0 only", or the "Server Side Public License, v 1".
- */
-
-package org.elasticsearch.common.io.stream;
-import org.elasticsearch.test.ESTestCase;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-
-public class BoundedOutputStreamTests extends ESTestCase {
-
- public void testWriteSingleBytes() throws IOException {
- ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
- int maxSize = randomIntBetween(0, 100);
- BoundedOutputStream boundedOutputStream = new BoundedOutputStream(byteArrayOutputStream, maxSize);
-
- byte[] values = new byte[maxSize];
- for (int i = 0; i < maxSize; i++) {
- byte value = randomByte();
- boundedOutputStream.write(value);
- values[i] = value;
- }
- assertArrayEquals(values, byteArrayOutputStream.toByteArray());
-
- // Subsequent writes all fail since the size is exceeded
- for (int i = 0; i < randomIntBetween(5, 15); i++) {
- assertThrows(
- BoundedOutputStreamFailedWriteException.class,
- () -> boundedOutputStream.write(randomByte())
- );
- }
- }
-
- public void testWriteByteArray() throws IOException {
- ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
- int maxSize = randomIntBetween(0, 100);
- BoundedOutputStream boundedOutputStream = new BoundedOutputStream(byteArrayOutputStream, maxSize);
-
- byte[] values = randomByteArrayOfLength(maxSize);
- boundedOutputStream.write(values);
- assertArrayEquals(values, byteArrayOutputStream.toByteArray());
- }
-
- public void testWriteByteArrayWithArrayLengthGreaterThanMaxSize() {
- ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
- int maxSize = randomIntBetween(0, 100);
- BoundedOutputStream boundedOutputStream = new BoundedOutputStream(byteArrayOutputStream, maxSize);
-
- assertThrows(
- BoundedOutputStreamFailedWriteException.class,
- () -> boundedOutputStream.write(randomByteArrayOfLength(maxSize + randomIntBetween(1, 100)))
- );
- }
-
- public void testWriteByteArrayWithOffset() throws IOException {
- ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
- int maxSize = randomIntBetween(50, 100);
- BoundedOutputStream boundedOutputStream = new BoundedOutputStream(byteArrayOutputStream, maxSize);
-
- byte[] values = new byte[maxSize];
- for (int i = 0; i < maxSize; i++) {
- values[i] = randomByte();
- }
-
- int offset = randomIntBetween(0, 50);
- int length = randomIntBetween(offset, maxSize);
- boundedOutputStream.write(values, offset, length);
- byte[] expectedResult = new byte[length];
- System.arraycopy(values, offset, expectedResult, 0, length);
- assertArrayEquals(expectedResult, byteArrayOutputStream.toByteArray());
- }
-
-
- public void testWriteByteArrayWithOffsetWithLengthExceedingMaxSize() {
- ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
-// int maxSize = randomIntBetween(0, 100);
- int maxSize = 1;
- BoundedOutputStream boundedOutputStream = new BoundedOutputStream(byteArrayOutputStream, maxSize);
-
-// int randomOffset = randomIntBetween(0, maxSize);
-// int randomLength = randomIntBetween(maxSize, 200);
- int randomOffset = 0;
- int randomLength = 1;
-
- logger.warn("maxsize: " + maxSize);
- logger.warn("offset: " + randomOffset);
- logger.warn("length: " + randomLength);
-
- assertThrows(
- BoundedOutputStreamFailedWriteException.class,
- () -> boundedOutputStream.write(randomByteArrayOfLength(randomLength), randomOffset, randomLength)
- );
- }
-
-
- public void testNoFurtherWritesAfterLimitExceeded() throws IOException {
- ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
- int maxSize = randomIntBetween(0, 100);
- BoundedOutputStream boundedOutputStream = new BoundedOutputStream(byteArrayOutputStream, maxSize);
-
- // Write until maxSize is reached
- for (int i = 0; i < maxSize; i++) {
- boundedOutputStream.write(randomByte());
- }
-
- assertThrows(BoundedOutputStreamFailedWriteException.class, () -> boundedOutputStream.write(randomByte()));
- assertThrows(BoundedOutputStreamFailedWriteException.class, () -> boundedOutputStream.write(randomByteArrayOfLength(randomIntBetween(0, 100))));
- assertThrows(BoundedOutputStreamFailedWriteException.class, () -> boundedOutputStream.write(randomByteArrayOfLength(randomIntBetween(0, 100)), randomIntBetween(0, 100), randomIntBetween(0, 100)));
- }
-}
diff --git a/server/src/test/java/org/elasticsearch/common/io/stream/TruncatedOutputStreamTests.java b/server/src/test/java/org/elasticsearch/common/io/stream/TruncatedOutputStreamTests.java
new file mode 100644
index 0000000000000..ab829a81055c5
--- /dev/null
+++ b/server/src/test/java/org/elasticsearch/common/io/stream/TruncatedOutputStreamTests.java
@@ -0,0 +1,122 @@
+/*
+ * 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", the "GNU Affero General Public License v3.0 only", 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", the "GNU Affero General Public
+ * License v3.0 only", or the "Server Side Public License, v 1".
+ */
+
+package org.elasticsearch.common.io.stream;
+
+import org.elasticsearch.test.ESTestCase;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class TruncatedOutputStreamTests extends ESTestCase {
+
+ public void testWriteSingleBytes() throws IOException {
+ ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+ int maxSize = randomIntBetween(0, 100);
+ TruncatedOutputStream truncatedOutputStream = new TruncatedOutputStream(
+ byteArrayOutputStream,
+ byteArrayOutputStream::size,
+ maxSize
+ );
+
+ byte[] values = new byte[maxSize];
+
+ // Write enough bytes within the defined maxSize
+ for (int i = 0; i < maxSize; i++) {
+ byte b = randomByte();
+ truncatedOutputStream.write(b);
+ values[i] = b;
+ }
+
+ // The stream should be truncated now that it is filled
+ for (int i = 0; i < randomIntBetween(0, 20); i++) {
+ truncatedOutputStream.write(randomByte());
+ }
+
+ assertArrayEquals(values, byteArrayOutputStream.toByteArray());
+ }
+
+ public void testWriteByteArray() throws IOException {
+ ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+ int maxSize = randomIntBetween(100, 200);
+ TruncatedOutputStream truncatedOutputStream = new TruncatedOutputStream(
+ byteArrayOutputStream,
+ byteArrayOutputStream::size,
+ maxSize
+ );
+
+ List values = new ArrayList<>();
+ int bytesWritten = 0;
+ // Write beyond the streams capacity
+ while (bytesWritten <= maxSize * 2) {
+ byte[] bytes = randomByteArrayOfLength(randomIntBetween(0, 20));
+ truncatedOutputStream.write(bytes);
+
+ // If there was capacity before writing, then the stream wrote the entire array
+ // even if that meant overflowing
+ if (bytesWritten < maxSize) {
+ for (byte b : bytes) {
+ values.add(b);
+ }
+ }
+
+ bytesWritten += bytes.length;
+ }
+
+ byte[] valuesAsByteArray = new byte[values.size()];
+ int i = 0;
+ for (byte b : values) {
+ valuesAsByteArray[i] = b;
+ i++;
+ }
+
+ assertArrayEquals(valuesAsByteArray, byteArrayOutputStream.toByteArray());
+ }
+
+ public void testWriteByteArrayWithOffsetAndLength() throws IOException {
+ ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+ int maxSize = randomIntBetween(100, 200);
+ TruncatedOutputStream truncatedOutputStream = new TruncatedOutputStream(
+ byteArrayOutputStream,
+ byteArrayOutputStream::size,
+ maxSize
+ );
+
+ List values = new ArrayList<>();
+ int bytesWritten = 0;
+ // Write beyond the streams capacity
+ while (bytesWritten <= maxSize * 2) {
+ byte[] bytes = randomByteArrayOfLength(randomIntBetween(0, 20));
+ int offset = randomIntBetween(0, bytes.length);
+ int length = randomIntBetween(0, bytes.length - offset);
+ truncatedOutputStream.write(bytes, offset, length);
+
+ // If there was capacity before writing, then the stream wrote the sub array
+ // even if that meant overflowing
+ if (bytesWritten < maxSize) {
+ for (int i = offset; i < offset + length; i++) {
+ values.add(bytes[i]);
+ }
+ }
+
+ bytesWritten += length;
+ }
+
+ byte[] valuesAsByteArray = new byte[values.size()];
+ int i = 0;
+ for (byte b : values) {
+ valuesAsByteArray[i] = b;
+ i++;
+ }
+
+ assertArrayEquals(valuesAsByteArray, byteArrayOutputStream.toByteArray());
+ }
+}
diff --git a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
index aeef37139c9f1..3fdde48b009a9 100644
--- a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
+++ b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
@@ -41,7 +41,6 @@
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.blobstore.OperationPurpose;
import org.elasticsearch.common.bytes.BytesArray;
-import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
@@ -79,7 +78,6 @@
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
-import java.util.Comparator;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -165,12 +163,12 @@ public void testRetrieveSnapshots() {
List snapshotIds = ESBlobStoreRepositoryIntegTestCase.getRepositoryData(repository)
.getSnapshotIds()
.stream()
- .sorted(Comparator.comparing(SnapshotId::getName))
+ .sorted((s1, s2) -> s1.getName().compareTo(s2.getName()))
.toList();
assertThat(snapshotIds, equalTo(originalSnapshots));
}
- public void testReadAndWriteSnapshotsThroughIndexFile() {
+ public void testReadAndWriteSnapshotsThroughIndexFile() throws Exception {
final BlobStoreRepository repository = setupRepo();
final long pendingGeneration = repository.metadata.pendingGeneration();
// write to and read from a index file with no entries
@@ -179,8 +177,8 @@ public void testReadAndWriteSnapshotsThroughIndexFile() {
writeIndexGen(repository, emptyData, emptyData.getGenId());
RepositoryData repoData = ESBlobStoreRepositoryIntegTestCase.getRepositoryData(repository);
assertEquals(repoData, emptyData);
- assertEquals(0, repoData.getIndices().size());
- assertEquals(0, repoData.getSnapshotIds().size());
+ assertEquals(repoData.getIndices().size(), 0);
+ assertEquals(repoData.getSnapshotIds().size(), 0);
assertEquals(pendingGeneration + 1L, repoData.getGenId());
// write to and read from an index file with snapshots but no indices
@@ -196,7 +194,7 @@ public void testReadAndWriteSnapshotsThroughIndexFile() {
public void testIndexGenerationalFiles() throws Exception {
final BlobStoreRepository repository = setupRepo();
- assertEquals(RepositoryData.EMPTY, ESBlobStoreRepositoryIntegTestCase.getRepositoryData(repository));
+ assertEquals(ESBlobStoreRepositoryIntegTestCase.getRepositoryData(repository), RepositoryData.EMPTY);
final long pendingGeneration = repository.metadata.pendingGeneration();
@@ -275,16 +273,16 @@ public ClusterState execute(ClusterState currentState) {
.map(
r -> r.name().equals(repositoryName)
? new RepositoryMetadata(
- r.name(),
- r.uuid(),
- r.type(),
- Settings.builder()
- .put(r.settings())
- .put(BlobStoreRepository.READONLY_SETTING_KEY, "true")
- .build(),
- r.generation(),
- r.pendingGeneration()
- )
+ r.name(),
+ r.uuid(),
+ r.type(),
+ Settings.builder()
+ .put(r.settings())
+ .put(BlobStoreRepository.READONLY_SETTING_KEY, "true")
+ .build(),
+ r.generation(),
+ r.pendingGeneration()
+ )
: r
)
.toList()
@@ -380,7 +378,7 @@ public void testBadChunksize() {
);
}
- public void testRepositoryDataDetails() {
+ public void testRepositoryDataDetails() throws Exception {
final BlobStoreRepository repository = setupRepo();
final String repositoryName = repository.getMetadata().name();
@@ -462,6 +460,7 @@ private BlobStoreRepository setupRepo() {
private BlobStoreRepository setupRepo(Settings.Builder repoSettings) {
final Client client = client();
final Path location = ESIntegTestCase.randomRepoPath(node().settings());
+
repoSettings.put(node().settings()).put("location", location);
boolean compress = randomBoolean();
if (compress == false) {
@@ -783,7 +782,7 @@ public void testShardBlobsToDeleteWithSufficientHeapSpace() {
final var shardId = shard;
final var shardGeneration = new ShardGeneration(randomUUID());
expectedShardGenerations.put(indexId, shard, shardGeneration);
- final var blobsToDelete = generateRandomBlobsToDelete();
+ final var blobsToDelete = generateRandomBlobsToDelete(0, 100);
blobCount += blobsToDelete.size();
final var indexPath = repo.basePath()
@@ -809,6 +808,7 @@ public void testShardBlobsToDeleteWithSufficientHeapSpace() {
safeAwait(countDownLatch);
assertEquals(expectedShardGenerations.build(), shardBlobsToDelete.getUpdatedShardGenerations());
shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
+ // All writes succeeded, and all blobPaths are returned
assertThat(expectedBlobsToDelete, empty());
assertThat(shardBlobsToDelete.sizeInBytes(), lessThanOrEqualTo(Math.max(ByteSizeUnit.KB.toIntBytes(1), 20 * blobCount)));
mockLog.assertAllExpectationsMatched();
@@ -817,136 +817,70 @@ public void testShardBlobsToDeleteWithSufficientHeapSpace() {
}
/*
- There is sufficient heap space for the first write, but not the second
+ There is sufficient heap space for some writes, but not all
*/
-// @TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
-// public void testShardBlobsToDeleteWithLimitedHeapSpace() {
-// TestShardSnapshotMetaDeleteResult blob1Result = generateTestShardSnapshotMetaDeleteResult(0, 20);
-// // blob1Result can be compressed up to 4x when written to shardsDeleteResults so make the second blob
-// // at least 4x larger than the first to guarantee this blob cannot also be written
-// TestShardSnapshotMetaDeleteResult blob2Result = generateTestShardSnapshotMetaDeleteResult(80, 100);
-//
-// final var expectedShardGenerations = ShardGenerations.builder()
-// .put(blob1Result.indexId, blob1Result.shardId, blob1Result.shardGeneration)
-// .put(blob2Result.indexId, blob2Result.shardId, blob2Result.shardGeneration)
-// .build();
-//
-// int totalBytesRequired = calculateBytesRequiredToWriteShardSnapshotMetaDeleteResult(blob1Result);
-// Settings.Builder settings = Settings.builder()
-// .put("repositories.blobstore.max_shard_delete_results_size", totalBytesRequired + "b");
-// final var repo = setupRepo(settings);
-// try (var shardBlobsToDelete = repo.new ShardBlobsToDelete(settings.build())) {
-// try (var mockLog = MockLog.capture(BlobStoreRepository.class)) {
-// addFailureToCleanUpDanglingBlobsSeenExpectation(mockLog);
-//
-// final var expectedBlobsToDelete = new HashSet();
-// var countDownLatch = new CountDownLatch(1);
-// try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
-// final var indexPath1 = generateRepoPath(repo, blob1Result);
-// for (final var blobToDelete : blob1Result.blobsToDelete) {
-// expectedBlobsToDelete.add(indexPath1 + blobToDelete);
-// }
-// repo.threadPool()
-// .generic()
-// .execute(
-// ActionRunnable.run(
-// refs.acquireListener(),
-// () -> shardBlobsToDelete.addShardDeleteResult(
-// blob1Result.indexId,
-// blob1Result.shardId,
-// blob1Result.shardGeneration,
-// blob1Result.blobsToDelete
-// )
-// )
-// );
-// }
-// safeAwait(countDownLatch);
-//
-// countDownLatch = new CountDownLatch(1);
-// try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
-// final var indexPath2 = generateRepoPath(repo, blob2Result);
-// for (final var blobToDelete : blob2Result.blobsToDelete) {
-// expectedBlobsToDelete.add(indexPath2 + blobToDelete);
-// }
-// repo.threadPool()
-// .generic()
-// .execute(
-// ActionRunnable.run(
-// refs.acquireListener(),
-// () -> shardBlobsToDelete.addShardDeleteResult(
-// blob2Result.indexId,
-// blob2Result.shardId,
-// blob2Result.shardGeneration,
-// blob2Result.blobsToDelete
-// )
-// )
-// );
-// }
-//
-// safeAwait(countDownLatch);
-// assertEquals(expectedShardGenerations, shardBlobsToDelete.getUpdatedShardGenerations());
-// shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
-// // None of the second writes succeeded and are therefore not returned by blobsToDelete()
-// assertEquals(expectedBlobsToDelete.size(), blob2Result.blobsToDelete.size());
-// mockLog.assertAllExpectationsMatched();
-// }
-// }
-// }
-//
-// /*
-// There is heap space, but it's insufficient for any write requests
-// */
-// @TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
-// public void testShardBlobsToDeleteWithSmallHeapSpace() {
-// TestShardSnapshotMetaDeleteResult result = generateTestShardSnapshotMetaDeleteResult(0, 20);
-// final var expectedShardGenerations = ShardGenerations.builder().put(result.indexId, result.shardId, result.shardGeneration).build();
-//
-// int totalBytesRequired = calculateBytesRequiredToWriteShardSnapshotMetaDeleteResult(result);
-// int heapSpace = randomIntBetween(1, totalBytesRequired - 1);
-// Settings.Builder settings = Settings.builder().put("repositories.blobstore.max_shard_delete_results_size", heapSpace + "b");
-// final var repo = setupRepo(settings);
-// try (var shardBlobsToDelete = repo.new ShardBlobsToDelete(settings.build())) {
-// try (var mockLog = MockLog.capture(BlobStoreRepository.class)) {
-// addFailureToCleanUpDanglingBlobsSeenExpectation(mockLog);
-//
-// final var expectedBlobsToDelete = new HashSet();
-// var countDownLatch = new CountDownLatch(1);
-// try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
-// final var indexPath = generateRepoPath(repo, result);
-// for (final var blobToDelete : result.blobsToDelete) {
-// expectedBlobsToDelete.add(indexPath + blobToDelete);
-// }
-// repo.threadPool()
-// .generic()
-// .execute(
-// ActionRunnable.run(
-// refs.acquireListener(),
-// () -> shardBlobsToDelete.addShardDeleteResult(
-// result.indexId,
-// result.shardId,
-// result.shardGeneration,
-// result.blobsToDelete
-// )
-// )
-// );
-//
-// }
-// safeAwait(countDownLatch);
-// assertEquals(expectedShardGenerations, shardBlobsToDelete.getUpdatedShardGenerations());
-// shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
-// assertEquals(result.blobsToDelete.size(), expectedBlobsToDelete.size());
-// mockLog.assertAllExpectationsMatched();
-// }
-// }
-// }
+ @TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
+ public void testShardBlobsToDeleteWithLimitedHeapSpace() {
+ // Limit the heap size so we force it to truncate the stream
+ int totalBytesRequired = randomIntBetween(1000, 10000);
+ Settings.Builder settings = Settings.builder()
+ .put("repositories.blobstore.max_shard_delete_results_size", totalBytesRequired + "b");
+ final var repo = setupRepo(settings);
+ try (var shardBlobsToDelete = repo.new ShardBlobsToDelete(settings.build())) {
+ try (var mockLog = MockLog.capture(BlobStoreRepository.class)) {
+ // We expect some writes to fail
+ addFailureToCleanUpDanglingBlobsSeenExpectation(mockLog);
+
+ final var expectedShardGenerations = ShardGenerations.builder();
+ final var expectedBlobsToDelete = new HashSet();
+
+ final var countDownLatch = new CountDownLatch(1);
+ try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
+ for (int index = between(200, 1000); index > 0; index--) {
+ final var indexId = new IndexId(randomIdentifier(), randomUUID());
+ for (int shard = between(5, 30); shard > 0; shard--) {
+ final var shardId = shard;
+ final var shardGeneration = new ShardGeneration(randomUUID());
+ expectedShardGenerations.put(indexId, shard, shardGeneration);
+ final var blobsToDelete = generateRandomBlobsToDelete(20, 100);
+
+ final var indexPath = repo.basePath()
+ .add("indices")
+ .add(indexId.getId())
+ .add(Integer.toString(shard))
+ .buildAsString();
+ for (final var blobToDelete : blobsToDelete) {
+ expectedBlobsToDelete.add(indexPath + blobToDelete);
+ }
+
+ repo.threadPool().generic().execute(ActionRunnable.run(refs.acquireListener(), () -> {
+ shardBlobsToDelete.addShardDeleteResult(indexId, shardId, shardGeneration, blobsToDelete);
+ }));
+ }
+ }
+ }
+
+ safeAwait(countDownLatch);
+ assertEquals(expectedShardGenerations.build(), shardBlobsToDelete.getUpdatedShardGenerations());
+ shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
+ // Some writes were truncated, and so not all blobPaths are returned
+ assertFalse(expectedBlobsToDelete.isEmpty());
+ mockLog.assertAllExpectationsMatched();
+ }
+ }
+ }
/*
There's no heap space, so we don't even attempt to write
*/
@TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
public void testShardBlobsToDeleteWithOutHeapSpace() {
- TestShardSnapshotMetaDeleteResult result = generateTestShardSnapshotMetaDeleteResult(0, 20);
- final var expectedShardGenerations = ShardGenerations.builder().put(result.indexId, result.shardId, result.shardGeneration).build();
+ IndexId indexId = new IndexId(randomIdentifier(), randomUUID());
+ int shardId = between(1, 30);
+ Collection blobIds = generateRandomBlobsToDelete(0, 20);
+ ShardGeneration shardGeneration = new ShardGeneration(randomUUID());
+
+ final var expectedShardGenerations = ShardGenerations.builder().put(indexId, shardId, shardGeneration).build();
Settings.Builder settings = Settings.builder().put("repositories.blobstore.max_shard_delete_results_size", "0b");
final var repo = setupRepo(settings);
@@ -957,8 +891,12 @@ public void testShardBlobsToDeleteWithOutHeapSpace() {
final var expectedBlobsToDelete = new HashSet();
var countDownLatch = new CountDownLatch(1);
try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
- final var indexPath = generateRepoPath(repo, result);
- for (final var blobToDelete : result.blobsToDelete) {
+ final var indexPath = repo.basePath()
+ .add("indices")
+ .add(indexId.getId())
+ .add(Integer.toString(shardId))
+ .buildAsString();
+ for (final var blobToDelete : blobIds) {
expectedBlobsToDelete.add(indexPath + blobToDelete);
}
repo.threadPool()
@@ -966,49 +904,23 @@ public void testShardBlobsToDeleteWithOutHeapSpace() {
.execute(
ActionRunnable.run(
refs.acquireListener(),
- () -> shardBlobsToDelete.addShardDeleteResult(
- result.indexId,
- result.shardId,
- result.shardGeneration,
- result.blobsToDelete
- )
+ () -> shardBlobsToDelete.addShardDeleteResult(indexId, shardId, shardGeneration, blobIds)
)
);
}
safeAwait(countDownLatch);
+ // We should still update the shard generations
assertEquals(expectedShardGenerations, shardBlobsToDelete.getUpdatedShardGenerations());
+ // We expect an empty blobPath iterator to be returned
shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
- assertEquals(result.blobsToDelete.size(), expectedBlobsToDelete.size());
+ assertEquals(blobIds.size(), expectedBlobsToDelete.size());
+ assertEquals(0, shardBlobsToDelete.sizeInBytes());
mockLog.assertAllExpectationsMatched();
}
}
}
- private record TestShardSnapshotMetaDeleteResult(
- IndexId indexId,
- int shardId,
- Collection blobsToDelete,
- ShardGeneration shardGeneration
- ) {}
-
- private TestShardSnapshotMetaDeleteResult generateTestShardSnapshotMetaDeleteResult(int minBlobSize, int maxBlobSize) {
- return new TestShardSnapshotMetaDeleteResult(
- new IndexId(randomIdentifier(), randomUUID()),
- between(1, 30),
- generateRandomBlobsToDelete(minBlobSize, maxBlobSize),
- new ShardGeneration(randomUUID())
- );
- }
-
- private String generateRepoPath(BlobStoreRepository repo, TestShardSnapshotMetaDeleteResult result) {
- return repo.basePath().add("indices").add(result.indexId.getId()).add(Integer.toString(result.shardId)).buildAsString();
- }
-
- private List generateRandomBlobsToDelete() {
- return generateRandomBlobsToDelete(0, 100);
- }
-
private List generateRandomBlobsToDelete(int minSize, int maxSize) {
return randomList(
minSize,
@@ -1023,7 +935,7 @@ private void addFailureToCleanUpDanglingBlobsUnseenExpectation(MockLog mockLog)
"failure to clean up dangling blobs warn logs",
"org.elasticsearch.repositories.blobstore.BlobStoreRepository",
Level.WARN,
- "Failure to clean up the following dangling blobs"
+ "Failure to clean up"
)
);
}
@@ -1034,7 +946,7 @@ private void addFailureToCleanUpDanglingBlobsSeenExpectation(MockLog mockLog) {
"failure to clean up dangling blobs warn logs",
"org.elasticsearch.repositories.blobstore.BlobStoreRepository",
Level.WARN,
- "Failure to clean up the following dangling blobs"
+ "Failure to clean up"
)
);
}
From 0fa50993999d3bc19610fcfaeb23b10ee3508096 Mon Sep 17 00:00:00 2001
From: Joshua Adams
Date: Fri, 12 Sep 2025 12:31:39 +0100
Subject: [PATCH 17/25] Spotless commit
---
.../blobstore/BlobStoreRepository.java | 48 +++++++++----------
.../io/stream/TruncatedOutputStreamTests.java | 18 +++----
.../blobstore/BlobStoreRepositoryTests.java | 20 ++++----
3 files changed, 43 insertions(+), 43 deletions(-)
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 39ba5d3770fac..8ab971ef5969b 100644
--- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
+++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
@@ -1440,12 +1440,12 @@ private static List unusedBlobs(
.filter(
blob -> blob.startsWith(SNAPSHOT_INDEX_PREFIX)
|| (blob.startsWith(SNAPSHOT_PREFIX)
- && blob.endsWith(METADATA_BLOB_NAME_SUFFIX)
- && survivingSnapshotUUIDs.contains(
- blob.substring(SNAPSHOT_PREFIX.length(), blob.length() - METADATA_BLOB_NAME_SUFFIX.length())
- ) == false)
+ && blob.endsWith(METADATA_BLOB_NAME_SUFFIX)
+ && survivingSnapshotUUIDs.contains(
+ blob.substring(SNAPSHOT_PREFIX.length(), blob.length() - METADATA_BLOB_NAME_SUFFIX.length())
+ ) == false)
|| (blob.startsWith(UPLOADED_DATA_BLOB_PREFIX)
- && updatedSnapshots.findNameFile(canonicalName(blob)) == null)
+ && updatedSnapshots.findNameFile(canonicalName(blob)) == null)
|| FsBlobContainer.isTempBlobName(blob)
)
.toList();
@@ -2623,10 +2623,10 @@ private void cacheRepositoryData(RepositoryData repositoryData, IndexVersion ver
toCache = repositoryData.withoutShardGenerations();
assert repositoryData.indexMetaDataGenerations().equals(IndexMetaDataGenerations.EMPTY)
: "repository data should not contain index generations at version ["
- + version.toReleaseVersion()
- + "] but saw ["
- + repositoryData.indexMetaDataGenerations()
- + "]";
+ + version.toReleaseVersion()
+ + "] but saw ["
+ + repositoryData.indexMetaDataGenerations()
+ + "]";
}
assert toCache.getGenId() >= 0 : "No need to cache abstract generations but attempted to cache [" + toCache.getGenId() + "]";
latestKnownRepositoryData.updateAndGet(known -> {
@@ -2650,10 +2650,10 @@ private static String previousWriterMessage(@Nullable Tuple previo
return previousWriterInfo == null
? ""
: " The last cluster to write to this repository was ["
- + previousWriterInfo.v2()
- + "] at generation ["
- + previousWriterInfo.v1()
- + "].";
+ + previousWriterInfo.v2()
+ + "] at generation ["
+ + previousWriterInfo.v1()
+ + "].";
}
/**
@@ -2863,10 +2863,10 @@ public ClusterState execute(ClusterState currentState) {
newGen = uninitializedMeta ? Math.max(expectedGen + 1, nextPendingGen) : nextPendingGen;
assert newGen > latestKnownRepoGen.get()
: "Attempted new generation ["
- + newGen
- + "] must be larger than latest known generation ["
- + latestKnownRepoGen.get()
- + "]";
+ + newGen
+ + "] must be larger than latest known generation ["
+ + latestKnownRepoGen.get()
+ + "]";
return ClusterState.builder(currentState)
.putProjectMetadata(
ProjectMetadata.builder(project)
@@ -3482,9 +3482,9 @@ private void doSnapshotShard(SnapshotShardContext context) {
.max()
.orElse(-1L) < Long.parseLong(indexGeneration.toString())
: "Tried to delete an index-N blob newer than the current generation ["
- + indexGeneration
- + "] when deleting index-N blobs "
- + blobsToDelete;
+ + indexGeneration
+ + "] when deleting index-N blobs "
+ + blobsToDelete;
final var finalFilesInShardMetadataCount = filesInShardMetadataCount;
final var finalFilesInShardMetadataSize = filesInShardMetadataSize;
@@ -4161,10 +4161,10 @@ private Tuple buildBlobStoreIndexShardSnapsh
.anyMatch(
b -> b.startsWith(SNAPSHOT_PREFIX) || b.startsWith(SNAPSHOT_INDEX_PREFIX) || b.startsWith(UPLOADED_DATA_BLOB_PREFIX)
)) {
- logger.warn(
- "Could not find a readable index-N file in a non-empty shard snapshot directory [" + shardContainer.path() + "]"
- );
- }
+ logger.warn(
+ "Could not find a readable index-N file in a non-empty shard snapshot directory [" + shardContainer.path() + "]"
+ );
+ }
return new Tuple<>(BlobStoreIndexShardSnapshots.EMPTY, latest);
}
diff --git a/server/src/test/java/org/elasticsearch/common/io/stream/TruncatedOutputStreamTests.java b/server/src/test/java/org/elasticsearch/common/io/stream/TruncatedOutputStreamTests.java
index ab829a81055c5..bf6540e81cab9 100644
--- a/server/src/test/java/org/elasticsearch/common/io/stream/TruncatedOutputStreamTests.java
+++ b/server/src/test/java/org/elasticsearch/common/io/stream/TruncatedOutputStreamTests.java
@@ -22,9 +22,9 @@ public void testWriteSingleBytes() throws IOException {
ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
int maxSize = randomIntBetween(0, 100);
TruncatedOutputStream truncatedOutputStream = new TruncatedOutputStream(
- byteArrayOutputStream,
- byteArrayOutputStream::size,
- maxSize
+ byteArrayOutputStream,
+ byteArrayOutputStream::size,
+ maxSize
);
byte[] values = new byte[maxSize];
@@ -48,9 +48,9 @@ public void testWriteByteArray() throws IOException {
ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
int maxSize = randomIntBetween(100, 200);
TruncatedOutputStream truncatedOutputStream = new TruncatedOutputStream(
- byteArrayOutputStream,
- byteArrayOutputStream::size,
- maxSize
+ byteArrayOutputStream,
+ byteArrayOutputStream::size,
+ maxSize
);
List values = new ArrayList<>();
@@ -85,9 +85,9 @@ public void testWriteByteArrayWithOffsetAndLength() throws IOException {
ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
int maxSize = randomIntBetween(100, 200);
TruncatedOutputStream truncatedOutputStream = new TruncatedOutputStream(
- byteArrayOutputStream,
- byteArrayOutputStream::size,
- maxSize
+ byteArrayOutputStream,
+ byteArrayOutputStream::size,
+ maxSize
);
List values = new ArrayList<>();
diff --git a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
index 3fdde48b009a9..e5145397b6156 100644
--- a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
+++ b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
@@ -273,16 +273,16 @@ public ClusterState execute(ClusterState currentState) {
.map(
r -> r.name().equals(repositoryName)
? new RepositoryMetadata(
- r.name(),
- r.uuid(),
- r.type(),
- Settings.builder()
- .put(r.settings())
- .put(BlobStoreRepository.READONLY_SETTING_KEY, "true")
- .build(),
- r.generation(),
- r.pendingGeneration()
- )
+ r.name(),
+ r.uuid(),
+ r.type(),
+ Settings.builder()
+ .put(r.settings())
+ .put(BlobStoreRepository.READONLY_SETTING_KEY, "true")
+ .build(),
+ r.generation(),
+ r.pendingGeneration()
+ )
: r
)
.toList()
From d55893d5cc3de63dcad1fcf8c1936ed19a696088 Mon Sep 17 00:00:00 2001
From: Joshua Adams
Date: Fri, 12 Sep 2025 15:51:07 +0100
Subject: [PATCH 18/25] Add skippedResultsCount
---
.../blobstore/BlobStoreRepository.java | 20 ++++++++++++++-----
.../blobstore/BlobStoreRepositoryTests.java | 4 ++--
2 files changed, 17 insertions(+), 7 deletions(-)
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 8ab971ef5969b..0ae5113dcc0bc 100644
--- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
+++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
@@ -1688,6 +1688,7 @@ void writeTo(StreamOutput out) throws IOException {
private final BytesStreamOutput shardDeleteResults;
private int resultCount = 0;
+ private int skippedResultsCount = 0;
private final StreamOutput compressed;
@@ -1751,8 +1752,13 @@ synchronized void addShardDeleteResult(
}
try {
shardGenerationsBuilder.put(indexId, shardId, newGeneration);
- new ShardSnapshotMetaDeleteResult(Objects.requireNonNull(indexId.getId()), shardId, blobsToDelete).writeTo(compressed);
- resultCount += 1;
+ // Only write if we have capacity
+ if (shardDeleteResults.size() < this.shardDeleteResultsMaxSize) {
+ new ShardSnapshotMetaDeleteResult(Objects.requireNonNull(indexId.getId()), shardId, blobsToDelete).writeTo(compressed);
+ resultCount += 1;
+ } else {
+ skippedResultsCount += 1;
+ }
} catch (IOException e) {
assert false : e; // no IO actually happens here
throw new UncheckedIOException(e);
@@ -1789,9 +1795,13 @@ public Iterator getBlobPaths() {
try {
shardResult = new ShardSnapshotMetaDeleteResult(input);
} catch (EOFException ex) {
- // There was insufficient stream space to write this blob
- // All subsequent blobs will likewise be dangling
- logger.warn("Failure to clean up {} dangling blobs", resultCount - i);
+ // There was insufficient stream space to write this blob so it cannot be parsed
+ // All further write attempts were then skipped
+ logger.warn(
+ "Failed to clean up {} shards because there was insufficient heap space to track them all. "
+ + "These shards will be cleaned during subsequent delete operations.",
+ skippedResultsCount + 1
+ );
break;
} catch (IOException e) {
throw new UncheckedIOException(e);
diff --git a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
index e5145397b6156..717c65067c8bd 100644
--- a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
+++ b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
@@ -935,7 +935,7 @@ private void addFailureToCleanUpDanglingBlobsUnseenExpectation(MockLog mockLog)
"failure to clean up dangling blobs warn logs",
"org.elasticsearch.repositories.blobstore.BlobStoreRepository",
Level.WARN,
- "Failure to clean up"
+ "Failed to clean up"
)
);
}
@@ -946,7 +946,7 @@ private void addFailureToCleanUpDanglingBlobsSeenExpectation(MockLog mockLog) {
"failure to clean up dangling blobs warn logs",
"org.elasticsearch.repositories.blobstore.BlobStoreRepository",
Level.WARN,
- "Failure to clean up"
+ "Failed to clean up"
)
);
}
From 3725a3c4b287b0b318218bc874f7bb21b44a470a Mon Sep 17 00:00:00 2001
From: Joshua Adams
Date: Thu, 18 Sep 2025 12:09:25 +0100
Subject: [PATCH 19/25] Rewrite the unit tests
---
.../blobstore/BlobStoreRepository.java | 81 ++---
.../blobstore/BlobStoreRepositoryTests.java | 298 ++++++++----------
2 files changed, 169 insertions(+), 210 deletions(-)
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 0ae5113dcc0bc..17c34ce81e533 100644
--- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
+++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
@@ -1688,7 +1688,6 @@ void writeTo(StreamOutput out) throws IOException {
private final BytesStreamOutput shardDeleteResults;
private int resultCount = 0;
- private int skippedResultsCount = 0;
private final StreamOutput compressed;
@@ -1696,6 +1695,7 @@ void writeTo(StreamOutput out) throws IOException {
private final ShardGenerations.Builder shardGenerationsBuilder = ShardGenerations.builder();
+ // Gets 25% of the heap size to be allocated to the shard_delete_results stream
public final Setting MAX_SHARD_DELETE_RESULTS_SIZE_SETTING = Setting.memorySizeSetting(
"repositories.blobstore.max_shard_delete_results_size",
"25%",
@@ -1704,39 +1704,36 @@ void writeTo(StreamOutput out) throws IOException {
ShardBlobsToDelete(Settings settings) {
this.shardDeleteResultsMaxSize = calculateMaximumShardDeleteResultsSize(settings);
- if (this.shardDeleteResultsMaxSize > 0) {
- this.shardDeleteResults = new ReleasableBytesStreamOutput(bigArrays);
- this.compressed = new OutputStreamStreamOutput(
- new TruncatedOutputStream(
- new BufferedOutputStream(
- new DeflaterOutputStream(Streams.flushOnCloseStream(shardDeleteResults)),
- DeflateCompressor.BUFFER_SIZE
- ),
- shardDeleteResults::size,
- this.shardDeleteResultsMaxSize
- )
- );
- resources.add(compressed);
- resources.add(LeakTracker.wrap((Releasable) shardDeleteResults));
- } else {
- this.shardDeleteResults = null;
- this.compressed = null;
- }
+ this.shardDeleteResults = new ReleasableBytesStreamOutput(bigArrays);
+ this.compressed = new OutputStreamStreamOutput(
+ new TruncatedOutputStream(
+ new BufferedOutputStream(
+ new DeflaterOutputStream(Streams.flushOnCloseStream(shardDeleteResults)),
+ DeflateCompressor.BUFFER_SIZE
+ ),
+ shardDeleteResults::size,
+ this.shardDeleteResultsMaxSize
+ )
+ );
+ resources.add(compressed);
+ resources.add(LeakTracker.wrap((Releasable) shardDeleteResults));
}
/**
* Calculates the maximum size of the shardDeleteResults BytesStreamOutput.
- * The size should at most be 2GB, but no more than 25% of the total remaining heap space.
+ * The size cannot exceed 2GB, without {@code BytesStreamOutput} throwing an IAE,
+ * but should also be no more than 25% of the total remaining heap space.
* A buffer of 1MB is maintained, so that even if the stream is of max size, there is room to flush
* @return The maximum number of bytes the shardDeleteResults BytesStreamOutput can consume in the heap
*/
int calculateMaximumShardDeleteResultsSize(Settings settings) {
- long maxSizeInBytes = MAX_SHARD_DELETE_RESULTS_SIZE_SETTING.get(settings).getBytes();
+ long maxHeapSizeInBytes = MAX_SHARD_DELETE_RESULTS_SIZE_SETTING.get(settings).getBytes();
int oneMBBuffer = 1024 * 1024;
- if (maxSizeInBytes > Integer.MAX_VALUE) {
- return Integer.MAX_VALUE - oneMBBuffer;
+ int maxShardDeleteResultsSize = Integer.MAX_VALUE - oneMBBuffer;
+ if (maxHeapSizeInBytes > maxShardDeleteResultsSize) {
+ return maxShardDeleteResultsSize;
}
- return (int) maxSizeInBytes;
+ return (int) maxHeapSizeInBytes;
}
synchronized void addShardDeleteResult(
@@ -1745,19 +1742,23 @@ synchronized void addShardDeleteResult(
ShardGeneration newGeneration,
Collection blobsToDelete
) {
- if (compressed == null) {
- // No output stream: skip writing, but still update generations
- shardGenerationsBuilder.put(indexId, shardId, newGeneration);
- return;
- }
try {
shardGenerationsBuilder.put(indexId, shardId, newGeneration);
// Only write if we have capacity
if (shardDeleteResults.size() < this.shardDeleteResultsMaxSize) {
new ShardSnapshotMetaDeleteResult(Objects.requireNonNull(indexId.getId()), shardId, blobsToDelete).writeTo(compressed);
- resultCount += 1;
+ // We only want to read this shard delete result if we were able to write the entire object.
+ // Otherwise, for partial writes, an EOFException will be thrown upon reading
+ if (shardDeleteResults.size() < this.shardDeleteResultsMaxSize) {
+ resultCount += 1;
+ }
} else {
- skippedResultsCount += 1;
+ logger.warn(
+ "Failure to clean up the following dangling blobs, {}, for index {} and shard {}",
+ blobsToDelete,
+ indexId,
+ shardId
+ );
}
} catch (IOException e) {
assert false : e; // no IO actually happens here
@@ -1770,10 +1771,6 @@ public ShardGenerations getUpdatedShardGenerations() {
}
public Iterator getBlobPaths() {
- if (compressed == null || shardDeleteResults == null) {
- // No output stream: nothing to return
- return Collections.emptyIterator();
- }
final StreamInput input;
try {
compressed.close();
@@ -1794,15 +1791,6 @@ public Iterator getBlobPaths() {
ShardSnapshotMetaDeleteResult shardResult;
try {
shardResult = new ShardSnapshotMetaDeleteResult(input);
- } catch (EOFException ex) {
- // There was insufficient stream space to write this blob so it cannot be parsed
- // All further write attempts were then skipped
- logger.warn(
- "Failed to clean up {} shards because there was insufficient heap space to track them all. "
- + "These shards will be cleaned during subsequent delete operations.",
- skippedResultsCount + 1
- );
- break;
} catch (IOException e) {
throw new UncheckedIOException(e);
}
@@ -1816,9 +1804,6 @@ public Iterator getBlobPaths() {
@Override
public void close() {
- if (resources.isEmpty()) {
- return;
- }
try {
IOUtils.close(resources);
} catch (IOException e) {
@@ -1829,7 +1814,7 @@ public void close() {
// exposed for tests
int sizeInBytes() {
- return shardDeleteResults == null ? 0 : shardDeleteResults.size();
+ return shardDeleteResults.size();
}
}
diff --git a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
index 717c65067c8bd..9ce5722982d00 100644
--- a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
+++ b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
@@ -10,6 +10,7 @@
package org.elasticsearch.repositories.blobstore;
import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.core.LogEvent;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRunnable;
import org.elasticsearch.action.admin.cluster.repositories.delete.DeleteRepositoryRequest;
@@ -41,6 +42,7 @@
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.blobstore.OperationPurpose;
import org.elasticsearch.common.bytes.BytesArray;
+import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
@@ -64,6 +66,7 @@
import org.elasticsearch.repositories.fs.FsRepository;
import org.elasticsearch.snapshots.AbstractSnapshotIntegTestCase;
import org.elasticsearch.snapshots.SnapshotId;
+import org.elasticsearch.snapshots.SnapshotShutdownProgressTracker;
import org.elasticsearch.snapshots.SnapshotState;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESSingleNodeTestCase;
@@ -753,201 +756,172 @@ public void testUuidCreationLogging() {
);
}
- // =============== Shard Blobs to Delete Tests =================
+ private MockLog mockLog;
- /*
- There is sufficient heap space to perform all operations on shardBlobsToDelete
+ public void setUp() throws Exception {
+ super.setUp();
+ mockLog = MockLog.capture(BlobStoreRepository.class);
+ }
+
+ private void resetMockLog() {
+ mockLog.close();
+ mockLog = MockLog.capture(BlobStoreRepository.class);
+ }
+
+ public void tearDown() throws Exception {
+ mockLog.close();
+ super.tearDown();
+ }
+
+ /**
+ * Tests writing multiple blobs to ShardBlobToDelete when it has a variable sized stream.
+ * Initially, if there is capacity, we write N blobs to ShardBlobToDelete. We expect each of them to be compressed
+ * and written to the underlying stream.
+ * Once capacity is reached, we write M subsequent blobs, but expect that they will not be written to the
+ * underlying stream, and a WARN log should be thrown for each.
+ * When we read from the stream, we expect only the successful writes to be returned
*/
@TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
- public void testShardBlobsToDeleteWithSufficientHeapSpace() {
- // When the heap memory is above 2GB (Integer.MAX_VALUE) then we expect it to be limited to 2GB
- long heapMemory = randomLongBetween(Integer.MAX_VALUE / 2, Integer.MAX_VALUE * 2L);
+ public void testWriteToShardBlobToDelete() {
+ resetMockLog();
+ int heapMemory = randomIntBetween(0, 20000);
Settings.Builder settings = Settings.builder().put("repositories.blobstore.max_shard_delete_results_size", heapMemory + "b");
final var repo = setupRepo(settings);
try (var shardBlobsToDelete = repo.new ShardBlobsToDelete(settings.build())) {
- // Ensure the logging is as expected
- try (var mockLog = MockLog.capture(BlobStoreRepository.class)) {
- // We expect every write to succeed
- addFailureToCleanUpDanglingBlobsUnseenExpectation(mockLog);
+ final var expectedShardGenerations = ShardGenerations.builder();
+ final var expectedBlobsToDelete = new HashSet();
+ CountDownLatch countDownLatch;
+ int blobCount = 0;
+
+ // First, we write as many blobs as we have capacity for
+
+ // Do not expect to see any WARN logs about dangling blobs as there should be capacity for them all
+ mockLog.addExpectation(
+ new MockLog.UnseenEventExpectation(
+ "failure to clean up dangling blobs warn logs",
+ "org.elasticsearch.repositories.blobstore.BlobStoreRepository",
+ Level.WARN,
+ "Failure to clean up the following dangling blobs"
+ )
+ );
- final var expectedShardGenerations = ShardGenerations.builder();
- final var expectedBlobsToDelete = new HashSet();
+ // Write while we have capacity
+ while (shardBlobsToDelete.sizeInBytes() < heapMemory) {
+ // Generate the next blob to write
+ final var indexId = new IndexId(randomIdentifier(), randomUUID());
+ final var shardId = between(1, 30);
+ final var shardGeneration = new ShardGeneration(randomUUID());
+ final var blobsToDelete = randomList(
+ 100,
+ () -> randomFrom(METADATA_PREFIX, INDEX_FILE_PREFIX, SNAPSHOT_PREFIX) + randomUUID() + randomFrom(
+ "",
+ METADATA_BLOB_NAME_SUFFIX
+ )
+ );
- final var countDownLatch = new CountDownLatch(1);
- int blobCount = 0;
+ expectedShardGenerations.put(indexId, shardId, shardGeneration);
+ final var indexPath = repo.basePath()
+ .add("indices")
+ .add(indexId.getId())
+ .add(Integer.toString(shardId))
+ .buildAsString();
+
+ countDownLatch = new CountDownLatch(1);
try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
- for (int index = between(0, 1000); index > 0; index--) {
- final var indexId = new IndexId(randomIdentifier(), randomUUID());
- for (int shard = between(1, 30); shard > 0; shard--) {
- final var shardId = shard;
- final var shardGeneration = new ShardGeneration(randomUUID());
- expectedShardGenerations.put(indexId, shard, shardGeneration);
- final var blobsToDelete = generateRandomBlobsToDelete(0, 100);
-
- blobCount += blobsToDelete.size();
- final var indexPath = repo.basePath()
- .add("indices")
- .add(indexId.getId())
- .add(Integer.toString(shard))
- .buildAsString();
- for (final var blobToDelete : blobsToDelete) {
- expectedBlobsToDelete.add(indexPath + blobToDelete);
- }
-
- repo.threadPool()
- .generic()
- .execute(
- ActionRunnable.run(
- refs.acquireListener(),
- () -> shardBlobsToDelete.addShardDeleteResult(indexId, shardId, shardGeneration, blobsToDelete)
- )
- );
- }
- }
+ repo.threadPool()
+ .generic()
+ .execute(
+ ActionRunnable.run(
+ refs.acquireListener(),
+ () -> shardBlobsToDelete.addShardDeleteResult(indexId, shardId, shardGeneration, blobsToDelete)
+ )
+ );
}
safeAwait(countDownLatch);
- assertEquals(expectedShardGenerations.build(), shardBlobsToDelete.getUpdatedShardGenerations());
- shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
- // All writes succeeded, and all blobPaths are returned
- assertThat(expectedBlobsToDelete, empty());
- assertThat(shardBlobsToDelete.sizeInBytes(), lessThanOrEqualTo(Math.max(ByteSizeUnit.KB.toIntBytes(1), 20 * blobCount)));
- mockLog.assertAllExpectationsMatched();
+
+ // Only if the entire blob was written to memory do we expect to see it returned
+ if (shardBlobsToDelete.sizeInBytes() < heapMemory) {
+ for (final var blobToDelete : blobsToDelete) {
+ expectedBlobsToDelete.add(indexPath + blobToDelete);
+ }
+ blobCount += blobsToDelete.size();
+ }
}
- }
- }
- /*
- There is sufficient heap space for some writes, but not all
- */
- @TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
- public void testShardBlobsToDeleteWithLimitedHeapSpace() {
- // Limit the heap size so we force it to truncate the stream
- int totalBytesRequired = randomIntBetween(1000, 10000);
- Settings.Builder settings = Settings.builder()
- .put("repositories.blobstore.max_shard_delete_results_size", totalBytesRequired + "b");
- final var repo = setupRepo(settings);
- try (var shardBlobsToDelete = repo.new ShardBlobsToDelete(settings.build())) {
- try (var mockLog = MockLog.capture(BlobStoreRepository.class)) {
- // We expect some writes to fail
- addFailureToCleanUpDanglingBlobsSeenExpectation(mockLog);
+ assertEquals(expectedShardGenerations.build(), shardBlobsToDelete.getUpdatedShardGenerations());
+ shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
+ assertThat(expectedBlobsToDelete, empty());
+ assertThat(shardBlobsToDelete.sizeInBytes(), lessThanOrEqualTo(Math.max(ByteSizeUnit.KB.toIntBytes(1), 20 * blobCount)));
- final var expectedShardGenerations = ShardGenerations.builder();
- final var expectedBlobsToDelete = new HashSet();
+ mockLog.assertAllExpectationsMatched();
+ resetMockLog();
- final var countDownLatch = new CountDownLatch(1);
- try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
- for (int index = between(200, 1000); index > 0; index--) {
- final var indexId = new IndexId(randomIdentifier(), randomUUID());
- for (int shard = between(5, 30); shard > 0; shard--) {
- final var shardId = shard;
- final var shardGeneration = new ShardGeneration(randomUUID());
- expectedShardGenerations.put(indexId, shard, shardGeneration);
- final var blobsToDelete = generateRandomBlobsToDelete(20, 100);
-
- final var indexPath = repo.basePath()
- .add("indices")
- .add(indexId.getId())
- .add(Integer.toString(shard))
- .buildAsString();
- for (final var blobToDelete : blobsToDelete) {
- expectedBlobsToDelete.add(indexPath + blobToDelete);
- }
-
- repo.threadPool().generic().execute(ActionRunnable.run(refs.acquireListener(), () -> {
- shardBlobsToDelete.addShardDeleteResult(indexId, shardId, shardGeneration, blobsToDelete);
- }));
- }
- }
- }
+ // Second, now we're at capacity, we test whether we can accept subsequent writes without throwing an error
+ int numberOfOverflowedWrites = randomIntBetween(1, 20);
- safeAwait(countDownLatch);
- assertEquals(expectedShardGenerations.build(), shardBlobsToDelete.getUpdatedShardGenerations());
- shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
- // Some writes were truncated, and so not all blobPaths are returned
- assertFalse(expectedBlobsToDelete.isEmpty());
- mockLog.assertAllExpectationsMatched();
- }
- }
- }
+ // Expect each write to throw a WARN log
+ mockLog.addExpectation(new MockLog.LoggingExpectation() {
+ int count = 0;
- /*
- There's no heap space, so we don't even attempt to write
- */
- @TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")
- public void testShardBlobsToDeleteWithOutHeapSpace() {
- IndexId indexId = new IndexId(randomIdentifier(), randomUUID());
- int shardId = between(1, 30);
- Collection blobIds = generateRandomBlobsToDelete(0, 20);
- ShardGeneration shardGeneration = new ShardGeneration(randomUUID());
+ @Override
+ public void match(LogEvent event) {
+ if (event.getLevel() != Level.WARN) {
+ return;
+ }
+ if (event.getLoggerName().equals(BlobStoreRepository.class.getCanonicalName()) == false) {
+ return;
+ }
- final var expectedShardGenerations = ShardGenerations.builder().put(indexId, shardId, shardGeneration).build();
+ String msg = event.getMessage().getFormattedMessage();
+ if (msg.matches("Failure to clean up the following dangling blobs, .*, for index .+ and shard .+")) {
+ count++;
+ }
+ }
- Settings.Builder settings = Settings.builder().put("repositories.blobstore.max_shard_delete_results_size", "0b");
- final var repo = setupRepo(settings);
- try (var shardBlobsToDelete = repo.new ShardBlobsToDelete(settings.build())) {
- try (var mockLog = MockLog.capture(BlobStoreRepository.class)) {
- addFailureToCleanUpDanglingBlobsUnseenExpectation(mockLog);
+ @Override
+ public void assertMatched() {
+ assertEquals(numberOfOverflowedWrites, count);
+ }
+ });
+
+ for (int i = 0; i < numberOfOverflowedWrites; i++) {
+ // Generate the next blob to write
+ final var indexId = new IndexId(randomIdentifier(), randomUUID());
+ final var shardId = between(1, 30);
+ final var shardGeneration = new ShardGeneration(randomUUID());
+ final var blobsToDelete = randomList(
+ 100,
+ () -> randomFrom(METADATA_PREFIX, INDEX_FILE_PREFIX, SNAPSHOT_PREFIX) + randomUUID() + randomFrom(
+ "",
+ METADATA_BLOB_NAME_SUFFIX
+ )
+ );
+ expectedShardGenerations.put(indexId, shardId, shardGeneration);
- final var expectedBlobsToDelete = new HashSet();
- var countDownLatch = new CountDownLatch(1);
+ countDownLatch = new CountDownLatch(1);
try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
- final var indexPath = repo.basePath()
- .add("indices")
- .add(indexId.getId())
- .add(Integer.toString(shardId))
- .buildAsString();
- for (final var blobToDelete : blobIds) {
- expectedBlobsToDelete.add(indexPath + blobToDelete);
- }
repo.threadPool()
.generic()
.execute(
ActionRunnable.run(
refs.acquireListener(),
- () -> shardBlobsToDelete.addShardDeleteResult(indexId, shardId, shardGeneration, blobIds)
+ () -> shardBlobsToDelete.addShardDeleteResult(indexId, shardId, shardGeneration, blobsToDelete)
)
);
-
}
safeAwait(countDownLatch);
- // We should still update the shard generations
- assertEquals(expectedShardGenerations, shardBlobsToDelete.getUpdatedShardGenerations());
- // We expect an empty blobPath iterator to be returned
- shardBlobsToDelete.getBlobPaths().forEachRemaining(s -> assertTrue(expectedBlobsToDelete.remove(s)));
- assertEquals(blobIds.size(), expectedBlobsToDelete.size());
- assertEquals(0, shardBlobsToDelete.sizeInBytes());
- mockLog.assertAllExpectationsMatched();
}
- }
- }
- private List generateRandomBlobsToDelete(int minSize, int maxSize) {
- return randomList(
- minSize,
- maxSize,
- () -> randomFrom(METADATA_PREFIX, INDEX_FILE_PREFIX, SNAPSHOT_PREFIX) + randomUUID() + randomFrom("", METADATA_BLOB_NAME_SUFFIX)
- );
- }
+ // We expect no shard generations
+ assertEquals(expectedShardGenerations.build(), shardBlobsToDelete.getUpdatedShardGenerations());
- private void addFailureToCleanUpDanglingBlobsUnseenExpectation(MockLog mockLog) {
- mockLog.addExpectation(
- new MockLog.UnseenEventExpectation(
- "failure to clean up dangling blobs warn logs",
- "org.elasticsearch.repositories.blobstore.BlobStoreRepository",
- Level.WARN,
- "Failed to clean up"
- )
- );
- }
+ // We expect the blob paths to be only those that were written before we exceeded the threshold
+ List blobPaths = new ArrayList<>();
+ shardBlobsToDelete.getBlobPaths().forEachRemaining(blobPaths::add);
+ assertEquals(blobCount, blobPaths.size());
- private void addFailureToCleanUpDanglingBlobsSeenExpectation(MockLog mockLog) {
- mockLog.addExpectation(
- new MockLog.SeenEventExpectation(
- "failure to clean up dangling blobs warn logs",
- "org.elasticsearch.repositories.blobstore.BlobStoreRepository",
- Level.WARN,
- "Failed to clean up"
- )
- );
+ mockLog.assertAllExpectationsMatched();
+ resetMockLog();
+ }
}
}
From ed00f1a105e320b715550b3ecdaeb248cb8fac24 Mon Sep 17 00:00:00 2001
From: Joshua Adams
Date: Thu, 18 Sep 2025 12:11:59 +0100
Subject: [PATCH 20/25] Spotless apply
---
.../repositories/blobstore/BlobStoreRepository.java | 1 -
.../repositories/blobstore/BlobStoreRepositoryTests.java | 8 +-------
2 files changed, 1 insertion(+), 8 deletions(-)
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 17c34ce81e533..187fb026723fb 100644
--- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
+++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
@@ -147,7 +147,6 @@
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
import java.io.Closeable;
-import java.io.EOFException;
import java.io.FilterInputStream;
import java.io.IOException;
import java.io.InputStream;
diff --git a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
index 9ce5722982d00..636113637f948 100644
--- a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
+++ b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
@@ -42,7 +42,6 @@
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.blobstore.OperationPurpose;
import org.elasticsearch.common.bytes.BytesArray;
-import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
@@ -66,7 +65,6 @@
import org.elasticsearch.repositories.fs.FsRepository;
import org.elasticsearch.snapshots.AbstractSnapshotIntegTestCase;
import org.elasticsearch.snapshots.SnapshotId;
-import org.elasticsearch.snapshots.SnapshotShutdownProgressTracker;
import org.elasticsearch.snapshots.SnapshotState;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESSingleNodeTestCase;
@@ -820,11 +818,7 @@ public void testWriteToShardBlobToDelete() {
);
expectedShardGenerations.put(indexId, shardId, shardGeneration);
- final var indexPath = repo.basePath()
- .add("indices")
- .add(indexId.getId())
- .add(Integer.toString(shardId))
- .buildAsString();
+ final var indexPath = repo.basePath().add("indices").add(indexId.getId()).add(Integer.toString(shardId)).buildAsString();
countDownLatch = new CountDownLatch(1);
try (var refs = new RefCountingRunnable(countDownLatch::countDown)) {
From 37404a5c7a8bc332ef1912ec06871a7b46679afc Mon Sep 17 00:00:00 2001
From: elasticsearchmachine
Date: Thu, 2 Oct 2025 07:40:13 +0000
Subject: [PATCH 21/25] [CI] Update transport version definitions
---
server/src/main/resources/transport/upper_bounds/8.18.csv | 2 +-
server/src/main/resources/transport/upper_bounds/8.19.csv | 2 +-
server/src/main/resources/transport/upper_bounds/9.0.csv | 2 +-
server/src/main/resources/transport/upper_bounds/9.1.csv | 2 +-
server/src/main/resources/transport/upper_bounds/9.2.csv | 2 +-
server/src/main/resources/transport/upper_bounds/9.3.csv | 1 +
6 files changed, 6 insertions(+), 5 deletions(-)
create mode 100644 server/src/main/resources/transport/upper_bounds/9.3.csv
diff --git a/server/src/main/resources/transport/upper_bounds/8.18.csv b/server/src/main/resources/transport/upper_bounds/8.18.csv
index 4eb5140004ea6..266bfbbd3bf78 100644
--- a/server/src/main/resources/transport/upper_bounds/8.18.csv
+++ b/server/src/main/resources/transport/upper_bounds/8.18.csv
@@ -1 +1 @@
-initial_elasticsearch_8_18_6,8840008
+transform_check_for_dangling_tasks,8840011
diff --git a/server/src/main/resources/transport/upper_bounds/8.19.csv b/server/src/main/resources/transport/upper_bounds/8.19.csv
index 476468b203875..3600b3f8c633a 100644
--- a/server/src/main/resources/transport/upper_bounds/8.19.csv
+++ b/server/src/main/resources/transport/upper_bounds/8.19.csv
@@ -1 +1 @@
-initial_elasticsearch_8_19_3,8841067
+transform_check_for_dangling_tasks,8841070
diff --git a/server/src/main/resources/transport/upper_bounds/9.0.csv b/server/src/main/resources/transport/upper_bounds/9.0.csv
index f8f50cc6d7839..c11e6837bb813 100644
--- a/server/src/main/resources/transport/upper_bounds/9.0.csv
+++ b/server/src/main/resources/transport/upper_bounds/9.0.csv
@@ -1 +1 @@
-initial_elasticsearch_9_0_6,9000015
+transform_check_for_dangling_tasks,9000018
diff --git a/server/src/main/resources/transport/upper_bounds/9.1.csv b/server/src/main/resources/transport/upper_bounds/9.1.csv
index 5a65f2e578156..80b97d85f7511 100644
--- a/server/src/main/resources/transport/upper_bounds/9.1.csv
+++ b/server/src/main/resources/transport/upper_bounds/9.1.csv
@@ -1 +1 @@
-initial_elasticsearch_9_1_4,9112007
+transform_check_for_dangling_tasks,9112009
diff --git a/server/src/main/resources/transport/upper_bounds/9.2.csv b/server/src/main/resources/transport/upper_bounds/9.2.csv
index e24f914a1d1ca..2147eab66c207 100644
--- a/server/src/main/resources/transport/upper_bounds/9.2.csv
+++ b/server/src/main/resources/transport/upper_bounds/9.2.csv
@@ -1 +1 @@
-ml_inference_endpoint_cache,9157000
+initial_9.2.0,9185000
diff --git a/server/src/main/resources/transport/upper_bounds/9.3.csv b/server/src/main/resources/transport/upper_bounds/9.3.csv
new file mode 100644
index 0000000000000..2147eab66c207
--- /dev/null
+++ b/server/src/main/resources/transport/upper_bounds/9.3.csv
@@ -0,0 +1 @@
+initial_9.2.0,9185000
From d1e81f7abe476d4478f8954a4d457a572bc4e8e0 Mon Sep 17 00:00:00 2001
From: Joshua Adams
Date: Tue, 7 Oct 2025 15:31:36 +0100
Subject: [PATCH 22/25] David comments
---
.../io/stream/TruncatedOutputStream.java | 2 +-
.../common/settings/ClusterSettings.java | 4 +-
.../blobstore/BlobStoreRepository.java | 90 ++++++++++---------
3 files changed, 52 insertions(+), 44 deletions(-)
diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/TruncatedOutputStream.java b/server/src/main/java/org/elasticsearch/common/io/stream/TruncatedOutputStream.java
index 756dd04be8130..0e7fc704a2ef5 100644
--- a/server/src/main/java/org/elasticsearch/common/io/stream/TruncatedOutputStream.java
+++ b/server/src/main/java/org/elasticsearch/common/io/stream/TruncatedOutputStream.java
@@ -30,7 +30,7 @@ public TruncatedOutputStream(OutputStream out, IntSupplier currentSizeSupplier,
this.maxSize = maxSize;
}
- private boolean hasCapacity() {
+ public boolean hasCapacity() {
if (hasCapacity) {
hasCapacity = currentSizeSupplier.getAsInt() < maxSize;
}
diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java
index 9c2d6fab10368..42c5f7c528cb4 100644
--- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java
+++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java
@@ -119,6 +119,7 @@
import org.elasticsearch.persistent.decider.EnableAssignmentDecider;
import org.elasticsearch.plugins.PluginsService;
import org.elasticsearch.readiness.ReadinessService;
+import org.elasticsearch.repositories.blobstore.BlobStoreRepository;
import org.elasticsearch.repositories.fs.FsRepository;
import org.elasticsearch.rest.BaseRestHandler;
import org.elasticsearch.script.ScriptService;
@@ -648,6 +649,7 @@ public void apply(Settings value, Settings current, Settings previous) {
WriteLoadConstraintSettings.WRITE_LOAD_DECIDER_HIGH_UTILIZATION_THRESHOLD_SETTING,
WriteLoadConstraintSettings.WRITE_LOAD_DECIDER_HIGH_UTILIZATION_DURATION_SETTING,
WriteLoadConstraintSettings.WRITE_LOAD_DECIDER_QUEUE_LATENCY_THRESHOLD_SETTING,
- WriteLoadConstraintSettings.WRITE_LOAD_DECIDER_REROUTE_INTERVAL_SETTING
+ WriteLoadConstraintSettings.WRITE_LOAD_DECIDER_REROUTE_INTERVAL_SETTING,
+ BlobStoreRepository.MAX_HEAP_SIZE_FOR_SNAPSHOT_DELETION_SETTING
);
}
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 187fb026723fb..4c77dc249bfbb 100644
--- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
+++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
@@ -433,6 +433,14 @@ public static String getRepositoryDataBlobName(long repositoryGeneration) {
Setting.Property.NodeScope
);
+ // Defines the max size of the shard_delete_results stream as a percentage of available heap memory
+ public static final Setting MAX_HEAP_SIZE_FOR_SNAPSHOT_DELETION_SETTING = Setting.memorySizeSetting(
+ "repositories.blobstore.max_shard_delete_results_size",
+ "25%",
+ Setting.Property.Dynamic,
+ Setting.Property.NodeScope
+ );
+
/**
* Repository settings that can be updated dynamically without having to create a new repository.
*/
@@ -1671,7 +1679,6 @@ void writeTo(StreamOutput out) throws IOException {
}
}
- private final int shardDeleteResultsMaxSize;
/**
*
* Shard-level results, i.e. a sequence of {@link ShardSnapshotMetaDeleteResult} objects, except serialized, concatenated, and
@@ -1685,35 +1692,27 @@ void writeTo(StreamOutput out) throws IOException {
*
*/
private final BytesStreamOutput shardDeleteResults;
-
- private int resultCount = 0;
-
+ private final TruncatedOutputStream truncatedShardDeleteResultsOutputStream;
private final StreamOutput compressed;
+ private final int shardDeleteResultsMaxSize;
+ private int successfullyWrittenBlobsCount = 0;
+ private int leakedBlobsCount = 0;
private final ArrayList resources = new ArrayList<>();
-
private final ShardGenerations.Builder shardGenerationsBuilder = ShardGenerations.builder();
- // Gets 25% of the heap size to be allocated to the shard_delete_results stream
- public final Setting MAX_SHARD_DELETE_RESULTS_SIZE_SETTING = Setting.memorySizeSetting(
- "repositories.blobstore.max_shard_delete_results_size",
- "25%",
- Setting.Property.NodeScope
- );
-
ShardBlobsToDelete(Settings settings) {
this.shardDeleteResultsMaxSize = calculateMaximumShardDeleteResultsSize(settings);
this.shardDeleteResults = new ReleasableBytesStreamOutput(bigArrays);
- this.compressed = new OutputStreamStreamOutput(
- new TruncatedOutputStream(
- new BufferedOutputStream(
- new DeflaterOutputStream(Streams.flushOnCloseStream(shardDeleteResults)),
- DeflateCompressor.BUFFER_SIZE
- ),
- shardDeleteResults::size,
- this.shardDeleteResultsMaxSize
- )
+ this.truncatedShardDeleteResultsOutputStream = new TruncatedOutputStream(
+ new BufferedOutputStream(
+ new DeflaterOutputStream(Streams.flushOnCloseStream(shardDeleteResults)),
+ DeflateCompressor.BUFFER_SIZE
+ ),
+ shardDeleteResults::size,
+ this.shardDeleteResultsMaxSize
);
+ this.compressed = new OutputStreamStreamOutput(this.truncatedShardDeleteResultsOutputStream);
resources.add(compressed);
resources.add(LeakTracker.wrap((Releasable) shardDeleteResults));
}
@@ -1726,13 +1725,10 @@ void writeTo(StreamOutput out) throws IOException {
* @return The maximum number of bytes the shardDeleteResults BytesStreamOutput can consume in the heap
*/
int calculateMaximumShardDeleteResultsSize(Settings settings) {
- long maxHeapSizeInBytes = MAX_SHARD_DELETE_RESULTS_SIZE_SETTING.get(settings).getBytes();
+ long maxHeapSizeInBytes = MAX_HEAP_SIZE_FOR_SNAPSHOT_DELETION_SETTING.get(settings).getBytes();
int oneMBBuffer = 1024 * 1024;
int maxShardDeleteResultsSize = Integer.MAX_VALUE - oneMBBuffer;
- if (maxHeapSizeInBytes > maxShardDeleteResultsSize) {
- return maxShardDeleteResultsSize;
- }
- return (int) maxHeapSizeInBytes;
+ return Math.toIntExact(Math.min(maxHeapSizeInBytes, maxShardDeleteResultsSize));
}
synchronized void addShardDeleteResult(
@@ -1743,21 +1739,25 @@ synchronized void addShardDeleteResult(
) {
try {
shardGenerationsBuilder.put(indexId, shardId, newGeneration);
- // Only write if we have capacity
- if (shardDeleteResults.size() < this.shardDeleteResultsMaxSize) {
+ // There is a minimum of 1 byte available for writing
+ if (this.truncatedShardDeleteResultsOutputStream.hasCapacity()) {
new ShardSnapshotMetaDeleteResult(Objects.requireNonNull(indexId.getId()), shardId, blobsToDelete).writeTo(compressed);
// We only want to read this shard delete result if we were able to write the entire object.
// Otherwise, for partial writes, an EOFException will be thrown upon reading
- if (shardDeleteResults.size() < this.shardDeleteResultsMaxSize) {
- resultCount += 1;
+ if (this.truncatedShardDeleteResultsOutputStream.hasCapacity()) {
+ successfullyWrittenBlobsCount += 1;
+ } else {
+ leakedBlobsCount += 1;
}
} else {
- logger.warn(
- "Failure to clean up the following dangling blobs, {}, for index {} and shard {}",
+ logger.debug(
+ "Unable to clean up the following dangling blobs, {}, for index {} and shard {} " +
+ "due to insufficient heap space on the master node.",
blobsToDelete,
indexId,
shardId
);
+ leakedBlobsCount += 1;
}
} catch (IOException e) {
assert false : e; // no IO actually happens here
@@ -1785,20 +1785,26 @@ public Iterator getBlobPaths() {
throw new UncheckedIOException(e);
}
- List blobPaths = new ArrayList<>();
- for (int i = 0; i < resultCount; i++) {
- ShardSnapshotMetaDeleteResult shardResult;
+ if (leakedBlobsCount > 0) {
+ logger.warn(
+ "Skipped cleanup of {} dangling snapshot blobs due to memory constraints on the master node. " +
+ "These blobs will be cleaned up automatically by future snapshot deletions. " +
+ "If you routinely delete large snapshots, consider increasing the master node's heap size " +
+ "to allow for more efficient cleanup.",
+ leakedBlobsCount
+ );
+ }
+
+ return Iterators.flatMap(Iterators.forRange(0, successfullyWrittenBlobsCount, i -> {
try {
- shardResult = new ShardSnapshotMetaDeleteResult(input);
+ return new ShardSnapshotMetaDeleteResult(input);
} catch (IOException e) {
throw new UncheckedIOException(e);
}
- String shardPath = shardPath(new IndexId("_na_", shardResult.indexId), shardResult.shardId).buildAsString();
- for (String blob : shardResult.blobsToDelete) {
- blobPaths.add(shardPath + blob);
- }
- }
- return blobPaths.iterator();
+ }), shardResult -> {
+ final var shardPath = shardPath(new IndexId("_na_", shardResult.indexId), shardResult.shardId).buildAsString();
+ return Iterators.map(shardResult.blobsToDelete.iterator(), blob -> shardPath + blob);
+ });
}
@Override
From 2f0ea30ed0306234ab113358f0bdcfeeb1adc636 Mon Sep 17 00:00:00 2001
From: Joshua Adams
Date: Tue, 7 Oct 2025 15:58:30 +0100
Subject: [PATCH 23/25] Fix test
---
.../blobstore/BlobStoreRepositoryTests.java | 54 ++++++++-----------
1 file changed, 22 insertions(+), 32 deletions(-)
diff --git a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
index 636113637f948..9be059d8d4770 100644
--- a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
+++ b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
@@ -10,7 +10,6 @@
package org.elasticsearch.repositories.blobstore;
import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.core.LogEvent;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRunnable;
import org.elasticsearch.action.admin.cluster.repositories.delete.DeleteRepositoryRequest;
@@ -791,15 +790,17 @@ public void testWriteToShardBlobToDelete() {
CountDownLatch countDownLatch;
int blobCount = 0;
- // First, we write as many blobs as we have capacity for
+ // First, write blobs until capacity is exceeded
- // Do not expect to see any WARN logs about dangling blobs as there should be capacity for them all
mockLog.addExpectation(
- new MockLog.UnseenEventExpectation(
- "failure to clean up dangling blobs warn logs",
- "org.elasticsearch.repositories.blobstore.BlobStoreRepository",
+ new MockLog.SeenEventExpectation(
+ "skipped cleanup warning",
+ BlobStoreRepository.class.getCanonicalName(),
Level.WARN,
- "Failure to clean up the following dangling blobs"
+ "*Skipped cleanup of 1 dangling snapshot blobs due to memory constraints "
+ + "on the master node. These blobs will be cleaned up automatically by future snapshot deletions. "
+ + "If you routinely delete large snapshots, consider increasing the master node's heap size to allow "
+ + "for more efficient cleanup."
)
);
@@ -851,32 +852,21 @@ public void testWriteToShardBlobToDelete() {
resetMockLog();
// Second, now we're at capacity, we test whether we can accept subsequent writes without throwing an error
- int numberOfOverflowedWrites = randomIntBetween(1, 20);
-
- // Expect each write to throw a WARN log
- mockLog.addExpectation(new MockLog.LoggingExpectation() {
- int count = 0;
-
- @Override
- public void match(LogEvent event) {
- if (event.getLevel() != Level.WARN) {
- return;
- }
- if (event.getLoggerName().equals(BlobStoreRepository.class.getCanonicalName()) == false) {
- return;
- }
-
- String msg = event.getMessage().getFormattedMessage();
- if (msg.matches("Failure to clean up the following dangling blobs, .*, for index .+ and shard .+")) {
- count++;
- }
- }
- @Override
- public void assertMatched() {
- assertEquals(numberOfOverflowedWrites, count);
- }
- });
+ int numberOfOverflowedWrites = randomIntBetween(1, 20);
+ mockLog.addExpectation(
+ new MockLog.SeenEventExpectation(
+ "skipped cleanup warning",
+ BlobStoreRepository.class.getCanonicalName(),
+ Level.WARN,
+ "*Skipped cleanup of "
+ + (numberOfOverflowedWrites + 1)
+ + " dangling snapshot blobs due to memory constraints "
+ + "on the master node. These blobs will be cleaned up automatically by future snapshot deletions. "
+ + "If you routinely delete large snapshots, consider increasing the master node's heap size to allow "
+ + "for more efficient cleanup."
+ )
+ );
for (int i = 0; i < numberOfOverflowedWrites; i++) {
// Generate the next blob to write
From 0d01264b6deead8e99e55e966a66ad4827058c60 Mon Sep 17 00:00:00 2001
From: Joshua Adams
Date: Tue, 7 Oct 2025 15:59:30 +0100
Subject: [PATCH 24/25] spotless
---
.../repositories/blobstore/BlobStoreRepository.java | 12 ++++++------
1 file changed, 6 insertions(+), 6 deletions(-)
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 4c77dc249bfbb..e036a4af12818 100644
--- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
+++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
@@ -1751,8 +1751,8 @@ synchronized void addShardDeleteResult(
}
} else {
logger.debug(
- "Unable to clean up the following dangling blobs, {}, for index {} and shard {} " +
- "due to insufficient heap space on the master node.",
+ "Unable to clean up the following dangling blobs, {}, for index {} and shard {} "
+ + "due to insufficient heap space on the master node.",
blobsToDelete,
indexId,
shardId
@@ -1787,10 +1787,10 @@ public Iterator getBlobPaths() {
if (leakedBlobsCount > 0) {
logger.warn(
- "Skipped cleanup of {} dangling snapshot blobs due to memory constraints on the master node. " +
- "These blobs will be cleaned up automatically by future snapshot deletions. " +
- "If you routinely delete large snapshots, consider increasing the master node's heap size " +
- "to allow for more efficient cleanup.",
+ "Skipped cleanup of {} dangling snapshot blobs due to memory constraints on the master node. "
+ + "These blobs will be cleaned up automatically by future snapshot deletions. "
+ + "If you routinely delete large snapshots, consider increasing the master node's heap size "
+ + "to allow for more efficient cleanup.",
leakedBlobsCount
);
}
From d73ffef9f99e1f399667d2fe66bac16ef54965c6 Mon Sep 17 00:00:00 2001
From: Joshua Adams
Date: Tue, 7 Oct 2025 16:09:29 +0100
Subject: [PATCH 25/25] Modify comment
---
.../repositories/blobstore/BlobStoreRepositoryTests.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
index 9be059d8d4770..0299bcd80f99b 100644
--- a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
+++ b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
@@ -775,7 +775,7 @@ public void tearDown() throws Exception {
* Initially, if there is capacity, we write N blobs to ShardBlobToDelete. We expect each of them to be compressed
* and written to the underlying stream.
* Once capacity is reached, we write M subsequent blobs, but expect that they will not be written to the
- * underlying stream, and a WARN log should be thrown for each.
+ * underlying stream.
* When we read from the stream, we expect only the successful writes to be returned
*/
@TestLogging(reason = "test includes assertions about logging", value = "org.elasticsearch.repositories.blobstore:WARN")