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:
parent
e14d9c9514
commit
5df74cc888
|
@ -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);
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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
|
||||
+ "-"
|
||||
|
|
|
@ -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()) {
|
||||
|
|
Loading…
Reference in New Issue