-
Notifications
You must be signed in to change notification settings - Fork 25k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Adjust the length of blob cache docs for Lucene metadata files #69431
Changes from 1 commit
b035670
276838a
1d9cda0
40f7557
03fc26a
0cc9e9d
6e70ff3
5d28df8
b5f2463
a76a065
40fca31
73a055a
986f392
424dc2c
68290e4
8942b93
36e1758
701a3a7
208fc4a
0894cd2
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Large diffs are not rendered by default.
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -10,6 +10,7 @@ | |
import org.apache.logging.log4j.LogManager; | ||
import org.apache.logging.log4j.Logger; | ||
import org.apache.logging.log4j.message.ParameterizedMessage; | ||
import org.apache.lucene.index.IndexFileNames; | ||
import org.elasticsearch.ElasticsearchTimeoutException; | ||
import org.elasticsearch.ExceptionsHelper; | ||
import org.elasticsearch.Version; | ||
|
@@ -25,13 +26,16 @@ | |
import org.elasticsearch.cluster.block.ClusterBlockException; | ||
import org.elasticsearch.common.bytes.BytesReference; | ||
import org.elasticsearch.common.unit.ByteSizeUnit; | ||
import org.elasticsearch.common.unit.ByteSizeValue; | ||
import org.elasticsearch.common.xcontent.ToXContent; | ||
import org.elasticsearch.common.xcontent.XContentBuilder; | ||
import org.elasticsearch.node.NodeClosedException; | ||
import org.elasticsearch.threadpool.ThreadPool; | ||
import org.elasticsearch.transport.ConnectTransportException; | ||
import org.elasticsearch.xpack.searchablesnapshots.cache.ByteRange; | ||
|
||
import java.time.Instant; | ||
import java.util.Set; | ||
import java.util.concurrent.TimeUnit; | ||
|
||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; | ||
|
@@ -41,7 +45,7 @@ public class BlobStoreCacheService { | |
|
||
private static final Logger logger = LogManager.getLogger(BlobStoreCacheService.class); | ||
|
||
public static final int DEFAULT_CACHED_BLOB_SIZE = ByteSizeUnit.KB.toIntBytes(4); | ||
public static final int DEFAULT_CACHED_BLOB_SIZE = ByteSizeUnit.KB.toIntBytes(1); | ||
|
||
private final ThreadPool threadPool; | ||
private final Client client; | ||
|
@@ -155,4 +159,46 @@ public void onFailure(Exception e) { | |
listener.onFailure(e); | ||
} | ||
} | ||
|
||
private static final Set<String> METADATA_FILES_EXTENSIONS = Set.of( | ||
ywelsch marked this conversation as resolved.
Show resolved
Hide resolved
|
||
"cfe", // compound file's entry table | ||
"dvm", // doc values metadata file | ||
"fdm", // stored fields metadata file | ||
"fnm", // field names metadata file | ||
"kdm", // Lucene 8.6 point format metadata file | ||
"nvm", // norms metadata file | ||
"tmd", // Lucene 8.6 terms metadata file | ||
"tvm", // terms vectors metadata file | ||
"vem" // Lucene 9.0 indexed vectors metadata | ||
); | ||
|
||
/** | ||
* Computes the {@link ByteRange} corresponding to the header of a Lucene file. This range can vary depending of the type of the file | ||
* which is indicated by the file's extension. The returned byte range can never be larger than the file's length but it can be smaller. | ||
* | ||
* For files that are declared as metadata files in {@link #METADATA_FILES_EXTENSIONS}, the header can be as large as the specified | ||
* maximum metadata length parameter {@code maxMetadataLength}. Non-metadata files have a fixed length header of maximum 1KB. | ||
* | ||
* @param fileName the name of the file | ||
* @param fileLength the length of the file | ||
* @param maxMetadataLength the maximum accepted length for metadata files | ||
* | ||
* @return the header {@link ByteRange} | ||
*/ | ||
public static ByteRange computeBlobCacheByteRange(String fileName, long fileLength, ByteSizeValue maxMetadataLength) { | ||
final String fileExtension = IndexFileNames.getExtension(fileName); | ||
if (METADATA_FILES_EXTENSIONS.contains(fileExtension)) { | ||
final long maxAllowedLengthInBytes = maxMetadataLength.getBytes(); | ||
if (fileLength > maxAllowedLengthInBytes) { | ||
logger.warn( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe too verbose? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let's use a |
||
"file of type [{}] and length [{}] is larger than the max. length allowed [{}] to cache metadata files in blob cache", | ||
fileExtension, | ||
fileLength, | ||
maxMetadataLength | ||
); | ||
} | ||
return ByteRange.of(0L, Math.min(fileLength, maxAllowedLengthInBytes)); | ||
} | ||
return ByteRange.of(0L, Math.min(fileLength, DEFAULT_CACHED_BLOB_SIZE)); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -42,6 +42,7 @@ | |
import java.util.function.Consumer; | ||
import java.util.stream.IntStream; | ||
|
||
import static org.elasticsearch.blobstore.cache.BlobStoreCacheService.computeBlobCacheByteRange; | ||
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsUtils.toIntBytes; | ||
|
||
public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexInput { | ||
|
@@ -56,7 +57,6 @@ public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexIn | |
private static final Logger logger = LogManager.getLogger(CachedBlobContainerIndexInput.class); | ||
private static final int COPY_BUFFER_SIZE = ByteSizeUnit.KB.toIntBytes(8); | ||
|
||
private final SearchableSnapshotDirectory directory; | ||
private final CacheFileReference cacheFileReference; | ||
private final int defaultRangeSize; | ||
private final int recoveryRangeSize; | ||
|
@@ -84,7 +84,8 @@ public CachedBlobContainerIndexInput( | |
fileInfo.length(), | ||
new CacheFileReference(directory, fileInfo.physicalName(), fileInfo.length()), | ||
rangeSize, | ||
recoveryRangeSize | ||
recoveryRangeSize, | ||
computeBlobCacheByteRange(fileInfo.physicalName(), fileInfo.length(), directory.getBlobStoreCacheMaxLength()) | ||
); | ||
assert getBufferSize() <= BlobStoreCacheService.DEFAULT_CACHED_BLOB_SIZE; // must be able to cache at least one buffer's worth | ||
stats.incrementOpenCount(); | ||
|
@@ -100,10 +101,10 @@ private CachedBlobContainerIndexInput( | |
long length, | ||
CacheFileReference cacheFileReference, | ||
int rangeSize, | ||
int recoveryRangeSize | ||
int recoveryRangeSize, | ||
ByteRange blobCacheByteRange | ||
) { | ||
super(logger, resourceDesc, directory.blobContainer(), fileInfo, context, stats, offset, length); | ||
this.directory = directory; | ||
super(logger, resourceDesc, directory, fileInfo, context, stats, offset, length, blobCacheByteRange); | ||
this.cacheFileReference = cacheFileReference; | ||
this.lastReadPosition = this.offset; | ||
this.lastSeekPosition = this.offset; | ||
|
@@ -162,25 +163,18 @@ protected void doReadInternal(ByteBuffer b) throws IOException { | |
|
||
// We try to use the cache index if: | ||
// - the file is small enough to be fully cached | ||
ywelsch marked this conversation as resolved.
Show resolved
Hide resolved
|
||
final boolean canBeFullyCached = fileInfo.length() <= BlobStoreCacheService.DEFAULT_CACHED_BLOB_SIZE * 2; | ||
// - we're reading the first N bytes of the file | ||
final boolean isStartOfFile = (position + length <= BlobStoreCacheService.DEFAULT_CACHED_BLOB_SIZE); | ||
|
||
if (canBeFullyCached || isStartOfFile) { | ||
final CachedBlob cachedBlob = directory.getCachedBlob(fileInfo.physicalName(), 0L, length); | ||
if (blobCacheByteRange.contains(position, position + length)) { | ||
final CachedBlob cachedBlob = directory.getCachedBlob(fileInfo.physicalName(), blobCacheByteRange.start(), length); | ||
|
||
if (cachedBlob == CachedBlob.CACHE_MISS || cachedBlob == CachedBlob.CACHE_NOT_READY) { | ||
if (cachedBlob == CachedBlob.CACHE_MISS | ||
|| cachedBlob == CachedBlob.CACHE_NOT_READY | ||
|| cachedBlob.from() != blobCacheByteRange.start() | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why do we leave it to the caller to check this and not just declare this a cache_miss? |
||
|| cachedBlob.to() != blobCacheByteRange.end()) { | ||
// We would have liked to find a cached entry but we did not find anything: the cache on the disk will be requested | ||
// so we compute the region of the file we would like to have the next time. The region is expressed as a tuple of | ||
// {start, end} where positions are relative to the whole file. | ||
|
||
if (canBeFullyCached) { | ||
// if the index input is smaller than twice the size of the blob cache, it will be fully indexed | ||
indexCacheMiss = ByteRange.of(0L, fileInfo.length()); | ||
} else { | ||
// the index input is too large to fully cache, so just cache the initial range | ||
indexCacheMiss = ByteRange.of(0L, BlobStoreCacheService.DEFAULT_CACHED_BLOB_SIZE); | ||
} | ||
indexCacheMiss = blobCacheByteRange; | ||
|
||
// We must fill in a cache miss even if CACHE_NOT_READY since the cache index is only created on the first put. | ||
// TODO TBD use a different trigger for creating the cache index and avoid a put in the CACHE_NOT_READY case. | ||
|
@@ -274,11 +268,12 @@ protected void doReadInternal(ByteBuffer b) throws IOException { | |
final Future<Integer> readFuture = cacheFile.readIfAvailableOrPending(indexCacheMiss, channel -> { | ||
final int indexCacheMissLength = toIntBytes(indexCacheMiss.length()); | ||
|
||
// TODO: should we use BigArrays? | ||
ywelsch marked this conversation as resolved.
Show resolved
Hide resolved
|
||
// We assume that we only cache small portions of blobs so that we do not need to: | ||
// - use a BigArrays for allocation | ||
// - use an intermediate copy buffer to read the file in sensibly-sized chunks | ||
// - release the buffer once the indexing operation is complete | ||
assert indexCacheMissLength <= COPY_BUFFER_SIZE : indexCacheMiss; | ||
// assert indexCacheMissLength <= COPY_BUFFER_SIZE : indexCacheMiss; | ||
ywelsch marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
final ByteBuffer byteBuffer = ByteBuffer.allocate(indexCacheMissLength); | ||
Channels.readFromFileChannelWithEofException(channel, indexCacheMiss.start(), byteBuffer); | ||
|
@@ -600,7 +595,8 @@ public IndexInput slice(String sliceDescription, long offset, long length) { | |
length, | ||
cacheFileReference, | ||
defaultRangeSize, | ||
recoveryRangeSize | ||
recoveryRangeSize, | ||
ByteRange.EMPTY // TODO implement blob cache for CFS | ||
ywelsch marked this conversation as resolved.
Show resolved
Hide resolved
|
||
); | ||
slice.isClone = true; | ||
return slice; | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
How will this change affect documents in the blob cache that have been created with a previous ES version?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry for the delayed response, I had to deal with other duties.
Looking at the current code in 7.x, reducing the size of cached blob might throw an exception in Cached/FrozenIndexInput at this line:
in the case
position
is larger than one or two buffered reads.I suggest