Skip to content

Commit 5c2b15a

Browse files
authored
Adjust the length of blob cache docs for Lucene metadata files (#69431)
Today searchable snapshots IndexInput implementations use the blob store cache to cache the first 4096 bytes of every Lucene files. After some experiments we think that we could adjust the length of the cached data depending of the Lucene file that is read, caching up to 64KB for Lucene metadata files (ie files that are fully read when a Directory is opened) and only 1KB for other files. The files that are cached up to 64KB are the following extensions: "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 The 64KB limit can be configured on a per index basis through a new index setting. This change is extracted from #69283 and does not address the caching of CFS files.
1 parent 6c6443a commit 5c2b15a

22 files changed

+875
-257
lines changed

x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java

Lines changed: 164 additions & 134 deletions
Large diffs are not rendered by default.

x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java

Lines changed: 13 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@
2626
import org.elasticsearch.snapshots.SnapshotsService;
2727
import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotAction;
2828
import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotRequest;
29+
import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotRequest.Storage;
2930
import org.elasticsearch.xpack.searchablesnapshots.cache.CacheService;
3031
import org.elasticsearch.xpack.searchablesnapshots.cache.FrozenCacheService;
3132

@@ -116,6 +117,17 @@ protected void mountSnapshot(
116117
String indexName,
117118
String restoredIndexName,
118119
Settings restoredIndexSettings
120+
) throws Exception {
121+
mountSnapshot(repositoryName, snapshotName, indexName, restoredIndexName, restoredIndexSettings, Storage.FULL_COPY);
122+
}
123+
124+
protected void mountSnapshot(
125+
String repositoryName,
126+
String snapshotName,
127+
String indexName,
128+
String restoredIndexName,
129+
Settings restoredIndexSettings,
130+
final Storage storage
119131
) throws Exception {
120132
final MountSearchableSnapshotRequest mountRequest = new MountSearchableSnapshotRequest(
121133
restoredIndexName,
@@ -128,7 +140,7 @@ protected void mountSnapshot(
128140
.build(),
129141
Strings.EMPTY_ARRAY,
130142
true,
131-
MountSearchableSnapshotRequest.Storage.FULL_COPY
143+
storage
132144
);
133145

134146
final RestoreSnapshotResponse restoreResponse = client().execute(MountSearchableSnapshotAction.INSTANCE, mountRequest).get();

x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java

Lines changed: 130 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@
1010
import org.apache.logging.log4j.LogManager;
1111
import org.apache.logging.log4j.Logger;
1212
import org.apache.logging.log4j.message.ParameterizedMessage;
13+
import org.apache.lucene.index.IndexFileNames;
1314
import org.elasticsearch.ElasticsearchTimeoutException;
1415
import org.elasticsearch.ExceptionsHelper;
1516
import org.elasticsearch.Version;
@@ -23,15 +24,24 @@
2324
import org.elasticsearch.client.Client;
2425
import org.elasticsearch.client.OriginSettingClient;
2526
import org.elasticsearch.cluster.block.ClusterBlockException;
27+
import org.elasticsearch.cluster.service.ClusterService;
2628
import org.elasticsearch.common.bytes.BytesReference;
29+
import org.elasticsearch.common.cache.Cache;
30+
import org.elasticsearch.common.cache.CacheBuilder;
2731
import org.elasticsearch.common.unit.ByteSizeUnit;
32+
import org.elasticsearch.common.unit.ByteSizeValue;
33+
import org.elasticsearch.common.unit.TimeValue;
34+
import org.elasticsearch.common.util.set.Sets;
2835
import org.elasticsearch.common.xcontent.ToXContent;
2936
import org.elasticsearch.common.xcontent.XContentBuilder;
3037
import org.elasticsearch.node.NodeClosedException;
3138
import org.elasticsearch.threadpool.ThreadPool;
3239
import org.elasticsearch.transport.ConnectTransportException;
40+
import org.elasticsearch.xpack.searchablesnapshots.cache.ByteRange;
3341

3442
import java.time.Instant;
43+
import java.util.Set;
44+
import java.util.concurrent.ExecutionException;
3545
import java.util.concurrent.TimeUnit;
3646

3747
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
@@ -41,14 +51,24 @@ public class BlobStoreCacheService {
4151

4252
private static final Logger logger = LogManager.getLogger(BlobStoreCacheService.class);
4353

44-
public static final int DEFAULT_CACHED_BLOB_SIZE = ByteSizeUnit.KB.toIntBytes(4);
54+
/**
55+
* Before 8.0.0 blobs were cached using a 4KB or 8KB maximum length.
56+
*/
57+
private static final Version OLD_CACHED_BLOB_SIZE_VERSION = Version.V_8_0_0; // TODO adjust after backport
4558

59+
public static final int DEFAULT_CACHED_BLOB_SIZE = ByteSizeUnit.KB.toIntBytes(1);
60+
private static final Cache<String, String> LOG_EXCEEDING_FILES_CACHE = CacheBuilder.<String, String>builder()
61+
.setExpireAfterAccess(TimeValue.timeValueMinutes(60L))
62+
.build();
63+
64+
private final ClusterService clusterService;
4665
private final ThreadPool threadPool;
4766
private final Client client;
4867
private final String index;
4968

50-
public BlobStoreCacheService(ThreadPool threadPool, Client client, String index) {
69+
public BlobStoreCacheService(ClusterService clusterService, ThreadPool threadPool, Client client, String index) {
5170
this.client = new OriginSettingClient(client, SEARCHABLE_SNAPSHOTS_ORIGIN);
71+
this.clusterService = clusterService;
5272
this.threadPool = threadPool;
5373
this.index = index;
5474
}
@@ -155,4 +175,112 @@ public void onFailure(Exception e) {
155175
listener.onFailure(e);
156176
}
157177
}
178+
179+
private static final Set<String> METADATA_FILES_EXTENSIONS;
180+
private static final Set<String> OTHER_FILES_EXTENSIONS;
181+
static {
182+
// List of Lucene file extensions that are considered as "metadata" and should therefore be fully cached in the blob store cache.
183+
// Those files are usually fully read by Lucene when it opens a Directory.
184+
METADATA_FILES_EXTENSIONS = Set.of(
185+
"cfe", // compound file's entry table
186+
"dvm", // doc values metadata file
187+
"fdm", // stored fields metadata file
188+
"fnm", // field names metadata file
189+
"kdm", // Lucene 8.6 point format metadata file
190+
"nvm", // norms metadata file
191+
"tmd", // Lucene 8.6 terms metadata file
192+
"tvm", // terms vectors metadata file
193+
"vem" // Lucene 9.0 indexed vectors metadata
194+
);
195+
196+
// List of extensions for which Lucene usually only reads the first 1024 byte and checks a header checksum when opening a Directory.
197+
OTHER_FILES_EXTENSIONS = Set.of(
198+
"cfs",
199+
"dii",
200+
"dim",
201+
"doc",
202+
"dvd",
203+
"fdt",
204+
"fdx",
205+
"kdd",
206+
"kdi",
207+
"liv",
208+
"nvd",
209+
"pay",
210+
"pos",
211+
"tim",
212+
"tip",
213+
"tvd",
214+
"tvx",
215+
"vec"
216+
);
217+
assert Sets.intersection(METADATA_FILES_EXTENSIONS, OTHER_FILES_EXTENSIONS).isEmpty();
218+
}
219+
220+
/**
221+
* Computes the {@link ByteRange} corresponding to the header of a Lucene file. This range can vary depending of the type of the file
222+
* 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.
223+
*
224+
* For files that are declared as metadata files in {@link #METADATA_FILES_EXTENSIONS}, the header can be as large as the specified
225+
* maximum metadata length parameter {@code maxMetadataLength}. Non-metadata files have a fixed length header of maximum 1KB.
226+
*
227+
* @param fileName the name of the file
228+
* @param fileLength the length of the file
229+
* @param maxMetadataLength the maximum accepted length for metadata files
230+
*
231+
* @return the header {@link ByteRange}
232+
*/
233+
public ByteRange computeBlobCacheByteRange(String fileName, long fileLength, ByteSizeValue maxMetadataLength) {
234+
final String fileExtension = IndexFileNames.getExtension(fileName);
235+
assert fileExtension == null || METADATA_FILES_EXTENSIONS.contains(fileExtension) || OTHER_FILES_EXTENSIONS.contains(fileExtension)
236+
: "unknown Lucene file extension [" + fileExtension + "] - should it be considered a metadata file?";
237+
238+
if (useLegacyCachedBlobSizes()) {
239+
if (fileLength <= ByteSizeUnit.KB.toBytes(8L)) {
240+
return ByteRange.of(0L, fileLength);
241+
} else {
242+
return ByteRange.of(0L, ByteSizeUnit.KB.toBytes(4L));
243+
}
244+
}
245+
246+
if (METADATA_FILES_EXTENSIONS.contains(fileExtension)) {
247+
final long maxAllowedLengthInBytes = maxMetadataLength.getBytes();
248+
if (fileLength > maxAllowedLengthInBytes) {
249+
logExceedingFile(fileExtension, fileLength, maxMetadataLength);
250+
}
251+
return ByteRange.of(0L, Math.min(fileLength, maxAllowedLengthInBytes));
252+
}
253+
return ByteRange.of(0L, Math.min(fileLength, DEFAULT_CACHED_BLOB_SIZE));
254+
}
255+
256+
protected boolean useLegacyCachedBlobSizes() {
257+
final Version minNodeVersion = clusterService.state().nodes().getMinNodeVersion();
258+
return minNodeVersion.before(OLD_CACHED_BLOB_SIZE_VERSION);
259+
}
260+
261+
private static void logExceedingFile(String extension, long length, ByteSizeValue maxAllowedLength) {
262+
if (logger.isWarnEnabled()) {
263+
try {
264+
// Use of a cache to prevent too many log traces per hour
265+
LOG_EXCEEDING_FILES_CACHE.computeIfAbsent(extension, key -> {
266+
logger.warn(
267+
"file with extension [{}] is larger ([{}]) than the max. length allowed [{}] to cache metadata files in blob cache",
268+
extension,
269+
length,
270+
maxAllowedLength
271+
);
272+
return key;
273+
});
274+
} catch (ExecutionException e) {
275+
logger.warn(
276+
() -> new ParameterizedMessage(
277+
"Failed to log information about exceeding file type [{}] with length [{}]",
278+
extension,
279+
length
280+
),
281+
e
282+
);
283+
}
284+
}
285+
}
158286
}

x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CachedBlob.java

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -187,4 +187,27 @@ public static CachedBlob fromSource(final Map<String, Object> source) {
187187
to.longValue()
188188
);
189189
}
190+
191+
@Override
192+
public String toString() {
193+
return "CachedBlob ["
194+
+ "creationTime="
195+
+ creationTime
196+
+ ", version="
197+
+ version
198+
+ ", repository='"
199+
+ repository
200+
+ '\''
201+
+ ", name='"
202+
+ name
203+
+ '\''
204+
+ ", path='"
205+
+ path
206+
+ '\''
207+
+ ", from="
208+
+ from
209+
+ ", to="
210+
+ to
211+
+ ']';
212+
}
190213
}

x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java

Lines changed: 11 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@
1515
import org.elasticsearch.index.snapshots.blobstore.SlicedInputStream;
1616
import org.elasticsearch.threadpool.ThreadPool;
1717
import org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants;
18+
import org.elasticsearch.xpack.searchablesnapshots.cache.ByteRange;
1819

1920
import java.io.IOException;
2021
import java.io.InputStream;
@@ -29,34 +30,41 @@
2930
public abstract class BaseSearchableSnapshotIndexInput extends BufferedIndexInput {
3031

3132
protected final Logger logger;
33+
protected final SearchableSnapshotDirectory directory;
3234
protected final BlobContainer blobContainer;
3335
protected final FileInfo fileInfo;
3436
protected final IOContext context;
3537
protected final IndexInputStats stats;
3638
protected final long offset;
3739
protected final long length;
3840

41+
/** Range of bytes that should be cached in the blob cache for the current index input **/
42+
protected final ByteRange blobCacheByteRange;
43+
3944
// the following are only mutable so they can be adjusted after cloning/slicing
4045
protected volatile boolean isClone;
4146
private AtomicBoolean closed;
4247

4348
public BaseSearchableSnapshotIndexInput(
4449
Logger logger,
4550
String resourceDesc,
46-
BlobContainer blobContainer,
51+
SearchableSnapshotDirectory directory,
4752
FileInfo fileInfo,
4853
IOContext context,
4954
IndexInputStats stats,
5055
long offset,
51-
long length
56+
long length,
57+
ByteRange blobCacheByteRange
5258
) {
5359
super(resourceDesc, context);
5460
this.logger = Objects.requireNonNull(logger);
55-
this.blobContainer = Objects.requireNonNull(blobContainer);
61+
this.directory = Objects.requireNonNull(directory);
62+
this.blobContainer = Objects.requireNonNull(directory.blobContainer());
5663
this.fileInfo = Objects.requireNonNull(fileInfo);
5764
this.context = Objects.requireNonNull(context);
5865
assert fileInfo.metadata().hashEqualsContents() == false
5966
: "this method should only be used with blobs that are NOT stored in metadata's hash field " + "(fileInfo: " + fileInfo + ')';
67+
this.blobCacheByteRange = Objects.requireNonNull(blobCacheByteRange);
6068
this.stats = Objects.requireNonNull(stats);
6169
this.offset = offset;
6270
this.length = length;

x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java

Lines changed: 19 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@
3434
import org.elasticsearch.common.collect.Tuple;
3535
import org.elasticsearch.common.lucene.store.ByteArrayIndexInput;
3636
import org.elasticsearch.common.settings.Settings;
37+
import org.elasticsearch.common.unit.ByteSizeValue;
3738
import org.elasticsearch.common.unit.TimeValue;
3839
import org.elasticsearch.common.util.LazyInitializable;
3940
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
@@ -59,6 +60,7 @@
5960
import org.elasticsearch.threadpool.ThreadPool;
6061
import org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots;
6162
import org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants;
63+
import org.elasticsearch.xpack.searchablesnapshots.cache.ByteRange;
6264
import org.elasticsearch.xpack.searchablesnapshots.cache.CacheService;
6365
import org.elasticsearch.xpack.searchablesnapshots.cache.FrozenCacheService;
6466
import org.elasticsearch.xpack.searchablesnapshots.cache.FrozenCacheService.FrozenCacheFile;
@@ -86,6 +88,7 @@
8688

8789
import static org.apache.lucene.store.BufferedIndexInput.bufferSize;
8890
import static org.elasticsearch.index.IndexModule.INDEX_STORE_TYPE_SETTING;
91+
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_BLOB_CACHE_METADATA_FILES_MAX_LENGTH_SETTING;
8992
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_CACHE_ENABLED_SETTING;
9093
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_CACHE_EXCLUDED_FILE_TYPES_SETTING;
9194
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_CACHE_PREWARM_ENABLED_SETTING;
@@ -136,6 +139,7 @@ public class SearchableSnapshotDirectory extends BaseDirectory {
136139
private final AtomicBoolean closed;
137140
private final boolean partial;
138141
private final FrozenCacheService frozenCacheService;
142+
private final ByteSizeValue blobStoreCacheMaxLength;
139143

140144
// volatile fields are updated once under `this` lock, all together, iff loaded is not true.
141145
private volatile BlobStoreIndexShardSnapshot snapshot;
@@ -179,6 +183,7 @@ public SearchableSnapshotDirectory(
179183
this.excludedFileTypes = new HashSet<>(SNAPSHOT_CACHE_EXCLUDED_FILE_TYPES_SETTING.get(indexSettings));
180184
this.uncachedChunkSize = SNAPSHOT_UNCACHED_CHUNK_SIZE_SETTING.get(indexSettings).getBytes();
181185
this.blobStoreCachePath = String.join("/", snapshotId.getUUID(), indexId.getId(), String.valueOf(shardId.id()));
186+
this.blobStoreCacheMaxLength = SNAPSHOT_BLOB_CACHE_METADATA_FILES_MAX_LENGTH_SETTING.get(indexSettings);
182187
this.threadPool = threadPool;
183188
this.loaded = false;
184189
this.frozenCacheService = frozenCacheService;
@@ -419,14 +424,7 @@ public IndexInput openInput(final String name, final IOContext context) throws I
419424
);
420425
}
421426
} else {
422-
return new DirectBlobContainerIndexInput(
423-
blobContainer(),
424-
fileInfo,
425-
context,
426-
inputStats,
427-
getUncachedChunkSize(),
428-
bufferSize(context)
429-
);
427+
return new DirectBlobContainerIndexInput(this, fileInfo, context, inputStats, getUncachedChunkSize(), bufferSize(context));
430428
}
431429
}
432430

@@ -679,10 +677,19 @@ public static SearchableSnapshotDirectory unwrapDirectory(Directory dir) {
679677
return null;
680678
}
681679

682-
public CachedBlob getCachedBlob(String name, long offset, int length) {
683-
final CachedBlob cachedBlob = blobStoreCacheService.get(repository, name, blobStoreCachePath, offset);
684-
assert cachedBlob == CachedBlob.CACHE_MISS || cachedBlob == CachedBlob.CACHE_NOT_READY || cachedBlob.from() <= offset;
685-
assert cachedBlob == CachedBlob.CACHE_MISS || cachedBlob == CachedBlob.CACHE_NOT_READY || offset + length <= cachedBlob.to();
680+
public ByteRange getBlobCacheByteRange(String fileName, long fileLength) {
681+
return blobStoreCacheService.computeBlobCacheByteRange(fileName, fileLength, blobStoreCacheMaxLength);
682+
}
683+
684+
public CachedBlob getCachedBlob(String name, ByteRange range) {
685+
final CachedBlob cachedBlob = blobStoreCacheService.get(repository, name, blobStoreCachePath, range.start());
686+
if (cachedBlob == CachedBlob.CACHE_MISS || cachedBlob == CachedBlob.CACHE_NOT_READY) {
687+
return cachedBlob;
688+
} else if (cachedBlob.from() != range.start() || cachedBlob.to() != range.end()) {
689+
// expected range in cache might differ with the returned cached blob; this can happen if the range to put in cache is changed
690+
// between versions or through the index setting. In this case we assume it is a cache miss to force the blob to be cached again
691+
return CachedBlob.CACHE_MISS;
692+
}
686693
return cachedBlob;
687694
}
688695

0 commit comments

Comments
 (0)