Replace Math.toIntExact with toIntBytes (#61604)

We convert longs to ints using `Math.toIntExact` in places where we're
sure there will be no overflow, but this doesn't explain the intent of
these conversions very well. This commit introduces a dedicated method
for these conversions, and adds an assertion that we never overflow.
This commit is contained in:
David Turner 2020-08-27 08:15:12 +01:00
parent e14d9c9514
commit 5df74cc888
14 changed files with 99 additions and 51 deletions

View File

@ -276,6 +276,22 @@ public enum ByteSizeUnit implements Writeable {
return d * m;
}
/**
* Convert to an {@code int} number of bytes. Callers are expected to be certain this will not overflow.
* @throws IllegalArgumentException on overflow, unless assertions are enabled in which case it throws an {@link AssertionError}.
* @return The number of bytes represented as an {@code int}.
*/
public final int toIntBytes(long size) {
final long l = toBytes(size);
final int i = (int) l;
if (i != l) {
final String message = "could not convert [" + size + " " + this + "] to an int";
assert false : message;
throw new IllegalArgumentException(message);
}
return i;
}
public abstract long toBytes(long size);
public abstract long toKB(long size);

View File

@ -52,6 +52,13 @@ public class ByteSizeValueTests extends AbstractWireSerializingTestCase<ByteSize
assertThat(ByteSizeUnit.PB.toPB(10), is(new ByteSizeValue(10, ByteSizeUnit.PB).getPb()));
}
public void testToIntBytes() {
assertThat(ByteSizeUnit.BYTES.toIntBytes(4), equalTo(4));
assertThat(ByteSizeUnit.KB.toIntBytes(4), equalTo(4096));
assertThat(expectThrows(AssertionError.class, () -> ByteSizeUnit.GB.toIntBytes(4)).getMessage(),
containsString("could not convert [4 GB] to an int"));
}
public void testEquality() {
String[] equalValues = new String[]{"1GB", "1024MB", "1048576KB", "1073741824B"};
ByteSizeValue value1 = ByteSizeValue.parseBytesSizeValue(randomFrom(equalValues), "equalTest");

View File

@ -7,6 +7,7 @@ package org.elasticsearch.xpack.searchablesnapshots;
import org.elasticsearch.Build;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import static org.elasticsearch.index.IndexModule.INDEX_STORE_TYPE_SETTING;
@ -44,4 +45,15 @@ public class SearchableSnapshotsConstants {
public static final String CACHE_PREWARMING_THREAD_POOL_SETTING = "xpack.searchable_snapshots.cache_prewarming_thread_pool";
public static final String SNAPSHOT_BLOB_CACHE_INDEX = ".snapshot-blob-cache";
/**
* We use {@code long} to represent offsets and lengths of files since they may be larger than 2GB, but {@code int} to represent
* offsets and lengths of arrays in memory which are limited to 2GB in size. We quite often need to convert from the file-based world
* of {@code long}s into the memory-based world of {@code int}s, knowing for certain that the result will not overflow. This method
* should be used to clarify that we're doing this.
*/
public static int toIntBytes(long l) {
return ByteSizeUnit.BYTES.toIntBytes(l);
}
}

View File

@ -51,7 +51,7 @@ public class BlobStoreCacheService extends AbstractLifecycleComponent implements
private static final Logger logger = LogManager.getLogger(BlobStoreCacheService.class);
public static final int DEFAULT_CACHED_BLOB_SIZE = Math.toIntExact(ByteSizeUnit.KB.toBytes(4L));
public static final int DEFAULT_CACHED_BLOB_SIZE = ByteSizeUnit.KB.toIntBytes(4);
private final ClusterService clusterService;
private final ThreadPool threadPool;

View File

@ -24,6 +24,7 @@ import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.Channels;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo;
import org.elasticsearch.index.snapshots.blobstore.SlicedInputStream;
import org.elasticsearch.index.store.BaseSearchableSnapshotIndexInput;
@ -45,6 +46,7 @@ import java.util.function.Predicate;
import java.util.stream.IntStream;
import static org.elasticsearch.index.store.checksum.ChecksumBlobContainerIndexInput.checksumToBytesArray;
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.toIntBytes;
public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexInput {
@ -56,7 +58,7 @@ public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexIn
public static final IOContext CACHE_WARMING_CONTEXT = new IOContext();
private static final Logger logger = LogManager.getLogger(CachedBlobContainerIndexInput.class);
private static final int COPY_BUFFER_SIZE = 8192;
private static final int COPY_BUFFER_SIZE = ByteSizeUnit.KB.toIntBytes(8);
private final SearchableSnapshotDirectory directory;
private final CacheFileReference cacheFileReference;
@ -219,7 +221,7 @@ public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexIn
);
stats.addIndexCacheBytesRead(cachedBlob.length());
final BytesRefIterator cachedBytesIterator = cachedBlob.bytes().slice(Math.toIntExact(position), length).iterator();
final BytesRefIterator cachedBytesIterator = cachedBlob.bytes().slice(toIntBytes(position), length).iterator();
BytesRef bytesRef;
while ((bytesRef = cachedBytesIterator.next()) != null) {
b.put(bytesRef.bytes, bytesRef.offset, bytesRef.length);
@ -235,7 +237,7 @@ public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexIn
(channel, from, to, progressUpdater) -> {
final long startTimeNanos = stats.currentTimeNanos();
final BytesRefIterator iterator = cachedBlob.bytes()
.slice(Math.toIntExact(from - cachedBlob.from()), Math.toIntExact(to - from))
.slice(toIntBytes(from - cachedBlob.from()), toIntBytes(to - from))
.iterator();
long writePosition = from;
BytesRef current;
@ -298,7 +300,7 @@ public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexIn
final int read;
if ((rangeToRead.v2() - rangeToRead.v1()) < b.remaining()) {
final ByteBuffer duplicate = b.duplicate();
duplicate.limit(duplicate.position() + Math.toIntExact(rangeToRead.v2() - rangeToRead.v1()));
duplicate.limit(duplicate.position() + toIntBytes(rangeToRead.v2() - rangeToRead.v1()));
read = readCacheFile(channel, position, duplicate);
assert duplicate.position() <= b.limit();
b.position(duplicate.position());
@ -311,7 +313,7 @@ public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexIn
if (indexCacheMiss != null) {
final Releasable onCacheFillComplete = stats.addIndexCacheFill();
final CompletableFuture<Integer> readFuture = cacheFile.readIfAvailableOrPending(indexCacheMiss, channel -> {
final int indexCacheMissLength = Math.toIntExact(indexCacheMiss.v2() - indexCacheMiss.v1());
final int indexCacheMissLength = toIntBytes(indexCacheMiss.v2() - indexCacheMiss.v1());
// We assume that we only cache small portions of blobs so that we do not need to:
// - use a BigArrays for allocation
@ -373,7 +375,7 @@ public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexIn
try {
// cache file was evicted during the range fetching, read bytes directly from blob container
final long length = b.remaining();
final byte[] copyBuffer = new byte[Math.toIntExact(Math.min(COPY_BUFFER_SIZE, length))];
final byte[] copyBuffer = new byte[toIntBytes(Math.min(COPY_BUFFER_SIZE, length))];
logger.trace(
() -> new ParameterizedMessage(
"direct reading of range [{}-{}] for cache file [{}]",
@ -481,7 +483,7 @@ public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexIn
final FileChannel fc = cacheFile.getChannel();
assert assertFileChannelOpen(fc);
final byte[] copyBuffer = new byte[Math.toIntExact(Math.min(COPY_BUFFER_SIZE, rangeLength))];
final byte[] copyBuffer = new byte[toIntBytes(Math.min(COPY_BUFFER_SIZE, rangeLength))];
long totalBytesRead = 0L;
final AtomicLong totalBytesWritten = new AtomicLong();
@ -507,8 +509,8 @@ public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexIn
(channel, start, end, progressUpdater) -> {
final ByteBuffer byteBuffer = ByteBuffer.wrap(
copyBuffer,
Math.toIntExact(start - readStart),
Math.toIntExact(end - start)
toIntBytes(start - readStart),
toIntBytes(end - start)
);
final int writtenBytes = positionalWrite(channel, start, byteBuffer);
logger.trace(
@ -557,7 +559,7 @@ public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexIn
long remaining,
CacheFileReference cacheFileReference
) throws IOException {
final int len = (remaining < copyBuffer.length) ? Math.toIntExact(remaining) : copyBuffer.length;
final int len = (remaining < copyBuffer.length) ? toIntBytes(remaining) : copyBuffer.length;
final int bytesRead = inputStream.read(copyBuffer, 0, len);
if (bytesRead == -1) {
throw new EOFException(
@ -616,7 +618,7 @@ public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexIn
assert assertFileChannelOpen(fc);
assert assertCurrentThreadMayWriteCacheFile();
final long length = end - start;
final byte[] copyBuffer = new byte[Math.toIntExact(Math.min(COPY_BUFFER_SIZE, length))];
final byte[] copyBuffer = new byte[toIntBytes(Math.min(COPY_BUFFER_SIZE, length))];
logger.trace(() -> new ParameterizedMessage("writing range [{}-{}] to cache file [{}]", start, end, cacheFileReference));
long bytesCopied = 0L;
@ -704,7 +706,7 @@ public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexIn
}
private long getLengthOfPart(long part) {
return fileInfo.partBytes(Math.toIntExact(part));
return fileInfo.partBytes(toIntBytes(part));
}
private void ensureValidPosition(long position) {

View File

@ -17,6 +17,8 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.Objects;
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.toIntBytes;
/**
* A {@link IndexInput} that can only be used to verify footer checksums.
*/
@ -108,7 +110,7 @@ public class ChecksumBlobContainerIndexInput extends IndexInput {
assert false : "unexpected read or seek at position [" + pos + "] but checksum starts at [" + checksumPosition + ']';
throw new IllegalArgumentException("Can't read or seek before footer checksum");
}
return Math.toIntExact(checksum.length - (length - pos));
return toIntBytes(checksum.length - (length - pos));
}
private static void ensureReadOnceChecksumContext(IOContext context) {

View File

@ -25,6 +25,8 @@ import java.nio.ByteBuffer;
import java.util.Objects;
import java.util.concurrent.atomic.LongAdder;
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.toIntBytes;
/**
* A {@link DirectBlobContainerIndexInput} instance corresponds to a single file from a Lucene directory that has been snapshotted. Because
* large Lucene file might be split into multiple parts during the snapshot, {@link DirectBlobContainerIndexInput} requires a
@ -110,9 +112,9 @@ public class DirectBlobContainerIndexInput extends BaseSearchableSnapshotIndexIn
int currentPart = Math.toIntExact(position / fileInfo.partSize().getBytes());
int remainingBytesInPart;
if (currentPart < (fileInfo.numberOfParts() - 1)) {
remainingBytesInPart = Math.toIntExact(((currentPart + 1L) * fileInfo.partSize().getBytes()) - position);
remainingBytesInPart = toIntBytes(((currentPart + 1L) * fileInfo.partSize().getBytes()) - position);
} else {
remainingBytesInPart = Math.toIntExact(fileInfo.length() - position);
remainingBytesInPart = toIntBytes(fileInfo.length() - position);
}
final int read = Math.min(b.remaining(), remainingBytesInPart);
readInternalBytes(currentPart, position % fileInfo.partSize().getBytes(), b, read);

View File

@ -24,6 +24,8 @@ import java.nio.file.Path;
import java.util.Objects;
import java.util.function.Predicate;
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.toIntBytes;
/**
* {@link CacheService} maintains a cache entry for all files read from searchable snapshot directories (
* see {@link org.elasticsearch.index.store.SearchableSnapshotDirectory})
@ -107,7 +109,7 @@ public class CacheService extends AbstractLifecycleComponent {
* @return the cache range size (in bytes)
*/
public int getRangeSize() {
return Math.toIntExact(rangeSize.getBytes());
return toIntBytes(rangeSize.getBytes());
}
public CacheFile get(final CacheKey cacheKey, final long fileLength, final Path cacheDir) throws Exception {

View File

@ -11,10 +11,11 @@ import java.util.function.LongSupplier;
import static org.elasticsearch.index.store.IndexInputStats.SEEKING_THRESHOLD;
import static org.elasticsearch.index.store.cache.TestUtils.assertCounter;
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.toIntBytes;
public class IndexInputStatsTests extends ESTestCase {
private static LongSupplier FAKE_CLOCK = () -> {
private static final LongSupplier FAKE_CLOCK = () -> {
assert false : "should not be called";
return -1L;
};
@ -32,7 +33,7 @@ public class IndexInputStatsTests extends ESTestCase {
for (int i = 0; i < randomIntBetween(1, 50); i++) {
final long currentPosition = randomLongBetween(0L, inputStats.getFileLength() - 1L);
final long previousPosition = randomBoolean() ? currentPosition : randomLongBetween(0L, inputStats.getFileLength() - 1L);
final int bytesRead = randomIntBetween(1, Math.toIntExact(Math.max(1L, inputStats.getFileLength() - currentPosition)));
final int bytesRead = randomIntBetween(1, toIntBytes(Math.max(1L, inputStats.getFileLength() - currentPosition)));
inputStats.incrementBytesRead(previousPosition, currentPosition, bytesRead);

View File

@ -28,7 +28,6 @@ import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.F
import org.elasticsearch.index.store.cache.TestUtils;
import org.elasticsearch.index.store.cache.TestUtils.NoopBlobStoreCacheService;
import org.elasticsearch.indices.recovery.RecoveryState;
import org.elasticsearch.indices.recovery.RecoveryState;
import org.elasticsearch.indices.recovery.SearchableSnapshotRecoveryState;
import org.elasticsearch.repositories.IndexId;
import org.elasticsearch.snapshots.SnapshotId;
@ -52,6 +51,7 @@ import static org.elasticsearch.index.store.cache.TestUtils.singleBlobContainer;
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_CACHE_ENABLED_SETTING;
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_CACHE_PREWARM_ENABLED_SETTING;
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_UNCACHED_CHUNK_SIZE_SETTING;
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.toIntBytes;
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
@ -119,7 +119,7 @@ public class SearchableSnapshotDirectoryStatsTests extends ESIndexInputTestCase
final IndexInputStats inputStats = directory.getStats(fileName);
assertThat(inputStats, notNullValue());
final byte[] result = randomReadAndSlice(input, Math.toIntExact(length));
final byte[] result = randomReadAndSlice(input, toIntBytes(length));
assertArrayEquals(fileContent, result);
final long cachedBytesWriteCount = TestUtils.numberOfRanges(length, rangeSize.getBytes());
@ -171,7 +171,7 @@ public class SearchableSnapshotDirectoryStatsTests extends ESIndexInputTestCase
final IndexInputStats inputStats = directory.getStats(fileName);
assertThat(inputStats, notNullValue());
final byte[] result = randomReadAndSlice(input, Math.toIntExact(length));
final byte[] result = randomReadAndSlice(input, toIntBytes(length));
assertArrayEquals(fileContent, result);
assertThat(inputStats.getCachedBytesWritten(), notNullValue());
@ -213,7 +213,7 @@ public class SearchableSnapshotDirectoryStatsTests extends ESIndexInputTestCase
// read all index input sequentially as it simplifies testing
final byte[] readBuffer = new byte[512];
for (long i = 0L; i < input.length();) {
int size = between(1, Math.toIntExact(Math.min(readBuffer.length, input.length() - input.getFilePointer())));
int size = between(1, toIntBytes(Math.min(readBuffer.length, input.length() - input.getFilePointer())));
input.readBytes(readBuffer, 0, size);
i += size;
@ -331,7 +331,7 @@ public class SearchableSnapshotDirectoryStatsTests extends ESIndexInputTestCase
// read the input input sequentially
for (long bytesRead = 0L; bytesRead < input.length();) {
int size = between(1, Math.toIntExact(Math.min(readBuffer.length, input.length() - bytesRead)));
int size = between(1, toIntBytes(Math.min(readBuffer.length, input.length() - bytesRead)));
input.readBytes(readBuffer, 0, size);
bytesRead += size;
@ -381,7 +381,7 @@ public class SearchableSnapshotDirectoryStatsTests extends ESIndexInputTestCase
input.seek(randomPosition);
final byte[] readBuffer = new byte[512];
int size = between(1, Math.toIntExact(Math.min(readBuffer.length, input.length() - randomPosition)));
int size = between(1, toIntBytes(Math.min(readBuffer.length, input.length() - randomPosition)));
input.readBytes(readBuffer, 0, size);
// BufferedIndexInput tries to read as much bytes as possible

View File

@ -128,6 +128,7 @@ import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SN
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_REPOSITORY_SETTING;
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_SNAPSHOT_ID_SETTING;
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_SNAPSHOT_NAME_SETTING;
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.toIntBytes;
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
@ -318,7 +319,7 @@ public class SearchableSnapshotDirectoryTests extends ESTestCase {
IndexInput::getFilePointer
);
int available = Math.toIntExact(indexInput.length() - indexInput.getFilePointer());
int available = toIntBytes(indexInput.length() - indexInput.getFilePointer());
if (available == 0) {
expectThrows(EOFException.class, () -> snapshotIndexInput.readBytes(snapshotBuffer, 0, snapshotBuffer.length));
return;
@ -721,7 +722,7 @@ public class SearchableSnapshotDirectoryTests extends ESTestCase {
final byte[] buffer = new byte[1024];
for (int i = 0; i < randomIntBetween(10, 50); i++) {
final BlobStoreIndexShardSnapshot.FileInfo fileInfo = randomFrom(randomFiles);
final int fileLength = Math.toIntExact(fileInfo.length());
final int fileLength = toIntBytes(fileInfo.length());
try (IndexInput input = directory.openInput(fileInfo.physicalName(), newIOContext(random()))) {
assertThat(input.length(), equalTo((long) fileLength));
@ -730,7 +731,7 @@ public class SearchableSnapshotDirectoryTests extends ESTestCase {
input.seek(start);
while (input.getFilePointer() < end) {
input.readBytes(buffer, 0, Math.toIntExact(Math.min(buffer.length, end - input.getFilePointer())));
input.readBytes(buffer, 0, toIntBytes(Math.min(buffer.length, end - input.getFilePointer())));
}
}
assertListOfFiles(cacheDir, allOf(greaterThan(0), lessThanOrEqualTo(nbRandomFiles)), greaterThan(0L));

View File

@ -21,6 +21,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Consumer;
import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentSet;
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.toIntBytes;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
@ -137,7 +138,7 @@ public class SparseFileTrackerTests extends ESTestCase {
final long end = randomLongBetween(start, fileContents.length);
boolean pending = false;
for (long i = start; i < end; i++) {
if (fileContents[Math.toIntExact(i)] == UNAVAILABLE) {
if (fileContents[toIntBytes(i)] == UNAVAILABLE) {
pending = true;
break;
}
@ -158,8 +159,8 @@ public class SparseFileTrackerTests extends ESTestCase {
// listener is notified when the last gap is completed
final AtomicBoolean shouldNotifyListener = new AtomicBoolean();
for (long i = gap.start(); i < gap.end(); i++) {
assertThat(fileContents[Math.toIntExact(i)], equalTo(UNAVAILABLE));
fileContents[Math.toIntExact(i)] = AVAILABLE;
assertThat(fileContents[toIntBytes(i)], equalTo(UNAVAILABLE));
fileContents[toIntBytes(i)] = AVAILABLE;
// listener is notified when the progress reached the last byte of the last gap
if ((gapIndex == gaps.size() - 1) && (i == gap.end() - 1L)) {
assertTrue(shouldNotifyListener.compareAndSet(false, true));
@ -213,7 +214,7 @@ public class SparseFileTrackerTests extends ESTestCase {
boolean pending = false;
for (long i = subRange.v1(); i < subRange.v2(); i++) {
if (fileContents[Math.toIntExact(i)] == UNAVAILABLE) {
if (fileContents[toIntBytes(i)] == UNAVAILABLE) {
pending = true;
}
}
@ -240,8 +241,8 @@ public class SparseFileTrackerTests extends ESTestCase {
assertThat(gap.end(), lessThanOrEqualTo(range.v2()));
for (long i = gap.start(); i < gap.end(); i++) {
assertThat(fileContents[Math.toIntExact(i)], equalTo(UNAVAILABLE));
fileContents[Math.toIntExact(i)] = AVAILABLE;
assertThat(fileContents[toIntBytes(i)], equalTo(UNAVAILABLE));
fileContents[toIntBytes(i)] = AVAILABLE;
gap.onProgress(i + 1L);
}
gap.onCompletion();
@ -269,7 +270,7 @@ public class SparseFileTrackerTests extends ESTestCase {
long triggeringProgress = -1L;
for (long i = subRange.v1(); i < subRange.v2(); i++) {
if (fileContents[Math.toIntExact(i)] == UNAVAILABLE) {
if (fileContents[toIntBytes(i)] == UNAVAILABLE) {
triggeringProgress = i;
}
}
@ -280,8 +281,8 @@ public class SparseFileTrackerTests extends ESTestCase {
assertThat(gap.end(), lessThanOrEqualTo(range.v2()));
for (long i = gap.start(); i < gap.end(); i++) {
assertThat(fileContents[Math.toIntExact(i)], equalTo(UNAVAILABLE));
fileContents[Math.toIntExact(i)] = AVAILABLE;
assertThat(fileContents[toIntBytes(i)], equalTo(UNAVAILABLE));
fileContents[toIntBytes(i)] = AVAILABLE;
if (triggeringProgress == i) {
assertFalse(expectNotification.getAndSet(true));
}
@ -414,22 +415,22 @@ public class SparseFileTrackerTests extends ESTestCase {
final Tuple<Long, Long> freeRange = sparseFileTracker.getAbsentRangeWithin(checkStart, checkEnd);
if (freeRange == null) {
for (long i = checkStart; i < checkEnd; i++) {
assertThat(fileContents[Math.toIntExact(i)], equalTo(AVAILABLE));
assertThat(fileContents[toIntBytes(i)], equalTo(AVAILABLE));
}
} else {
assertThat(freeRange.v1(), greaterThanOrEqualTo(checkStart));
assertTrue(freeRange.toString(), freeRange.v1() < freeRange.v2());
assertThat(freeRange.v2(), lessThanOrEqualTo(checkEnd));
for (long i = checkStart; i < freeRange.v1(); i++) {
assertThat(fileContents[Math.toIntExact(i)], equalTo(AVAILABLE));
assertThat(fileContents[toIntBytes(i)], equalTo(AVAILABLE));
}
for (long i = freeRange.v2(); i < checkEnd; i++) {
assertThat(fileContents[Math.toIntExact(i)], equalTo(AVAILABLE));
assertThat(fileContents[toIntBytes(i)], equalTo(AVAILABLE));
}
if (expectExact) {
// without concurrent activity, the returned range is as small as possible
assertThat(fileContents[Math.toIntExact(freeRange.v1())], equalTo(UNAVAILABLE));
assertThat(fileContents[Math.toIntExact(freeRange.v2() - 1)], equalTo(UNAVAILABLE));
assertThat(fileContents[toIntBytes(freeRange.v1())], equalTo(UNAVAILABLE));
assertThat(fileContents[toIntBytes(freeRange.v2() - 1)], equalTo(UNAVAILABLE));
}
}
}
@ -454,7 +455,7 @@ public class SparseFileTrackerTests extends ESTestCase {
@Override
public void onResponse(Void aVoid) {
for (long i = subRangeStart; i < subRangeEnd; i++) {
assertThat(fileContents[Math.toIntExact(i)], equalTo(AVAILABLE));
assertThat(fileContents[toIntBytes(i)], equalTo(AVAILABLE));
}
assertTrue(listenerCalled.compareAndSet(false, true));
}
@ -474,7 +475,7 @@ public class SparseFileTrackerTests extends ESTestCase {
for (final SparseFileTracker.Gap gap : gaps) {
for (long i = gap.start(); i < gap.end(); i++) {
assertThat(Long.toString(i), fileContents[Math.toIntExact(i)], equalTo(UNAVAILABLE));
assertThat(Long.toString(i), fileContents[toIntBytes(i)], equalTo(UNAVAILABLE));
}
gapConsumer.accept(gap);
}
@ -488,14 +489,14 @@ public class SparseFileTrackerTests extends ESTestCase {
private static void processGap(byte[] fileContents, SparseFileTracker.Gap gap) {
for (long i = gap.start(); i < gap.end(); i++) {
assertThat(fileContents[Math.toIntExact(i)], equalTo(UNAVAILABLE));
assertThat(fileContents[toIntBytes(i)], equalTo(UNAVAILABLE));
}
if (randomBoolean()) {
gap.onFailure(new ElasticsearchException("simulated"));
} else {
for (long i = gap.start(); i < gap.end(); i++) {
fileContents[Math.toIntExact(i)] = AVAILABLE;
fileContents[toIntBytes(i)] = AVAILABLE;
gap.onProgress(i + 1L);
}
gap.onCompletion();

View File

@ -32,6 +32,7 @@ import java.util.Random;
import static com.carrotsearch.randomizedtesting.generators.RandomNumbers.randomIntBetween;
import static com.carrotsearch.randomizedtesting.generators.RandomPicks.randomFrom;
import static java.util.Arrays.asList;
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.toIntBytes;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;
import static org.mockito.Mockito.mock;
@ -62,7 +63,7 @@ public final class TestUtils {
}
public static long numberOfRanges(long fileSize, long rangeSize) {
return numberOfRanges(Math.toIntExact(fileSize), Math.toIntExact(rangeSize));
return numberOfRanges(toIntBytes(fileSize), toIntBytes(rangeSize));
}
static long numberOfRanges(int fileSize, int rangeSize) {
@ -107,7 +108,7 @@ public final class TestUtils {
throw new FileNotFoundException("Blob not found: " + name);
}
return Streams.limitStream(
new ByteArrayInputStream(blobContent, Math.toIntExact(position), blobContent.length - Math.toIntExact(position)),
new ByteArrayInputStream(blobContent, toIntBytes(position), blobContent.length - toIntBytes(position)),
length
);
}
@ -133,7 +134,7 @@ public final class TestUtils {
+ partSize
+ "]";
final int partNumber = Integer.parseInt(name.substring(prefix.length()));
final int positionInBlob = Math.toIntExact(position) + partSize * partNumber;
final int positionInBlob = toIntBytes(position) + partSize * partNumber;
assert positionInBlob + length <= blobContent.length : "cannot read ["
+ positionInBlob
+ "-"

View File

@ -24,6 +24,7 @@ import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.util.concurrent.atomic.AtomicInteger;
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.toIntBytes;
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
@ -80,7 +81,7 @@ public class DirectBlobContainerIndexInputTests extends ESIndexInputTestCase {
final InputStream stream;
if (fileInfo.numberOfParts() == 1L) {
assertThat("Unexpected blob name [" + name + "]", name, equalTo(fileInfo.name()));
stream = new ByteArrayInputStream(input, Math.toIntExact(position), Math.toIntExact(length));
stream = new ByteArrayInputStream(input, toIntBytes(position), toIntBytes(length));
} else {
assertThat("Unexpected blob name [" + name + "]", name, allOf(startsWith(fileInfo.name()), containsString(".part")));
@ -92,7 +93,7 @@ public class DirectBlobContainerIndexInputTests extends ESIndexInputTestCase {
allOf(greaterThanOrEqualTo(0L), lessThan(fileInfo.numberOfParts()))
);
stream = new ByteArrayInputStream(input, Math.toIntExact(partNumber * partSize + position), Math.toIntExact(length));
stream = new ByteArrayInputStream(input, toIntBytes(partNumber * partSize + position), toIntBytes(length));
}
if (randomBoolean()) {