Skip to content

Add file type-based exclusion setting for searchable snapshots cache #53492

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

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@
*/
package org.elasticsearch.index.store;

import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.store.BaseDirectory;
import org.apache.lucene.store.BufferedIndexInput;
import org.apache.lucene.store.Directory;
Expand Down Expand Up @@ -37,17 +38,20 @@
import java.nio.file.Path;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.LongSupplier;

import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_CACHE_EXCLUDED_FILE_TYPES_SETTING;
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_CACHE_ENABLED_SETTING;
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_INDEX_ID_SETTING;
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.SearchableSnapshots.SNAPSHOT_UNCACHED_CHUNK_SIZE_SETTING;

/**
* Implementation of {@link Directory} that exposes files from a snapshot as a Lucene directory. Because snapshot are immutable this
Expand All @@ -71,6 +75,8 @@ public class SearchableSnapshotDirectory extends BaseDirectory {
private final Map<String, IndexInputStats> stats;
private final CacheService cacheService;
private final boolean useCache;
private final Set<String> excludedFileTypes;
private final long uncachedChunkSize;
private final Path cacheDir;
private final AtomicBoolean closed;

Expand All @@ -97,6 +103,10 @@ public SearchableSnapshotDirectory(
this.cacheDir = Objects.requireNonNull(cacheDir);
this.closed = new AtomicBoolean(false);
this.useCache = SNAPSHOT_CACHE_ENABLED_SETTING.get(indexSettings);
this.excludedFileTypes = new HashSet<>(SNAPSHOT_CACHE_EXCLUDED_FILE_TYPES_SETTING.get(indexSettings));
this.uncachedChunkSize = SNAPSHOT_UNCACHED_CHUNK_SIZE_SETTING.get(indexSettings).getBytes() < 0 ?
blobContainer.readBlobPreferredLength() :
SNAPSHOT_UNCACHED_CHUNK_SIZE_SETTING.get(indexSettings).getBytes();
}

public BlobContainer blobContainer() {
Expand Down Expand Up @@ -223,14 +233,18 @@ public IndexInput openInput(final String name, final IOContext context) throws I
ensureOpen();
final BlobStoreIndexShardSnapshot.FileInfo fileInfo = fileInfo(name);
final IndexInputStats inputStats = stats.computeIfAbsent(name, n -> createIndexInputStats(fileInfo.length()));
if (useCache) {
if (useCache && isExcludedFromCache(name) == false) {
return new CacheBufferedIndexInput(this, fileInfo, context, inputStats);
} else {
long preferredLength = blobContainer.readBlobPreferredLength();
return new SearchableSnapshotIndexInput(blobContainer, fileInfo, context, preferredLength, BufferedIndexInput.BUFFER_SIZE);
return new SearchableSnapshotIndexInput(blobContainer, fileInfo, context, uncachedChunkSize, BufferedIndexInput.BUFFER_SIZE);
}
}

private boolean isExcludedFromCache(String name) {
final String ext = IndexFileNames.getExtension(name);
return ext != null && excludedFileTypes.contains(ext);
}

@Override
public String toString() {
return this.getClass().getSimpleName() + "@" + snapshot.snapshot() + " lockFactory=" + lockFactory;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsFilter;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.env.Environment;
import org.elasticsearch.env.NodeEnvironment;
Expand Down Expand Up @@ -77,6 +79,13 @@ public class SearchableSnapshots extends Plugin implements IndexStorePlugin, Rep
Setting.simpleString("index.store.snapshot.index_uuid", Setting.Property.IndexScope, Setting.Property.PrivateIndex);
public static final Setting<Boolean> SNAPSHOT_CACHE_ENABLED_SETTING =
Setting.boolSetting("index.store.snapshot.cache.enabled", true, Setting.Property.IndexScope);
// The file extensions that are excluded from the cache
public static final Setting<List<String>> SNAPSHOT_CACHE_EXCLUDED_FILE_TYPES_SETTING =
Setting.listSetting("index.store.snapshot.cache.excluded_file_types", Collections.emptyList(), Function.identity(),
Setting.Property.IndexScope, Setting.Property.NodeScope);
public static final Setting<ByteSizeValue> SNAPSHOT_UNCACHED_CHUNK_SIZE_SETTING =
Setting.byteSizeSetting("index.store.snapshot.uncached_chunk_size", new ByteSizeValue(-1, ByteSizeUnit.BYTES),
Setting.Property.IndexScope, Setting.Property.NodeScope);

public static final String SNAPSHOT_DIRECTORY_FACTORY_KEY = "snapshot";

Expand All @@ -97,6 +106,8 @@ public List<Setting<?>> getSettings() {
SNAPSHOT_SNAPSHOT_ID_SETTING,
SNAPSHOT_INDEX_ID_SETTING,
SNAPSHOT_CACHE_ENABLED_SETTING,
SNAPSHOT_CACHE_EXCLUDED_FILE_TYPES_SETTING,
SNAPSHOT_UNCACHED_CHUNK_SIZE_SETTING,
CacheService.SNAPSHOT_CACHE_SIZE_SETTING,
CacheService.SNAPSHOT_CACHE_RANGE_SIZE_SETTING
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@

import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
Expand Down Expand Up @@ -122,12 +123,23 @@ public void testCreateAndRestoreSearchableSnapshot() throws Exception {
final boolean cacheEnabled = randomBoolean();
logger.info("--> restoring index [{}] with cache [{}]", restoredIndexName, cacheEnabled ? "enabled" : "disabled");

Settings.Builder indexSettingsBuilder = Settings.builder()
.put(SearchableSnapshots.SNAPSHOT_CACHE_ENABLED_SETTING.getKey(), cacheEnabled)
.put(IndexSettings.INDEX_CHECK_ON_STARTUP.getKey(), Boolean.FALSE.toString());
final List<String> nonCachedExtensions;
if (randomBoolean()) {
nonCachedExtensions = randomSubsetOf(Arrays.asList("fdt", "fdx", "nvd", "dvd", "tip", "cfs", "dim"));
indexSettingsBuilder.putList(SearchableSnapshots.SNAPSHOT_CACHE_EXCLUDED_FILE_TYPES_SETTING.getKey(), nonCachedExtensions);
} else {
nonCachedExtensions = Collections.emptyList();
}
if (randomBoolean()) {
indexSettingsBuilder.put(SearchableSnapshots.SNAPSHOT_UNCACHED_CHUNK_SIZE_SETTING.getKey(),
new ByteSizeValue(randomLongBetween(10, 100_000)));
}
final MountSearchableSnapshotRequest req = new MountSearchableSnapshotRequest(restoredIndexName, fsRepoName,
snapshotInfo.snapshotId().getName(), indexName,
Settings.builder()
.put(SearchableSnapshots.SNAPSHOT_CACHE_ENABLED_SETTING.getKey(), cacheEnabled)
.put(IndexSettings.INDEX_CHECK_ON_STARTUP.getKey(), Boolean.FALSE.toString())
.build(), Strings.EMPTY_ARRAY, true);
indexSettingsBuilder.build(), Strings.EMPTY_ARRAY, true);

final RestoreSnapshotResponse restoreSnapshotResponse = client().execute(MountSearchableSnapshotAction.INSTANCE, req).get();
assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), equalTo(0));
Expand All @@ -151,6 +163,13 @@ public void testCreateAndRestoreSearchableSnapshot() throws Exception {
for (SearchableSnapshotShardStats stats : statsResponse.getStats()) {
assertThat(stats.getShardRouting().getIndexName(), equalTo(restoredIndexName));
assertThat(stats.getStats().size(), greaterThan(0));
for (SearchableSnapshotShardStats.CacheIndexInputStats indexInputStats : stats.getStats()) {
for (String ext : nonCachedExtensions) {
if (indexInputStats.getFileName().endsWith(ext)) {
assertEquals(indexInputStats.getFileName(), 0, indexInputStats.getOpenCount());
}
}
}
}
} else {
expectThrows(ResourceNotFoundException.class, future::actionGet);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,11 @@ public InputStream readBlob(String name, long position, long length) throws IOEx
return Streams.limitStream(new ByteArrayInputStream(blobContent, Math.toIntExact(position), blobContent.length), length);
}

@Override
public long readBlobPreferredLength() {
return Long.MAX_VALUE;
}

@Override
public Map<String, BlobMetaData> listBlobs() {
throw unsupportedException();
Expand Down