From d0014198b691d898a85ddfccf1cb7262a849c568 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Fri, 31 Jul 2020 16:10:18 +0200 Subject: [PATCH 01/63] Introduce index based snapshot blob cache for Searchable Snapshots --- .../test/rest/ESRestTestCase.java | 1 + .../xpack/core/ClientHelper.java | 1 + .../SearchableSnapshotsConstants.java | 2 + .../cache/BlobStoreCacheService.java | 305 ++++++++ .../blobstore/cache/CachedBlob.java | 179 +++++ .../cache/CopyOnReadInputStream.java | 118 ++++ .../BaseSearchableSnapshotIndexInput.java | 104 ++- .../store/SearchableSnapshotDirectory.java | 57 +- .../cache/CachedBlobContainerIndexInput.java | 76 +- .../ChecksumBlobContainerIndexInput.java | 8 +- .../SearchableSnapshotIndexEventListener.java | 2 +- .../SearchableSnapshots.java | 33 +- .../cache/CopyOnReadInputStreamTests.java | 218 ++++++ ...ableSnapshotsBlobStoreCacheIntegTests.java | 659 ++++++++++++++++++ ...SearchableSnapshotDirectoryStatsTests.java | 35 +- .../SearchableSnapshotDirectoryTests.java | 51 +- .../CachedBlobContainerIndexInputTests.java | 50 +- .../index/store/cache/TestUtils.java | 30 + .../BaseSearchableSnapshotsIntegTestCase.java | 63 ++ 19 files changed, 1968 insertions(+), 24 deletions(-) create mode 100644 x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java create mode 100644 x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CachedBlob.java create mode 100644 x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStream.java create mode 100644 x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStreamTests.java create mode 100644 x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index 80ea7f439e66c..b719047dbeda3 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -1223,6 +1223,7 @@ protected static boolean isXPackTemplate(String name) { case "metrics": case "metrics-settings": case "metrics-mappings": + case ".snapshot-blob-cache": return true; default: return false; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ClientHelper.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ClientHelper.java index eaec3d0b1e25e..d5c817f33e95b 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ClientHelper.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ClientHelper.java @@ -69,6 +69,7 @@ public static Map filterSecurityHeaders(Map head public static final String ASYNC_SEARCH_ORIGIN = "async_search"; public static final String IDP_ORIGIN = "idp"; public static final String STACK_ORIGIN = "stack"; + public static final String SEARCHABLE_SNAPSHOTS_ORIGIN = "searchable_snapshots"; private ClientHelper() {} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsConstants.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsConstants.java index 96d376787b82c..5dc4c8ba15a97 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsConstants.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsConstants.java @@ -40,4 +40,6 @@ public static boolean isSearchableSnapshotStore(Settings indexSettings) { public static final String CACHE_PREWARMING_THREAD_POOL_NAME = "searchable_snapshots_cache_prewarming"; 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"; } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java new file mode 100644 index 0000000000000..8373578c0fe4d --- /dev/null +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -0,0 +1,305 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.blobstore.cache; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.ResourceAlreadyExistsException; +import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.NoShardAvailableActionException; +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.OriginSettingClient; +import org.elasticsearch.cluster.ClusterChangedEvent; +import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.cluster.routing.IndexRoutingTable; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.bytes.ReleasableBytesReference; +import org.elasticsearch.common.component.AbstractLifecycleComponent; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.threadpool.ThreadPool; + +import java.io.IOException; +import java.time.Instant; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.index.mapper.MapperService.SINGLE_MAPPING_NAME; +import static org.elasticsearch.xpack.core.ClientHelper.SEARCHABLE_SNAPSHOTS_ORIGIN; + +public class BlobStoreCacheService extends AbstractLifecycleComponent implements ClusterStateListener { + + private static final Logger logger = LogManager.getLogger(BlobStoreCacheService.class); + + public static final int DEFAULT_SIZE = Math.toIntExact(ByteSizeUnit.KB.toBytes(4L)); + + private final ClusterService clusterService; + private final ThreadPool threadPool; + private final AtomicBoolean ready; + private final Client client; + private final String index; + + public BlobStoreCacheService(ClusterService clusterService, ThreadPool threadPool, Client client, String index) { + this.client = new OriginSettingClient(client, SEARCHABLE_SNAPSHOTS_ORIGIN); + this.ready = new AtomicBoolean(false); + this.clusterService = clusterService; + this.threadPool = threadPool; + this.index = index; + } + + @Override + protected void doStart() { + clusterService.addListener(this); + } + + @Override + protected void doStop() { + clusterService.removeListener(this); + } + + @Override + protected void doClose() {} + + @Override + public void clusterChanged(ClusterChangedEvent event) { + if (lifecycle.started() == false || event.routingTableChanged() == false) { + return; + } + if (event.indexRoutingTableChanged(index)) { + final IndexRoutingTable indexRoutingTable = event.state().routingTable().index(index); + if (indexRoutingTable == null) { + ready.set(false); + return; + } + ready.set(indexRoutingTable.allPrimaryShardsActive()); + } + } + + public boolean isReady() { + return lifecycle.started() && ready.get(); + } + + private void createIndexIfNecessary(ActionListener listener) { + if (clusterService.state().routingTable().hasIndex(index)) { + listener.onResponse(index); + return; + } + try { + client.admin() + .indices() + .prepareCreate(index) + .setSettings(settings()) + .setMapping(mappings()) + .execute(ActionListener.wrap(success -> listener.onResponse(index), e -> { + if (e instanceof ResourceAlreadyExistsException + || ExceptionsHelper.unwrapCause(e) instanceof ResourceAlreadyExistsException) { + listener.onResponse(index); + } else { + listener.onFailure(e); + } + })); + } catch (Exception e) { + listener.onFailure(e); + } + } + + private static Settings settings() { + return Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, "0-1") + .put(IndexMetadata.SETTING_PRIORITY, "900") + .build(); + } + + private static XContentBuilder mappings() throws IOException { + final XContentBuilder builder = jsonBuilder(); + { + builder.startObject(); + { + builder.startObject(SINGLE_MAPPING_NAME); + builder.field("dynamic", "false"); + { + builder.startObject("_meta"); + builder.field("version", Version.CURRENT); + builder.endObject(); + } + { + builder.startObject("properties"); + { + builder.startObject("type"); + builder.field("type", "keyword"); + builder.endObject(); + } + { + builder.startObject("creation_time"); + builder.field("type", "date"); + builder.field("format", "epoch_millis"); + builder.endObject(); + } + { + builder.startObject("accessed_time"); + builder.field("type", "date"); + builder.field("format", "epoch_millis"); + builder.endObject(); + } + { + builder.startObject("version"); + builder.field("type", "integer"); + builder.endObject(); + } + { + builder.startObject("repository"); + builder.field("type", "keyword"); + builder.endObject(); + } + { + builder.startObject("blob"); + builder.field("type", "object"); + { + builder.startObject("properties"); + { + builder.startObject("name"); + builder.field("type", "keyword"); + builder.endObject(); + builder.startObject("path"); + builder.field("type", "keyword"); + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + } + { + builder.startObject("data"); + builder.field("type", "object"); + { + builder.startObject("properties"); + { + builder.startObject("content"); + builder.field("type", "binary"); + builder.endObject(); + } + { + builder.startObject("length"); + builder.field("type", "long"); + builder.endObject(); + } + { + builder.startObject("from"); + builder.field("type", "long"); + builder.endObject(); + } + { + builder.startObject("to"); + builder.field("type", "long"); + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + } + return builder; + } + + public CachedBlob get(String repository, String name, String path, long offset) { + final PlainActionFuture future = PlainActionFuture.newFuture(); + getAsync(repository, name, path, offset, future); + return future.actionGet(); + } + + protected void getAsync(String repository, String name, String path, long offset, ActionListener listener) { + if (isReady() == false) { + logger.debug("blob cache system index [{}] is not ready", index); + listener.onResponse(null); + return; + } + try { + final GetRequest request = new GetRequest(index).id(CachedBlob.generateId(repository, name, path, offset)); + client.get(request, ActionListener.wrap(response -> { + if (response.isExists()) { + logger.debug("found cached blob with id [{}] in cache", request.id()); + assert response.isSourceEmpty() == false; + + final CachedBlob cachedBlob = CachedBlob.fromSource(response.getSource()); + assert response.getId().equals(cachedBlob.generatedId()); + listener.onResponse(cachedBlob); + } else { + logger.debug("no cached blob found with id [{}] in cache", request.id()); + listener.onResponse(null); + } + }, e -> { + logger.warn(() -> new ParameterizedMessage("failed to retrieve cached blob from system index [{}]", index), e); + if (e instanceof IndexNotFoundException || e instanceof NoShardAvailableActionException) { + // In case the blob cache system index got unavailable, we pretend we didn't find a cache entry and we move on. + // Failing here might bubble up the exception and fail the searchable snapshot shard which is potentially recovering. + listener.onResponse(null); + } else { + listener.onFailure(e); + } + })); + } catch (Exception e) { + listener.onFailure(e); + } + } + + public void putAsync( + String repository, + String name, + String path, + long offset, + ReleasableBytesReference content, + ActionListener listener + ) { + createIndexIfNecessary(ActionListener.wrap(index -> { + try { + final CachedBlob cachedBlob = new CachedBlob( + Instant.ofEpochMilli(threadPool.absoluteTimeInMillis()), + Version.CURRENT, + repository, + name, + path, + content, + offset + ); + final IndexRequest request = new IndexRequest(index).id(cachedBlob.generatedId()); + try (XContentBuilder builder = jsonBuilder()) { + request.source(cachedBlob.toXContent(builder, ToXContent.EMPTY_PARAMS)); + } + client.index(request, ActionListener.wrap(response -> { + if (response.status() == RestStatus.CREATED) { + logger.trace("cached blob [{}] successfully indexed in [{}]", request, index); + } + }, listener::onFailure)); + } catch (IOException e) { + logger.warn("failed to index cached blob in cache", e); + } finally { + IOUtils.closeWhileHandlingException(content); + } + }, e -> { + logger.error(() -> new ParameterizedMessage("failed to create blob cache system index [{}]", index), e); + IOUtils.closeWhileHandlingException(content); + })); + } +} diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CachedBlob.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CachedBlob.java new file mode 100644 index 0000000000000..111cd335d0496 --- /dev/null +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CachedBlob.java @@ -0,0 +1,179 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.blobstore.cache; + +import org.elasticsearch.Version; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; + +import java.io.IOException; +import java.time.Instant; +import java.util.Base64; +import java.util.Map; + +public class CachedBlob implements ToXContent { + + private static final String TYPE = "blob"; + + private final Instant creationTime; + private final Version version; + private final String repository; + private final String name; + private final String path; + + private final BytesReference bytes; + private final long from; + private final long to; + + public CachedBlob( + Instant creationTime, + Version version, + String repository, + String name, + String path, + BytesReference content, + long offset + ) { + this(creationTime, version, repository, name, path, content, offset, offset + (content == null ? 0 : content.length())); + } + + private CachedBlob( + Instant creationTime, + Version version, + String repository, + String name, + String path, + BytesReference content, + long from, + long to + ) { + this.creationTime = creationTime; + this.version = version; + this.repository = repository; + this.name = name; + this.path = path; + this.bytes = content; + this.from = from; + this.to = to; + assert this.to == this.from + this.bytes.length(); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + { + builder.field("type", TYPE); + builder.field("creation_time", creationTime.toEpochMilli()); + builder.field("version", version.id); + builder.field("repository", repository); + builder.startObject("blob"); + { + builder.field("name", name); + builder.field("path", path); + } + builder.endObject(); + builder.startObject("data"); + { + builder.field("content", BytesReference.toBytes(bytes)); + builder.field("length", bytes.length()); + builder.field("from", from); + builder.field("to", to); + } + builder.endObject(); + } + return builder.endObject(); + } + + public String generatedId() { + return generateId(repository, name, path, from); + } + + public long from() { + return from; + } + + public long to() { + return to; + } + + public int length() { + return bytes.length(); + } + + public BytesReference bytes() { + return bytes; + } + + public static String generateId(String repository, String name, String path, long offset) { + return String.join("/", repository, path, name, "@" + offset); + } + + @SuppressWarnings("unchecked") + public static CachedBlob fromSource(final Map source) { + final Long creationTimeEpochMillis = (Long) source.get("creation_time"); + if (creationTimeEpochMillis == null) { + throw new IllegalStateException("cached blob document does not have the [creation_time] field"); + } + final Version version = Version.fromId((Integer) source.get("version")); + if (version == null) { + throw new IllegalStateException("cached blob document does not have the [version] field"); + } + final String repository = (String) source.get("repository"); + if (repository == null) { + throw new IllegalStateException("cached blob document does not have the [repository] field"); + } + final Map blob = (Map) source.get("blob"); + if (blob == null || blob.isEmpty()) { + throw new IllegalStateException("cached blob document does not have the [blob] object"); + } + final String name = (String) blob.get("name"); + if (name == null) { + throw new IllegalStateException("cached blob document does not have the [blob.name] field"); + } + final String path = (String) blob.get("path"); + if (path == null) { + throw new IllegalStateException("cached blob document does not have the [blob.path] field"); + } + final Map data = (Map) source.get("data"); + if (data == null || data.isEmpty()) { + throw new IllegalStateException("cached blob document does not have the [data] fobjectield"); + } + final String encodedContent = (String) data.get("content"); + if (encodedContent == null) { + throw new IllegalStateException("cached blob document does not have the [data.content] field"); + } + final Integer length = (Integer) data.get("length"); + if (length == null) { + throw new IllegalStateException("cached blob document does not have the [data.length] field"); + } + final byte[] content = Base64.getDecoder().decode(encodedContent); + if (content.length != length) { + throw new IllegalStateException("cached blob document content length does not match [data.length] field"); + } + final Number from = (Number) data.get("from"); + if (from == null) { + throw new IllegalStateException("cached blob document does not have the [data.from] field"); + } + final Number to = (Number) data.get("to"); + if (to == null) { + throw new IllegalStateException("cached blob document does not have the [data.to] field"); + } + // TODO add exhaustive verifications (from/to/content.length, version supported, id == recomputed id etc) + return new CachedBlob( + Instant.ofEpochMilli(creationTimeEpochMillis), + version, + repository, + name, + path, + new BytesArray(content), + from.longValue(), + to.longValue() + ); + } +} diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStream.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStream.java new file mode 100644 index 0000000000000..6d87f8c3ff048 --- /dev/null +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStream.java @@ -0,0 +1,118 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.blobstore.cache; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.CheckedSupplier; +import org.elasticsearch.common.bytes.PagedBytesReference; +import org.elasticsearch.common.bytes.ReleasableBytesReference; +import org.elasticsearch.common.util.ByteArray; + +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * A {@link FilterInputStream} that copies over all the bytes read from the original input stream to a given {@link ByteArray}. The + * number of bytes copied cannot exceed the size of the {@link ByteArray}. + */ +public class CopyOnReadInputStream extends FilterInputStream { + + private final ActionListener listener; + private final AtomicBoolean closed; + private final ByteArray bytes; + + private IOException failure; + private long count; + private long mark; + + public CopyOnReadInputStream(InputStream in, ByteArray byteArray, ActionListener listener) { + super(in); + this.listener = Objects.requireNonNull(listener); + this.bytes = Objects.requireNonNull(byteArray); + this.closed = new AtomicBoolean(false); + } + + private T handleFailure(CheckedSupplier supplier) throws IOException { + try { + return supplier.get(); + } catch (IOException e) { + assert failure == null; + failure = e; + throw e; + } + } + + public int read() throws IOException { + final int result = handleFailure(super::read); + if (result != -1) { + if (count < bytes.size()) { + bytes.set(count, (byte) result); + } + count++; + } + return result; + } + + public int read(byte[] b, int off, int len) throws IOException { + final int result = handleFailure(() -> super.read(b, off, len)); + if (result != -1) { + if (count < bytes.size()) { + bytes.set(count, b, off, Math.toIntExact(Math.min(bytes.size() - count, result))); + } + count += result; + } + return result; + } + + @Override + public long skip(long n) throws IOException { + final long skip = handleFailure(() -> super.skip(n)); + if (skip > 0L) { + count += skip; + } + return skip; + } + + @Override + public synchronized void mark(int readlimit) { + super.mark(readlimit); + mark = count; + } + + @Override + public synchronized void reset() throws IOException { + handleFailure(() -> { + super.reset(); + return null; + }); + count = mark; + } + + @Override + public final void close() throws IOException { + if (closed.compareAndSet(false, true)) { + boolean success = false; + try { + if (failure == null) { + PagedBytesReference reference = new PagedBytesReference(bytes, Math.toIntExact(Math.min(count, bytes.size()))); + listener.onResponse(new ReleasableBytesReference(reference, bytes)); + success = true; + } else { + listener.onFailure(failure); + } + } finally { + if (success == false) { + bytes.close(); + } + in.close(); + } + } + } +} diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java index 9b4ed2db5981e..59880249ac241 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java @@ -5,9 +5,19 @@ */ package org.elasticsearch.index.store; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.store.BufferedIndexInput; import org.apache.lucene.store.IOContext; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.blobstore.cache.CopyOnReadInputStream; +import org.elasticsearch.common.TriConsumer; import org.elasticsearch.common.blobstore.BlobContainer; +import org.elasticsearch.common.bytes.ReleasableBytesReference; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.io.Streams; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo; import org.elasticsearch.index.snapshots.blobstore.SlicedInputStream; import org.elasticsearch.threadpool.ThreadPool; @@ -15,11 +25,20 @@ import java.io.IOException; import java.io.InputStream; +import java.io.SequenceInputStream; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; public abstract class BaseSearchableSnapshotIndexInput extends BufferedIndexInput { + public static final int DEFAULT_BUFFER_SIZE = Math.toIntExact(ByteSizeUnit.KB.toBytes(4L)); + protected final BlobContainer blobContainer; protected final FileInfo fileInfo; protected final IOContext context; @@ -40,7 +59,7 @@ public BaseSearchableSnapshotIndexInput( long offset, long length ) { - super(resourceDesc, context); + super(resourceDesc, DEFAULT_BUFFER_SIZE); // TODO align buffer size with block size on disk and length of content cached for blobs this.blobContainer = Objects.requireNonNull(blobContainer); this.fileInfo = Objects.requireNonNull(fileInfo); this.context = Objects.requireNonNull(context); @@ -129,6 +148,89 @@ protected InputStream openSlice(long slice) throws IOException { } } + protected InputStream openInputStream( + final long position, + final long length, + final List> regions, + final TriConsumer blobStoreCacher, + final Logger logger + ) throws IOException { + final InputStream stream = openInputStream(position, length); + if (regions == null || regions.isEmpty()) { + return stream; + } + + // + // TODO I'm so sorry. This should be done differently, maybe using a smarter CopyOnReadInputStream + // + // The idea is to build a SequenceInputStream that wraps the stream from the blob store repository + // into multiple limited streams that copy over the bytes of the regions. + // + // If while reading the stream we saw interesting regions to cache, we index them. It means that + // we first have to sort the regions and exclude the ones that we're not going to see anyway. + // + // TODO we should check overlapping regions too + + final Iterator> sortedRegions = regions.stream() + .filter(region -> position <= region.v1()) + .filter(region -> region.v1() + region.v2() <= position + length) + .sorted(Comparator.comparing(Tuple::v1)) + .collect(Collectors.toList()) + .iterator(); + + final List streams = new ArrayList<>(); + for (long p = position; p < position + length;) { + if (sortedRegions.hasNext()) { + final Tuple nextRegion = sortedRegions.next(); + if (p < nextRegion.v1()) { + long limit = nextRegion.v1() - p; + streams.add(Streams.limitStream(Streams.noCloseStream(stream), limit)); + p += limit; + } + assert p == nextRegion.v1(); + + long limit = nextRegion.v2(); + streams.add( + new CopyOnReadInputStream( + Streams.limitStream(p + limit < length ? Streams.noCloseStream(stream) : stream, limit), + BigArrays.NON_RECYCLING_INSTANCE.newByteArray(limit), + ActionListener.wrap(releasable -> { + logger.trace( + () -> new ParameterizedMessage( + "indexing bytes of file [{}] for region [{}-{}] in blob cache index", + fileInfo.physicalName(), + nextRegion.v1(), + nextRegion.v1() + nextRegion.v2() + ) + ); + blobStoreCacher.apply(fileInfo.physicalName(), nextRegion.v1(), releasable); + }, e -> { + logger.trace( + () -> new ParameterizedMessage( + "fail to index bytes of file [{}] for region [{}-{}] in blob cache index", + fileInfo.physicalName(), + nextRegion.v1(), + nextRegion.v1() + nextRegion.v2() + ), + e + ); + }) + ) + ); + p += limit; + assert p == nextRegion.v1() + nextRegion.v2(); + } else if (p < position + length) { + long limit = position + length - p; + streams.add(Streams.limitStream(stream, limit)); + p += limit; + } + } + if (streams.size() == 1) { + return streams.get(0); + } + return new SequenceInputStream(Collections.enumeration(streams)); + } + protected final boolean assertCurrentThreadMayAccessBlobStore() { final String threadName = Thread.currentThread().getName(); assert threadName.contains('[' + ThreadPool.Names.SNAPSHOT + ']') diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java index 0ba394f06e541..ba81a1a5b1372 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java @@ -20,11 +20,14 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.support.GroupedActionListener; +import org.elasticsearch.blobstore.cache.BlobStoreCacheService; +import org.elasticsearch.blobstore.cache.CachedBlob; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.CheckedRunnable; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.support.FilterBlobContainer; +import org.elasticsearch.common.bytes.ReleasableBytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lucene.store.ByteArrayIndexInput; import org.elasticsearch.common.settings.Settings; @@ -41,6 +44,7 @@ import org.elasticsearch.index.store.cache.CachedBlobContainerIndexInput; import org.elasticsearch.index.store.checksum.ChecksumBlobContainerIndexInput; import org.elasticsearch.index.store.direct.DirectBlobContainerIndexInput; +import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.Repository; @@ -102,6 +106,9 @@ public class SearchableSnapshotDirectory extends BaseDirectory { private final Supplier blobContainerSupplier; private final Supplier snapshotSupplier; + private final BlobStoreCacheService blobStoreCacheService; + private final String blobStoreCachePath; + private final String repository; private final SnapshotId snapshotId; private final IndexId indexId; private final ShardId shardId; @@ -115,16 +122,20 @@ public class SearchableSnapshotDirectory extends BaseDirectory { private final long uncachedChunkSize; // if negative use BlobContainer#readBlobPreferredLength, see #getUncachedChunkSize() private final Path cacheDir; private final ShardPath shardPath; + private final AtomicBoolean recovered; private final AtomicBoolean closed; // volatile fields are updated once under `this` lock, all together, iff loaded is not true. private volatile BlobStoreIndexShardSnapshot snapshot; private volatile BlobContainer blobContainer; + private volatile RecoveryState recoveryState; private volatile boolean loaded; public SearchableSnapshotDirectory( Supplier blobContainer, Supplier snapshot, + BlobStoreCacheService blobStoreCacheService, + String repository, SnapshotId snapshotId, IndexId indexId, ShardId shardId, @@ -138,6 +149,8 @@ public SearchableSnapshotDirectory( super(new SingleInstanceLockFactory()); this.snapshotSupplier = Objects.requireNonNull(snapshot); this.blobContainerSupplier = Objects.requireNonNull(blobContainer); + this.blobStoreCacheService = Objects.requireNonNull(blobStoreCacheService); + this.repository = Objects.requireNonNull(repository); this.snapshotId = Objects.requireNonNull(snapshotId); this.indexId = Objects.requireNonNull(indexId); this.shardId = Objects.requireNonNull(shardId); @@ -146,11 +159,13 @@ public SearchableSnapshotDirectory( this.cacheService = Objects.requireNonNull(cacheService); this.cacheDir = Objects.requireNonNull(cacheDir); this.shardPath = Objects.requireNonNull(shardPath); + this.recovered = new AtomicBoolean(false); this.closed = new AtomicBoolean(false); this.useCache = SNAPSHOT_CACHE_ENABLED_SETTING.get(indexSettings); this.prewarmCache = useCache ? SNAPSHOT_CACHE_PREWARM_ENABLED_SETTING.get(indexSettings) : false; this.excludedFileTypes = new HashSet<>(SNAPSHOT_CACHE_EXCLUDED_FILE_TYPES_SETTING.get(indexSettings)); this.uncachedChunkSize = SNAPSHOT_UNCACHED_CHUNK_SIZE_SETTING.get(indexSettings).getBytes(); + this.blobStoreCachePath = String.join("/", snapshotId.getUUID(), indexId.getId(), String.valueOf(shardId.id())); this.threadPool = threadPool; this.loaded = false; assert invariant(); @@ -159,6 +174,7 @@ public SearchableSnapshotDirectory( private synchronized boolean invariant() { assert loaded != (snapshot == null); assert loaded != (blobContainer == null); + assert loaded != (recoveryState == null); return true; } @@ -176,7 +192,7 @@ protected final boolean assertCurrentThreadMayLoadSnapshot() { * * @return true if the snapshot was loaded by executing this method, false otherwise */ - public boolean loadSnapshot() { + public boolean loadSnapshot(RecoveryState recoveryState) { assert assertCurrentThreadMayLoadSnapshot(); boolean alreadyLoaded = this.loaded; if (alreadyLoaded == false) { @@ -185,8 +201,8 @@ public boolean loadSnapshot() { if (alreadyLoaded == false) { this.blobContainer = blobContainerSupplier.get(); this.snapshot = snapshotSupplier.get(); + this.recoveryState = recoveryState; this.loaded = true; - cleanExistingRegularShardFiles(); prewarmCache(); } } @@ -209,6 +225,17 @@ public BlobStoreIndexShardSnapshot snapshot() { return snapshot; } + public boolean isRecoveryDone() { + if (recovered.get()) { + return true; + } + final RecoveryState recoveryState = this.recoveryState; + if (recoveryState != null && recoveryState.getStage() == RecoveryState.Stage.DONE) { + return recovered.compareAndSet(false, true); + } + return false; + } + private List files() { if (loaded == false) { return List.of(); @@ -461,7 +488,8 @@ public static Directory create( IndexSettings indexSettings, ShardPath shardPath, LongSupplier currentTimeNanosSupplier, - ThreadPool threadPool + ThreadPool threadPool, + BlobStoreCacheService blobStoreCacheService ) throws IOException { if (SNAPSHOT_REPOSITORY_SETTING.exists(indexSettings.getSettings()) == false @@ -489,7 +517,8 @@ public static Directory create( ); } - final Repository repository = repositories.repository(SNAPSHOT_REPOSITORY_SETTING.get(indexSettings.getSettings())); + final String repositoryName = SNAPSHOT_REPOSITORY_SETTING.get(indexSettings.getSettings()); + final Repository repository = repositories.repository(repositoryName); if (repository instanceof BlobStoreRepository == false) { throw new IllegalArgumentException("Repository [" + repository + "] is not searchable"); } @@ -519,6 +548,8 @@ public static Directory create( new SearchableSnapshotDirectory( lazyBlobContainer::getOrCompute, lazySnapshot::getOrCompute, + blobStoreCacheService, + repositoryName, snapshotId, indexId, shardPath.getShardId(), @@ -558,6 +589,24 @@ public static SearchableSnapshotDirectory unwrapDirectory(Directory dir) { return null; } + public CachedBlob getCachedBlob(String name, long offset, int length) { + final CachedBlob cachedBlob = blobStoreCacheService.get(repository, name, blobStoreCachePath, offset); + assert cachedBlob == null || cachedBlob.from() <= offset; + assert cachedBlob == null || offset + length <= cachedBlob.to(); + return cachedBlob; + } + + public void putCachedBlob(String name, long offset, ReleasableBytesReference content) { + blobStoreCacheService.putAsync( + repository, + name, + blobStoreCachePath, + offset, + content, + ActionListener.wrap(success -> {}, failure -> {}) + ); + } + /** * A {@link FilterBlobContainer} that uses {@link BlobStoreRepository#maybeRateLimitRestores(InputStream)} to limit the rate at which * blobs are read from the repository. diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index 4ee8a191fd5d7..6c83e06cc5c0f 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -9,11 +9,15 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; +import org.elasticsearch.blobstore.cache.BlobStoreCacheService; +import org.elasticsearch.blobstore.cache.CachedBlob; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.SuppressForbidden; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.Channels; import org.elasticsearch.common.lease.Releasable; @@ -28,6 +32,7 @@ import java.io.InputStream; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; +import java.util.List; import java.util.Locale; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -35,6 +40,8 @@ import java.util.function.Predicate; import java.util.stream.IntStream; +import static org.elasticsearch.index.store.checksum.ChecksumBlobContainerIndexInput.checksumToBytesArray; + public class CachedBlobContainerIndexInput extends BaseSearchableSnapshotIndexInput { /** @@ -136,6 +143,58 @@ protected void readInternal(ByteBuffer b) throws IOException { final long position = getFilePointer() + this.offset; final int length = b.remaining(); + // We can detect that we're going to read the last 16 bytes (that contains the footer checksum) of the file. Such reads are often + // executed when opening a Directory and since we have the checksum in the snapshot metadata we can use it to fill the ByteBuffer. + if (length == CodecUtil.footerLength()) { + // we're comparing the global position (file pointer + offset) with the total file length (not the length of the index input + // which can be a slice) so the following code only applies when reading the footer of non-sliced index inputs (we're asserting + // that we are not reading from a clone as it would be surprising that Lucene uses a slice to verify the footer of a .cfs file) + if (position == fileInfo.length() - length) { + logger.trace("reading footer of file [{}] at position [{}], bypassing all caches", fileInfo.physicalName(), position); + b.put(checksumToBytesArray(fileInfo.checksum())); + assert b.remaining() == 0L; + assert isClone == false; + return; // TODO we should add this to DirectBlobContainerIndexInput too. + } + } + + final List> regions; + + // We prefer to use the index cache if the recovery is not done yet + if (directory.isRecoveryDone() == false) { + // We try to use the snapshot blob cache if: + // - we're reading the first N bytes of the file + final boolean isStartOfFile = (position + length <= BlobStoreCacheService.DEFAULT_SIZE); + // - the file is small enough to be fully cached in the blob cache + final boolean canBeFullyCached = (fileInfo.length() <= (BlobStoreCacheService.DEFAULT_SIZE * 2)); + + if (canBeFullyCached || isStartOfFile) { + final CachedBlob cachedBlob = directory.getCachedBlob(fileInfo.physicalName(), 0L, length); + if (cachedBlob != null) { + logger.trace( + "reading [{}] bytes of file [{}] at position [{}] using index cache", + length, + fileInfo.physicalName(), + position + ); + b.put(BytesReference.toBytes(cachedBlob.bytes().slice(Math.toIntExact(position), length))); + return; + } + } + + // 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 regions of the file we would like to have the next time. The regions are expressed as tuple of + // {position, length} where position is relative to the file. + if (canBeFullyCached) { + // if the index input is smaller than twice the size of the blob cache, it will be fully indexed + regions = List.of(Tuple.tuple(0L, Math.toIntExact(fileInfo.length()))); + } else { + regions = List.of(Tuple.tuple(0L, BlobStoreCacheService.DEFAULT_SIZE)); + } + } else { + regions = List.of(); + } + int totalBytesRead = 0; while (totalBytesRead < length) { final long pos = position + totalBytesRead; @@ -159,7 +218,10 @@ protected void readInternal(ByteBuffer b) throws IOException { read = readCacheFile(channel, pos, b); } return read; - }, this::writeCacheFile, directory.cacheFetchAsyncExecutor()).get(); + }, + (channel, from, to, progressUpdater) -> writeCacheFile(channel, from, to, progressUpdater, regions, logger), + directory.cacheFetchAsyncExecutor() + ).get(); } } catch (final Exception e) { if (e instanceof AlreadyClosedException || (e.getCause() != null && e.getCause() instanceof AlreadyClosedException)) { @@ -346,8 +408,14 @@ private int readCacheFile(final FileChannel fc, final long position, final ByteB return bytesRead; } - private void writeCacheFile(final FileChannel fc, final long start, final long end, final Consumer progressUpdater) - throws IOException { + private void writeCacheFile( + final FileChannel fc, + final long start, + final long end, + final Consumer progressUpdater, + final List> cacheableRegions, + final Logger logger + ) throws IOException { assert assertFileChannelOpen(fc); assert assertCurrentThreadMayWriteCacheFile(); final long length = end - start; @@ -357,7 +425,7 @@ private void writeCacheFile(final FileChannel fc, final long start, final long e long bytesCopied = 0L; long remaining = end - start; final long startTimeNanos = stats.currentTimeNanos(); - try (InputStream input = openInputStream(start, length)) { + try (InputStream input = openInputStream(start, length, cacheableRegions, directory::putCachedBlob, logger)) { while (remaining > 0L) { final int bytesRead = readSafe(input, copyBuffer, start, end, remaining, cacheFileReference); positionalWrite(fc, start + bytesCopied, ByteBuffer.wrap(copyBuffer, 0, bytesRead)); diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/checksum/ChecksumBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/checksum/ChecksumBlobContainerIndexInput.java index 76c9017ba84e1..af9c6e69813fc 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/checksum/ChecksumBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/checksum/ChecksumBlobContainerIndexInput.java @@ -131,14 +131,18 @@ private static void ensureReadOnceChecksumContext(IOContext context) { * @throws IOException if something goes wrong when creating the {@link ChecksumBlobContainerIndexInput} */ public static ChecksumBlobContainerIndexInput create(String name, long length, String checksum, IOContext context) throws IOException { + return new ChecksumBlobContainerIndexInput(name, length, checksumToBytesArray(checksum), context); + } + + public static byte[] checksumToBytesArray(String checksum) throws IOException { final ByteBuffersDataOutput out = new ByteBuffersDataOutput(); - try (IndexOutput output = new ByteBuffersIndexOutput(out, "tmp", name)) { + try (IndexOutput output = new ByteBuffersIndexOutput(out, "footerChecksumToBytesArray", "tmp")) { // reverse CodecUtil.writeFooter() output.writeInt(CodecUtil.FOOTER_MAGIC); output.writeInt(0); output.writeLong(Long.parseLong(checksum, Character.MAX_RADIX)); output.close(); - return new ChecksumBlobContainerIndexInput(name, length, out.toArrayCopy(), context); + return out.toArrayCopy(); } } } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotIndexEventListener.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotIndexEventListener.java index 71183bfba389a..2e4e96b929d49 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotIndexEventListener.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotIndexEventListener.java @@ -34,7 +34,7 @@ private static void ensureSnapshotIsLoaded(IndexShard indexShard) { final SearchableSnapshotDirectory directory = SearchableSnapshotDirectory.unwrapDirectory(indexShard.store().directory()); assert directory != null; - final boolean success = directory.loadSnapshot(); + final boolean success = directory.loadSnapshot(indexShard.recoveryState()); assert directory.listAll().length > 0 : "expecting directory listing to be non-empty"; assert success || indexShard.routingEntry() diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshots.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshots.java index f84f64a8beaa8..45bae91232e88 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshots.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshots.java @@ -8,6 +8,7 @@ import org.apache.lucene.util.SetOnce; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.blobstore.cache.BlobStoreCacheService; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.node.DiscoveryNodes; @@ -31,6 +32,7 @@ import org.elasticsearch.index.engine.ReadOnlyEngine; import org.elasticsearch.index.store.SearchableSnapshotDirectory; import org.elasticsearch.index.translog.TranslogStats; +import org.elasticsearch.indices.SystemIndexDescriptor; import org.elasticsearch.license.LicenseUtils; import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.plugins.ActionPlugin; @@ -38,6 +40,7 @@ import org.elasticsearch.plugins.EnginePlugin; import org.elasticsearch.plugins.IndexStorePlugin; import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.SystemIndexPlugin; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestHandler; @@ -76,12 +79,13 @@ import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.CACHE_PREWARMING_THREAD_POOL_SETTING; import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.SEARCHABLE_SNAPSHOTS_FEATURE_ENABLED; import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.CACHE_FETCH_ASYNC_THREAD_POOL_NAME; +import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.SNAPSHOT_BLOB_CACHE_INDEX; import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.SNAPSHOT_DIRECTORY_FACTORY_KEY; /** * Plugin for Searchable Snapshots feature */ -public class SearchableSnapshots extends Plugin implements IndexStorePlugin, EnginePlugin, ActionPlugin, ClusterPlugin { +public class SearchableSnapshots extends Plugin implements IndexStorePlugin, EnginePlugin, ActionPlugin, ClusterPlugin, SystemIndexPlugin { public static final Setting SNAPSHOT_REPOSITORY_SETTING = Setting.simpleString( "index.store.snapshot.repository_name", @@ -137,6 +141,7 @@ public class SearchableSnapshots extends Plugin implements IndexStorePlugin, Eng ); private volatile Supplier repositoriesServiceSupplier; + private final SetOnce blobStoreCacheService = new SetOnce<>(); private final SetOnce cacheService = new SetOnce<>(); private final SetOnce threadPool = new SetOnce<>(); private final Settings settings; @@ -190,7 +195,14 @@ public Collection createComponents( this.cacheService.set(cacheService); this.repositoriesServiceSupplier = repositoriesServiceSupplier; this.threadPool.set(threadPool); - return List.of(cacheService); + final BlobStoreCacheService blobStoreCacheService = new BlobStoreCacheService( + clusterService, + threadPool, + client, + SNAPSHOT_BLOB_CACHE_INDEX + ); + this.blobStoreCacheService.set(blobStoreCacheService); + return List.of(cacheService, blobStoreCacheService); } else { this.repositoriesServiceSupplier = () -> { assert false : "searchable snapshots are disabled"; @@ -207,6 +219,11 @@ public void onIndexModule(IndexModule indexModule) { } } + @Override + public Collection getSystemIndexDescriptors(Settings settings) { + return List.of(new SystemIndexDescriptor(SNAPSHOT_BLOB_CACHE_INDEX, "Contains cached data of blob store repositories")); + } + @Override public Map getDirectoryFactories() { if (SEARCHABLE_SNAPSHOTS_FEATURE_ENABLED) { @@ -217,7 +234,17 @@ public Map getDirectoryFactories() { assert cache != null; final ThreadPool threadPool = this.threadPool.get(); assert threadPool != null; - return SearchableSnapshotDirectory.create(repositories, cache, indexSettings, shardPath, System::nanoTime, threadPool); + final BlobStoreCacheService blobCache = blobStoreCacheService.get(); + assert blobCache != null; + return SearchableSnapshotDirectory.create( + repositories, + cache, + indexSettings, + shardPath, + System::nanoTime, + threadPool, + blobCache + ); }); } else { return Map.of(); diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStreamTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStreamTests.java new file mode 100644 index 0000000000000..0f9553290b1a5 --- /dev/null +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStreamTests.java @@ -0,0 +1,218 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.blobstore.cache; + +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.bytes.ReleasableBytesReference; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.ByteArray; +import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.common.util.MockPageCacheRecycler; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.test.ESTestCase; + +import java.io.ByteArrayInputStream; +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Arrays; + +import static org.elasticsearch.blobstore.cache.BlobStoreCacheService.DEFAULT_SIZE; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; + +public class CopyOnReadInputStreamTests extends ESTestCase { + + private final MockBigArrays bigArrays = new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), new NoneCircuitBreakerService()); + + public void testCopyOnReadInputStream() throws Exception { + final SetOnce onSuccess = new SetOnce<>(); + final SetOnce onFailure = new SetOnce<>(); + final ActionListener listener = ActionListener.wrap(onSuccess::set, onFailure::set); + + final byte[] blobContent = randomByteArray(); + final ByteArray byteArray = bigArrays.newByteArray(DEFAULT_SIZE); + + final int maxBytesToRead = randomIntBetween(0, blobContent.length); + final InputStream stream = new CopyOnReadInputStream(new ByteArrayInputStream(blobContent), byteArray, listener); + randomReads(stream, maxBytesToRead); + stream.close(); + + final ReleasableBytesReference releasable = onSuccess.get(); + assertThat(releasable, notNullValue()); + assertThat(releasable.length(), equalTo((int) Math.min(maxBytesToRead, byteArray.size()))); + assertArrayEquals(Arrays.copyOfRange(blobContent, 0, releasable.length()), BytesReference.toBytes(releasable)); + releasable.close(); + + final Exception failure = onFailure.get(); + assertThat(failure, nullValue()); + } + + public void testCopyOnReadInputStreamDoesNotCopyMoreThanByteArraySize() throws Exception { + final SetOnce onSuccess = new SetOnce<>(); + final SetOnce onFailure = new SetOnce<>(); + final ActionListener listener = ActionListener.wrap(onSuccess::set, onFailure::set); + + final byte[] blobContent = randomByteArray(); + + final ByteArray byteArray = bigArrays.newByteArray(randomIntBetween(0, DEFAULT_SIZE)); + final InputStream stream = new CopyOnReadInputStream(new ByteArrayInputStream(blobContent), byteArray, listener); + randomReads(stream, blobContent.length); + stream.close(); + + final ReleasableBytesReference releasable = onSuccess.get(); + assertThat(releasable, notNullValue()); + assertThat(releasable.length(), equalTo(Math.toIntExact(Math.min(blobContent.length, byteArray.size())))); + assertArrayEquals(Arrays.copyOfRange(blobContent, 0, releasable.length()), BytesReference.toBytes(releasable)); + releasable.close(); + + final Exception failure = onFailure.get(); + assertThat(failure, nullValue()); + } + + public void testCopyOnReadWithFailure() throws Exception { + final SetOnce onSuccess = new SetOnce<>(); + final SetOnce onFailure = new SetOnce<>(); + final ActionListener listener = ActionListener.wrap(onSuccess::set, onFailure::set); + + final byte[] blobContent = new byte[0]; + randomByteArray(); + + final int failAfterNBytesRead = randomIntBetween(0, Math.max(0, blobContent.length - 1)); + final InputStream erroneousStream = new FailAfterNBytesInputStream(new ByteArrayInputStream(blobContent), failAfterNBytesRead); + + final int byteSize = randomIntBetween(0, DEFAULT_SIZE); + try (InputStream stream = new CopyOnReadInputStream(erroneousStream, bigArrays.newByteArray(byteSize), listener)) { + IOException exception = expectThrows(IOException.class, () -> randomReads(stream, Math.max(1, blobContent.length))); + assertThat(exception.getMessage(), containsString("Fail to access bytes")); + } + + if (failAfterNBytesRead < byteSize) { + Exception failure = onFailure.get(); + assertThat(failure, notNullValue()); + assertThat(failure.getMessage(), containsString("Fail to access bytes")); + assertThat(onSuccess.get(), nullValue()); + + } else { + ReleasableBytesReference releasable = onSuccess.get(); + assertThat(releasable, notNullValue()); + assertArrayEquals(Arrays.copyOfRange(blobContent, 0, byteSize), BytesReference.toBytes(releasable)); + assertThat(onFailure.get(), nullValue()); + releasable.close(); + } + } + + private static byte[] randomByteArray() { + return randomByteArrayOfLength(randomIntBetween(0, frequently() ? DEFAULT_SIZE : 1 << 20)); // rarely up to 1mb; + } + + private void randomReads(final InputStream stream, final int maxBytesToRead) throws IOException { + int remaining = maxBytesToRead; + while (remaining > 0) { + int read; + switch (randomInt(3)) { + case 0: // single byte read + read = stream.read(); + if (read != -1) { + remaining--; + } + break; + case 1: // buffered read with fixed buffer offset/length + read = stream.read(new byte[randomIntBetween(1, remaining)]); + if (read != -1) { + remaining -= read; + } + break; + case 2: // buffered read with random buffer offset/length + final byte[] tmp = new byte[randomIntBetween(1, remaining)]; + final int off = randomIntBetween(0, tmp.length - 1); + read = stream.read(tmp, off, randomIntBetween(1, Math.min(1, tmp.length - off))); + if (read != -1) { + remaining -= read; + } + break; + + case 3: // mark & reset with intermediate skip() + final int toSkip = randomIntBetween(1, remaining); + stream.mark(toSkip); + stream.skip(toSkip); + stream.reset(); + break; + default: + fail("Unsupported test condition in " + getTestName()); + } + } + } + + /** + * InputStream that throws an IOException once byte at position N is read or skipped + */ + private static class FailAfterNBytesInputStream extends FilterInputStream { + + private final long failAfter; + + private long reads; + private long mark; + + FailAfterNBytesInputStream(InputStream in, long failAfter) { + super(in); + this.failAfter = failAfter; + } + + void ensureCanAccessMoreBytes() throws IOException { + if (failAfter <= reads) { + throw new IOException("Fail to access bytes after [" + failAfter + "]"); + } + } + + @Override + public int read() throws IOException { + ensureCanAccessMoreBytes(); + final int read = super.read(); + if (read != -1) { + reads++; + } + return read; + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + ensureCanAccessMoreBytes(); + final int read = super.read(b, off, Math.min(len, Math.toIntExact(failAfter - reads))); + if (read != -1) { + reads += read; + } + return read; + } + + @Override + public long skip(long n) throws IOException { + ensureCanAccessMoreBytes(); + final long skipped = super.skip(Math.min(n, Math.toIntExact(failAfter - reads))); + if (skipped > 0L) { + reads += skipped; + } + return skipped; + } + + @Override + public synchronized void reset() throws IOException { + super.reset(); + reads = mark; + } + + @Override + public synchronized void mark(int readlimit) { + super.mark(readlimit); + mark = reads; + } + } +} diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java new file mode 100644 index 0000000000000..73362087747fa --- /dev/null +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java @@ -0,0 +1,659 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.blobstore.cache; + +import org.apache.lucene.index.IndexFileNames; +import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse; +import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.OriginSettingClient; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.IndexRoutingTable; +import org.elasticsearch.cluster.routing.RoutingNode; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.ShardsIterator; +import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; +import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider; +import org.elasticsearch.cluster.routing.allocation.decider.Decision; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.blobstore.BlobContainer; +import org.elasticsearch.common.blobstore.BlobPath; +import org.elasticsearch.common.blobstore.BlobStore; +import org.elasticsearch.common.blobstore.support.FilterBlobContainer; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.io.Streams; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +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.index.IndexNotFoundException; +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; +import org.elasticsearch.indices.recovery.RecoverySettings; +import org.elasticsearch.plugins.ClusterPlugin; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.PluginsService; +import org.elasticsearch.plugins.RepositoryPlugin; +import org.elasticsearch.repositories.Repository; +import org.elasticsearch.repositories.fs.FsRepository; +import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.test.InternalTestCluster; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.core.ClientHelper; +import org.elasticsearch.xpack.searchablesnapshots.BaseSearchableSnapshotsIntegTestCase; +import org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots; +import org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants; +import org.elasticsearch.xpack.searchablesnapshots.cache.CacheService; + +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.function.BiConsumer; + +import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.INDEX_SHARD_SNAPSHOT_FORMAT; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; +import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.CACHE_FETCH_ASYNC_THREAD_POOL_NAME; +import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.CACHE_PREWARMING_THREAD_POOL_NAME; +import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.SNAPSHOT_BLOB_CACHE_INDEX; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; + +public class SearchableSnapshotsBlobStoreCacheIntegTests extends BaseSearchableSnapshotsIntegTestCase { + + @Override + protected Collection> nodePlugins() { + final List> plugins = new ArrayList<>(); + plugins.add(WaitForSnapshotBlobCacheShardsActivePlugin.class); + plugins.add(TrackingRepositoryPlugin.class); + plugins.addAll(super.nodePlugins()); + return List.copyOf(plugins); + } + + @Override + protected int numberOfReplicas() { + return 0; + } + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal)) + // Use a cache range size setting aligned with BufferedIndexInput's buffer size and BlobStoreCacheService's default size + // TODO randomized this + .put( + CacheService.SNAPSHOT_CACHE_RANGE_SIZE_SETTING.getKey(), + new ByteSizeValue(BlobStoreCacheService.DEFAULT_SIZE, ByteSizeUnit.BYTES) + ) + .put(CacheService.SNAPSHOT_CACHE_SIZE_SETTING.getKey(), new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES)) + .build(); + } + + public void testBlobStoreCache() throws Exception { + final String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName); + + final List indexRequestBuilders = new ArrayList<>(); + for (int i = scaledRandomIntBetween(0, 10_000); i >= 0; i--) { + indexRequestBuilders.add(client().prepareIndex(indexName).setSource("text", randomUnicodeOfLength(10), "num", i)); + } + indexRandom(true, false, true, indexRequestBuilders); + final long numberOfDocs = indexRequestBuilders.size(); + final NumShards numberOfShards = getNumShards(indexName); + + final ForceMergeResponse forceMergeResponse = client().admin().indices().prepareForceMerge(indexName).setMaxNumSegments(1).get(); + assertThat(forceMergeResponse.getSuccessfulShards(), equalTo(numberOfShards.totalNumShards)); + assertThat(forceMergeResponse.getFailedShards(), equalTo(0)); + + final String repositoryName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + final Path repositoryLocation = randomRepoPath(); + createFsRepository(repositoryName, repositoryLocation); + + final SnapshotId snapshot = createSnapshot(repositoryName, List.of(indexName)); + assertAcked(client().admin().indices().prepareDelete(indexName)); + + // extract the list of blobs per shard from the snapshot directory on disk + final Map blobsInSnapshot = blobsInSnapshot(repositoryLocation, snapshot.getUUID()); + assertThat("Failed to load all shard snapshot metadata files", blobsInSnapshot.size(), equalTo(numberOfShards.numPrimaries)); + + // register a new repository that can track blob read operations + assertAcked(client().admin().cluster().prepareDeleteRepository(repositoryName)); + createRepository( + repositoryName, + TrackingRepositoryPlugin.TRACKING, + Settings.builder().put(FsRepository.LOCATION_SETTING.getKey(), repositoryLocation).build(), + false + ); + assertBusy(this::ensureClusterStateConsistency); + + expectThrows( + IndexNotFoundException.class, + ".snapshot-blob-cache system index should not be created yet", + () -> systemClient().admin().indices().prepareGetIndex().addIndices(SNAPSHOT_BLOB_CACHE_INDEX).get() + ); + + final boolean usePrewarming = false; // TODO randomize this and adapt test + + logger.info("--> mount snapshot [{}] as an index for the first time", snapshot); + final String restoredIndex = mountSnapshot( + repositoryName, + snapshot.getName(), + indexName, + Settings.builder() + .put(SearchableSnapshots.SNAPSHOT_CACHE_ENABLED_SETTING.getKey(), true) + .put(SearchableSnapshots.SNAPSHOT_CACHE_PREWARM_ENABLED_SETTING.getKey(), usePrewarming) + .build() + ); + ensureGreen(restoredIndex); + ensureExecutorsAreIdle(); + + logger.info("--> verifying cached documents in system index [{}]", SNAPSHOT_BLOB_CACHE_INDEX); + assertCachedBlobsInSystemIndex(repositoryName, blobsInSnapshot); + + refreshSystemIndex(); + final long numberOfCachedBlobs = systemClient().prepareSearch(SNAPSHOT_BLOB_CACHE_INDEX).get().getHits().getTotalHits().value; + + ensureBlobStoreRepositoriesWithActiveShards( + restoredIndex, + (nodeId, blobStore) -> assertThat( + "Blob read operations should have been executed on node [" + nodeId + ']', + blobStore.numberOfReads(), + greaterThan(0L) + ) + ); + + logger.info("--> verifying documents in index [{}]", restoredIndex); + assertHitCount(client().prepareSearch(restoredIndex).setSize(0).setTrackTotalHits(true).get(), numberOfDocs); + assertHitCount( + client().prepareSearch(restoredIndex) + .setQuery(QueryBuilders.rangeQuery("num").lte(numberOfDocs)) + .setSize(0) + .setTrackTotalHits(true) + .get(), + numberOfDocs + ); + assertHitCount( + client().prepareSearch(restoredIndex) + .setQuery(QueryBuilders.rangeQuery("num").gt(numberOfDocs + 1)) + .setSize(0) + .setTrackTotalHits(true) + .get(), + 0L + ); + + assertAcked(client().admin().indices().prepareDelete(restoredIndex)); + resetTrackedFiles(); + + logger.info("--> mount snapshot [{}] as an index for the second time", snapshot); + final String restoredAgainIndex = mountSnapshot( + repositoryName, + snapshot.getName(), + indexName, + Settings.builder() + .put(SearchableSnapshots.SNAPSHOT_CACHE_ENABLED_SETTING.getKey(), true) + .put(SearchableSnapshots.SNAPSHOT_CACHE_PREWARM_ENABLED_SETTING.getKey(), usePrewarming) + .build() + ); + ensureGreen(restoredAgainIndex); + ensureExecutorsAreIdle(); + + logger.info("--> verifying documents in index [{}]", restoredAgainIndex); + assertHitCount(client().prepareSearch(restoredAgainIndex).setSize(0).setTrackTotalHits(true).get(), numberOfDocs); + assertHitCount( + client().prepareSearch(restoredAgainIndex) + .setQuery(QueryBuilders.rangeQuery("num").lte(numberOfDocs)) + .setSize(0) + .setTrackTotalHits(true) + .get(), + numberOfDocs + ); + assertHitCount( + client().prepareSearch(restoredAgainIndex) + .setQuery(QueryBuilders.rangeQuery("num").gt(numberOfDocs + 1)) + .setSize(0) + .setTrackTotalHits(true) + .get(), + 0L + ); + + logger.info("--> verifying cached documents (again) in system index [{}]", SNAPSHOT_BLOB_CACHE_INDEX); + assertCachedBlobsInSystemIndex(repositoryName, blobsInSnapshot); + + logger.info("--> verifying that no cached blobs were indexed in system index [{}]", SNAPSHOT_BLOB_CACHE_INDEX); + refreshSystemIndex(); + assertHitCount(systemClient().prepareSearch(SNAPSHOT_BLOB_CACHE_INDEX).setSize(0).get(), numberOfCachedBlobs); + + logger.info("--> verifying blobs read from the repository"); + assertBlobsReadFromRemoteRepository(restoredAgainIndex, blobsInSnapshot); + + resetTrackedFiles(); + + logger.info("--> restarting cluster"); + internalCluster().fullRestart(new InternalTestCluster.RestartCallback() { + @Override + public Settings onNodeStopped(String nodeName) throws Exception { + return Settings.builder() + .put(super.onNodeStopped(nodeName)) + .put(WaitForSnapshotBlobCacheShardsActivePlugin.ENABLED.getKey(), true) + .build(); + } + }); + ensureGreen(restoredAgainIndex); + ensureExecutorsAreIdle(); + + logger.info("--> verifying documents in index [{}]", restoredAgainIndex); + assertHitCount(client().prepareSearch(restoredAgainIndex).setSize(0).setTrackTotalHits(true).get(), numberOfDocs); + assertHitCount(client().prepareSearch(restoredAgainIndex).setSize(0).setTrackTotalHits(true).get(), numberOfDocs); + assertHitCount( + client().prepareSearch(restoredAgainIndex) + .setQuery(QueryBuilders.rangeQuery("num").lte(numberOfDocs)) + .setSize(0) + .setTrackTotalHits(true) + .get(), + numberOfDocs + ); + assertHitCount( + client().prepareSearch(restoredAgainIndex) + .setQuery(QueryBuilders.rangeQuery("num").gt(numberOfDocs + 1)) + .setSize(0) + .setTrackTotalHits(true) + .get(), + 0L + ); + + logger.info("--> verifying cached documents (after restart) in system index [{}]", SNAPSHOT_BLOB_CACHE_INDEX); + assertCachedBlobsInSystemIndex(repositoryName, blobsInSnapshot); + + logger.info("--> verifying that no cached blobs were indexed in system index [{}] after restart", SNAPSHOT_BLOB_CACHE_INDEX); + assertHitCount(systemClient().prepareSearch(SNAPSHOT_BLOB_CACHE_INDEX).setSize(0).get(), numberOfCachedBlobs); + + logger.info("--> verifying blobs read from the repository after restart"); + // Without the WaitForSnapshotBlobCacheShardsActivePlugin this would fail + assertBlobsReadFromRemoteRepository(restoredAgainIndex, blobsInSnapshot); + + // TODO would be great to test when the index is frozen + } + + /** + * @return a {@link Client} that can be used to query the blob store cache system index + */ + private Client systemClient() { + return new OriginSettingClient(client(), ClientHelper.SEARCHABLE_SNAPSHOTS_ORIGIN); + } + + private void refreshSystemIndex() { + final RefreshResponse refreshResponse = systemClient().admin().indices().prepareRefresh(SNAPSHOT_BLOB_CACHE_INDEX).get(); + assertThat(refreshResponse.getSuccessfulShards(), greaterThan(0)); + assertThat(refreshResponse.getFailedShards(), equalTo(0)); + } + + /** + * Reads a repository location on disk and extracts the list of blobs for each shards + */ + private Map blobsInSnapshot(Path repositoryLocation, String snapshotId) throws IOException { + final Map blobsPerShard = new HashMap<>(); + Files.walkFileTree(repositoryLocation.resolve("indices"), new SimpleFileVisitor<>() { + @Override + public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException { + final String fileName = file.getFileName().toString(); + if (fileName.equals("snap-" + snapshotId + ".dat")) { + blobsPerShard.put( + String.join( + "/", + snapshotId, + file.getParent().getParent().getFileName().toString(), + file.getParent().getFileName().toString() + ), + INDEX_SHARD_SNAPSHOT_FORMAT.deserialize(fileName, xContentRegistry(), Streams.readFully(Files.newInputStream(file))) + ); + } + return FileVisitResult.CONTINUE; + } + }); + return Map.copyOf(blobsPerShard); + } + + private void ensureExecutorsAreIdle() throws Exception { + assertBusy(() -> { + for (ThreadPool threadPool : internalCluster().getDataNodeInstances(ThreadPool.class)) { + for (String threadPoolName : List.of(CACHE_FETCH_ASYNC_THREAD_POOL_NAME, CACHE_PREWARMING_THREAD_POOL_NAME)) { + final ThreadPoolExecutor executor = (ThreadPoolExecutor) threadPool.executor(threadPoolName); + assertThat(threadPoolName, executor.getQueue().size(), equalTo(0)); + assertThat(threadPoolName, executor.getActiveCount(), equalTo(0)); + } + } + }); + } + + private void assertCachedBlobsInSystemIndex(final String repositoryName, final Map blobsInSnapshot) + throws Exception { + assertBusy(() -> { + refreshSystemIndex(); + + long numberOfCachedBlobs = 0L; + for (Map.Entry blob : blobsInSnapshot.entrySet()) { + for (BlobStoreIndexShardSnapshot.FileInfo fileInfo : blob.getValue().indexFiles()) { + if (fileInfo.name().startsWith("__") == false) { + continue; + } + + final String path = String.join("/", repositoryName, blob.getKey(), fileInfo.physicalName()); + if (fileInfo.length() <= BlobStoreCacheService.DEFAULT_SIZE * 2) { + // file has been fully cached + final GetResponse getResponse = systemClient().prepareGet(SNAPSHOT_BLOB_CACHE_INDEX, path + "/@0").get(); + assertThat( + "Blob [" + fileInfo + "] should have been indexed in the blob cache system index as a single document", + getResponse.isExists(), + is(true) + ); + final CachedBlob cachedBlob = CachedBlob.fromSource(getResponse.getSourceAsMap()); + assertThat(cachedBlob.from(), equalTo(0L)); + assertThat(cachedBlob.to(), equalTo(fileInfo.length())); + assertThat((long) cachedBlob.length(), equalTo(fileInfo.length())); + numberOfCachedBlobs += 1; + + } else { + // first region of file has been cached + GetResponse getResponse = systemClient().prepareGet(SNAPSHOT_BLOB_CACHE_INDEX, path + "/@0").get(); + assertThat( + "First region of blob [" + fileInfo + "] should have been indexed in the blob cache system index", + getResponse.isExists(), + is(true) + ); + + CachedBlob cachedBlob = CachedBlob.fromSource(getResponse.getSourceAsMap()); + assertThat(cachedBlob.from(), equalTo(0L)); + assertThat(cachedBlob.to(), equalTo((long) BlobStoreCacheService.DEFAULT_SIZE)); + assertThat(cachedBlob.length(), equalTo(BlobStoreCacheService.DEFAULT_SIZE)); + numberOfCachedBlobs += 1; + } + } + } + + refreshSystemIndex(); + assertHitCount(systemClient().prepareSearch(SNAPSHOT_BLOB_CACHE_INDEX).setSize(0).get(), numberOfCachedBlobs); + }); + } + + private void assertBlobsReadFromRemoteRepository( + final String indexName, + final Map blobsInSnapshot + ) { + ensureBlobStoreRepositoriesWithActiveShards(indexName, (nodeId, blobStore) -> { + for (Map.Entry>> blob : blobStore.blobs.entrySet()) { + final String blobName = blob.getKey(); + + if (blobName.endsWith(".dat") || blobName.equals("index-0")) { + // The snapshot metadata files are accessed when recovering from the snapshot during restore and do not benefit from + // the snapshot blob cache as the files are accessed outside of a searchable snapshot directory + assertThat( + blobName + " should be fully read from the beginning", + blob.getValue().stream().allMatch(read -> read.v1() == 0L), + is(true) + ); + // TODO assert it is read til the end + + } else { + BlobStoreIndexShardSnapshot.FileInfo blobInfo = null; + for (BlobStoreIndexShardSnapshot blobs : blobsInSnapshot.values()) { + for (BlobStoreIndexShardSnapshot.FileInfo fileInfo : blobs.indexFiles()) { + for (int i = 0; i < fileInfo.numberOfParts(); i++) { + if (blobName.endsWith(fileInfo.partName(i))) { + blobInfo = fileInfo; + break; + } + } + } + } + assertThat("Unable to find blob " + blobName + " in the blobs on disk", blobInfo, notNullValue()); + + final String fileExtension = IndexFileNames.getExtension(blobInfo.physicalName()); + assertThat( + "Only compound files can be read from the blob store after blob store cache is populated", + fileExtension, + equalTo("cfs") + ); + } + } + }); + } + + /** + * Returns the {@link TrackingRepositoryPlugin} instance on a given node. + */ + private TrackingRepositoryPlugin getTrackingRepositoryInstance(String node) { + DiscoveryNode discoveryNode = clusterService().state().nodes().resolveNode(node); + assertThat("Cannot find node " + node, discoveryNode, notNullValue()); + + PluginsService pluginsService = internalCluster().getInstance(PluginsService.class, discoveryNode.getName()); + assertThat("Cannot find PluginsService on node " + node, pluginsService, notNullValue()); + + List trackingRepositoryPlugins = pluginsService.filterPlugins(TrackingRepositoryPlugin.class); + assertThat("List of TrackingRepositoryPlugin is null on node " + node, trackingRepositoryPlugins, notNullValue()); + assertThat("List of TrackingRepositoryPlugin is empty on node " + node, trackingRepositoryPlugins, hasSize(1)); + + TrackingRepositoryPlugin trackingRepositoryPlugin = trackingRepositoryPlugins.get(0); + assertThat("TrackingRepositoryPlugin is null on node " + node, trackingRepositoryPlugin, notNullValue()); + return trackingRepositoryPlugin; + } + + private void resetTrackedFiles() { + for (String nodeName : internalCluster().getNodeNames()) { + final TrackingRepositoryPlugin tracker = getTrackingRepositoryInstance(nodeName); + tracker.reset(); + assertThat(tracker.numberOfReads(), equalTo(0L)); + assertThat(tracker.blobs.size(), equalTo(0)); + } + } + + private void ensureBlobStoreRepositoriesWithActiveShards(String indexName, BiConsumer consumer) { + final ClusterState clusterState = clusterService().state(); + assertTrue(clusterState.metadata().hasIndex(indexName)); + assertTrue(SearchableSnapshotsConstants.isSearchableSnapshotStore(clusterState.metadata().index(indexName).getSettings())); + final IndexRoutingTable indexRoutingTable = clusterState.routingTable().index(indexName); + assertThat(indexRoutingTable, notNullValue()); + + final ShardsIterator shardsIterator = indexRoutingTable.randomAllActiveShardsIt(); + assertThat(shardsIterator.size(), greaterThanOrEqualTo(getNumShards(indexName).numPrimaries)); + + for (ShardRouting shardRouting : shardsIterator) { + consumer.accept(shardRouting.currentNodeId(), getTrackingRepositoryInstance(shardRouting.currentNodeId())); + } + } + + /** + * A plugin that allows to track the read operations on blobs + */ + public static class TrackingRepositoryPlugin extends Plugin implements RepositoryPlugin { + + static final String TRACKING = "tracking"; + + private final Map>> blobs = new ConcurrentHashMap<>(); + + long numberOfReads() { + return blobs.values().stream().flatMap(Collection::stream).mapToLong(Tuple::v2).sum(); + } + + void reset() { + blobs.clear(); + } + + @Override + public Map getRepositories( + Environment env, + NamedXContentRegistry namedXContentRegistry, + ClusterService clusterService, + RecoverySettings recoverySettings + ) { + return Collections.singletonMap( + TRACKING, + (metadata) -> new FsRepository(metadata, env, namedXContentRegistry, clusterService, recoverySettings) { + + @Override + protected BlobStore createBlobStore() throws Exception { + final BlobStore delegate = super.createBlobStore(); + return new BlobStore() { + @Override + public BlobContainer blobContainer(BlobPath path) { + return new TrackingFilesBlobContainer(delegate.blobContainer(path)); + } + + @Override + public void close() throws IOException { + delegate.close(); + } + }; + } + } + ); + } + + class TrackingFilesBlobContainer extends FilterBlobContainer { + + TrackingFilesBlobContainer(BlobContainer delegate) { + super(delegate); + } + + @Override + public InputStream readBlob(String blobName) throws IOException { + return new CountingInputStream(buildPath(blobName), 0L, super.readBlob(blobName)); + } + + @Override + public InputStream readBlob(String blobName, long position, long length) throws IOException { + return new CountingInputStream(buildPath(blobName), position, super.readBlob(blobName, position, length)); + } + + private String buildPath(String name) { + return path().buildAsString() + name; + } + + @Override + protected BlobContainer wrapChild(BlobContainer child) { + return new TrackingFilesBlobContainer(child); + } + } + + class CountingInputStream extends FilterInputStream { + + private final String name; + private final long offset; + + long bytesRead = 0L; + + protected CountingInputStream(String name, long offset, InputStream in) { + super(in); + this.name = name; + this.offset = offset; + } + + @Override + public int read() throws IOException { + final int result = in.read(); + if (result == -1) { + return result; + } + bytesRead += 1L; + return result; + } + + @Override + public int read(byte[] b, int offset, int len) throws IOException { + final int result = in.read(b, offset, len); + if (result == -1) { + return result; + } + bytesRead += len; + return result; + } + + @Override + public void close() throws IOException { + blobs.computeIfAbsent(name, n -> Collections.synchronizedList(new ArrayList<>())).add(Tuple.tuple(offset, bytesRead)); + super.close(); + } + } + } + + /** + * This plugin declares an {@link AllocationDecider} that forces searchable snapshot shards to be allocated after + * the primary shards of the snapshot blob cache index are started. This way we can ensure that searchable snapshot + * shards can use the snapshot blob cache index after the cluster restarted. + */ + public static class WaitForSnapshotBlobCacheShardsActivePlugin extends Plugin implements ClusterPlugin { + + public static Setting ENABLED = Setting.boolSetting( + "wait_for_snapshot_blob_cache_shards_active.enabled", + false, + Setting.Property.NodeScope + ); + + @Override + public List> getSettings() { + return List.of(ENABLED); + } + + @Override + public Collection createAllocationDeciders(Settings settings, ClusterSettings clusterSettings) { + if (ENABLED.get(settings) == false) { + return List.of(); + } + final String name = "wait_for_snapshot_blob_cache_shards_active"; + return List.of(new AllocationDecider() { + + @Override + public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { + return canAllocate(shardRouting, allocation); + } + + @Override + public Decision canAllocate(ShardRouting shardRouting, RoutingAllocation allocation) { + final IndexMetadata indexMetadata = allocation.metadata().index(shardRouting.index()); + if (SearchableSnapshotsConstants.isSearchableSnapshotStore(indexMetadata.getSettings()) == false) { + return allocation.decision(Decision.YES, name, "index is not a searchable snapshot shard - can allocate"); + } + if (allocation.metadata().hasIndex(SNAPSHOT_BLOB_CACHE_INDEX) == false) { + return allocation.decision(Decision.YES, name, SNAPSHOT_BLOB_CACHE_INDEX + " is not created yet"); + } + if (allocation.routingTable().hasIndex(SNAPSHOT_BLOB_CACHE_INDEX) == false) { + return allocation.decision(Decision.THROTTLE, name, SNAPSHOT_BLOB_CACHE_INDEX + " is not active yet"); + } + final IndexRoutingTable indexRoutingTable = allocation.routingTable().index(SNAPSHOT_BLOB_CACHE_INDEX); + if (indexRoutingTable.allPrimaryShardsActive() == false) { + return allocation.decision(Decision.THROTTLE, name, SNAPSHOT_BLOB_CACHE_INDEX + " is not active yet"); + } + return allocation.decision(Decision.YES, name, "primary shard for this replica is already active"); + } + }); + } + } +} diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryStatsTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryStatsTests.java index 7e5b1e7613695..91961d3eb301f 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryStatsTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryStatsTests.java @@ -9,7 +9,12 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.elasticsearch.Version; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.RecoverySource; +import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.common.TriConsumer; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.lucene.store.ESIndexInputTestCase; import org.elasticsearch.common.settings.Settings; @@ -22,7 +27,10 @@ import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo; import org.elasticsearch.index.store.cache.TestUtils; +import org.elasticsearch.index.store.cache.TestUtils.NoopBlobStoreCacheService; +import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; @@ -199,7 +207,7 @@ public void testDirectBytesReadsWithCache() { final IndexInputStats inputStats = directory.getStats(fileName); // account for internal buffered reads - final long bufferSize = BufferedIndexInput.bufferSize(ioContext); + final long bufferSize = BaseSearchableSnapshotIndexInput.DEFAULT_BUFFER_SIZE; final long remaining = input.length() % bufferSize; final long expectedTotal = input.length(); final long expectedCount = input.length() / bufferSize + (remaining > 0L ? 1L : 0L); @@ -316,7 +324,7 @@ public void testReadBytesContiguously() { final IndexInputStats inputStats = cacheDirectory.getStats(fileName); // account for the CacheBufferedIndexInput internal buffer - final long bufferSize = (long) BufferedIndexInput.bufferSize(ioContext); + final long bufferSize = BaseSearchableSnapshotIndexInput.DEFAULT_BUFFER_SIZE; final long remaining = input.length() % bufferSize; final long expectedTotal = input.length(); final long expectedCount = input.length() / bufferSize + (remaining > 0L ? 1L : 0L); @@ -381,7 +389,7 @@ public void testReadBytesNonContiguously() { input.readBytes(readBuffer, 0, size); // BufferedIndexInput tries to read as much bytes as possible - final long bytesRead = Math.min(BufferedIndexInput.bufferSize(ioContext), input.length() - randomPosition); + final long bytesRead = Math.min(BaseSearchableSnapshotIndexInput.DEFAULT_BUFFER_SIZE, input.length() - randomPosition); lastReadPosition = randomPosition + bytesRead; totalBytesRead += bytesRead; minBytesRead = (bytesRead < minBytesRead) ? bytesRead : minBytesRead; @@ -608,6 +616,23 @@ private static void executeTestCase( throw new UncheckedIOException(e); } final ShardPath shardPath = new ShardPath(false, shardDir, shardDir, shardId); + final DiscoveryNode discoveryNode = new DiscoveryNode("_id", buildNewFakeTransportAddress(), Version.CURRENT); + final RecoveryState recoveryState = new RecoveryState( + TestShardRouting.newShardRouting( + shardId, + discoveryNode.getId(), + true, + ShardRoutingState.INITIALIZING, + new RecoverySource.SnapshotRecoverySource( + UUIDs.randomBase64UUID(), + new Snapshot("_repo", snapshotId), + Version.CURRENT, + indexId + ) + ), + discoveryNode, + null + ); final Path cacheDir = createTempDir(); try ( @@ -615,6 +640,8 @@ private static void executeTestCase( SearchableSnapshotDirectory directory = new SearchableSnapshotDirectory( () -> blobContainer, () -> snapshot, + new NoopBlobStoreCacheService(), + "_repo", snapshotId, indexId, shardId, @@ -637,7 +664,7 @@ protected IndexInputStats createIndexInputStats(long fileLength) { cacheService.start(); assertThat(directory.getStats(fileName), nullValue()); - final boolean loaded = directory.loadSnapshot(); + final boolean loaded = directory.loadSnapshot(recoveryState); assertThat("Failed to load snapshot", loaded, is(true)); assertThat("Snapshot should be loaded", directory.snapshot(), notNullValue()); assertThat("BlobContainer should be loaded", directory.blobContainer(), notNullValue()); diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryTests.java index 519233f26915b..13f04bf73e5ab 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryTests.java @@ -38,6 +38,10 @@ import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.RepositoryMetadata; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.RecoverySource; +import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.common.CheckedBiConsumer; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.UUIDs; @@ -67,13 +71,16 @@ import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; import org.elasticsearch.index.store.cache.TestUtils; +import org.elasticsearch.index.store.cache.TestUtils.NoopBlobStoreCacheService; import org.elasticsearch.index.store.checksum.ChecksumBlobContainerIndexInput; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.recovery.RecoverySettings; +import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.blobstore.BlobStoreRepository; import org.elasticsearch.repositories.blobstore.BlobStoreTestUtil; import org.elasticsearch.repositories.fs.FsRepository; +import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ESTestCase; @@ -558,6 +565,23 @@ protected void assertSnapshotOrGenericThread() { throw new UncheckedIOException(e); } final ShardPath shardPath = new ShardPath(false, shardDir, shardDir, shardId); + final DiscoveryNode discoveryNode = new DiscoveryNode("_id", buildNewFakeTransportAddress(), Version.CURRENT); + final RecoveryState recoveryState = new RecoveryState( + TestShardRouting.newShardRouting( + shardId, + discoveryNode.getId(), + true, + ShardRoutingState.INITIALIZING, + new RecoverySource.SnapshotRecoverySource( + UUIDs.randomBase64UUID(), + new Snapshot("_repo", snapshotId), + Version.CURRENT, + indexId + ) + ), + discoveryNode, + null + ); final Path cacheDir = createTempDir(); final CacheService cacheService = TestUtils.createDefaultCacheService(); releasables.add(cacheService); @@ -567,6 +591,8 @@ protected void assertSnapshotOrGenericThread() { SearchableSnapshotDirectory snapshotDirectory = new SearchableSnapshotDirectory( () -> blobContainer, () -> snapshot, + new NoopBlobStoreCacheService(), + "_repo", snapshotId, indexId, shardId, @@ -581,7 +607,7 @@ protected void assertSnapshotOrGenericThread() { threadPool ) ) { - final boolean loaded = snapshotDirectory.loadSnapshot(); + final boolean loaded = snapshotDirectory.loadSnapshot(recoveryState); assertThat("Failed to load snapshot", loaded, is(true)); assertThat("Snapshot should be loaded", snapshotDirectory.snapshot(), sameInstance(snapshot)); assertThat("BlobContainer should be loaded", snapshotDirectory.blobContainer(), sameInstance(blobContainer)); @@ -655,12 +681,31 @@ public void testClearCache() throws Exception { throw new UncheckedIOException(e); } final ShardPath shardPath = new ShardPath(false, shardDir, shardDir, shardId); + final DiscoveryNode discoveryNode = new DiscoveryNode("_id", buildNewFakeTransportAddress(), Version.CURRENT); + final RecoveryState recoveryState = new RecoveryState( + TestShardRouting.newShardRouting( + shardId, + discoveryNode.getId(), + true, + ShardRoutingState.INITIALIZING, + new RecoverySource.SnapshotRecoverySource( + UUIDs.randomBase64UUID(), + new Snapshot("_repo", snapshotId), + Version.CURRENT, + indexId + ) + ), + discoveryNode, + null + ); final Path cacheDir = createTempDir(); final ThreadPool threadPool = new TestThreadPool(getTestName(), SearchableSnapshots.executorBuilders()); try ( SearchableSnapshotDirectory directory = new SearchableSnapshotDirectory( () -> blobContainer, () -> snapshot, + new NoopBlobStoreCacheService(), + "_repo", snapshotId, indexId, shardId, @@ -678,7 +723,7 @@ public void testClearCache() throws Exception { ) ) { - final boolean loaded = directory.loadSnapshot(); + final boolean loaded = directory.loadSnapshot(recoveryState); assertThat("Failed to load snapshot", loaded, is(true)); assertThat("Snapshot should be loaded", directory.snapshot(), sameInstance(snapshot)); assertThat("BlobContainer should be loaded", directory.blobContainer(), sameInstance(blobContainer)); @@ -731,7 +776,7 @@ public void testRequiresAdditionalSettings() { final IndexSettings indexSettings = new IndexSettings(IndexMetadata.builder("test").settings(settings).build(), Settings.EMPTY); expectThrows( IllegalArgumentException.class, - () -> SearchableSnapshotDirectory.create(null, null, indexSettings, null, null, null) + () -> SearchableSnapshotDirectory.create(null, null, indexSettings, null, null, null, null) ); } } diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInputTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInputTests.java index 08cc9bd8fd2fc..5ac12c051274a 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInputTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInputTests.java @@ -7,6 +7,11 @@ import org.apache.lucene.store.IndexInput; import org.elasticsearch.Version; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.RecoverySource; +import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.TestShardRouting; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.support.FilterBlobContainer; import org.elasticsearch.common.lucene.store.ESIndexInputTestCase; @@ -18,7 +23,10 @@ import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; import org.elasticsearch.index.store.SearchableSnapshotDirectory; import org.elasticsearch.index.store.StoreFileMetadata; +import org.elasticsearch.index.store.cache.TestUtils.NoopBlobStoreCacheService; +import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; @@ -93,11 +101,30 @@ public void testRandomReads() throws IOException { throw new UncheckedIOException(e); } final ShardPath shardPath = new ShardPath(false, shardDir, shardDir, shardId); + final DiscoveryNode discoveryNode = new DiscoveryNode("_id", buildNewFakeTransportAddress(), Version.CURRENT); + final RecoveryState recoveryState = new RecoveryState( + TestShardRouting.newShardRouting( + shardId, + discoveryNode.getId(), + true, + ShardRoutingState.INITIALIZING, + new RecoverySource.SnapshotRecoverySource( + UUIDs.randomBase64UUID(), + new Snapshot("_repo", snapshotId), + Version.CURRENT, + indexId + ) + ), + discoveryNode, + null + ); final Path cacheDir = createTempDir(); try ( SearchableSnapshotDirectory directory = new SearchableSnapshotDirectory( () -> blobContainer, () -> snapshot, + new NoopBlobStoreCacheService(), + "_repo", snapshotId, indexId, shardId, @@ -112,7 +139,7 @@ public void testRandomReads() throws IOException { threadPool ) ) { - final boolean loaded = directory.loadSnapshot(); + final boolean loaded = directory.loadSnapshot(recoveryState); assertThat("Failed to load snapshot", loaded, is(true)); assertThat("Snapshot should be loaded", directory.snapshot(), notNullValue()); assertThat("BlobContainer should be loaded", directory.blobContainer(), notNullValue()); @@ -176,11 +203,30 @@ public void testThrowsEOFException() throws IOException { throw new UncheckedIOException(e); } final ShardPath shardPath = new ShardPath(false, shardDir, shardDir, shardId); + final DiscoveryNode discoveryNode = new DiscoveryNode("_id", buildNewFakeTransportAddress(), Version.CURRENT); + final RecoveryState recoveryState = new RecoveryState( + TestShardRouting.newShardRouting( + shardId, + discoveryNode.getId(), + true, + ShardRoutingState.INITIALIZING, + new RecoverySource.SnapshotRecoverySource( + UUIDs.randomBase64UUID(), + new Snapshot("_repo", snapshotId), + Version.CURRENT, + indexId + ) + ), + discoveryNode, + null + ); final Path cacheDir = createTempDir(); try ( SearchableSnapshotDirectory searchableSnapshotDirectory = new SearchableSnapshotDirectory( () -> blobContainer, () -> snapshot, + new NoopBlobStoreCacheService(), + "_repo", snapshotId, indexId, shardId, @@ -192,7 +238,7 @@ public void testThrowsEOFException() throws IOException { threadPool ) ) { - final boolean loaded = searchableSnapshotDirectory.loadSnapshot(); + final boolean loaded = searchableSnapshotDirectory.loadSnapshot(recoveryState); assertThat("Failed to load snapshot", loaded, is(true)); assertThat("Snapshot should be loaded", searchableSnapshotDirectory.snapshot(), notNullValue()); assertThat("BlobContainer should be loaded", searchableSnapshotDirectory.blobContainer(), notNullValue()); diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/TestUtils.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/TestUtils.java index 5d71c085cf1a5..47c39b14ceb3d 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/TestUtils.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/TestUtils.java @@ -5,10 +5,15 @@ */ package org.elasticsearch.index.store.cache; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.blobstore.cache.BlobStoreCacheService; +import org.elasticsearch.blobstore.cache.CachedBlob; +import org.elasticsearch.client.Client; import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.BlobMetadata; import org.elasticsearch.common.blobstore.BlobPath; import org.elasticsearch.common.blobstore.DeleteResult; +import org.elasticsearch.common.bytes.ReleasableBytesReference; import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; @@ -28,6 +33,7 @@ import static com.carrotsearch.randomizedtesting.generators.RandomPicks.randomFrom; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.mock; public final class TestUtils { private TestUtils() {} @@ -209,4 +215,28 @@ private UnsupportedOperationException unsupportedException() { return new UnsupportedOperationException("This operation is not supported"); } } + + public static class NoopBlobStoreCacheService extends BlobStoreCacheService { + + public NoopBlobStoreCacheService() { + super(null, null, mock(Client.class), null); + } + + @Override + protected void getAsync(String repository, String name, String path, long offset, ActionListener listener) { + listener.onResponse(null); + } + + @Override + public void putAsync( + String repository, + String name, + String path, + long offset, + ReleasableBytesReference content, + ActionListener listener + ) { + listener.onResponse(null); + } + } } diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java index 5f95cb6f48d27..bc33ff8c78de8 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java @@ -24,23 +24,34 @@ */ package org.elasticsearch.xpack.searchablesnapshots; +import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; +import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.license.LicenseService; import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.repositories.fs.FsRepository; +import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin; +import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotAction; +import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotRequest; import org.elasticsearch.xpack.searchablesnapshots.cache.CacheService; import java.nio.file.Path; import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.Locale; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; public abstract class BaseSearchableSnapshotsIntegTestCase extends ESIntegTestCase { @Override @@ -78,6 +89,58 @@ protected Settings nodeSettings(int nodeOrdinal) { return builder.build(); } + protected void createFsRepository(String repositoryName, Path location) { + createRepository( + repositoryName, + FsRepository.TYPE, + Settings.builder().put(FsRepository.LOCATION_SETTING.getKey(), location).build(), + true + ); + } + + protected void createRepository(String repositoryName, String repositoryType, Settings repositorySettings, boolean verify) { + assertAcked( + client().admin() + .cluster() + .preparePutRepository(repositoryName) + .setType(repositoryType) + .setSettings(repositorySettings) + .setVerify(verify) + ); + } + + protected SnapshotId createSnapshot(String repositoryName, List indices) { + final CreateSnapshotResponse response = client().admin() + .cluster() + .prepareCreateSnapshot(repositoryName, randomAlphaOfLength(10).toLowerCase(Locale.ROOT)) + .setIndices(indices.toArray(Strings.EMPTY_ARRAY)) + .setWaitForCompletion(true) + .get(); + + final SnapshotInfo snapshotInfo = response.getSnapshotInfo(); + assertThat(snapshotInfo.successfulShards(), greaterThan(0)); + assertThat(snapshotInfo.failedShards(), equalTo(0)); + return snapshotInfo.snapshotId(); + } + + protected String mountSnapshot(String repositoryName, String snapshotName, String indexName, Settings indexSettings) throws Exception { + final String restoredIndexName = randomBoolean() ? indexName : randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + final MountSearchableSnapshotRequest mountRequest = new MountSearchableSnapshotRequest( + restoredIndexName, + repositoryName, + snapshotName, + indexName, + Settings.builder().put(IndexSettings.INDEX_CHECK_ON_STARTUP.getKey(), Boolean.FALSE.toString()).put(indexSettings).build(), + Strings.EMPTY_ARRAY, + true + ); + + final RestoreSnapshotResponse restoreResponse = client().execute(MountSearchableSnapshotAction.INSTANCE, mountRequest).get(); + assertThat(restoreResponse.getRestoreInfo().successfulShards(), equalTo(getNumShards(restoredIndexName).numPrimaries)); + assertThat(restoreResponse.getRestoreInfo().failedShards(), equalTo(0)); + return restoredIndexName; + } + protected void createRepo(String fsRepoName) { final Path repo = randomRepoPath(); assertAcked( From c4f251c97d6e23a1f358f1d5aac9f0624fd700b2 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 4 Aug 2020 16:04:26 +0100 Subject: [PATCH 02/63] WIP adjust some logging and back out cache-size-times-two optimisation --- .../cache/BlobStoreCacheService.java | 10 ++-- .../BaseSearchableSnapshotIndexInput.java | 7 ++- .../store/SearchableSnapshotDirectory.java | 2 +- .../cache/CachedBlobContainerIndexInput.java | 7 ++- ...ableSnapshotsBlobStoreCacheIntegTests.java | 51 +++++++++---------- 5 files changed, 40 insertions(+), 37 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index 8373578c0fe4d..0a553de8b5de6 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -231,7 +231,7 @@ public CachedBlob get(String repository, String name, String path, long offset) protected void getAsync(String repository, String name, String path, long offset, ActionListener listener) { if (isReady() == false) { - logger.debug("blob cache system index [{}] is not ready", index); + logger.debug("not ready : [{}]", CachedBlob.generateId(repository, name, path, offset)); listener.onResponse(null); return; } @@ -239,14 +239,14 @@ protected void getAsync(String repository, String name, String path, long offset final GetRequest request = new GetRequest(index).id(CachedBlob.generateId(repository, name, path, offset)); client.get(request, ActionListener.wrap(response -> { if (response.isExists()) { - logger.debug("found cached blob with id [{}] in cache", request.id()); + logger.debug("cache hit : [{}]", request.id()); assert response.isSourceEmpty() == false; final CachedBlob cachedBlob = CachedBlob.fromSource(response.getSource()); assert response.getId().equals(cachedBlob.generatedId()); listener.onResponse(cachedBlob); } else { - logger.debug("no cached blob found with id [{}] in cache", request.id()); + logger.debug("cache miss: [{}]", request.id()); listener.onResponse(null); } }, e -> { @@ -289,11 +289,11 @@ public void putAsync( } client.index(request, ActionListener.wrap(response -> { if (response.status() == RestStatus.CREATED) { - logger.trace("cached blob [{}] successfully indexed in [{}]", request, index); + logger.trace("cache fill: [{}]", request.id()); } }, listener::onFailure)); } catch (IOException e) { - logger.warn("failed to index cached blob in cache", e); + logger.warn(new ParameterizedMessage("cache fill failure: [{}]", CachedBlob.generateId(repository, name, path, offset)), e); } finally { IOUtils.closeWhileHandlingException(content); } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java index 59880249ac241..8f1505eee41b0 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java @@ -157,6 +157,7 @@ protected InputStream openInputStream( ) throws IOException { final InputStream stream = openInputStream(position, length); if (regions == null || regions.isEmpty()) { + logger.trace("returning bare stream for [{}]", fileInfo); return stream; } @@ -171,6 +172,8 @@ protected InputStream openInputStream( // // TODO we should check overlapping regions too + logger.trace("returning caching stream for [{}] with regions [{}]", this, regions); + final Iterator> sortedRegions = regions.stream() .filter(region -> position <= region.v1()) .filter(region -> region.v1() + region.v2() <= position + length) @@ -198,7 +201,7 @@ protected InputStream openInputStream( logger.trace( () -> new ParameterizedMessage( "indexing bytes of file [{}] for region [{}-{}] in blob cache index", - fileInfo.physicalName(), + this, nextRegion.v1(), nextRegion.v1() + nextRegion.v2() ) @@ -208,7 +211,7 @@ protected InputStream openInputStream( logger.trace( () -> new ParameterizedMessage( "fail to index bytes of file [{}] for region [{}-{}] in blob cache index", - fileInfo.physicalName(), + this, nextRegion.v1(), nextRegion.v1() + nextRegion.v2() ), diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java index ba81a1a5b1372..896e1740febfd 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java @@ -403,7 +403,7 @@ private boolean isExcludedFromCache(String name) { @Override public String toString() { - return this.getClass().getSimpleName() + "@snapshotId=" + snapshotId + " lockFactory=" + lockFactory; + return this.getClass().getSimpleName() + "@snapshotId=" + snapshotId + " lockFactory=" + lockFactory + " shard=" + shardId; } private void cleanExistingRegularShardFiles() { diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index 6c83e06cc5c0f..dda690dc6a5ee 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -143,6 +143,8 @@ protected void readInternal(ByteBuffer b) throws IOException { final long position = getFilePointer() + this.offset; final int length = b.remaining(); + logger.trace("readInternal: read [{}-{}] from [{}]", position, position + length, this); + // We can detect that we're going to read the last 16 bytes (that contains the footer checksum) of the file. Such reads are often // executed when opening a Directory and since we have the checksum in the snapshot metadata we can use it to fill the ByteBuffer. if (length == CodecUtil.footerLength()) { @@ -166,7 +168,7 @@ protected void readInternal(ByteBuffer b) throws IOException { // - we're reading the first N bytes of the file final boolean isStartOfFile = (position + length <= BlobStoreCacheService.DEFAULT_SIZE); // - the file is small enough to be fully cached in the blob cache - final boolean canBeFullyCached = (fileInfo.length() <= (BlobStoreCacheService.DEFAULT_SIZE * 2)); + final boolean canBeFullyCached = (fileInfo.length() <= (BlobStoreCacheService.DEFAULT_SIZE)); if (canBeFullyCached || isStartOfFile) { final CachedBlob cachedBlob = directory.getCachedBlob(fileInfo.physicalName(), 0L, length); @@ -191,6 +193,7 @@ protected void readInternal(ByteBuffer b) throws IOException { } else { regions = List.of(Tuple.tuple(0L, BlobStoreCacheService.DEFAULT_SIZE)); } + logger.trace("recovery cache miss for [{}], falling through with regions [{}]", this, regions); } else { regions = List.of(); } @@ -499,6 +502,8 @@ public String toString() { + getFilePointer() + ", rangeSize=" + getDefaultRangeSize() + + ", directory=" + + directory + '}'; } diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java index 73362087747fa..25d4c0d89cd13 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java @@ -6,7 +6,6 @@ package org.elasticsearch.blobstore.cache; -import org.apache.lucene.index.IndexFileNames; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse; import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; import org.elasticsearch.action.get.GetResponse; @@ -367,14 +366,10 @@ private void assertCachedBlobsInSystemIndex(final String repositoryName, final M } final String path = String.join("/", repositoryName, blob.getKey(), fileInfo.physicalName()); - if (fileInfo.length() <= BlobStoreCacheService.DEFAULT_SIZE * 2) { + if (fileInfo.length() <= BlobStoreCacheService.DEFAULT_SIZE) { // file has been fully cached final GetResponse getResponse = systemClient().prepareGet(SNAPSHOT_BLOB_CACHE_INDEX, path + "/@0").get(); - assertThat( - "Blob [" + fileInfo + "] should have been indexed in the blob cache system index as a single document", - getResponse.isExists(), - is(true) - ); + assertThat("not cached: [" + path + "/@0] for blob [" + fileInfo + "]", getResponse.isExists(), is(true)); final CachedBlob cachedBlob = CachedBlob.fromSource(getResponse.getSourceAsMap()); assertThat(cachedBlob.from(), equalTo(0L)); assertThat(cachedBlob.to(), equalTo(fileInfo.length())); @@ -385,7 +380,7 @@ private void assertCachedBlobsInSystemIndex(final String repositoryName, final M // first region of file has been cached GetResponse getResponse = systemClient().prepareGet(SNAPSHOT_BLOB_CACHE_INDEX, path + "/@0").get(); assertThat( - "First region of blob [" + fileInfo + "] should have been indexed in the blob cache system index", + "not cached: [" + path + "/@0] for first region of blob [" + fileInfo + "]", getResponse.isExists(), is(true) ); @@ -422,26 +417,26 @@ private void assertBlobsReadFromRemoteRepository( ); // TODO assert it is read til the end - } else { - BlobStoreIndexShardSnapshot.FileInfo blobInfo = null; - for (BlobStoreIndexShardSnapshot blobs : blobsInSnapshot.values()) { - for (BlobStoreIndexShardSnapshot.FileInfo fileInfo : blobs.indexFiles()) { - for (int i = 0; i < fileInfo.numberOfParts(); i++) { - if (blobName.endsWith(fileInfo.partName(i))) { - blobInfo = fileInfo; - break; - } - } - } - } - assertThat("Unable to find blob " + blobName + " in the blobs on disk", blobInfo, notNullValue()); - - final String fileExtension = IndexFileNames.getExtension(blobInfo.physicalName()); - assertThat( - "Only compound files can be read from the blob store after blob store cache is populated", - fileExtension, - equalTo("cfs") - ); + // } else { + // BlobStoreIndexShardSnapshot.FileInfo blobInfo = null; + // for (BlobStoreIndexShardSnapshot blobs : blobsInSnapshot.values()) { + // for (BlobStoreIndexShardSnapshot.FileInfo fileInfo : blobs.indexFiles()) { + // for (int i = 0; i < fileInfo.numberOfParts(); i++) { + // if (blobName.endsWith(fileInfo.partName(i))) { + // blobInfo = fileInfo; + // break; + // } + // } + // } + // } + // assertThat("Unable to find blob " + blobName + " in the blobs on disk", blobInfo, notNullValue()); + // + // final String fileExtension = IndexFileNames.getExtension(blobInfo.physicalName()); + // assertThat( + // "Only compound files can be read from the blob store after blob store cache is populated, not " + blobInfo, + // fileExtension, + // equalTo("cfs") + // ); } } }); From 1f30c42b6f094b9f4bf4aae9c185cdfa38c680d7 Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 6 Aug 2020 16:19:18 +0100 Subject: [PATCH 03/63] Avoid ActionListener#wrap --- .../cache/BlobStoreCacheService.java | 127 +++++++++++------- 1 file changed, 75 insertions(+), 52 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index 0a553de8b5de6..300444a26c33b 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -14,7 +14,9 @@ import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.NoShardAvailableActionException; +import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.client.Client; @@ -101,19 +103,23 @@ private void createIndexIfNecessary(ActionListener listener) { return; } try { - client.admin() - .indices() - .prepareCreate(index) - .setSettings(settings()) - .setMapping(mappings()) - .execute(ActionListener.wrap(success -> listener.onResponse(index), e -> { + client.admin().indices().prepareCreate(index).setSettings(settings()).setMapping(mappings()).execute(new ActionListener<>() { + @Override + public void onResponse(CreateIndexResponse createIndexResponse) { + assert createIndexResponse.index().equals(index); + listener.onResponse(createIndexResponse.index()); + } + + @Override + public void onFailure(Exception e) { if (e instanceof ResourceAlreadyExistsException || ExceptionsHelper.unwrapCause(e) instanceof ResourceAlreadyExistsException) { listener.onResponse(index); } else { listener.onFailure(e); } - })); + } + }); } catch (Exception e) { listener.onFailure(e); } @@ -237,28 +243,36 @@ protected void getAsync(String repository, String name, String path, long offset } try { final GetRequest request = new GetRequest(index).id(CachedBlob.generateId(repository, name, path, offset)); - client.get(request, ActionListener.wrap(response -> { - if (response.isExists()) { - logger.debug("cache hit : [{}]", request.id()); - assert response.isSourceEmpty() == false; + client.get(request, new ActionListener<>() { + @Override + public void onResponse(GetResponse response) { + if (response.isExists()) { + logger.debug("cache hit : [{}]", request.id()); + assert response.isSourceEmpty() == false; - final CachedBlob cachedBlob = CachedBlob.fromSource(response.getSource()); - assert response.getId().equals(cachedBlob.generatedId()); - listener.onResponse(cachedBlob); - } else { - logger.debug("cache miss: [{}]", request.id()); - listener.onResponse(null); + final CachedBlob cachedBlob = CachedBlob.fromSource(response.getSource()); + assert response.getId().equals(cachedBlob.generatedId()); + listener.onResponse(cachedBlob); + } else { + logger.debug("cache miss: [{}]", request.id()); + listener.onResponse(null); + } } - }, e -> { - logger.warn(() -> new ParameterizedMessage("failed to retrieve cached blob from system index [{}]", index), e); - if (e instanceof IndexNotFoundException || e instanceof NoShardAvailableActionException) { - // In case the blob cache system index got unavailable, we pretend we didn't find a cache entry and we move on. - // Failing here might bubble up the exception and fail the searchable snapshot shard which is potentially recovering. - listener.onResponse(null); - } else { - listener.onFailure(e); + + @Override + public void onFailure(Exception e) { + if (e instanceof IndexNotFoundException || e instanceof NoShardAvailableActionException) { + // In case the blob cache system index got unavailable, we pretend we didn't find a cache entry and we move on. + // Failing here might bubble up the exception and fail the searchable snapshot shard which is potentially + // recovering. + logger.debug(() -> new ParameterizedMessage("failed to retrieve cached blob from system index [{}]", index), e); + listener.onResponse(null); + } else { + logger.warn(() -> new ParameterizedMessage("failed to retrieve cached blob from system index [{}]", index), e); + listener.onFailure(e); + } } - })); + }); } catch (Exception e) { listener.onFailure(e); } @@ -272,34 +286,43 @@ public void putAsync( ReleasableBytesReference content, ActionListener listener ) { - createIndexIfNecessary(ActionListener.wrap(index -> { - try { - final CachedBlob cachedBlob = new CachedBlob( - Instant.ofEpochMilli(threadPool.absoluteTimeInMillis()), - Version.CURRENT, - repository, - name, - path, - content, - offset - ); - final IndexRequest request = new IndexRequest(index).id(cachedBlob.generatedId()); - try (XContentBuilder builder = jsonBuilder()) { - request.source(cachedBlob.toXContent(builder, ToXContent.EMPTY_PARAMS)); - } - client.index(request, ActionListener.wrap(response -> { - if (response.status() == RestStatus.CREATED) { - logger.trace("cache fill: [{}]", request.id()); + createIndexIfNecessary(new ActionListener<>() { + @Override + public void onResponse(String s) { + try { + final CachedBlob cachedBlob = new CachedBlob( + Instant.ofEpochMilli(threadPool.absoluteTimeInMillis()), + Version.CURRENT, + repository, + name, + path, + content, + offset + ); + final IndexRequest request = new IndexRequest(index).id(cachedBlob.generatedId()); + try (XContentBuilder builder = jsonBuilder()) { + request.source(cachedBlob.toXContent(builder, ToXContent.EMPTY_PARAMS)); } - }, listener::onFailure)); - } catch (IOException e) { - logger.warn(new ParameterizedMessage("cache fill failure: [{}]", CachedBlob.generateId(repository, name, path, offset)), e); - } finally { + client.index(request, ActionListener.wrap(response -> { + if (response.status() == RestStatus.CREATED) { + logger.trace("cache fill: [{}]", request.id()); + } + }, listener::onFailure)); + } catch (IOException e) { + logger.warn( + new ParameterizedMessage("cache fill failure: [{}]", CachedBlob.generateId(repository, name, path, offset)), + e + ); + } finally { + IOUtils.closeWhileHandlingException(content); + } + } + + @Override + public void onFailure(Exception e) { + logger.error(() -> new ParameterizedMessage("failed to create blob cache system index [{}]", index), e); IOUtils.closeWhileHandlingException(content); } - }, e -> { - logger.error(() -> new ParameterizedMessage("failed to create blob cache system index [{}]", index), e); - IOUtils.closeWhileHandlingException(content); - })); + }); } } From 02a8ed12cbd5c2503081c8062cb3a47c1c351d3d Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 6 Aug 2020 16:24:03 +0100 Subject: [PATCH 04/63] Add TODO --- .../blobstore/cache/BlobStoreCacheService.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index 300444a26c33b..a7549a7e4365d 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -93,10 +93,6 @@ public void clusterChanged(ClusterChangedEvent event) { } } - public boolean isReady() { - return lifecycle.started() && ready.get(); - } - private void createIndexIfNecessary(ActionListener listener) { if (clusterService.state().routingTable().hasIndex(index)) { listener.onResponse(index); @@ -236,7 +232,9 @@ public CachedBlob get(String repository, String name, String path, long offset) } protected void getAsync(String repository, String name, String path, long offset, ActionListener listener) { - if (isReady() == false) { + if ((lifecycle.started() && ready.get()) == false) { + // TODO TBD can we just execute the GET request and let it fail if the index isn't ready yet? + // We might get lucky and hit a started shard anyway. logger.debug("not ready : [{}]", CachedBlob.generateId(repository, name, path, offset)); listener.onResponse(null); return; From 76b97ab793ce90a80a5af1a600eee4e11100f2da Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 6 Aug 2020 16:55:41 +0100 Subject: [PATCH 05/63] Move special openInputStream impl --- .../BaseSearchableSnapshotIndexInput.java | 102 ----------------- .../cache/CachedBlobContainerIndexInput.java | 104 ++++++++++++++++++ 2 files changed, 104 insertions(+), 102 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java index 8f1505eee41b0..11b941218cf57 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java @@ -5,19 +5,10 @@ */ package org.elasticsearch.index.store; -import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.store.BufferedIndexInput; import org.apache.lucene.store.IOContext; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.blobstore.cache.CopyOnReadInputStream; -import org.elasticsearch.common.TriConsumer; import org.elasticsearch.common.blobstore.BlobContainer; -import org.elasticsearch.common.bytes.ReleasableBytesReference; -import org.elasticsearch.common.collect.Tuple; -import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.unit.ByteSizeUnit; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo; import org.elasticsearch.index.snapshots.blobstore.SlicedInputStream; import org.elasticsearch.threadpool.ThreadPool; @@ -25,15 +16,8 @@ import java.io.IOException; import java.io.InputStream; -import java.io.SequenceInputStream; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; public abstract class BaseSearchableSnapshotIndexInput extends BufferedIndexInput { @@ -148,92 +132,6 @@ protected InputStream openSlice(long slice) throws IOException { } } - protected InputStream openInputStream( - final long position, - final long length, - final List> regions, - final TriConsumer blobStoreCacher, - final Logger logger - ) throws IOException { - final InputStream stream = openInputStream(position, length); - if (regions == null || regions.isEmpty()) { - logger.trace("returning bare stream for [{}]", fileInfo); - return stream; - } - - // - // TODO I'm so sorry. This should be done differently, maybe using a smarter CopyOnReadInputStream - // - // The idea is to build a SequenceInputStream that wraps the stream from the blob store repository - // into multiple limited streams that copy over the bytes of the regions. - // - // If while reading the stream we saw interesting regions to cache, we index them. It means that - // we first have to sort the regions and exclude the ones that we're not going to see anyway. - // - // TODO we should check overlapping regions too - - logger.trace("returning caching stream for [{}] with regions [{}]", this, regions); - - final Iterator> sortedRegions = regions.stream() - .filter(region -> position <= region.v1()) - .filter(region -> region.v1() + region.v2() <= position + length) - .sorted(Comparator.comparing(Tuple::v1)) - .collect(Collectors.toList()) - .iterator(); - - final List streams = new ArrayList<>(); - for (long p = position; p < position + length;) { - if (sortedRegions.hasNext()) { - final Tuple nextRegion = sortedRegions.next(); - if (p < nextRegion.v1()) { - long limit = nextRegion.v1() - p; - streams.add(Streams.limitStream(Streams.noCloseStream(stream), limit)); - p += limit; - } - assert p == nextRegion.v1(); - - long limit = nextRegion.v2(); - streams.add( - new CopyOnReadInputStream( - Streams.limitStream(p + limit < length ? Streams.noCloseStream(stream) : stream, limit), - BigArrays.NON_RECYCLING_INSTANCE.newByteArray(limit), - ActionListener.wrap(releasable -> { - logger.trace( - () -> new ParameterizedMessage( - "indexing bytes of file [{}] for region [{}-{}] in blob cache index", - this, - nextRegion.v1(), - nextRegion.v1() + nextRegion.v2() - ) - ); - blobStoreCacher.apply(fileInfo.physicalName(), nextRegion.v1(), releasable); - }, e -> { - logger.trace( - () -> new ParameterizedMessage( - "fail to index bytes of file [{}] for region [{}-{}] in blob cache index", - this, - nextRegion.v1(), - nextRegion.v1() + nextRegion.v2() - ), - e - ); - }) - ) - ); - p += limit; - assert p == nextRegion.v1() + nextRegion.v2(); - } else if (p < position + length) { - long limit = position + length - p; - streams.add(Streams.limitStream(stream, limit)); - p += limit; - } - } - if (streams.size() == 1) { - return streams.get(0); - } - return new SequenceInputStream(Collections.enumeration(streams)); - } - protected final boolean assertCurrentThreadMayAccessBlobStore() { final String threadName = Thread.currentThread().getName(); assert threadName.contains('[' + ThreadPool.Names.SNAPSHOT + ']') diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index dda690dc6a5ee..91e450e4087c1 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -13,14 +13,20 @@ import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.blobstore.cache.BlobStoreCacheService; import org.elasticsearch.blobstore.cache.CachedBlob; +import org.elasticsearch.blobstore.cache.CopyOnReadInputStream; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.SuppressForbidden; +import org.elasticsearch.common.TriConsumer; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.bytes.ReleasableBytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.Channels; +import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo; import org.elasticsearch.index.store.BaseSearchableSnapshotIndexInput; import org.elasticsearch.index.store.IndexInputStats; @@ -30,14 +36,20 @@ import java.io.EOFException; import java.io.IOException; import java.io.InputStream; +import java.io.SequenceInputStream; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; import java.util.List; import java.util.Locale; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.function.Predicate; +import java.util.stream.Collectors; import java.util.stream.IntStream; import static org.elasticsearch.index.store.checksum.ChecksumBlobContainerIndexInput.checksumToBytesArray; @@ -441,6 +453,98 @@ private void writeCacheFile( } } + private InputStream openInputStream( + final long position, + final long length, + final List> regions, + final TriConsumer blobStoreCacher, + final Logger logger + ) throws IOException { + final InputStream stream = openInputStream(position, length); + if (regions == null || regions.isEmpty()) { + logger.trace("returning bare stream for [{}]", fileInfo); + return stream; + } + + // + // TODO I'm so sorry. This should be done differently, maybe using a smarter CopyOnReadInputStream + // + // The idea is to build a SequenceInputStream that wraps the stream from the blob store repository + // into multiple limited streams that copy over the bytes of the regions. + // + // If while reading the stream we saw interesting regions to cache, we index them. It means that + // we first have to sort the regions and exclude the ones that we're not going to see anyway. + // + // TODO we should check overlapping regions too + + logger.trace("returning caching stream for [{}] with regions [{}]", this, regions); + + final Iterator> sortedRegions = regions.stream() + .filter(region -> position <= region.v1()) + .filter(region -> region.v1() + region.v2() <= position + length) + .sorted(Comparator.comparing(Tuple::v1)) + .collect(Collectors.toList()) + .iterator(); + + final List streams = new ArrayList<>(); + for (long p = position; p < position + length;) { + if (sortedRegions.hasNext()) { + final Tuple nextRegion = sortedRegions.next(); + if (p < nextRegion.v1()) { + long limit = nextRegion.v1() - p; + streams.add(Streams.limitStream(Streams.noCloseStream(stream), limit)); + p += limit; + } + assert p == nextRegion.v1(); + + long limit = nextRegion.v2(); + streams.add( + new CopyOnReadInputStream( + Streams.limitStream(p + limit < length ? Streams.noCloseStream(stream) : stream, limit), + BigArrays.NON_RECYCLING_INSTANCE.newByteArray(limit), + new ActionListener<>() { + @Override + public void onResponse(ReleasableBytesReference releasableBytesReference) { + logger.trace( + () -> new ParameterizedMessage( + "indexing bytes of file [{}] for region [{}-{}] in blob cache index", + this, + nextRegion.v1(), + nextRegion.v1() + nextRegion.v2() + ) + ); + blobStoreCacher.apply(fileInfo.physicalName(), nextRegion.v1(), releasableBytesReference); + } + + @Override + public void onFailure(Exception e) { + logger.trace( + () -> new ParameterizedMessage( + "fail to index bytes of file [{}] for region [{}-{}] in blob cache index", + this, + nextRegion.v1(), + nextRegion.v1() + nextRegion.v2() + ), + e + ); + } + } + ) + ); + p += limit; + assert p == nextRegion.v1() + nextRegion.v2(); + } else if (p < position + length) { + long limit = position + length - p; + streams.add(Streams.limitStream(stream, limit)); + p += limit; + } + } + if (streams.size() == 1) { + return streams.get(0); + } + return new SequenceInputStream(Collections.enumeration(streams)); + } + @Override protected void seekInternal(long pos) throws IOException { if (pos > length()) { From df7c0b5c81eef57c7150e620431fdbb166c69f58 Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 6 Aug 2020 17:01:35 +0100 Subject: [PATCH 06/63] Add TODOs --- .../elasticsearch/blobstore/cache/CopyOnReadInputStream.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStream.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStream.java index 6d87f8c3ff048..4dccf019b325a 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStream.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStream.java @@ -74,6 +74,7 @@ public int read(byte[] b, int off, int len) throws IOException { @Override public long skip(long n) throws IOException { final long skip = handleFailure(() -> super.skip(n)); + // TODO BUG read the skipped bytes if they should go in the byte array if (skip > 0L) { count += skip; } @@ -102,6 +103,8 @@ public final void close() throws IOException { try { if (failure == null) { PagedBytesReference reference = new PagedBytesReference(bytes, Math.toIntExact(Math.min(count, bytes.size()))); + // TODO notify listener as soon as bytes are available; likewise notify on failure if not; no need to fail if we + // got the bytes we wanted listener.onResponse(new ReleasableBytesReference(reference, bytes)); success = true; } else { From 922c381a1de1860f53a7bb03daa458ee2bca7146 Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 6 Aug 2020 17:09:04 +0100 Subject: [PATCH 07/63] Add TODOs --- .../index/store/cache/CachedBlobContainerIndexInput.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index 91e450e4087c1..25a6742f2d527 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -502,6 +502,7 @@ private InputStream openInputStream( new CopyOnReadInputStream( Streams.limitStream(p + limit < length ? Streams.noCloseStream(stream) : stream, limit), BigArrays.NON_RECYCLING_INSTANCE.newByteArray(limit), + // TODO use proper BigArrays, also let the CopyOnReadInputStream allocate this new ActionListener<>() { @Override public void onResponse(ReleasableBytesReference releasableBytesReference) { From c24d78e6b6f92fdeb159a14ebde2be53f4bd0181 Mon Sep 17 00:00:00 2001 From: David Turner Date: Fri, 7 Aug 2020 08:28:36 +0100 Subject: [PATCH 08/63] Rework implementation With this commit we fill in any blob index cache misses from the cache file, which may in turn be populated from the blob store. It also renames/comments some methods and moves some methods down the class hierarchy to their usage site. --- .../cache/BlobStoreCacheService.java | 33 +- .../cache/CopyOnReadInputStream.java | 121 ------- .../BaseSearchableSnapshotIndexInput.java | 55 --- .../store/SearchableSnapshotDirectory.java | 14 +- .../index/store/cache/CacheFile.java | 10 +- .../cache/CachedBlobContainerIndexInput.java | 342 ++++++++++-------- .../cache/CopyOnReadInputStreamTests.java | 218 ----------- ...ableSnapshotsBlobStoreCacheIntegTests.java | 2 +- ...SearchableSnapshotDirectoryStatsTests.java | 17 +- .../CachedBlobContainerIndexInputTests.java | 30 +- .../index/store/cache/TestUtils.java | 4 +- .../SearchableSnapshotsIntegTests.java | 11 +- .../security/authz/AuthorizationUtils.java | 2 + 13 files changed, 243 insertions(+), 616 deletions(-) delete mode 100644 x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStream.java delete mode 100644 x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStreamTests.java diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index a7549a7e4365d..45ac3b24fb43c 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -18,6 +18,7 @@ import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.client.Client; import org.elasticsearch.client.OriginSettingClient; @@ -26,13 +27,12 @@ import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.bytes.ReleasableBytesReference; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.threadpool.ThreadPool; @@ -276,14 +276,7 @@ public void onFailure(Exception e) { } } - public void putAsync( - String repository, - String name, - String path, - long offset, - ReleasableBytesReference content, - ActionListener listener - ) { + public void putAsync(String repository, String name, String path, long offset, BytesReference content, ActionListener listener) { createIndexIfNecessary(new ActionListener<>() { @Override public void onResponse(String s) { @@ -301,25 +294,31 @@ public void onResponse(String s) { try (XContentBuilder builder = jsonBuilder()) { request.source(cachedBlob.toXContent(builder, ToXContent.EMPTY_PARAMS)); } - client.index(request, ActionListener.wrap(response -> { - if (response.status() == RestStatus.CREATED) { - logger.trace("cache fill: [{}]", request.id()); + client.index(request, new ActionListener<>() { + @Override + public void onResponse(IndexResponse indexResponse) { + if (indexResponse.status() == RestStatus.CREATED) { + logger.trace("cache fill: [{}]", request.id()); + } + listener.onResponse(null); + } + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); } - }, listener::onFailure)); + }); } catch (IOException e) { logger.warn( new ParameterizedMessage("cache fill failure: [{}]", CachedBlob.generateId(repository, name, path, offset)), e ); - } finally { - IOUtils.closeWhileHandlingException(content); } } @Override public void onFailure(Exception e) { logger.error(() -> new ParameterizedMessage("failed to create blob cache system index [{}]", index), e); - IOUtils.closeWhileHandlingException(content); } }); } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStream.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStream.java deleted file mode 100644 index 4dccf019b325a..0000000000000 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStream.java +++ /dev/null @@ -1,121 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License; - * you may not use this file except in compliance with the Elastic License. - */ - -package org.elasticsearch.blobstore.cache; - -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.common.CheckedSupplier; -import org.elasticsearch.common.bytes.PagedBytesReference; -import org.elasticsearch.common.bytes.ReleasableBytesReference; -import org.elasticsearch.common.util.ByteArray; - -import java.io.FilterInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.util.Objects; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * A {@link FilterInputStream} that copies over all the bytes read from the original input stream to a given {@link ByteArray}. The - * number of bytes copied cannot exceed the size of the {@link ByteArray}. - */ -public class CopyOnReadInputStream extends FilterInputStream { - - private final ActionListener listener; - private final AtomicBoolean closed; - private final ByteArray bytes; - - private IOException failure; - private long count; - private long mark; - - public CopyOnReadInputStream(InputStream in, ByteArray byteArray, ActionListener listener) { - super(in); - this.listener = Objects.requireNonNull(listener); - this.bytes = Objects.requireNonNull(byteArray); - this.closed = new AtomicBoolean(false); - } - - private T handleFailure(CheckedSupplier supplier) throws IOException { - try { - return supplier.get(); - } catch (IOException e) { - assert failure == null; - failure = e; - throw e; - } - } - - public int read() throws IOException { - final int result = handleFailure(super::read); - if (result != -1) { - if (count < bytes.size()) { - bytes.set(count, (byte) result); - } - count++; - } - return result; - } - - public int read(byte[] b, int off, int len) throws IOException { - final int result = handleFailure(() -> super.read(b, off, len)); - if (result != -1) { - if (count < bytes.size()) { - bytes.set(count, b, off, Math.toIntExact(Math.min(bytes.size() - count, result))); - } - count += result; - } - return result; - } - - @Override - public long skip(long n) throws IOException { - final long skip = handleFailure(() -> super.skip(n)); - // TODO BUG read the skipped bytes if they should go in the byte array - if (skip > 0L) { - count += skip; - } - return skip; - } - - @Override - public synchronized void mark(int readlimit) { - super.mark(readlimit); - mark = count; - } - - @Override - public synchronized void reset() throws IOException { - handleFailure(() -> { - super.reset(); - return null; - }); - count = mark; - } - - @Override - public final void close() throws IOException { - if (closed.compareAndSet(false, true)) { - boolean success = false; - try { - if (failure == null) { - PagedBytesReference reference = new PagedBytesReference(bytes, Math.toIntExact(Math.min(count, bytes.size()))); - // TODO notify listener as soon as bytes are available; likewise notify on failure if not; no need to fail if we - // got the bytes we wanted - listener.onResponse(new ReleasableBytesReference(reference, bytes)); - success = true; - } else { - listener.onFailure(failure); - } - } finally { - if (success == false) { - bytes.close(); - } - in.close(); - } - } - } -} diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java index 11b941218cf57..a6b0ede20ae44 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java @@ -10,12 +10,10 @@ import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo; -import org.elasticsearch.index.snapshots.blobstore.SlicedInputStream; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants; import java.io.IOException; -import java.io.InputStream; import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; @@ -104,34 +102,6 @@ public final void close() throws IOException { public abstract void innerClose() throws IOException; - protected InputStream openInputStream(final long position, final long length) throws IOException { - assert assertCurrentThreadMayAccessBlobStore(); - if (fileInfo.numberOfParts() == 1L) { - assert position + length <= fileInfo.partBytes(0) : "cannot read [" - + position - + "-" - + (position + length) - + "] from [" - + fileInfo - + "]"; - return blobContainer.readBlob(fileInfo.partName(0L), position, length); - } else { - final long startPart = getPartNumberForPosition(position); - final long endPart = getPartNumberForPosition(position + length); - return new SlicedInputStream(endPart - startPart + 1L) { - @Override - protected InputStream openSlice(long slice) throws IOException { - final long currentPart = startPart + slice; - final long startInPart = (currentPart == startPart) ? getRelativePositionInPart(position) : 0L; - final long endInPart = (currentPart == endPart) - ? getRelativePositionInPart(position + length) - : getLengthOfPart(currentPart); - return blobContainer.readBlob(fileInfo.partName(currentPart), startInPart, endInPart - startInPart); - } - }; - } - } - protected final boolean assertCurrentThreadMayAccessBlobStore() { final String threadName = Thread.currentThread().getName(); assert threadName.contains('[' + ThreadPool.Names.SNAPSHOT + ']') @@ -152,29 +122,4 @@ protected final boolean assertCurrentThreadMayAccessBlobStore() { return true; } - private long getPartNumberForPosition(long position) { - ensureValidPosition(position); - final long part = position / fileInfo.partSize().getBytes(); - assert part <= fileInfo.numberOfParts() : "part number [" + part + "] exceeds number of parts: " + fileInfo.numberOfParts(); - assert part >= 0L : "part number [" + part + "] is negative"; - return part; - } - - private long getRelativePositionInPart(long position) { - ensureValidPosition(position); - final long pos = position % fileInfo.partSize().getBytes(); - assert pos < fileInfo.partBytes((int) getPartNumberForPosition(pos)) : "position in part [" + pos + "] exceeds part's length"; - assert pos >= 0L : "position in part [" + pos + "] is negative"; - return pos; - } - - private long getLengthOfPart(long part) { - return fileInfo.partBytes(Math.toIntExact(part)); - } - - private void ensureValidPosition(long position) { - if (position < 0L || position > fileInfo.length()) { - throw new IllegalArgumentException("Position [" + position + "] is invalid"); - } - } } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java index 896e1740febfd..3353008954ce2 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java @@ -27,7 +27,7 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.support.FilterBlobContainer; -import org.elasticsearch.common.bytes.ReleasableBytesReference; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lucene.store.ByteArrayIndexInput; import org.elasticsearch.common.settings.Settings; @@ -203,6 +203,7 @@ public boolean loadSnapshot(RecoveryState recoveryState) { this.snapshot = snapshotSupplier.get(); this.recoveryState = recoveryState; this.loaded = true; + cleanExistingRegularShardFiles(); prewarmCache(); } } @@ -596,15 +597,8 @@ public CachedBlob getCachedBlob(String name, long offset, int length) { return cachedBlob; } - public void putCachedBlob(String name, long offset, ReleasableBytesReference content) { - blobStoreCacheService.putAsync( - repository, - name, - blobStoreCachePath, - offset, - content, - ActionListener.wrap(success -> {}, failure -> {}) - ); + public void putCachedBlob(String name, long offset, BytesReference content) { + blobStoreCacheService.putAsync(repository, name, blobStoreCachePath, offset, content, ActionListener.wrap(() -> {})); } /** diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CacheFile.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CacheFile.java index c6e0557bb483d..f71c48cba89f0 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CacheFile.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CacheFile.java @@ -270,7 +270,15 @@ interface RangeMissingHandler { void fillCacheRange(FileChannel channel, long from, long to, Consumer progressUpdater) throws IOException; } - CompletableFuture fetchAsync( + /** + * Populates any missing ranges within {@code rangeToWrite} using the {@link RangeMissingHandler}, and notifies the + * {@link RangeAvailableHandler} when {@code rangeToRead} is available to read from the file. If {@code rangeToRead} is already + * available then the {@link RangeAvailableHandler} is called synchronously by this method; if not then the given {@link Executor} + * processes the missing ranges and notifies the {@link RangeAvailableHandler}. + * + * @return a future which returns the result of the {@link RangeAvailableHandler} once it has completed. + */ + CompletableFuture populateAndRead( final Tuple rangeToWrite, final Tuple rangeToRead, final RangeAvailableHandler reader, diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index 25a6742f2d527..741f2f1928d19 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -13,21 +13,17 @@ import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.blobstore.cache.BlobStoreCacheService; import org.elasticsearch.blobstore.cache.CachedBlob; -import org.elasticsearch.blobstore.cache.CopyOnReadInputStream; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.SuppressForbidden; -import org.elasticsearch.common.TriConsumer; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.bytes.ReleasableBytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.Channels; -import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo; +import org.elasticsearch.index.snapshots.blobstore.SlicedInputStream; import org.elasticsearch.index.store.BaseSearchableSnapshotIndexInput; import org.elasticsearch.index.store.IndexInputStats; import org.elasticsearch.index.store.SearchableSnapshotDirectory; @@ -36,20 +32,15 @@ import java.io.EOFException; import java.io.IOException; import java.io.InputStream; -import java.io.SequenceInputStream; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; import java.util.List; import java.util.Locale; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.function.Predicate; -import java.util.stream.Collectors; import java.util.stream.IntStream; import static org.elasticsearch.index.store.checksum.ChecksumBlobContainerIndexInput.checksumToBytesArray; @@ -159,20 +150,15 @@ protected void readInternal(ByteBuffer b) throws IOException { // We can detect that we're going to read the last 16 bytes (that contains the footer checksum) of the file. Such reads are often // executed when opening a Directory and since we have the checksum in the snapshot metadata we can use it to fill the ByteBuffer. - if (length == CodecUtil.footerLength()) { - // we're comparing the global position (file pointer + offset) with the total file length (not the length of the index input - // which can be a slice) so the following code only applies when reading the footer of non-sliced index inputs (we're asserting - // that we are not reading from a clone as it would be surprising that Lucene uses a slice to verify the footer of a .cfs file) - if (position == fileInfo.length() - length) { - logger.trace("reading footer of file [{}] at position [{}], bypassing all caches", fileInfo.physicalName(), position); - b.put(checksumToBytesArray(fileInfo.checksum())); - assert b.remaining() == 0L; - assert isClone == false; - return; // TODO we should add this to DirectBlobContainerIndexInput too. + if (length == CodecUtil.footerLength() && isClone == false && position == fileInfo.length() - length) { + if (readChecksumFromFileInfo(b)) { + logger.trace("read footer of file [{}] at position [{}], bypassing all caches", fileInfo.physicalName(), position); + return; } + assert b.remaining() == length; } - final List> regions; + final List> indexCacheMisses; // We prefer to use the index cache if the recovery is not done yet if (directory.isRecoveryDone() == false) { @@ -180,7 +166,7 @@ protected void readInternal(ByteBuffer b) throws IOException { // - we're reading the first N bytes of the file final boolean isStartOfFile = (position + length <= BlobStoreCacheService.DEFAULT_SIZE); // - the file is small enough to be fully cached in the blob cache - final boolean canBeFullyCached = (fileInfo.length() <= (BlobStoreCacheService.DEFAULT_SIZE)); + final boolean canBeFullyCached = fileInfo.length() <= BlobStoreCacheService.DEFAULT_SIZE * 2; if (canBeFullyCached || isStartOfFile) { final CachedBlob cachedBlob = directory.getCachedBlob(fileInfo.physicalName(), 0L, length); @@ -194,6 +180,10 @@ protected void readInternal(ByteBuffer b) throws IOException { b.put(BytesReference.toBytes(cachedBlob.bytes().slice(Math.toIntExact(position), length))); return; } + + // Cache miss may be that the cache is completely unavailable (no point in populating it) or that the blob is + // definitely absent. TODO only bother populating the cache in the latter case. + } // We would have liked to find a cached entry but we did not find anything: the cache on the disk will be requested so @@ -201,27 +191,38 @@ protected void readInternal(ByteBuffer b) throws IOException { // {position, length} where position is relative to the file. if (canBeFullyCached) { // if the index input is smaller than twice the size of the blob cache, it will be fully indexed - regions = List.of(Tuple.tuple(0L, Math.toIntExact(fileInfo.length()))); + indexCacheMisses = List.of(Tuple.tuple(0L, fileInfo.length())); } else { - regions = List.of(Tuple.tuple(0L, BlobStoreCacheService.DEFAULT_SIZE)); + indexCacheMisses = List.of(Tuple.tuple(0L, (long) BlobStoreCacheService.DEFAULT_SIZE)); } - logger.trace("recovery cache miss for [{}], falling through with regions [{}]", this, regions); + logger.trace("recovery cache miss for [{}], falling through with regions [{}]", this, indexCacheMisses); } else { - regions = List.of(); + indexCacheMisses = List.of(); } int totalBytesRead = 0; while (totalBytesRead < length) { + // TODO lose this loop once confirmed that it really isn't necessary + assert totalBytesRead == 0 : "readInternal read only [" + totalBytesRead + "] of [" + length + "] bytes for " + this; final long pos = position + totalBytesRead; final int len = length - totalBytesRead; int bytesRead = 0; try { final CacheFile cacheFile = getCacheFileSafe(); try (Releasable ignored = cacheFile.fileLock()) { - final Tuple rangeToWrite = computeRange(pos); + + // Read all target ranges in one go, including any cache misses identified above. + final Tuple startRangeToWrite = computeRange(pos); + final Tuple endRangeToWrite = computeRange(pos + len - 1); + assert startRangeToWrite.v2() <= endRangeToWrite.v2() : startRangeToWrite + " vs " + endRangeToWrite; + final Tuple rangeToWrite = Tuple.tuple( + Math.min(startRangeToWrite.v1(), indexCacheMisses.stream().mapToLong(Tuple::v1).max().orElse(Long.MAX_VALUE)), + Math.max(endRangeToWrite.v2(), indexCacheMisses.stream().mapToLong(Tuple::v2).max().orElse(Long.MIN_VALUE)) + ); + final Tuple rangeToRead = Tuple.tuple(pos, Math.min(pos + len, rangeToWrite.v2())); - bytesRead = cacheFile.fetchAsync(rangeToWrite, rangeToRead, (channel) -> { + final CompletableFuture populateCacheFuture = cacheFile.populateAndRead(rangeToWrite, rangeToRead, channel -> { final int read; if ((rangeToRead.v2() - rangeToRead.v1()) < b.remaining()) { final ByteBuffer duplicate = b.duplicate(); @@ -233,10 +234,52 @@ protected void readInternal(ByteBuffer b) throws IOException { read = readCacheFile(channel, pos, b); } return read; - }, - (channel, from, to, progressUpdater) -> writeCacheFile(channel, from, to, progressUpdater, regions, logger), - directory.cacheFetchAsyncExecutor() - ).get(); + }, this::writeCacheFile, directory.cacheFetchAsyncExecutor()); + + for (Tuple indexCacheMiss : indexCacheMisses) { + cacheFile.populateAndRead(indexCacheMiss, indexCacheMiss, channel -> { + final int indexCacheMissLength = Math.toIntExact(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 + // - 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; + + final ByteBuffer byteBuffer = ByteBuffer.allocate(indexCacheMissLength); + Channels.readFromFileChannelWithEofException(channel, indexCacheMiss.v1(), byteBuffer); + // NB use Channels.readFromFileChannelWithEofException not readCacheFile() to avoid counting this in the stats + byteBuffer.flip(); + final BytesReference content = BytesReference.fromByteBuffer(byteBuffer); + directory.putCachedBlob(fileInfo.physicalName(), indexCacheMiss.v1(), content); + return indexCacheMissLength; + }, (channel, from, to, progressUpdater) -> { + // normally doesn't happen, we're already obtaining a range covering all cache misses above, but this + // can happen if the real populateAndRead call already failed to obtain this range of the file. In that + // case, we don't retry, we simply fail to populate the index cache. + logger.debug( + "failed to fill index cache miss [{}-{}] of {} due to earlier failure", + from, + to, + CachedBlobContainerIndexInput.this + ); + throw new IOException( + "failed to fill index cache miss [" + + from + + "-" + + to + + "] of [" + + CachedBlobContainerIndexInput.this + + "] due to earlier failure" + ); + }, + EsExecutors.newDirectExecutorService() // if ranges are still missing, fail immediately, so no need to fork + ); + + } + + bytesRead = populateCacheFuture.get(); + } } catch (final Exception e) { if (e instanceof AlreadyClosedException || (e.getCause() != null && e.getCause() instanceof AlreadyClosedException)) { @@ -260,6 +303,26 @@ protected void readInternal(ByteBuffer b) throws IOException { lastSeekPosition = lastReadPosition; } + private boolean readChecksumFromFileInfo(ByteBuffer b) throws IOException { + assert isClone == false; + byte[] footer; + try { + footer = checksumToBytesArray(fileInfo.checksum()); + } catch (NumberFormatException e) { + // tests disable this optimisation by passing an invalid checksum + footer = null; + } + if (footer == null) { + return false; + } + + b.put(footer); + assert b.remaining() == 0L; + return true; + + // TODO we should add this to DirectBlobContainerIndexInput too. + } + /** * Prefetches a complete part and writes it in cache. This method is used to prewarm the cache. */ @@ -309,7 +372,7 @@ public void prefetchPart(final int part) throws IOException { final AtomicLong totalBytesWritten = new AtomicLong(); long remainingBytes = rangeEnd - rangeStart; final long startTimeNanos = stats.currentTimeNanos(); - try (InputStream input = openInputStream(rangeStart, rangeLength)) { + try (InputStream input = openInputStreamFromBlobStore(rangeStart, rangeLength)) { while (remainingBytes > 0L) { assert totalBytesRead + remainingBytes == rangeLength; final int bytesRead = readSafe(input, copyBuffer, rangeStart, rangeEnd, remainingBytes, cacheFileReference); @@ -318,23 +381,33 @@ public void prefetchPart(final int part) throws IOException { final long readStart = rangeStart + totalBytesRead; final Tuple rangeToWrite = Tuple.tuple(readStart, readStart + bytesRead); - cacheFile.fetchAsync(rangeToWrite, rangeToWrite, (channel) -> bytesRead, (channel, start, end, progressUpdater) -> { - final ByteBuffer byteBuffer = ByteBuffer.wrap( - copyBuffer, - Math.toIntExact(start - readStart), - Math.toIntExact(end - start) - ); - final int writtenBytes = positionalWrite(channel, start, byteBuffer); - logger.trace( - "prefetchPart: writing range [{}-{}] of file [{}], [{}] bytes written", - start, - end, - fileInfo.physicalName(), - writtenBytes - ); - totalBytesWritten.addAndGet(writtenBytes); - progressUpdater.accept(start + writtenBytes); - }, directory.cacheFetchAsyncExecutor()).get(); + // We do not actually read anything, but we want to wait for the write to complete before proceeding. + // noinspection UnnecessaryLocalVariable + final Tuple rangeToRead = rangeToWrite; + + cacheFile.populateAndRead( + rangeToWrite, + rangeToRead, + (channel) -> bytesRead, + (channel, start, end, progressUpdater) -> { + final ByteBuffer byteBuffer = ByteBuffer.wrap( + copyBuffer, + Math.toIntExact(start - readStart), + Math.toIntExact(end - start) + ); + final int writtenBytes = positionalWrite(channel, start, byteBuffer); + logger.trace( + "prefetchPart: writing range [{}-{}] of file [{}], [{}] bytes written", + start, + end, + fileInfo.physicalName(), + writtenBytes + ); + totalBytesWritten.addAndGet(writtenBytes); + progressUpdater.accept(start + writtenBytes); + }, + directory.cacheFetchAsyncExecutor() + ).get(); totalBytesRead += bytesRead; remainingBytes -= bytesRead; } @@ -423,14 +496,8 @@ private int readCacheFile(final FileChannel fc, final long position, final ByteB return bytesRead; } - private void writeCacheFile( - final FileChannel fc, - final long start, - final long end, - final Consumer progressUpdater, - final List> cacheableRegions, - final Logger logger - ) throws IOException { + private void writeCacheFile(final FileChannel fc, final long start, final long end, final Consumer progressUpdater) + throws IOException { assert assertFileChannelOpen(fc); assert assertCurrentThreadMayWriteCacheFile(); final long length = end - start; @@ -440,7 +507,7 @@ private void writeCacheFile( long bytesCopied = 0L; long remaining = end - start; final long startTimeNanos = stats.currentTimeNanos(); - try (InputStream input = openInputStream(start, length, cacheableRegions, directory::putCachedBlob, logger)) { + try (InputStream input = openInputStreamFromBlobStore(start, length)) { while (remaining > 0L) { final int bytesRead = readSafe(input, copyBuffer, start, end, remaining, cacheFileReference); positionalWrite(fc, start + bytesCopied, ByteBuffer.wrap(copyBuffer, 0, bytesRead)); @@ -453,97 +520,74 @@ private void writeCacheFile( } } - private InputStream openInputStream( - final long position, - final long length, - final List> regions, - final TriConsumer blobStoreCacher, - final Logger logger - ) throws IOException { - final InputStream stream = openInputStream(position, length); - if (regions == null || regions.isEmpty()) { - logger.trace("returning bare stream for [{}]", fileInfo); - return stream; + /** + * Opens an {@link InputStream} for the given range of bytes which reads the data directly from the blob store. If the requested range + * spans multiple blobs then this stream will request them in turn. + * + * @param position The start of the range of bytes to read, relative to the start of the corresponding Lucene file. + * @param length The number of bytes to read + */ + private InputStream openInputStreamFromBlobStore(final long position, final long length) throws IOException { + assert assertCurrentThreadMayAccessBlobStore(); + if (fileInfo.numberOfParts() == 1L) { + assert position + length <= fileInfo.partBytes(0) : "cannot read [" + + position + + "-" + + (position + length) + + "] from [" + + fileInfo + + "]"; + return blobContainer.readBlob(fileInfo.partName(0L), position, length); + } else { + final long startPart = getPartNumberForPosition(position); + final long endPart = getPartNumberForPosition(position + length - 1); + return new SlicedInputStream(endPart - startPart + 1L) { + @Override + protected InputStream openSlice(long slice) throws IOException { + final long currentPart = startPart + slice; + final long startInPart = (currentPart == startPart) ? getRelativePositionInPart(position) : 0L; + final long endInPart = (currentPart == endPart) + ? getRelativePositionInPart(position + length - 1) + 1 + : getLengthOfPart(currentPart); + return blobContainer.readBlob(fileInfo.partName(currentPart), startInPart, endInPart - startInPart); + } + }; } + } - // - // TODO I'm so sorry. This should be done differently, maybe using a smarter CopyOnReadInputStream - // - // The idea is to build a SequenceInputStream that wraps the stream from the blob store repository - // into multiple limited streams that copy over the bytes of the regions. - // - // If while reading the stream we saw interesting regions to cache, we index them. It means that - // we first have to sort the regions and exclude the ones that we're not going to see anyway. - // - // TODO we should check overlapping regions too - - logger.trace("returning caching stream for [{}] with regions [{}]", this, regions); - - final Iterator> sortedRegions = regions.stream() - .filter(region -> position <= region.v1()) - .filter(region -> region.v1() + region.v2() <= position + length) - .sorted(Comparator.comparing(Tuple::v1)) - .collect(Collectors.toList()) - .iterator(); - - final List streams = new ArrayList<>(); - for (long p = position; p < position + length;) { - if (sortedRegions.hasNext()) { - final Tuple nextRegion = sortedRegions.next(); - if (p < nextRegion.v1()) { - long limit = nextRegion.v1() - p; - streams.add(Streams.limitStream(Streams.noCloseStream(stream), limit)); - p += limit; - } - assert p == nextRegion.v1(); - - long limit = nextRegion.v2(); - streams.add( - new CopyOnReadInputStream( - Streams.limitStream(p + limit < length ? Streams.noCloseStream(stream) : stream, limit), - BigArrays.NON_RECYCLING_INSTANCE.newByteArray(limit), - // TODO use proper BigArrays, also let the CopyOnReadInputStream allocate this - new ActionListener<>() { - @Override - public void onResponse(ReleasableBytesReference releasableBytesReference) { - logger.trace( - () -> new ParameterizedMessage( - "indexing bytes of file [{}] for region [{}-{}] in blob cache index", - this, - nextRegion.v1(), - nextRegion.v1() + nextRegion.v2() - ) - ); - blobStoreCacher.apply(fileInfo.physicalName(), nextRegion.v1(), releasableBytesReference); - } + /** + * Compute the part number that contains the byte at the given position in the corresponding Lucene file. + */ + private long getPartNumberForPosition(long position) { + ensureValidPosition(position); + final long part = position / fileInfo.partSize().getBytes(); + assert part <= fileInfo.numberOfParts() : "part number [" + part + "] exceeds number of parts: " + fileInfo.numberOfParts(); + assert part >= 0L : "part number [" + part + "] is negative"; + return part; + } - @Override - public void onFailure(Exception e) { - logger.trace( - () -> new ParameterizedMessage( - "fail to index bytes of file [{}] for region [{}-{}] in blob cache index", - this, - nextRegion.v1(), - nextRegion.v1() + nextRegion.v2() - ), - e - ); - } - } - ) - ); - p += limit; - assert p == nextRegion.v1() + nextRegion.v2(); - } else if (p < position + length) { - long limit = position + length - p; - streams.add(Streams.limitStream(stream, limit)); - p += limit; - } - } - if (streams.size() == 1) { - return streams.get(0); + /** + * Compute the position of the given byte relative to the start of its part. + * @param position the position of the required byte (within the corresponding Lucene file) + */ + private long getRelativePositionInPart(long position) { + ensureValidPosition(position); + final long pos = position % fileInfo.partSize().getBytes(); + assert pos < fileInfo.partBytes((int) getPartNumberForPosition(pos)) : "position in part [" + pos + "] exceeds part's length"; + assert pos >= 0L : "position in part [" + pos + "] is negative"; + return pos; + } + + private long getLengthOfPart(long part) { + return fileInfo.partBytes(Math.toIntExact(part)); + } + + private void ensureValidPosition(long position) { + assert position >= 0L && position < fileInfo.length() : position + " vs " + fileInfo.length(); + // noinspection ConstantConditions in case assertions are disabled + if (position < 0L || position >= fileInfo.length()) { + throw new IllegalArgumentException("Position [" + position + "] is invalid for a file of length [" + fileInfo.length() + "]"); } - return new SequenceInputStream(Collections.enumeration(streams)); } @Override @@ -619,7 +663,7 @@ private int readDirectly(long start, long end, ByteBuffer b) throws IOException int bytesCopied = 0; final long startTimeNanos = stats.currentTimeNanos(); - try (InputStream input = openInputStream(start, length)) { + try (InputStream input = openInputStreamFromBlobStore(start, length)) { long remaining = end - start; while (remaining > 0) { final int len = (remaining < copyBuffer.length) ? (int) remaining : copyBuffer.length; diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStreamTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStreamTests.java deleted file mode 100644 index 0f9553290b1a5..0000000000000 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/CopyOnReadInputStreamTests.java +++ /dev/null @@ -1,218 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License; - * you may not use this file except in compliance with the Elastic License. - */ - -package org.elasticsearch.blobstore.cache; - -import org.apache.lucene.util.SetOnce; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.bytes.ReleasableBytesReference; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.ByteArray; -import org.elasticsearch.common.util.MockBigArrays; -import org.elasticsearch.common.util.MockPageCacheRecycler; -import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; -import org.elasticsearch.test.ESTestCase; - -import java.io.ByteArrayInputStream; -import java.io.FilterInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.util.Arrays; - -import static org.elasticsearch.blobstore.cache.BlobStoreCacheService.DEFAULT_SIZE; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.notNullValue; -import static org.hamcrest.Matchers.nullValue; - -public class CopyOnReadInputStreamTests extends ESTestCase { - - private final MockBigArrays bigArrays = new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), new NoneCircuitBreakerService()); - - public void testCopyOnReadInputStream() throws Exception { - final SetOnce onSuccess = new SetOnce<>(); - final SetOnce onFailure = new SetOnce<>(); - final ActionListener listener = ActionListener.wrap(onSuccess::set, onFailure::set); - - final byte[] blobContent = randomByteArray(); - final ByteArray byteArray = bigArrays.newByteArray(DEFAULT_SIZE); - - final int maxBytesToRead = randomIntBetween(0, blobContent.length); - final InputStream stream = new CopyOnReadInputStream(new ByteArrayInputStream(blobContent), byteArray, listener); - randomReads(stream, maxBytesToRead); - stream.close(); - - final ReleasableBytesReference releasable = onSuccess.get(); - assertThat(releasable, notNullValue()); - assertThat(releasable.length(), equalTo((int) Math.min(maxBytesToRead, byteArray.size()))); - assertArrayEquals(Arrays.copyOfRange(blobContent, 0, releasable.length()), BytesReference.toBytes(releasable)); - releasable.close(); - - final Exception failure = onFailure.get(); - assertThat(failure, nullValue()); - } - - public void testCopyOnReadInputStreamDoesNotCopyMoreThanByteArraySize() throws Exception { - final SetOnce onSuccess = new SetOnce<>(); - final SetOnce onFailure = new SetOnce<>(); - final ActionListener listener = ActionListener.wrap(onSuccess::set, onFailure::set); - - final byte[] blobContent = randomByteArray(); - - final ByteArray byteArray = bigArrays.newByteArray(randomIntBetween(0, DEFAULT_SIZE)); - final InputStream stream = new CopyOnReadInputStream(new ByteArrayInputStream(blobContent), byteArray, listener); - randomReads(stream, blobContent.length); - stream.close(); - - final ReleasableBytesReference releasable = onSuccess.get(); - assertThat(releasable, notNullValue()); - assertThat(releasable.length(), equalTo(Math.toIntExact(Math.min(blobContent.length, byteArray.size())))); - assertArrayEquals(Arrays.copyOfRange(blobContent, 0, releasable.length()), BytesReference.toBytes(releasable)); - releasable.close(); - - final Exception failure = onFailure.get(); - assertThat(failure, nullValue()); - } - - public void testCopyOnReadWithFailure() throws Exception { - final SetOnce onSuccess = new SetOnce<>(); - final SetOnce onFailure = new SetOnce<>(); - final ActionListener listener = ActionListener.wrap(onSuccess::set, onFailure::set); - - final byte[] blobContent = new byte[0]; - randomByteArray(); - - final int failAfterNBytesRead = randomIntBetween(0, Math.max(0, blobContent.length - 1)); - final InputStream erroneousStream = new FailAfterNBytesInputStream(new ByteArrayInputStream(blobContent), failAfterNBytesRead); - - final int byteSize = randomIntBetween(0, DEFAULT_SIZE); - try (InputStream stream = new CopyOnReadInputStream(erroneousStream, bigArrays.newByteArray(byteSize), listener)) { - IOException exception = expectThrows(IOException.class, () -> randomReads(stream, Math.max(1, blobContent.length))); - assertThat(exception.getMessage(), containsString("Fail to access bytes")); - } - - if (failAfterNBytesRead < byteSize) { - Exception failure = onFailure.get(); - assertThat(failure, notNullValue()); - assertThat(failure.getMessage(), containsString("Fail to access bytes")); - assertThat(onSuccess.get(), nullValue()); - - } else { - ReleasableBytesReference releasable = onSuccess.get(); - assertThat(releasable, notNullValue()); - assertArrayEquals(Arrays.copyOfRange(blobContent, 0, byteSize), BytesReference.toBytes(releasable)); - assertThat(onFailure.get(), nullValue()); - releasable.close(); - } - } - - private static byte[] randomByteArray() { - return randomByteArrayOfLength(randomIntBetween(0, frequently() ? DEFAULT_SIZE : 1 << 20)); // rarely up to 1mb; - } - - private void randomReads(final InputStream stream, final int maxBytesToRead) throws IOException { - int remaining = maxBytesToRead; - while (remaining > 0) { - int read; - switch (randomInt(3)) { - case 0: // single byte read - read = stream.read(); - if (read != -1) { - remaining--; - } - break; - case 1: // buffered read with fixed buffer offset/length - read = stream.read(new byte[randomIntBetween(1, remaining)]); - if (read != -1) { - remaining -= read; - } - break; - case 2: // buffered read with random buffer offset/length - final byte[] tmp = new byte[randomIntBetween(1, remaining)]; - final int off = randomIntBetween(0, tmp.length - 1); - read = stream.read(tmp, off, randomIntBetween(1, Math.min(1, tmp.length - off))); - if (read != -1) { - remaining -= read; - } - break; - - case 3: // mark & reset with intermediate skip() - final int toSkip = randomIntBetween(1, remaining); - stream.mark(toSkip); - stream.skip(toSkip); - stream.reset(); - break; - default: - fail("Unsupported test condition in " + getTestName()); - } - } - } - - /** - * InputStream that throws an IOException once byte at position N is read or skipped - */ - private static class FailAfterNBytesInputStream extends FilterInputStream { - - private final long failAfter; - - private long reads; - private long mark; - - FailAfterNBytesInputStream(InputStream in, long failAfter) { - super(in); - this.failAfter = failAfter; - } - - void ensureCanAccessMoreBytes() throws IOException { - if (failAfter <= reads) { - throw new IOException("Fail to access bytes after [" + failAfter + "]"); - } - } - - @Override - public int read() throws IOException { - ensureCanAccessMoreBytes(); - final int read = super.read(); - if (read != -1) { - reads++; - } - return read; - } - - @Override - public int read(byte[] b, int off, int len) throws IOException { - ensureCanAccessMoreBytes(); - final int read = super.read(b, off, Math.min(len, Math.toIntExact(failAfter - reads))); - if (read != -1) { - reads += read; - } - return read; - } - - @Override - public long skip(long n) throws IOException { - ensureCanAccessMoreBytes(); - final long skipped = super.skip(Math.min(n, Math.toIntExact(failAfter - reads))); - if (skipped > 0L) { - reads += skipped; - } - return skipped; - } - - @Override - public synchronized void reset() throws IOException { - super.reset(); - reads = mark; - } - - @Override - public synchronized void mark(int readlimit) { - super.mark(readlimit); - mark = reads; - } - } -} diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java index 25d4c0d89cd13..69eb78a789ded 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java @@ -366,7 +366,7 @@ private void assertCachedBlobsInSystemIndex(final String repositoryName, final M } final String path = String.join("/", repositoryName, blob.getKey(), fileInfo.physicalName()); - if (fileInfo.length() <= BlobStoreCacheService.DEFAULT_SIZE) { + if (fileInfo.length() <= BlobStoreCacheService.DEFAULT_SIZE * 2) { // file has been fully cached final GetResponse getResponse = systemClient().prepareGet(SNAPSHOT_BLOB_CACHE_INDEX, path + "/@0").get(); assertThat("not cached: [" + path + "/@0] for blob [" + fileInfo + "]", getResponse.isExists(), is(true)); diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryStatsTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryStatsTests.java index 91961d3eb301f..14d965af9eabc 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryStatsTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryStatsTests.java @@ -127,20 +127,18 @@ public void testCachedBytesReadsAndWrites() { assertBusy(() -> { assertThat(inputStats.getCachedBytesWritten(), notNullValue()); assertThat(inputStats.getCachedBytesWritten().total(), equalTo(length)); - assertThat(inputStats.getCachedBytesWritten().count(), equalTo(cachedBytesWriteCount)); + final long actualWriteCount = inputStats.getCachedBytesWritten().count(); + assertThat(actualWriteCount, lessThanOrEqualTo(cachedBytesWriteCount)); assertThat(inputStats.getCachedBytesWritten().min(), greaterThan(0L)); - assertThat( - inputStats.getCachedBytesWritten().max(), - (length < rangeSize.getBytes()) ? equalTo(length) : equalTo(rangeSize.getBytes()) - ); + assertThat(inputStats.getCachedBytesWritten().max(), lessThanOrEqualTo(length)); assertThat( inputStats.getCachedBytesWritten().totalNanoseconds(), allOf( // each read takes at least FAKE_CLOCK_ADVANCE_NANOS time - greaterThanOrEqualTo(FAKE_CLOCK_ADVANCE_NANOS * cachedBytesWriteCount), + greaterThanOrEqualTo(FAKE_CLOCK_ADVANCE_NANOS * actualWriteCount), // worst case: we start all reads before finishing any of them - lessThanOrEqualTo(FAKE_CLOCK_ADVANCE_NANOS * cachedBytesWriteCount * cachedBytesWriteCount) + lessThanOrEqualTo(FAKE_CLOCK_ADVANCE_NANOS * actualWriteCount * actualWriteCount) ) ); }); @@ -149,10 +147,7 @@ public void testCachedBytesReadsAndWrites() { assertThat(inputStats.getCachedBytesRead().total(), greaterThanOrEqualTo(length)); assertThat(inputStats.getCachedBytesRead().count(), greaterThan(0L)); assertThat(inputStats.getCachedBytesRead().min(), greaterThan(0L)); - assertThat( - inputStats.getCachedBytesRead().max(), - (length < rangeSize.getBytes()) ? lessThanOrEqualTo(length) : lessThanOrEqualTo(rangeSize.getBytes()) - ); + assertThat(inputStats.getCachedBytesRead().max(), lessThanOrEqualTo(length)); assertCounter(inputStats.getDirectBytesRead(), 0L, 0L, 0L, 0L); assertThat(inputStats.getDirectBytesRead().totalNanoseconds(), equalTo(0L)); diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInputTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInputTests.java index 5ac12c051274a..b0b765fd55608 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInputTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInputTests.java @@ -53,6 +53,7 @@ import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_CACHE_PREWARM_ENABLED_SETTING; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.notNullValue; public class CachedBlobContainerIndexInputTests extends ESIndexInputTestCase { @@ -68,6 +69,7 @@ public void testRandomReads() throws IOException { for (int i = 0; i < 5; i++) { final String fileName = randomAlphaOfLength(10); + final byte[] input = randomUnicodeOfLength(randomIntBetween(1, 100_000)).getBytes(StandardCharsets.UTF_8); final String blobName = randomUnicodeOfLength(10); @@ -155,9 +157,9 @@ public void testRandomReads() throws IOException { if (blobContainer instanceof CountingBlobContainer) { long numberOfRanges = TestUtils.numberOfRanges(input.length, cacheService.getRangeSize()); assertThat( - "Expected " + numberOfRanges + " ranges fetched from the source", + "Expected at most " + numberOfRanges + " ranges fetched from the source", ((CountingBlobContainer) blobContainer).totalOpens.sum(), - equalTo(numberOfRanges) + lessThanOrEqualTo(numberOfRanges) ); assertThat( "All bytes should have been read from source", @@ -365,30 +367,6 @@ public int read(byte[] b, int offset, int len) throws IOException { @Override public void close() throws IOException { in.close(); - if (start % rangeSize != 0) { - throw new AssertionError("Read operation should start at the beginning of a range"); - } - if (end % rangeSize != 0) { - if (end != length) { - throw new AssertionError("Read operation should finish at the end of a range or the end of the file"); - } - } - if (length <= rangeSize) { - if (bytesRead != length) { - throw new AssertionError("All [" + length + "] bytes should have been read, no more no less but got:" + bytesRead); - } - } else { - if (bytesRead != rangeSize) { - if (end != length) { - throw new AssertionError("Expecting [" + rangeSize + "] bytes to be read but got:" + bytesRead); - - } - final long remaining = length % rangeSize; - if (bytesRead != remaining) { - throw new AssertionError("Expecting [" + remaining + "] bytes to be read but got:" + bytesRead); - } - } - } this.container.totalBytes.add(bytesRead); } } diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/TestUtils.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/TestUtils.java index 47c39b14ceb3d..c8114d50cbeb6 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/TestUtils.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/TestUtils.java @@ -13,7 +13,7 @@ import org.elasticsearch.common.blobstore.BlobMetadata; import org.elasticsearch.common.blobstore.BlobPath; import org.elasticsearch.common.blobstore.DeleteResult; -import org.elasticsearch.common.bytes.ReleasableBytesReference; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; @@ -233,7 +233,7 @@ public void putAsync( String name, String path, long offset, - ReleasableBytesReference content, + BytesReference content, ActionListener listener ) { listener.onResponse(null); diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java index 75c80c2d828f3..ee510c73b9cfd 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java @@ -768,11 +768,12 @@ private void assertSearchableSnapshotStats(String indexName, boolean cacheEnable equalTo(0L) ); } else if (nodeIdsWithLargeEnoughCache.contains(stats.getShardRouting().currentNodeId())) { - assertThat( - "Expected at least 1 cache read or write for " + fileName + " of shard " + shardRouting, - Math.max(indexInputStats.getCachedBytesRead().getCount(), indexInputStats.getCachedBytesWritten().getCount()), - greaterThan(0L) - ); + // not necessarily, it may have been entirely in blob cache TODO improve stats to handle this + // assertThat( + // "Expected at least 1 cache read or write for " + fileName + " of shard " + shardRouting, + // Math.max(indexInputStats.getCachedBytesRead().getCount(), indexInputStats.getCachedBytesWritten().getCount()), + // greaterThan(0L) + // ); assertThat( "Expected no optimized read for " + fileName + " of shard " + shardRouting, indexInputStats.getOptimizedBytesRead().getCount(), diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationUtils.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationUtils.java index c63e086921cbd..7e0911120df1b 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationUtils.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationUtils.java @@ -25,6 +25,7 @@ import static org.elasticsearch.xpack.core.ClientHelper.ASYNC_SEARCH_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.ENRICH_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.IDP_ORIGIN; +import static org.elasticsearch.xpack.core.ClientHelper.SEARCHABLE_SNAPSHOTS_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.STACK_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.TRANSFORM_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.DEPRECATION_ORIGIN; @@ -121,6 +122,7 @@ public static void switchUserBasedOnActionOriginAndExecute(ThreadContext threadC case IDP_ORIGIN: case INGEST_ORIGIN: case STACK_ORIGIN: + case SEARCHABLE_SNAPSHOTS_ORIGIN: case TASKS_ORIGIN: // TODO use a more limited user for tasks securityContext.executeAsUser(XPackUser.INSTANCE, consumer, Version.CURRENT); break; From 1b459aed4a79035dc9366f4ff19118463f745eaf Mon Sep 17 00:00:00 2001 From: David Turner Date: Fri, 7 Aug 2020 21:14:30 +0100 Subject: [PATCH 09/63] One-shot reading --- .../cache/CachedBlobContainerIndexInput.java | 262 +++++++++--------- 1 file changed, 133 insertions(+), 129 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index 741f2f1928d19..83007333fe9bc 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -183,12 +183,11 @@ protected void readInternal(ByteBuffer b) throws IOException { // Cache miss may be that the cache is completely unavailable (no point in populating it) or that the blob is // definitely absent. TODO only bother populating the cache in the latter case. - } // 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 regions of the file we would like to have the next time. The regions are expressed as tuple of - // {position, length} where position is relative to the file. + // we compute the regions of the file we would like to have the next time. The regions are expressed as tuples of + // {start, end} ranges 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 indexCacheMisses = List.of(Tuple.tuple(0L, fileInfo.length())); @@ -200,109 +199,148 @@ protected void readInternal(ByteBuffer b) throws IOException { indexCacheMisses = List.of(); } - int totalBytesRead = 0; - while (totalBytesRead < length) { - // TODO lose this loop once confirmed that it really isn't necessary - assert totalBytesRead == 0 : "readInternal read only [" + totalBytesRead + "] of [" + length + "] bytes for " + this; - final long pos = position + totalBytesRead; - final int len = length - totalBytesRead; - int bytesRead = 0; - try { - final CacheFile cacheFile = getCacheFileSafe(); - try (Releasable ignored = cacheFile.fileLock()) { - - // Read all target ranges in one go, including any cache misses identified above. - final Tuple startRangeToWrite = computeRange(pos); - final Tuple endRangeToWrite = computeRange(pos + len - 1); - assert startRangeToWrite.v2() <= endRangeToWrite.v2() : startRangeToWrite + " vs " + endRangeToWrite; - final Tuple rangeToWrite = Tuple.tuple( - Math.min(startRangeToWrite.v1(), indexCacheMisses.stream().mapToLong(Tuple::v1).max().orElse(Long.MAX_VALUE)), - Math.max(endRangeToWrite.v2(), indexCacheMisses.stream().mapToLong(Tuple::v2).max().orElse(Long.MIN_VALUE)) - ); + try { + final CacheFile cacheFile = getCacheFileSafe(); + try (Releasable ignored = cacheFile.fileLock()) { - final Tuple rangeToRead = Tuple.tuple(pos, Math.min(pos + len, rangeToWrite.v2())); - - final CompletableFuture populateCacheFuture = cacheFile.populateAndRead(rangeToWrite, rangeToRead, channel -> { - 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())); - read = readCacheFile(channel, pos, duplicate); - assert duplicate.position() <= b.limit(); - b.position(duplicate.position()); - } else { - read = readCacheFile(channel, pos, b); - } - return read; - }, this::writeCacheFile, directory.cacheFetchAsyncExecutor()); - - for (Tuple indexCacheMiss : indexCacheMisses) { - cacheFile.populateAndRead(indexCacheMiss, indexCacheMiss, channel -> { - final int indexCacheMissLength = Math.toIntExact(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 - // - 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; - - final ByteBuffer byteBuffer = ByteBuffer.allocate(indexCacheMissLength); - Channels.readFromFileChannelWithEofException(channel, indexCacheMiss.v1(), byteBuffer); - // NB use Channels.readFromFileChannelWithEofException not readCacheFile() to avoid counting this in the stats - byteBuffer.flip(); - final BytesReference content = BytesReference.fromByteBuffer(byteBuffer); - directory.putCachedBlob(fileInfo.physicalName(), indexCacheMiss.v1(), content); - return indexCacheMissLength; - }, (channel, from, to, progressUpdater) -> { - // normally doesn't happen, we're already obtaining a range covering all cache misses above, but this - // can happen if the real populateAndRead call already failed to obtain this range of the file. In that - // case, we don't retry, we simply fail to populate the index cache. - logger.debug( - "failed to fill index cache miss [{}-{}] of {} due to earlier failure", - from, - to, - CachedBlobContainerIndexInput.this - ); - throw new IOException( - "failed to fill index cache miss [" - + from - + "-" - + to - + "] of [" - + CachedBlobContainerIndexInput.this - + "] due to earlier failure" - ); - }, - EsExecutors.newDirectExecutorService() // if ranges are still missing, fail immediately, so no need to fork - ); + // Read all target ranges in one go, including any cache misses identified above. + final Tuple startRangeToWrite = computeRange(position); + final Tuple endRangeToWrite = computeRange(position + length - 1); + assert startRangeToWrite.v2() <= endRangeToWrite.v2() : startRangeToWrite + " vs " + endRangeToWrite; + final Tuple rangeToWrite = Tuple.tuple( + Math.min(startRangeToWrite.v1(), indexCacheMisses.stream().mapToLong(Tuple::v1).max().orElse(Long.MAX_VALUE)), + Math.max(endRangeToWrite.v2(), indexCacheMisses.stream().mapToLong(Tuple::v2).max().orElse(Long.MIN_VALUE)) + ); + assert rangeToWrite.v1() <= position && position + length <= rangeToWrite.v2() : "[" + + position + + "-" + + (position + length) + + "] vs " + + rangeToWrite; + final Tuple rangeToRead = Tuple.tuple(position, position + length); + + final CompletableFuture populateCacheFuture = cacheFile.populateAndRead(rangeToWrite, rangeToRead, channel -> { + 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())); + read = readCacheFile(channel, position, duplicate); + assert duplicate.position() <= b.limit(); + b.position(duplicate.position()); + } else { + read = readCacheFile(channel, position, b); } - - bytesRead = populateCacheFuture.get(); + return read; + }, this::writeCacheFile, directory.cacheFetchAsyncExecutor()); + + for (Tuple indexCacheMiss : indexCacheMisses) { + cacheFile.populateAndRead(indexCacheMiss, indexCacheMiss, channel -> { + final int indexCacheMissLength = Math.toIntExact(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 + // - 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; + + final ByteBuffer byteBuffer = ByteBuffer.allocate(indexCacheMissLength); + Channels.readFromFileChannelWithEofException(channel, indexCacheMiss.v1(), byteBuffer); + // NB use Channels.readFromFileChannelWithEofException not readCacheFile() to avoid counting this in the stats + byteBuffer.flip(); + final BytesReference content = BytesReference.fromByteBuffer(byteBuffer); + directory.putCachedBlob(fileInfo.physicalName(), indexCacheMiss.v1(), content); + return indexCacheMissLength; + }, (channel, from, to, progressUpdater) -> { + // normally doesn't happen, we're already obtaining a range covering all cache misses above, but this + // can happen if the real populateAndRead call already failed to obtain this range of the file. In that + // case, we don't retry, we simply fail to populate the index cache. + logger.debug( + "failed to fill index cache miss [{}-{}] of {} due to earlier failure", + from, + to, + CachedBlobContainerIndexInput.this + ); + throw new IOException( + "failed to fill index cache miss [" + + from + + "-" + + to + + "] of [" + + CachedBlobContainerIndexInput.this + + "] due to earlier failure" + ); + }, + EsExecutors.newDirectExecutorService() // if ranges are still missing, fail immediately, so no need to fork + ); } - } catch (final Exception e) { - if (e instanceof AlreadyClosedException || (e.getCause() != null && e.getCause() instanceof AlreadyClosedException)) { - try { - // cache file was evicted during the range fetching, read bytes directly from source - bytesRead = readDirectly(pos, pos + len, b); - continue; - } catch (Exception inner) { - e.addSuppressed(inner); - } - } - throw new IOException("Fail to read data from cache", e); - } finally { - totalBytesRead += bytesRead; + final int bytesRead = populateCacheFuture.get(); + assert bytesRead == length : bytesRead + " vs " + length; } + } catch (final Exception e) { + // may have partially filled the buffer before the exception was thrown, so try and get the remainder directly. + final int alreadyRead = length - b.remaining(); + final int bytesRead = readDirectlyIfAlreadyClosed(position + alreadyRead, b, e); + assert alreadyRead + bytesRead == length : alreadyRead + " + " + bytesRead + " vs " + length; } - assert totalBytesRead == length : "partial read operation, read [" + totalBytesRead + "] bytes of [" + length + "]"; - stats.incrementBytesRead(lastReadPosition, position, totalBytesRead); - lastReadPosition = position + totalBytesRead; + + stats.incrementBytesRead(lastReadPosition, position, length); + lastReadPosition = position + length; lastSeekPosition = lastReadPosition; } + private int readDirectlyIfAlreadyClosed(long position, ByteBuffer b, Exception e) throws IOException { + if (e instanceof AlreadyClosedException || (e.getCause() != null && e.getCause() instanceof AlreadyClosedException)) { + 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))]; + logger.trace( + () -> new ParameterizedMessage( + "direct reading of range [{}-{}] for cache file [{}]", + position, + position + length, + cacheFileReference + ) + ); + + int bytesCopied = 0; + final long startTimeNanos = stats.currentTimeNanos(); + try (InputStream input = openInputStreamFromBlobStore(position, length)) { + long remaining = length; + while (remaining > 0) { + final int len = (remaining < copyBuffer.length) ? (int) remaining : copyBuffer.length; + int bytesRead = input.read(copyBuffer, 0, len); + if (bytesRead == -1) { + throw new EOFException( + String.format( + Locale.ROOT, + "unexpected EOF reading [%d-%d] ([%d] bytes remaining) from %s", + position, + position + length, + remaining, + cacheFileReference + ) + ); + } + b.put(copyBuffer, 0, bytesRead); + bytesCopied += bytesRead; + remaining -= bytesRead; + assert remaining == b.remaining() : remaining + " vs " + b.remaining(); + } + final long endTimeNanos = stats.currentTimeNanos(); + stats.addDirectBytesRead(bytesCopied, endTimeNanos - startTimeNanos); + } + return bytesCopied; + } catch (Exception inner) { + e.addSuppressed(inner); + } + } + throw new IOException("failed to read data from cache", e); + } + private boolean readChecksumFromFileInfo(ByteBuffer b) throws IOException { assert isClone == false; byte[] footer; @@ -656,40 +694,6 @@ public String toString() { + '}'; } - private int readDirectly(long start, long end, ByteBuffer b) throws IOException { - final long length = end - start; - final byte[] copyBuffer = new byte[Math.toIntExact(Math.min(COPY_BUFFER_SIZE, length))]; - logger.trace(() -> new ParameterizedMessage("direct reading of range [{}-{}] for cache file [{}]", start, end, cacheFileReference)); - - int bytesCopied = 0; - final long startTimeNanos = stats.currentTimeNanos(); - try (InputStream input = openInputStreamFromBlobStore(start, length)) { - long remaining = end - start; - while (remaining > 0) { - final int len = (remaining < copyBuffer.length) ? (int) remaining : copyBuffer.length; - int bytesRead = input.read(copyBuffer, 0, len); - if (bytesRead == -1) { - throw new EOFException( - String.format( - Locale.ROOT, - "unexpected EOF reading [%d-%d] ([%d] bytes remaining) from %s", - start, - end, - remaining, - cacheFileReference - ) - ); - } - b.put(copyBuffer, 0, bytesRead); - bytesCopied += bytesRead; - remaining -= bytesRead; - } - final long endTimeNanos = stats.currentTimeNanos(); - stats.addDirectBytesRead(bytesCopied, endTimeNanos - startTimeNanos); - } - return bytesCopied; - } - private static class CacheFileReference implements CacheFile.EvictionListener { private final long fileLength; From f06e0ae49760a3fd12ca5ff271711df2cb12d0fb Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 10 Aug 2020 10:41:58 +0100 Subject: [PATCH 10/63] Try to fill the cache miss rather than immediately failing --- .../cache/CachedBlobContainerIndexInput.java | 25 ++++++------------- 1 file changed, 7 insertions(+), 18 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index 83007333fe9bc..2ea4f2a9ac721 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -21,7 +21,6 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.Channels; import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo; import org.elasticsearch.index.snapshots.blobstore.SlicedInputStream; import org.elasticsearch.index.store.BaseSearchableSnapshotIndexInput; @@ -252,28 +251,18 @@ protected void readInternal(ByteBuffer b) throws IOException { directory.putCachedBlob(fileInfo.physicalName(), indexCacheMiss.v1(), content); return indexCacheMissLength; }, (channel, from, to, progressUpdater) -> { - // normally doesn't happen, we're already obtaining a range covering all cache misses above, but this - // can happen if the real populateAndRead call already failed to obtain this range of the file. In that - // case, we don't retry, we simply fail to populate the index cache. + // Normally doesn't happen, we're already obtaining a range covering all cache misses above, but theoretically + // possible in the case that the real populateAndRead call already failed to obtain this range of the file. In that + // case, try and fill just the cache miss from the blob store because there may be other reads waiting on this + // range. logger.debug( - "failed to fill index cache miss [{}-{}] of {} due to earlier failure", + "directly filling index cache miss [{}-{}] of {} due to earlier failure", from, to, CachedBlobContainerIndexInput.this ); - throw new IOException( - "failed to fill index cache miss [" - + from - + "-" - + to - + "] of [" - + CachedBlobContainerIndexInput.this - + "] due to earlier failure" - ); - }, - EsExecutors.newDirectExecutorService() // if ranges are still missing, fail immediately, so no need to fork - ); - + writeCacheFile(channel, from, to, progressUpdater); + }, directory.cacheFetchAsyncExecutor()); } final int bytesRead = populateCacheFuture.get(); From c5b65bc4c58eea930382bc93a1fa9600fceeab1f Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 10 Aug 2020 10:51:28 +0100 Subject: [PATCH 11/63] No need for a list with <= 1 element --- .../cache/CachedBlobContainerIndexInput.java | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index 2ea4f2a9ac721..f5e0952999f27 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -33,7 +33,6 @@ import java.io.InputStream; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; -import java.util.List; import java.util.Locale; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; @@ -157,7 +156,7 @@ protected void readInternal(ByteBuffer b) throws IOException { assert b.remaining() == length; } - final List> indexCacheMisses; + final Tuple indexCacheMiss; // null if not a miss // We prefer to use the index cache if the recovery is not done yet if (directory.isRecoveryDone() == false) { @@ -189,13 +188,13 @@ protected void readInternal(ByteBuffer b) throws IOException { // {start, end} ranges 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 - indexCacheMisses = List.of(Tuple.tuple(0L, fileInfo.length())); + indexCacheMiss = Tuple.tuple(0L, fileInfo.length()); } else { - indexCacheMisses = List.of(Tuple.tuple(0L, (long) BlobStoreCacheService.DEFAULT_SIZE)); + indexCacheMiss = Tuple.tuple(0L, (long) BlobStoreCacheService.DEFAULT_SIZE); } - logger.trace("recovery cache miss for [{}], falling through with regions [{}]", this, indexCacheMisses); + logger.trace("recovery cache miss for [{}], falling through with cache miss [{}]", this, indexCacheMiss); } else { - indexCacheMisses = List.of(); + indexCacheMiss = null; } try { @@ -207,8 +206,8 @@ protected void readInternal(ByteBuffer b) throws IOException { final Tuple endRangeToWrite = computeRange(position + length - 1); assert startRangeToWrite.v2() <= endRangeToWrite.v2() : startRangeToWrite + " vs " + endRangeToWrite; final Tuple rangeToWrite = Tuple.tuple( - Math.min(startRangeToWrite.v1(), indexCacheMisses.stream().mapToLong(Tuple::v1).max().orElse(Long.MAX_VALUE)), - Math.max(endRangeToWrite.v2(), indexCacheMisses.stream().mapToLong(Tuple::v2).max().orElse(Long.MIN_VALUE)) + Math.min(startRangeToWrite.v1(), indexCacheMiss == null ? Long.MAX_VALUE : indexCacheMiss.v1()), + Math.max(endRangeToWrite.v2(), indexCacheMiss == null ? Long.MIN_VALUE : indexCacheMiss.v2()) ); assert rangeToWrite.v1() <= position && position + length <= rangeToWrite.v2() : "[" @@ -233,7 +232,7 @@ protected void readInternal(ByteBuffer b) throws IOException { return read; }, this::writeCacheFile, directory.cacheFetchAsyncExecutor()); - for (Tuple indexCacheMiss : indexCacheMisses) { + if (indexCacheMiss != null) { cacheFile.populateAndRead(indexCacheMiss, indexCacheMiss, channel -> { final int indexCacheMissLength = Math.toIntExact(indexCacheMiss.v2() - indexCacheMiss.v1()); From 974c09593f29b89546e751c42ad99f2983b1b7a4 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 10 Aug 2020 10:53:58 +0100 Subject: [PATCH 12/63] Reorder defs --- .../index/store/cache/CachedBlobContainerIndexInput.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index f5e0952999f27..6cbcdebe565c7 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -161,10 +161,10 @@ protected void readInternal(ByteBuffer b) throws IOException { // We prefer to use the index cache if the recovery is not done yet if (directory.isRecoveryDone() == false) { // We try to use the snapshot blob cache if: - // - we're reading the first N bytes of the file - final boolean isStartOfFile = (position + length <= BlobStoreCacheService.DEFAULT_SIZE); // - the file is small enough to be fully cached in the blob cache final boolean canBeFullyCached = fileInfo.length() <= BlobStoreCacheService.DEFAULT_SIZE * 2; + // - we're reading the first N bytes of the file + final boolean isStartOfFile = (position + length <= BlobStoreCacheService.DEFAULT_SIZE); if (canBeFullyCached || isStartOfFile) { final CachedBlob cachedBlob = directory.getCachedBlob(fileInfo.physicalName(), 0L, length); From 2abf12671472b3a86002c85d797ccea52a6e7f09 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 10 Aug 2020 11:02:51 +0100 Subject: [PATCH 13/63] Mapping tweaks --- .../blobstore/cache/BlobStoreCacheService.java | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index 45ac3b24fb43c..39c49f61bc3ff 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -135,7 +135,7 @@ private static XContentBuilder mappings() throws IOException { builder.startObject(); { builder.startObject(SINGLE_MAPPING_NAME); - builder.field("dynamic", "false"); + builder.field("dynamic", "strict"); { builder.startObject("_meta"); builder.field("version", Version.CURRENT); @@ -154,12 +154,6 @@ private static XContentBuilder mappings() throws IOException { builder.field("format", "epoch_millis"); builder.endObject(); } - { - builder.startObject("accessed_time"); - builder.field("type", "date"); - builder.field("format", "epoch_millis"); - builder.endObject(); - } { builder.startObject("version"); builder.field("type", "integer"); From 4c8ed359d350767d1123e095adfebe0fed28dad5 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 11 Aug 2020 10:08:06 +0100 Subject: [PATCH 14/63] No need to index fields, we only get by ID --- .../blobstore/cache/BlobStoreCacheService.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index 39c49f61bc3ff..d2191cf815ce0 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -146,22 +146,26 @@ private static XContentBuilder mappings() throws IOException { { builder.startObject("type"); builder.field("type", "keyword"); + builder.field("index", "false"); builder.endObject(); } { builder.startObject("creation_time"); builder.field("type", "date"); builder.field("format", "epoch_millis"); + builder.field("index", "false"); builder.endObject(); } { builder.startObject("version"); builder.field("type", "integer"); + builder.field("index", "false"); builder.endObject(); } { builder.startObject("repository"); builder.field("type", "keyword"); + builder.field("index", "false"); builder.endObject(); } { @@ -172,9 +176,11 @@ private static XContentBuilder mappings() throws IOException { { builder.startObject("name"); builder.field("type", "keyword"); + builder.field("index", "false"); builder.endObject(); builder.startObject("path"); builder.field("type", "keyword"); + builder.field("index", "false"); builder.endObject(); } builder.endObject(); @@ -194,16 +200,19 @@ private static XContentBuilder mappings() throws IOException { { builder.startObject("length"); builder.field("type", "long"); + builder.field("index", "false"); builder.endObject(); } { builder.startObject("from"); builder.field("type", "long"); + builder.field("index", "false"); builder.endObject(); } { builder.startObject("to"); builder.field("type", "long"); + builder.field("index", "false"); builder.endObject(); } builder.endObject(); From 4b6798a73a18c6488503d69a643ef661516632ee Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 11 Aug 2020 10:29:06 +0100 Subject: [PATCH 15/63] Comment --- .../index/store/cache/CachedBlobContainerIndexInput.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index 6cbcdebe565c7..eed72d21d238d 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -272,6 +272,9 @@ protected void readInternal(ByteBuffer b) throws IOException { final int alreadyRead = length - b.remaining(); final int bytesRead = readDirectlyIfAlreadyClosed(position + alreadyRead, b, e); assert alreadyRead + bytesRead == length : alreadyRead + " + " + bytesRead + " vs " + length; + + // In principle we could handle an index cache miss here too, ensuring that the direct read was large enough, but this is + // already a rare case caused by an overfull/undersized cache. } stats.incrementBytesRead(lastReadPosition, position, length); From 8a3b1742dda99cef89bc7ce2253a86b9b227f1e5 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 11 Aug 2020 11:18:26 +0100 Subject: [PATCH 16/63] Rename --- .../cache/BlobStoreCacheService.java | 37 +++++++++++-------- 1 file changed, 21 insertions(+), 16 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index d2191cf815ce0..09d4e18645177 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -99,29 +99,34 @@ private void createIndexIfNecessary(ActionListener listener) { return; } try { - client.admin().indices().prepareCreate(index).setSettings(settings()).setMapping(mappings()).execute(new ActionListener<>() { - @Override - public void onResponse(CreateIndexResponse createIndexResponse) { - assert createIndexResponse.index().equals(index); - listener.onResponse(createIndexResponse.index()); - } + client.admin() + .indices() + .prepareCreate(index) + .setSettings(indexSettings()) + .setMapping(mappings()) + .execute(new ActionListener<>() { + @Override + public void onResponse(CreateIndexResponse createIndexResponse) { + assert createIndexResponse.index().equals(index); + listener.onResponse(createIndexResponse.index()); + } - @Override - public void onFailure(Exception e) { - if (e instanceof ResourceAlreadyExistsException - || ExceptionsHelper.unwrapCause(e) instanceof ResourceAlreadyExistsException) { - listener.onResponse(index); - } else { - listener.onFailure(e); + @Override + public void onFailure(Exception e) { + if (e instanceof ResourceAlreadyExistsException + || ExceptionsHelper.unwrapCause(e) instanceof ResourceAlreadyExistsException) { + listener.onResponse(index); + } else { + listener.onFailure(e); + } } - } - }); + }); } catch (Exception e) { listener.onFailure(e); } } - private static Settings settings() { + private static Settings indexSettings() { return Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, "0-1") From 80e42ffb0bcc09279332d3b0d5b06a9770dd7041 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 11 Aug 2020 12:01:41 +0100 Subject: [PATCH 17/63] Assert no more indexing into cache after first startup --- .../cache/BlobStoreCacheService.java | 3 + .../cache/CachedBlobContainerIndexInput.java | 4 +- ...ableSnapshotsBlobStoreCacheIntegTests.java | 62 +++---------------- 3 files changed, 14 insertions(+), 55 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index 09d4e18645177..3a794eafdd4eb 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -307,12 +307,15 @@ public void onResponse(String s) { public void onResponse(IndexResponse indexResponse) { if (indexResponse.status() == RestStatus.CREATED) { logger.trace("cache fill: [{}]", request.id()); + } else { + logger.trace("not created: [{}] returned [{}]", request.id(), indexResponse.status()); } listener.onResponse(null); } @Override public void onFailure(Exception e) { + logger.debug(new ParameterizedMessage("failure in cache fill: [{}]", request.id()), e); listener.onFailure(e); } }); diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index eed72d21d238d..558f025e2c4db 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -189,8 +189,10 @@ protected void readInternal(ByteBuffer b) throws IOException { if (canBeFullyCached) { // if the index input is smaller than twice the size of the blob cache, it will be fully indexed indexCacheMiss = Tuple.tuple(0L, fileInfo.length()); - } else { + } else if (isStartOfFile) { indexCacheMiss = Tuple.tuple(0L, (long) BlobStoreCacheService.DEFAULT_SIZE); + } else { + indexCacheMiss = null; } logger.trace("recovery cache miss for [{}], falling through with cache miss [{}]", this, indexCacheMiss); } else { diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java index 69eb78a789ded..91c436ed006cd 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java @@ -107,11 +107,11 @@ protected int numberOfReplicas() { protected Settings nodeSettings(int nodeOrdinal) { return Settings.builder() .put(super.nodeSettings(nodeOrdinal)) - // Use a cache range size setting aligned with BufferedIndexInput's buffer size and BlobStoreCacheService's default size - // TODO randomized this .put( CacheService.SNAPSHOT_CACHE_RANGE_SIZE_SETTING.getKey(), - new ByteSizeValue(BlobStoreCacheService.DEFAULT_SIZE, ByteSizeUnit.BYTES) + randomLongBetween( + new ByteSizeValue(4, ByteSizeUnit.KB).getBytes(), + new ByteSizeValue(20, ByteSizeUnit.KB).getBytes()) + "b" ) .put(CacheService.SNAPSHOT_CACHE_SIZE_SETTING.getKey(), new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES)) .build(); @@ -180,6 +180,8 @@ public void testBlobStoreCache() throws Exception { refreshSystemIndex(); final long numberOfCachedBlobs = systemClient().prepareSearch(SNAPSHOT_BLOB_CACHE_INDEX).get().getHits().getTotalHits().value; + final long numberOfCacheWrites = systemClient().admin().indices().prepareStats(SNAPSHOT_BLOB_CACHE_INDEX).clear().setIndexing(true) + .get().getTotal().indexing.getTotal().getIndexCount(); ensureBlobStoreRepositoriesWithActiveShards( restoredIndex, @@ -247,12 +249,11 @@ public void testBlobStoreCache() throws Exception { logger.info("--> verifying cached documents (again) in system index [{}]", SNAPSHOT_BLOB_CACHE_INDEX); assertCachedBlobsInSystemIndex(repositoryName, blobsInSnapshot); - logger.info("--> verifying that no cached blobs were indexed in system index [{}]", SNAPSHOT_BLOB_CACHE_INDEX); + logger.info("--> verifying that no extra cached blobs were indexed [{}]", SNAPSHOT_BLOB_CACHE_INDEX); refreshSystemIndex(); assertHitCount(systemClient().prepareSearch(SNAPSHOT_BLOB_CACHE_INDEX).setSize(0).get(), numberOfCachedBlobs); - - logger.info("--> verifying blobs read from the repository"); - assertBlobsReadFromRemoteRepository(restoredAgainIndex, blobsInSnapshot); + assertThat(systemClient().admin().indices().prepareStats(SNAPSHOT_BLOB_CACHE_INDEX).clear().setIndexing(true) + .get().getTotal().indexing.getTotal().getIndexCount(), equalTo(numberOfCacheWrites)); resetTrackedFiles(); @@ -295,10 +296,6 @@ public Settings onNodeStopped(String nodeName) throws Exception { logger.info("--> verifying that no cached blobs were indexed in system index [{}] after restart", SNAPSHOT_BLOB_CACHE_INDEX); assertHitCount(systemClient().prepareSearch(SNAPSHOT_BLOB_CACHE_INDEX).setSize(0).get(), numberOfCachedBlobs); - logger.info("--> verifying blobs read from the repository after restart"); - // Without the WaitForSnapshotBlobCacheShardsActivePlugin this would fail - assertBlobsReadFromRemoteRepository(restoredAgainIndex, blobsInSnapshot); - // TODO would be great to test when the index is frozen } @@ -399,49 +396,6 @@ private void assertCachedBlobsInSystemIndex(final String repositoryName, final M }); } - private void assertBlobsReadFromRemoteRepository( - final String indexName, - final Map blobsInSnapshot - ) { - ensureBlobStoreRepositoriesWithActiveShards(indexName, (nodeId, blobStore) -> { - for (Map.Entry>> blob : blobStore.blobs.entrySet()) { - final String blobName = blob.getKey(); - - if (blobName.endsWith(".dat") || blobName.equals("index-0")) { - // The snapshot metadata files are accessed when recovering from the snapshot during restore and do not benefit from - // the snapshot blob cache as the files are accessed outside of a searchable snapshot directory - assertThat( - blobName + " should be fully read from the beginning", - blob.getValue().stream().allMatch(read -> read.v1() == 0L), - is(true) - ); - // TODO assert it is read til the end - - // } else { - // BlobStoreIndexShardSnapshot.FileInfo blobInfo = null; - // for (BlobStoreIndexShardSnapshot blobs : blobsInSnapshot.values()) { - // for (BlobStoreIndexShardSnapshot.FileInfo fileInfo : blobs.indexFiles()) { - // for (int i = 0; i < fileInfo.numberOfParts(); i++) { - // if (blobName.endsWith(fileInfo.partName(i))) { - // blobInfo = fileInfo; - // break; - // } - // } - // } - // } - // assertThat("Unable to find blob " + blobName + " in the blobs on disk", blobInfo, notNullValue()); - // - // final String fileExtension = IndexFileNames.getExtension(blobInfo.physicalName()); - // assertThat( - // "Only compound files can be read from the blob store after blob store cache is populated, not " + blobInfo, - // fileExtension, - // equalTo("cfs") - // ); - } - } - }); - } - /** * Returns the {@link TrackingRepositoryPlugin} instance on a given node. */ From b052d7e1b15a2405e004eb998d866c898755b5d1 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 11 Aug 2020 17:04:45 +0100 Subject: [PATCH 18/63] Stronger assertions about what is read from the blob store when the cache is in play --- .../SearchableSnapshotShardStats.java | 18 +++++- .../SearchableSnapshotShardStatsTests.java | 3 +- .../index/store/IndexInputStats.java | 10 +++ .../cache/CachedBlobContainerIndexInput.java | 60 +++++++++++++++++- .../direct/DirectBlobContainerIndexInput.java | 1 + ...ansportSearchableSnapshotsStatsAction.java | 3 +- ...ableSnapshotsBlobStoreCacheIntegTests.java | 63 ++++++++++++++++--- ...SearchableSnapshotsStatsResponseTests.java | 3 +- 8 files changed, 143 insertions(+), 18 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java index 4d66aa351332c..b2449f1ccd7cd 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java @@ -137,13 +137,15 @@ public static class CacheIndexInputStats implements Writeable, ToXContentObject private final TimedCounter cachedBytesWritten; private final TimedCounter directBytesRead; private final TimedCounter optimizedBytesRead; + private final Counter blobStoreBytesRequested; public CacheIndexInputStats(String fileName, long fileLength, long openCount, long closeCount, Counter forwardSmallSeeks, Counter backwardSmallSeeks, Counter forwardLargeSeeks, Counter backwardLargeSeeks, Counter contiguousReads, Counter nonContiguousReads, Counter cachedBytesRead, TimedCounter cachedBytesWritten, - TimedCounter directBytesRead, TimedCounter optimizedBytesRead) { + TimedCounter directBytesRead, TimedCounter optimizedBytesRead, + Counter blobStoreBytesRequested) { this.fileName = fileName; this.fileLength = fileLength; this.openCount = openCount; @@ -158,6 +160,7 @@ public CacheIndexInputStats(String fileName, long fileLength, long openCount, lo this.cachedBytesWritten = cachedBytesWritten; this.directBytesRead = directBytesRead; this.optimizedBytesRead = optimizedBytesRead; + this.blobStoreBytesRequested = blobStoreBytesRequested; } CacheIndexInputStats(final StreamInput in) throws IOException { @@ -175,6 +178,7 @@ public CacheIndexInputStats(String fileName, long fileLength, long openCount, lo this.cachedBytesWritten = new TimedCounter(in); this.directBytesRead = new TimedCounter(in); this.optimizedBytesRead = new TimedCounter(in); + this.blobStoreBytesRequested = new Counter(in); } @Override @@ -194,6 +198,7 @@ public void writeTo(StreamOutput out) throws IOException { cachedBytesWritten.writeTo(out); directBytesRead.writeTo(out); optimizedBytesRead.writeTo(out); + blobStoreBytesRequested.writeTo(out); } public String getFileName() { @@ -252,6 +257,10 @@ public TimedCounter getOptimizedBytesRead() { return optimizedBytesRead; } + public Counter getBlobStoreBytesRequested() { + return blobStoreBytesRequested; + } + @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); @@ -278,6 +287,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field("large", getBackwardLargeSeeks()); builder.endObject(); } + builder.field("blob_store_bytes_requested", getBlobStoreBytesRequested()); } return builder.endObject(); } @@ -304,7 +314,8 @@ public boolean equals(Object other) { && Objects.equals(cachedBytesRead, stats.cachedBytesRead) && Objects.equals(cachedBytesWritten, stats.cachedBytesWritten) && Objects.equals(directBytesRead, stats.directBytesRead) - && Objects.equals(optimizedBytesRead, stats.optimizedBytesRead); + && Objects.equals(optimizedBytesRead, stats.optimizedBytesRead) + && Objects.equals(blobStoreBytesRequested, stats.blobStoreBytesRequested); } @Override @@ -314,7 +325,8 @@ public int hashCode() { forwardLargeSeeks, backwardLargeSeeks, contiguousReads, nonContiguousReads, cachedBytesRead, cachedBytesWritten, - directBytesRead, optimizedBytesRead); + directBytesRead, optimizedBytesRead, + blobStoreBytesRequested); } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java index dade0a0ca4204..dbab8122453be 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java @@ -47,7 +47,8 @@ private CacheIndexInputStats randomCacheIndexInputStats() { randomCounter(), randomCounter(), randomCounter(), randomCounter(), randomCounter(), randomTimedCounter(), - randomTimedCounter(), randomTimedCounter()); + randomTimedCounter(), randomTimedCounter(), + randomCounter()); } private Counter randomCounter() { diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java index 8cc4b0e4f9d8f..33b3672df2684 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java @@ -45,6 +45,8 @@ public class IndexInputStats { private final Counter cachedBytesRead = new Counter(); private final TimedCounter cachedBytesWritten = new TimedCounter(); + private final Counter blobStoreBytesRequested = new Counter(); + public IndexInputStats(long fileLength, LongSupplier currentTimeNanos) { this(fileLength, SEEKING_THRESHOLD.getBytes(), currentTimeNanos); } @@ -112,6 +114,10 @@ public void incrementSeeks(long currentPosition, long newPosition) { } } + public void addBlobStoreBytesRequested(long bytesRequested) { + blobStoreBytesRequested.add(bytesRequested); + } + public long getFileLength() { return fileLength; } @@ -164,6 +170,10 @@ public TimedCounter getCachedBytesWritten() { return cachedBytesWritten; } + public Counter getBlobStoreBytesRequested() { + return blobStoreBytesRequested; + } + @SuppressForbidden(reason = "Handles Long.MIN_VALUE before using Math.abs()") public boolean isLargeSeek(long delta) { return delta != Long.MIN_VALUE && Math.abs(delta) > seekingThreshold; diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index 558f025e2c4db..fd09330a7c4bf 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -13,6 +13,8 @@ import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefIterator; import org.elasticsearch.blobstore.cache.BlobStoreCacheService; import org.elasticsearch.blobstore.cache.CachedBlob; import org.elasticsearch.common.Nullable; @@ -144,8 +146,6 @@ protected void readInternal(ByteBuffer b) throws IOException { final long position = getFilePointer() + this.offset; final int length = b.remaining(); - logger.trace("readInternal: read [{}-{}] from [{}]", position, position + length, this); - // We can detect that we're going to read the last 16 bytes (that contains the footer checksum) of the file. Such reads are often // executed when opening a Directory and since we have the checksum in the snapshot metadata we can use it to fill the ByteBuffer. if (length == CodecUtil.footerLength() && isClone == false && position == fileInfo.length() - length) { @@ -156,6 +156,8 @@ protected void readInternal(ByteBuffer b) throws IOException { assert b.remaining() == length; } + logger.trace("readInternal: read [{}-{}] ([{}] bytes) from [{}]", position, position + length, length, this); + final Tuple indexCacheMiss; // null if not a miss // We prefer to use the index cache if the recovery is not done yet @@ -176,6 +178,50 @@ protected void readInternal(ByteBuffer b) throws IOException { position ); b.put(BytesReference.toBytes(cachedBlob.bytes().slice(Math.toIntExact(position), length))); + + try { + final CacheFile cacheFile = getCacheFileSafe(); + try (Releasable ignored = cacheFile.fileLock()) { + final Tuple cachedRange = Tuple.tuple(cachedBlob.from(), cachedBlob.to()); + cacheFile.populateAndRead( + cachedRange, + cachedRange, + channel -> cachedBlob.length(), + (channel, from, to, progressUpdater) -> { + final long startTimeNanos = stats.currentTimeNanos(); + final BytesRefIterator iterator = cachedBlob.bytes() + .slice(Math.toIntExact(from - cachedBlob.from()), Math.toIntExact(to - from)) + .iterator(); + long writePosition = from; + BytesRef current; + while ((current = iterator.next()) != null) { + final ByteBuffer byteBuffer = ByteBuffer.wrap(current.bytes, current.offset, current.length); + while (byteBuffer.remaining() > 0) { + writePosition += positionalWrite(channel, writePosition, byteBuffer); + progressUpdater.accept(writePosition); + } + } + assert writePosition == to : writePosition + " vs " + to; + final long endTimeNanos = stats.currentTimeNanos(); + stats.addCachedBytesWritten(to - from, endTimeNanos - startTimeNanos); + logger.trace("copied bytes [{}-{}] of file [{}] from index cache to node cache", from, to, fileInfo); + }, + directory.cacheFetchAsyncExecutor() + ); + } + } catch (Exception e) { + logger.debug( + new ParameterizedMessage( + "failed to store bytes [{}-{}] of file [{}] obtained from index cache", + cachedBlob.from(), + cachedBlob.to(), + fileInfo + ), + e + ); + // oh well, no big deal, at least we can return them to the caller. + } + return; } @@ -568,10 +614,20 @@ private InputStream openInputStreamFromBlobStore(final long position, final long + "] from [" + fileInfo + "]"; + stats.addBlobStoreBytesRequested(length); return blobContainer.readBlob(fileInfo.partName(0L), position, length); } else { final long startPart = getPartNumberForPosition(position); final long endPart = getPartNumberForPosition(position + length - 1); + + for (long currentPart = startPart; currentPart <= endPart; currentPart++) { + final long startInPart = (currentPart == startPart) ? getRelativePositionInPart(position) : 0L; + final long endInPart = (currentPart == endPart) + ? getRelativePositionInPart(position + length - 1) + 1 + : getLengthOfPart(currentPart); + stats.addBlobStoreBytesRequested(endInPart - startInPart); + } + return new SlicedInputStream(endPart - startPart + 1L) { @Override protected InputStream openSlice(long slice) throws IOException { diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/direct/DirectBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/direct/DirectBlobContainerIndexInput.java index 18e779578a286..1386c67de671a 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/direct/DirectBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/direct/DirectBlobContainerIndexInput.java @@ -338,6 +338,7 @@ public String toString() { private InputStream openBlobStream(int part, long pos, long length) throws IOException { assert assertCurrentThreadMayAccessBlobStore(); + stats.addBlobStoreBytesRequested(length); return blobContainer.readBlob(fileInfo.partName(part), pos, length); } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java index a2e9ac0c1f478..47f4a77d0cbe9 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java @@ -110,7 +110,8 @@ private static CacheIndexInputStats toCacheIndexInputStats(final String fileName toCounter(inputStats.getCachedBytesRead()), toTimedCounter(inputStats.getCachedBytesWritten()), toTimedCounter(inputStats.getDirectBytesRead()), - toTimedCounter(inputStats.getOptimizedBytesRead()) + toTimedCounter(inputStats.getOptimizedBytesRead()), + toCounter(inputStats.getBlobStoreBytesRequested()) ); } diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java index 91c436ed006cd..6f71fb9e93e66 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java @@ -23,6 +23,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.BlobPath; import org.elasticsearch.common.blobstore.BlobStore; @@ -48,11 +49,15 @@ import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.InternalTestCluster; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.ClientHelper; +import org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotShardStats; import org.elasticsearch.xpack.searchablesnapshots.BaseSearchableSnapshotsIntegTestCase; import org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots; import org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants; +import org.elasticsearch.xpack.searchablesnapshots.action.SearchableSnapshotsStatsAction; +import org.elasticsearch.xpack.searchablesnapshots.action.SearchableSnapshotsStatsRequest; import org.elasticsearch.xpack.searchablesnapshots.cache.CacheService; import java.io.FilterInputStream; @@ -87,6 +92,7 @@ import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; +@TestLogging(reason = "nocommit", value = "org.elasticsearch.index.store.cache.CachedBlobContainerIndexInput:TRACE") public class SearchableSnapshotsBlobStoreCacheIntegTests extends BaseSearchableSnapshotsIntegTestCase { @Override @@ -109,9 +115,7 @@ protected Settings nodeSettings(int nodeOrdinal) { .put(super.nodeSettings(nodeOrdinal)) .put( CacheService.SNAPSHOT_CACHE_RANGE_SIZE_SETTING.getKey(), - randomLongBetween( - new ByteSizeValue(4, ByteSizeUnit.KB).getBytes(), - new ByteSizeValue(20, ByteSizeUnit.KB).getBytes()) + "b" + randomLongBetween(new ByteSizeValue(4, ByteSizeUnit.KB).getBytes(), new ByteSizeValue(20, ByteSizeUnit.KB).getBytes()) + "b" ) .put(CacheService.SNAPSHOT_CACHE_SIZE_SETTING.getKey(), new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES)) .build(); @@ -129,9 +133,12 @@ public void testBlobStoreCache() throws Exception { final long numberOfDocs = indexRequestBuilders.size(); final NumShards numberOfShards = getNumShards(indexName); - final ForceMergeResponse forceMergeResponse = client().admin().indices().prepareForceMerge(indexName).setMaxNumSegments(1).get(); - assertThat(forceMergeResponse.getSuccessfulShards(), equalTo(numberOfShards.totalNumShards)); - assertThat(forceMergeResponse.getFailedShards(), equalTo(0)); + if (randomBoolean()) { + logger.info("--> force-merging index before snapshotting"); + final ForceMergeResponse forceMergeResponse = client().admin().indices().prepareForceMerge(indexName).setMaxNumSegments(1).get(); + assertThat(forceMergeResponse.getSuccessfulShards(), equalTo(numberOfShards.totalNumShards)); + assertThat(forceMergeResponse.getFailedShards(), equalTo(0)); + } final String repositoryName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); final Path repositoryLocation = randomRepoPath(); @@ -175,13 +182,27 @@ public void testBlobStoreCache() throws Exception { ensureGreen(restoredIndex); ensureExecutorsAreIdle(); + for (final SearchableSnapshotShardStats shardStats : client().execute( + SearchableSnapshotsStatsAction.INSTANCE, + new SearchableSnapshotsStatsRequest() + ).actionGet().getStats()) { + for (final SearchableSnapshotShardStats.CacheIndexInputStats indexInputStats : shardStats.getStats()) { + assertThat(Strings.toString(indexInputStats), indexInputStats.getBlobStoreBytesRequested().getCount(), greaterThan(0L)); + } + } + logger.info("--> verifying cached documents in system index [{}]", SNAPSHOT_BLOB_CACHE_INDEX); assertCachedBlobsInSystemIndex(repositoryName, blobsInSnapshot); refreshSystemIndex(); final long numberOfCachedBlobs = systemClient().prepareSearch(SNAPSHOT_BLOB_CACHE_INDEX).get().getHits().getTotalHits().value; - final long numberOfCacheWrites = systemClient().admin().indices().prepareStats(SNAPSHOT_BLOB_CACHE_INDEX).clear().setIndexing(true) - .get().getTotal().indexing.getTotal().getIndexCount(); + final long numberOfCacheWrites = systemClient().admin() + .indices() + .prepareStats(SNAPSHOT_BLOB_CACHE_INDEX) + .clear() + .setIndexing(true) + .get() + .getTotal().indexing.getTotal().getIndexCount(); ensureBlobStoreRepositoriesWithActiveShards( restoredIndex, @@ -227,6 +248,24 @@ public void testBlobStoreCache() throws Exception { ensureGreen(restoredAgainIndex); ensureExecutorsAreIdle(); + logger.info("--> verifying shards of [{}] were started without using the blob store more than necessary", restoredAgainIndex); + for (final SearchableSnapshotShardStats shardStats : client().execute( + SearchableSnapshotsStatsAction.INSTANCE, + new SearchableSnapshotsStatsRequest() + ).actionGet().getStats()) { + for (final SearchableSnapshotShardStats.CacheIndexInputStats indexInputStats : shardStats.getStats()) { + final boolean mayReadMoreThanHeader + // we read the header of each file contained within the .cfs file, which could be anywhere + = indexInputStats.getFileName().endsWith(".cfs") + // we read a couple of longs at the end of the .fdt file (see https://issues.apache.org/jira/browse/LUCENE-9456) + // TODO revisit this when this issue is addressed in Lucene + || indexInputStats.getFileName().endsWith(".fdt"); + if (indexInputStats.getFileLength() <= BlobStoreCacheService.DEFAULT_SIZE * 2 || mayReadMoreThanHeader == false) { + assertThat(Strings.toString(indexInputStats), indexInputStats.getBlobStoreBytesRequested().getCount(), equalTo(0L)); + } + } + } + logger.info("--> verifying documents in index [{}]", restoredAgainIndex); assertHitCount(client().prepareSearch(restoredAgainIndex).setSize(0).setTrackTotalHits(true).get(), numberOfDocs); assertHitCount( @@ -252,8 +291,12 @@ public void testBlobStoreCache() throws Exception { logger.info("--> verifying that no extra cached blobs were indexed [{}]", SNAPSHOT_BLOB_CACHE_INDEX); refreshSystemIndex(); assertHitCount(systemClient().prepareSearch(SNAPSHOT_BLOB_CACHE_INDEX).setSize(0).get(), numberOfCachedBlobs); - assertThat(systemClient().admin().indices().prepareStats(SNAPSHOT_BLOB_CACHE_INDEX).clear().setIndexing(true) - .get().getTotal().indexing.getTotal().getIndexCount(), equalTo(numberOfCacheWrites)); + assertThat( + systemClient().admin().indices().prepareStats(SNAPSHOT_BLOB_CACHE_INDEX).clear().setIndexing(true).get().getTotal().indexing + .getTotal() + .getIndexCount(), + equalTo(numberOfCacheWrites) + ); resetTrackedFiles(); diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java index fd04ce4b7f226..b73b5f44e04f1 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java @@ -109,7 +109,8 @@ private static SearchableSnapshotShardStats.CacheIndexInputStats randomCacheInde randomCounter(), randomTimedCounter(), randomTimedCounter(), - randomTimedCounter() + randomTimedCounter(), + randomCounter() ); } From d4919fad9f2748e9e2c6e979baeaba0151e4c8bd Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 11 Aug 2020 17:10:14 +0100 Subject: [PATCH 19/63] No need for a fake blob store when we track accesses in the index input stats instead --- ...ableSnapshotsBlobStoreCacheIntegTests.java | 215 +----------------- 1 file changed, 12 insertions(+), 203 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java index 6f71fb9e93e66..3ba05fd43108d 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java @@ -12,44 +12,28 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.client.Client; import org.elasticsearch.client.OriginSettingClient; -import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetadata; -import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.routing.ShardsIterator; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.Decision; -import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.blobstore.BlobContainer; -import org.elasticsearch.common.blobstore.BlobPath; -import org.elasticsearch.common.blobstore.BlobStore; -import org.elasticsearch.common.blobstore.support.FilterBlobContainer; -import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; 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.index.IndexNotFoundException; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; -import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.plugins.ClusterPlugin; import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.plugins.PluginsService; -import org.elasticsearch.plugins.RepositoryPlugin; -import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.InternalTestCluster; -import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.ClientHelper; import org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotShardStats; @@ -60,9 +44,7 @@ import org.elasticsearch.xpack.searchablesnapshots.action.SearchableSnapshotsStatsRequest; import org.elasticsearch.xpack.searchablesnapshots.cache.CacheService; -import java.io.FilterInputStream; import java.io.IOException; -import java.io.InputStream; import java.nio.file.FileVisitResult; import java.nio.file.Files; import java.nio.file.Path; @@ -70,14 +52,11 @@ import java.nio.file.attribute.BasicFileAttributes; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ThreadPoolExecutor; -import java.util.function.BiConsumer; import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.INDEX_SHARD_SNAPSHOT_FORMAT; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -87,19 +66,14 @@ import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.SNAPSHOT_BLOB_CACHE_INDEX; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; -import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.notNullValue; -@TestLogging(reason = "nocommit", value = "org.elasticsearch.index.store.cache.CachedBlobContainerIndexInput:TRACE") public class SearchableSnapshotsBlobStoreCacheIntegTests extends BaseSearchableSnapshotsIntegTestCase { @Override protected Collection> nodePlugins() { final List> plugins = new ArrayList<>(); plugins.add(WaitForSnapshotBlobCacheShardsActivePlugin.class); - plugins.add(TrackingRepositoryPlugin.class); plugins.addAll(super.nodePlugins()); return List.copyOf(plugins); } @@ -135,7 +109,11 @@ public void testBlobStoreCache() throws Exception { if (randomBoolean()) { logger.info("--> force-merging index before snapshotting"); - final ForceMergeResponse forceMergeResponse = client().admin().indices().prepareForceMerge(indexName).setMaxNumSegments(1).get(); + final ForceMergeResponse forceMergeResponse = client().admin() + .indices() + .prepareForceMerge(indexName) + .setMaxNumSegments(1) + .get(); assertThat(forceMergeResponse.getSuccessfulShards(), equalTo(numberOfShards.totalNumShards)); assertThat(forceMergeResponse.getFailedShards(), equalTo(0)); } @@ -155,7 +133,7 @@ public void testBlobStoreCache() throws Exception { assertAcked(client().admin().cluster().prepareDeleteRepository(repositoryName)); createRepository( repositoryName, - TrackingRepositoryPlugin.TRACKING, + "fs", Settings.builder().put(FsRepository.LOCATION_SETTING.getKey(), repositoryLocation).build(), false ); @@ -183,8 +161,8 @@ public void testBlobStoreCache() throws Exception { ensureExecutorsAreIdle(); for (final SearchableSnapshotShardStats shardStats : client().execute( - SearchableSnapshotsStatsAction.INSTANCE, - new SearchableSnapshotsStatsRequest() + SearchableSnapshotsStatsAction.INSTANCE, + new SearchableSnapshotsStatsRequest() ).actionGet().getStats()) { for (final SearchableSnapshotShardStats.CacheIndexInputStats indexInputStats : shardStats.getStats()) { assertThat(Strings.toString(indexInputStats), indexInputStats.getBlobStoreBytesRequested().getCount(), greaterThan(0L)); @@ -204,15 +182,6 @@ public void testBlobStoreCache() throws Exception { .get() .getTotal().indexing.getTotal().getIndexCount(); - ensureBlobStoreRepositoriesWithActiveShards( - restoredIndex, - (nodeId, blobStore) -> assertThat( - "Blob read operations should have been executed on node [" + nodeId + ']', - blobStore.numberOfReads(), - greaterThan(0L) - ) - ); - logger.info("--> verifying documents in index [{}]", restoredIndex); assertHitCount(client().prepareSearch(restoredIndex).setSize(0).setTrackTotalHits(true).get(), numberOfDocs); assertHitCount( @@ -233,7 +202,6 @@ public void testBlobStoreCache() throws Exception { ); assertAcked(client().admin().indices().prepareDelete(restoredIndex)); - resetTrackedFiles(); logger.info("--> mount snapshot [{}] as an index for the second time", snapshot); final String restoredAgainIndex = mountSnapshot( @@ -250,13 +218,13 @@ public void testBlobStoreCache() throws Exception { logger.info("--> verifying shards of [{}] were started without using the blob store more than necessary", restoredAgainIndex); for (final SearchableSnapshotShardStats shardStats : client().execute( - SearchableSnapshotsStatsAction.INSTANCE, - new SearchableSnapshotsStatsRequest() + SearchableSnapshotsStatsAction.INSTANCE, + new SearchableSnapshotsStatsRequest() ).actionGet().getStats()) { for (final SearchableSnapshotShardStats.CacheIndexInputStats indexInputStats : shardStats.getStats()) { final boolean mayReadMoreThanHeader - // we read the header of each file contained within the .cfs file, which could be anywhere - = indexInputStats.getFileName().endsWith(".cfs") + // we read the header of each file contained within the .cfs file, which could be anywhere + = indexInputStats.getFileName().endsWith(".cfs") // we read a couple of longs at the end of the .fdt file (see https://issues.apache.org/jira/browse/LUCENE-9456) // TODO revisit this when this issue is addressed in Lucene || indexInputStats.getFileName().endsWith(".fdt"); @@ -298,8 +266,6 @@ public void testBlobStoreCache() throws Exception { equalTo(numberOfCacheWrites) ); - resetTrackedFiles(); - logger.info("--> restarting cluster"); internalCluster().fullRestart(new InternalTestCluster.RestartCallback() { @Override @@ -439,163 +405,6 @@ private void assertCachedBlobsInSystemIndex(final String repositoryName, final M }); } - /** - * Returns the {@link TrackingRepositoryPlugin} instance on a given node. - */ - private TrackingRepositoryPlugin getTrackingRepositoryInstance(String node) { - DiscoveryNode discoveryNode = clusterService().state().nodes().resolveNode(node); - assertThat("Cannot find node " + node, discoveryNode, notNullValue()); - - PluginsService pluginsService = internalCluster().getInstance(PluginsService.class, discoveryNode.getName()); - assertThat("Cannot find PluginsService on node " + node, pluginsService, notNullValue()); - - List trackingRepositoryPlugins = pluginsService.filterPlugins(TrackingRepositoryPlugin.class); - assertThat("List of TrackingRepositoryPlugin is null on node " + node, trackingRepositoryPlugins, notNullValue()); - assertThat("List of TrackingRepositoryPlugin is empty on node " + node, trackingRepositoryPlugins, hasSize(1)); - - TrackingRepositoryPlugin trackingRepositoryPlugin = trackingRepositoryPlugins.get(0); - assertThat("TrackingRepositoryPlugin is null on node " + node, trackingRepositoryPlugin, notNullValue()); - return trackingRepositoryPlugin; - } - - private void resetTrackedFiles() { - for (String nodeName : internalCluster().getNodeNames()) { - final TrackingRepositoryPlugin tracker = getTrackingRepositoryInstance(nodeName); - tracker.reset(); - assertThat(tracker.numberOfReads(), equalTo(0L)); - assertThat(tracker.blobs.size(), equalTo(0)); - } - } - - private void ensureBlobStoreRepositoriesWithActiveShards(String indexName, BiConsumer consumer) { - final ClusterState clusterState = clusterService().state(); - assertTrue(clusterState.metadata().hasIndex(indexName)); - assertTrue(SearchableSnapshotsConstants.isSearchableSnapshotStore(clusterState.metadata().index(indexName).getSettings())); - final IndexRoutingTable indexRoutingTable = clusterState.routingTable().index(indexName); - assertThat(indexRoutingTable, notNullValue()); - - final ShardsIterator shardsIterator = indexRoutingTable.randomAllActiveShardsIt(); - assertThat(shardsIterator.size(), greaterThanOrEqualTo(getNumShards(indexName).numPrimaries)); - - for (ShardRouting shardRouting : shardsIterator) { - consumer.accept(shardRouting.currentNodeId(), getTrackingRepositoryInstance(shardRouting.currentNodeId())); - } - } - - /** - * A plugin that allows to track the read operations on blobs - */ - public static class TrackingRepositoryPlugin extends Plugin implements RepositoryPlugin { - - static final String TRACKING = "tracking"; - - private final Map>> blobs = new ConcurrentHashMap<>(); - - long numberOfReads() { - return blobs.values().stream().flatMap(Collection::stream).mapToLong(Tuple::v2).sum(); - } - - void reset() { - blobs.clear(); - } - - @Override - public Map getRepositories( - Environment env, - NamedXContentRegistry namedXContentRegistry, - ClusterService clusterService, - RecoverySettings recoverySettings - ) { - return Collections.singletonMap( - TRACKING, - (metadata) -> new FsRepository(metadata, env, namedXContentRegistry, clusterService, recoverySettings) { - - @Override - protected BlobStore createBlobStore() throws Exception { - final BlobStore delegate = super.createBlobStore(); - return new BlobStore() { - @Override - public BlobContainer blobContainer(BlobPath path) { - return new TrackingFilesBlobContainer(delegate.blobContainer(path)); - } - - @Override - public void close() throws IOException { - delegate.close(); - } - }; - } - } - ); - } - - class TrackingFilesBlobContainer extends FilterBlobContainer { - - TrackingFilesBlobContainer(BlobContainer delegate) { - super(delegate); - } - - @Override - public InputStream readBlob(String blobName) throws IOException { - return new CountingInputStream(buildPath(blobName), 0L, super.readBlob(blobName)); - } - - @Override - public InputStream readBlob(String blobName, long position, long length) throws IOException { - return new CountingInputStream(buildPath(blobName), position, super.readBlob(blobName, position, length)); - } - - private String buildPath(String name) { - return path().buildAsString() + name; - } - - @Override - protected BlobContainer wrapChild(BlobContainer child) { - return new TrackingFilesBlobContainer(child); - } - } - - class CountingInputStream extends FilterInputStream { - - private final String name; - private final long offset; - - long bytesRead = 0L; - - protected CountingInputStream(String name, long offset, InputStream in) { - super(in); - this.name = name; - this.offset = offset; - } - - @Override - public int read() throws IOException { - final int result = in.read(); - if (result == -1) { - return result; - } - bytesRead += 1L; - return result; - } - - @Override - public int read(byte[] b, int offset, int len) throws IOException { - final int result = in.read(b, offset, len); - if (result == -1) { - return result; - } - bytesRead += len; - return result; - } - - @Override - public void close() throws IOException { - blobs.computeIfAbsent(name, n -> Collections.synchronizedList(new ArrayList<>())).add(Tuple.tuple(offset, bytesRead)); - super.close(); - } - } - } - /** * This plugin declares an {@link AllocationDecider} that forces searchable snapshot shards to be allocated after * the primary shards of the snapshot blob cache index are started. This way we can ensure that searchable snapshot From d2632ee1a7c5fc973f9155ea127d86a0d214327e Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 11 Aug 2020 17:22:19 +0100 Subject: [PATCH 20/63] Assert no indexing at all after restart --- ...archableSnapshotsBlobStoreCacheIntegTests.java | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java index 3ba05fd43108d..406644c714865 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java @@ -145,8 +145,6 @@ public void testBlobStoreCache() throws Exception { () -> systemClient().admin().indices().prepareGetIndex().addIndices(SNAPSHOT_BLOB_CACHE_INDEX).get() ); - final boolean usePrewarming = false; // TODO randomize this and adapt test - logger.info("--> mount snapshot [{}] as an index for the first time", snapshot); final String restoredIndex = mountSnapshot( repositoryName, @@ -154,7 +152,7 @@ public void testBlobStoreCache() throws Exception { indexName, Settings.builder() .put(SearchableSnapshots.SNAPSHOT_CACHE_ENABLED_SETTING.getKey(), true) - .put(SearchableSnapshots.SNAPSHOT_CACHE_PREWARM_ENABLED_SETTING.getKey(), usePrewarming) + .put(SearchableSnapshots.SNAPSHOT_CACHE_PREWARM_ENABLED_SETTING.getKey(), false) .build() ); ensureGreen(restoredIndex); @@ -210,7 +208,7 @@ public void testBlobStoreCache() throws Exception { indexName, Settings.builder() .put(SearchableSnapshots.SNAPSHOT_CACHE_ENABLED_SETTING.getKey(), true) - .put(SearchableSnapshots.SNAPSHOT_CACHE_PREWARM_ENABLED_SETTING.getKey(), usePrewarming) + .put(SearchableSnapshots.SNAPSHOT_CACHE_PREWARM_ENABLED_SETTING.getKey(), false) .build() ); ensureGreen(restoredAgainIndex); @@ -304,8 +302,15 @@ public Settings onNodeStopped(String nodeName) throws Exception { logger.info("--> verifying that no cached blobs were indexed in system index [{}] after restart", SNAPSHOT_BLOB_CACHE_INDEX); assertHitCount(systemClient().prepareSearch(SNAPSHOT_BLOB_CACHE_INDEX).setSize(0).get(), numberOfCachedBlobs); + assertThat( + systemClient().admin().indices().prepareStats(SNAPSHOT_BLOB_CACHE_INDEX).clear().setIndexing(true).get().getTotal().indexing + .getTotal() + .getIndexCount(), + equalTo(0L) + ); - // TODO would be great to test when the index is frozen + // TODO also test when the index is frozen + // TODO also test when prewarming is enabled } /** From 24f646f8a0ba141ebebefac1cc719ffffdafb895 Mon Sep 17 00:00:00 2001 From: David Turner Date: Fri, 14 Aug 2020 12:08:59 +0100 Subject: [PATCH 21/63] Block test until cache fills are complete --- .../SearchableSnapshotShardStats.java | 29 +++++++++++++++---- .../SearchableSnapshotShardStatsTests.java | 2 +- .../cache/BlobStoreCacheService.java | 7 ++--- .../index/store/IndexInputStats.java | 15 ++++++++++ .../store/SearchableSnapshotDirectory.java | 4 +-- .../cache/CachedBlobContainerIndexInput.java | 14 ++++++++- ...ansportSearchableSnapshotsStatsAction.java | 3 +- ...ableSnapshotsBlobStoreCacheIntegTests.java | 13 ++++++++- ...SearchableSnapshotsStatsResponseTests.java | 3 +- 9 files changed, 73 insertions(+), 17 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java index b2449f1ccd7cd..02b295d7534c3 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java @@ -5,6 +5,7 @@ */ package org.elasticsearch.xpack.core.searchablesnapshots; +import org.elasticsearch.Version; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -138,6 +139,7 @@ public static class CacheIndexInputStats implements Writeable, ToXContentObject private final TimedCounter directBytesRead; private final TimedCounter optimizedBytesRead; private final Counter blobStoreBytesRequested; + private final long currentIndexCacheFills; public CacheIndexInputStats(String fileName, long fileLength, long openCount, long closeCount, Counter forwardSmallSeeks, Counter backwardSmallSeeks, @@ -145,7 +147,7 @@ public CacheIndexInputStats(String fileName, long fileLength, long openCount, lo Counter contiguousReads, Counter nonContiguousReads, Counter cachedBytesRead, TimedCounter cachedBytesWritten, TimedCounter directBytesRead, TimedCounter optimizedBytesRead, - Counter blobStoreBytesRequested) { + Counter blobStoreBytesRequested, long currentIndexCacheFills) { this.fileName = fileName; this.fileLength = fileLength; this.openCount = openCount; @@ -161,6 +163,7 @@ public CacheIndexInputStats(String fileName, long fileLength, long openCount, lo this.directBytesRead = directBytesRead; this.optimizedBytesRead = optimizedBytesRead; this.blobStoreBytesRequested = blobStoreBytesRequested; + this.currentIndexCacheFills = currentIndexCacheFills; } CacheIndexInputStats(final StreamInput in) throws IOException { @@ -178,7 +181,14 @@ public CacheIndexInputStats(String fileName, long fileLength, long openCount, lo this.cachedBytesWritten = new TimedCounter(in); this.directBytesRead = new TimedCounter(in); this.optimizedBytesRead = new TimedCounter(in); - this.blobStoreBytesRequested = new Counter(in); + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + this.blobStoreBytesRequested = new Counter(in); + this.currentIndexCacheFills = in.readVLong(); + } else { + this.blobStoreBytesRequested = new Counter(0, 0, 0, 0); + this.currentIndexCacheFills = 0; + + } } @Override @@ -198,7 +208,10 @@ public void writeTo(StreamOutput out) throws IOException { cachedBytesWritten.writeTo(out); directBytesRead.writeTo(out); optimizedBytesRead.writeTo(out); - blobStoreBytesRequested.writeTo(out); + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + blobStoreBytesRequested.writeTo(out); + out.writeVLong(currentIndexCacheFills); + } } public String getFileName() { @@ -261,6 +274,10 @@ public Counter getBlobStoreBytesRequested() { return blobStoreBytesRequested; } + public long getCurrentIndexCacheFills() { + return currentIndexCacheFills; + } + @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); @@ -288,6 +305,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.endObject(); } builder.field("blob_store_bytes_requested", getBlobStoreBytesRequested()); + builder.field("current_index_cache_fills", getCurrentIndexCacheFills()); } return builder.endObject(); } @@ -315,7 +333,8 @@ public boolean equals(Object other) { && Objects.equals(cachedBytesWritten, stats.cachedBytesWritten) && Objects.equals(directBytesRead, stats.directBytesRead) && Objects.equals(optimizedBytesRead, stats.optimizedBytesRead) - && Objects.equals(blobStoreBytesRequested, stats.blobStoreBytesRequested); + && Objects.equals(blobStoreBytesRequested, stats.blobStoreBytesRequested) + && currentIndexCacheFills == stats.currentIndexCacheFills; } @Override @@ -326,7 +345,7 @@ public int hashCode() { contiguousReads, nonContiguousReads, cachedBytesRead, cachedBytesWritten, directBytesRead, optimizedBytesRead, - blobStoreBytesRequested); + blobStoreBytesRequested, currentIndexCacheFills); } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java index dbab8122453be..70eeab11c086e 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java @@ -48,7 +48,7 @@ private CacheIndexInputStats randomCacheIndexInputStats() { randomCounter(), randomCounter(), randomCounter(), randomTimedCounter(), randomTimedCounter(), randomTimedCounter(), - randomCounter()); + randomCounter(), randomNonNegativeLong()); } private Counter randomCounter() { diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index 3a794eafdd4eb..918f33129949c 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -31,6 +31,7 @@ import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.IndexNotFoundException; @@ -305,11 +306,7 @@ public void onResponse(String s) { client.index(request, new ActionListener<>() { @Override public void onResponse(IndexResponse indexResponse) { - if (indexResponse.status() == RestStatus.CREATED) { - logger.trace("cache fill: [{}]", request.id()); - } else { - logger.trace("not created: [{}] returned [{}]", request.id(), indexResponse.status()); - } + logger.trace("cache fill ({}): [{}]", indexResponse.status(), request.id()); listener.onResponse(null); } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java index 33b3672df2684..e087268e21f0e 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java @@ -6,6 +6,7 @@ package org.elasticsearch.index.store; import org.elasticsearch.common.SuppressForbidden; +import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.index.store.cache.CachedBlobContainerIndexInput; @@ -46,6 +47,7 @@ public class IndexInputStats { private final TimedCounter cachedBytesWritten = new TimedCounter(); private final Counter blobStoreBytesRequested = new Counter(); + private final AtomicLong currentIndexCacheFills = new AtomicLong(); public IndexInputStats(long fileLength, LongSupplier currentTimeNanos) { this(fileLength, SEEKING_THRESHOLD.getBytes(), currentTimeNanos); @@ -118,6 +120,15 @@ public void addBlobStoreBytesRequested(long bytesRequested) { blobStoreBytesRequested.add(bytesRequested); } + public Releasable addIndexCacheFill() { + final long openValue = currentIndexCacheFills.incrementAndGet(); + assert openValue > 0 : openValue; + return () -> { + final long closeValue = currentIndexCacheFills.decrementAndGet(); + assert closeValue >= 0 : closeValue; + }; + } + public long getFileLength() { return fileLength; } @@ -179,6 +190,10 @@ public boolean isLargeSeek(long delta) { return delta != Long.MIN_VALUE && Math.abs(delta) > seekingThreshold; } + public long getCurrentIndexCacheFills() { + return currentIndexCacheFills.get(); + } + public static class Counter { private final LongAdder count = new LongAdder(); diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java index 5397d70ff755c..3b2c8baa63fe9 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java @@ -622,8 +622,8 @@ public CachedBlob getCachedBlob(String name, long offset, int length) { return cachedBlob; } - public void putCachedBlob(String name, long offset, BytesReference content) { - blobStoreCacheService.putAsync(repository, name, blobStoreCachePath, offset, content, ActionListener.wrap(() -> {})); + public void putCachedBlob(String name, long offset, BytesReference content, ActionListener listener) { + blobStoreCacheService.putAsync(repository, name, blobStoreCachePath, offset, content, listener); } /** diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index fd09330a7c4bf..f03c3a042cabc 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -15,6 +15,7 @@ import org.apache.lucene.store.IndexInput; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.blobstore.cache.BlobStoreCacheService; import org.elasticsearch.blobstore.cache.CachedBlob; import org.elasticsearch.common.Nullable; @@ -281,6 +282,7 @@ protected void readInternal(ByteBuffer b) throws IOException { }, this::writeCacheFile, directory.cacheFetchAsyncExecutor()); if (indexCacheMiss != null) { + final Releasable onCacheFillComplete = stats.addIndexCacheFill(); cacheFile.populateAndRead(indexCacheMiss, indexCacheMiss, channel -> { final int indexCacheMissLength = Math.toIntExact(indexCacheMiss.v2() - indexCacheMiss.v1()); @@ -295,7 +297,17 @@ protected void readInternal(ByteBuffer b) throws IOException { // NB use Channels.readFromFileChannelWithEofException not readCacheFile() to avoid counting this in the stats byteBuffer.flip(); final BytesReference content = BytesReference.fromByteBuffer(byteBuffer); - directory.putCachedBlob(fileInfo.physicalName(), indexCacheMiss.v1(), content); + directory.putCachedBlob(fileInfo.physicalName(), indexCacheMiss.v1(), content, new ActionListener<>() { + @Override + public void onResponse(Void response) { + onCacheFillComplete.close(); + } + + @Override + public void onFailure(Exception e1) { + onCacheFillComplete.close(); + } + }); return indexCacheMissLength; }, (channel, from, to, progressUpdater) -> { // Normally doesn't happen, we're already obtaining a range covering all cache misses above, but theoretically diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java index 47f4a77d0cbe9..fd8bc8c559d02 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java @@ -111,7 +111,8 @@ private static CacheIndexInputStats toCacheIndexInputStats(final String fileName toTimedCounter(inputStats.getCachedBytesWritten()), toTimedCounter(inputStats.getDirectBytesRead()), toTimedCounter(inputStats.getOptimizedBytesRead()), - toCounter(inputStats.getBlobStoreBytesRequested()) + toCounter(inputStats.getBlobStoreBytesRequested()), + inputStats.getCurrentIndexCacheFills() ); } diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java index 406644c714865..915ac07c21c4a 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java @@ -158,6 +158,18 @@ public void testBlobStoreCache() throws Exception { ensureGreen(restoredIndex); ensureExecutorsAreIdle(); + // wait for all async cache fills to complete + assertBusy(() -> { + for (final SearchableSnapshotShardStats shardStats : client().execute( + SearchableSnapshotsStatsAction.INSTANCE, + new SearchableSnapshotsStatsRequest() + ).actionGet().getStats()) { + for (final SearchableSnapshotShardStats.CacheIndexInputStats indexInputStats : shardStats.getStats()) { + assertThat(Strings.toString(indexInputStats), indexInputStats.getCurrentIndexCacheFills(), equalTo(0L)); + } + } + }); + for (final SearchableSnapshotShardStats shardStats : client().execute( SearchableSnapshotsStatsAction.INSTANCE, new SearchableSnapshotsStatsRequest() @@ -170,7 +182,6 @@ public void testBlobStoreCache() throws Exception { logger.info("--> verifying cached documents in system index [{}]", SNAPSHOT_BLOB_CACHE_INDEX); assertCachedBlobsInSystemIndex(repositoryName, blobsInSnapshot); - refreshSystemIndex(); final long numberOfCachedBlobs = systemClient().prepareSearch(SNAPSHOT_BLOB_CACHE_INDEX).get().getHits().getTotalHits().value; final long numberOfCacheWrites = systemClient().admin() .indices() diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java index b73b5f44e04f1..723cba70b7cf5 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java @@ -110,7 +110,8 @@ private static SearchableSnapshotShardStats.CacheIndexInputStats randomCacheInde randomTimedCounter(), randomTimedCounter(), randomTimedCounter(), - randomCounter() + randomCounter(), + randomNonNegativeLong() ); } From 5e752dd926e663253929ce257faf9d64f90e100b Mon Sep 17 00:00:00 2001 From: David Turner Date: Fri, 14 Aug 2020 12:09:54 +0100 Subject: [PATCH 22/63] Add debugging --- .../SearchableSnapshotsBlobStoreCacheIntegTests.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java index 915ac07c21c4a..a31c44b98494e 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java @@ -34,6 +34,7 @@ import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.InternalTestCluster; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.ClientHelper; import org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotShardStats; @@ -68,6 +69,8 @@ import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.is; +@TestLogging(reason = "debugging", value = "org.elasticsearch.index.store.cache.CachedBlobContainerIndexInput:TRACE," + + "org.elasticsearch.blobstore.cache.BlobStoreCacheService:TRACE") // TODO remove this before merge public class SearchableSnapshotsBlobStoreCacheIntegTests extends BaseSearchableSnapshotsIntegTestCase { @Override @@ -83,6 +86,11 @@ protected int numberOfReplicas() { return 0; } + @Override + protected int numberOfShards() { + return 1; // TODO remove this before merge + } + @Override protected Settings nodeSettings(int nodeOrdinal) { return Settings.builder() From 61701fef9ed4409cedb48f5911c461cd0d57f421 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 18 Aug 2020 12:32:34 +0100 Subject: [PATCH 23/63] Revert --- .../elasticsearch/index/store/SearchableSnapshotDirectory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java index 3b2c8baa63fe9..1887a234da783 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java @@ -208,9 +208,9 @@ public boolean loadSnapshot(RecoveryState recoveryState) { if (alreadyLoaded == false) { this.blobContainer = blobContainerSupplier.get(); this.snapshot = snapshotSupplier.get(); - this.recoveryState = (SearchableSnapshotRecoveryState) recoveryState; this.loaded = true; cleanExistingRegularShardFiles(); + this.recoveryState = (SearchableSnapshotRecoveryState) recoveryState; prewarmCache(); } } From f5948c4dfb43fb280b0897dab9f5d2c944f299a3 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 18 Aug 2020 12:39:22 +0100 Subject: [PATCH 24/63] Moar revert --- .../index/store/SearchableSnapshotDirectory.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java index 1887a234da783..7d1599b40bc53 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java @@ -431,10 +431,8 @@ private void prewarmCache() { final BlockingQueue, CheckedRunnable>> queue = new LinkedBlockingQueue<>(); final Executor executor = prewarmExecutor(); - final GroupedActionListener completionListener = new GroupedActionListener<>(ActionListener.wrap(voids -> { - logger.trace("{} prewarming complete", shardId); - recoveryState.setPreWarmComplete(); - }, e -> {}), // Ignore pre-warm errors + final GroupedActionListener completionListener = new GroupedActionListener<>( + ActionListener.wrap(voids -> recoveryState.setPreWarmComplete(), e -> {}), // Ignore pre-warm errors snapshot().totalFileCount() ); @@ -487,9 +485,10 @@ private void prewarmCache() { } } + logger.debug("{} warming shard cache for [{}] files", shardId, queue.size()); + // Start as many workers as fit into the searchable snapshot pool at once at the most final int workers = Math.min(threadPool.info(CACHE_FETCH_ASYNC_THREAD_POOL_NAME).getMax(), queue.size()); - logger.debug("{} warming shard cache for [{}] files with [{}] workers", shardId, queue.size(), workers); for (int i = 0; i < workers; ++i) { prewarmNext(executor, queue); } From 51a81ac2fe9a57dffc10eab1d801e8384d6ace47 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 18 Aug 2020 14:01:35 +0100 Subject: [PATCH 25/63] Revert --- .../cache/SearchableSnapshotsBlobStoreCacheIntegTests.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java index a31c44b98494e..19e2b7d2b1eb5 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java @@ -86,11 +86,6 @@ protected int numberOfReplicas() { return 0; } - @Override - protected int numberOfShards() { - return 1; // TODO remove this before merge - } - @Override protected Settings nodeSettings(int nodeOrdinal) { return Settings.builder() From 7116f9ff990ae3a742cd6141063922c4eaf403df Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 18 Aug 2020 16:41:51 +0100 Subject: [PATCH 26/63] Revert buffer size increase --- .../blobstore/cache/BlobStoreCacheService.java | 5 ++--- .../store/BaseSearchableSnapshotIndexInput.java | 5 +---- .../store/cache/CachedBlobContainerIndexInput.java | 7 ++++--- ...SearchableSnapshotsBlobStoreCacheIntegTests.java | 13 +++++++------ .../SearchableSnapshotDirectoryStatsTests.java | 6 +++--- 5 files changed, 17 insertions(+), 19 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index 918f33129949c..bc19ec32f37d7 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -31,11 +31,10 @@ import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.IndexNotFoundException; -import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.index.store.BaseSearchableSnapshotIndexInput; import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; @@ -50,7 +49,7 @@ public class BlobStoreCacheService extends AbstractLifecycleComponent implements private static final Logger logger = LogManager.getLogger(BlobStoreCacheService.class); - public static final int DEFAULT_SIZE = Math.toIntExact(ByteSizeUnit.KB.toBytes(4L)); + public static final int DEFAULT_CACHED_BLOB_SIZE = Math.toIntExact(ByteSizeUnit.KB.toBytes(4L)); private final ClusterService clusterService; private final ThreadPool threadPool; diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java index a6b0ede20ae44..45e57e40cd855 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/BaseSearchableSnapshotIndexInput.java @@ -8,7 +8,6 @@ import org.apache.lucene.store.BufferedIndexInput; import org.apache.lucene.store.IOContext; import org.elasticsearch.common.blobstore.BlobContainer; -import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants; @@ -19,8 +18,6 @@ public abstract class BaseSearchableSnapshotIndexInput extends BufferedIndexInput { - public static final int DEFAULT_BUFFER_SIZE = Math.toIntExact(ByteSizeUnit.KB.toBytes(4L)); - protected final BlobContainer blobContainer; protected final FileInfo fileInfo; protected final IOContext context; @@ -41,7 +38,7 @@ public BaseSearchableSnapshotIndexInput( long offset, long length ) { - super(resourceDesc, DEFAULT_BUFFER_SIZE); // TODO align buffer size with block size on disk and length of content cached for blobs + super(resourceDesc, context); this.blobContainer = Objects.requireNonNull(blobContainer); this.fileInfo = Objects.requireNonNull(fileInfo); this.context = Objects.requireNonNull(context); diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index f03c3a042cabc..aa37ae3927877 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -85,6 +85,7 @@ public CachedBlobContainerIndexInput( new CacheFileReference(directory, fileInfo.physicalName(), fileInfo.length()), rangeSize ); + assert getBufferSize() <= BlobStoreCacheService.DEFAULT_CACHED_BLOB_SIZE; // must be able to cache at least one buffer's worth stats.incrementOpenCount(); } @@ -165,9 +166,9 @@ protected void readInternal(ByteBuffer b) throws IOException { if (directory.isRecoveryDone() == false) { // We try to use the snapshot blob cache if: // - the file is small enough to be fully cached in the blob cache - final boolean canBeFullyCached = fileInfo.length() <= BlobStoreCacheService.DEFAULT_SIZE * 2; + 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_SIZE); + final boolean isStartOfFile = (position + length <= BlobStoreCacheService.DEFAULT_CACHED_BLOB_SIZE); if (canBeFullyCached || isStartOfFile) { final CachedBlob cachedBlob = directory.getCachedBlob(fileInfo.physicalName(), 0L, length); @@ -237,7 +238,7 @@ protected void readInternal(ByteBuffer b) throws IOException { // if the index input is smaller than twice the size of the blob cache, it will be fully indexed indexCacheMiss = Tuple.tuple(0L, fileInfo.length()); } else if (isStartOfFile) { - indexCacheMiss = Tuple.tuple(0L, (long) BlobStoreCacheService.DEFAULT_SIZE); + indexCacheMiss = Tuple.tuple(0L, (long) BlobStoreCacheService.DEFAULT_CACHED_BLOB_SIZE); } else { indexCacheMiss = null; } diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java index 19e2b7d2b1eb5..da3c5c20ba8b3 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java @@ -164,8 +164,8 @@ public void testBlobStoreCache() throws Exception { // wait for all async cache fills to complete assertBusy(() -> { for (final SearchableSnapshotShardStats shardStats : client().execute( - SearchableSnapshotsStatsAction.INSTANCE, - new SearchableSnapshotsStatsRequest() + SearchableSnapshotsStatsAction.INSTANCE, + new SearchableSnapshotsStatsRequest() ).actionGet().getStats()) { for (final SearchableSnapshotShardStats.CacheIndexInputStats indexInputStats : shardStats.getStats()) { assertThat(Strings.toString(indexInputStats), indexInputStats.getCurrentIndexCacheFills(), equalTo(0L)); @@ -240,7 +240,8 @@ public void testBlobStoreCache() throws Exception { // we read a couple of longs at the end of the .fdt file (see https://issues.apache.org/jira/browse/LUCENE-9456) // TODO revisit this when this issue is addressed in Lucene || indexInputStats.getFileName().endsWith(".fdt"); - if (indexInputStats.getFileLength() <= BlobStoreCacheService.DEFAULT_SIZE * 2 || mayReadMoreThanHeader == false) { + if (indexInputStats.getFileLength() <= BlobStoreCacheService.DEFAULT_CACHED_BLOB_SIZE * 2 + || mayReadMoreThanHeader == false) { assertThat(Strings.toString(indexInputStats), indexInputStats.getBlobStoreBytesRequested().getCount(), equalTo(0L)); } } @@ -391,7 +392,7 @@ private void assertCachedBlobsInSystemIndex(final String repositoryName, final M } final String path = String.join("/", repositoryName, blob.getKey(), fileInfo.physicalName()); - if (fileInfo.length() <= BlobStoreCacheService.DEFAULT_SIZE * 2) { + if (fileInfo.length() <= BlobStoreCacheService.DEFAULT_CACHED_BLOB_SIZE * 2) { // file has been fully cached final GetResponse getResponse = systemClient().prepareGet(SNAPSHOT_BLOB_CACHE_INDEX, path + "/@0").get(); assertThat("not cached: [" + path + "/@0] for blob [" + fileInfo + "]", getResponse.isExists(), is(true)); @@ -412,8 +413,8 @@ private void assertCachedBlobsInSystemIndex(final String repositoryName, final M CachedBlob cachedBlob = CachedBlob.fromSource(getResponse.getSourceAsMap()); assertThat(cachedBlob.from(), equalTo(0L)); - assertThat(cachedBlob.to(), equalTo((long) BlobStoreCacheService.DEFAULT_SIZE)); - assertThat(cachedBlob.length(), equalTo(BlobStoreCacheService.DEFAULT_SIZE)); + assertThat(cachedBlob.to(), equalTo((long) BlobStoreCacheService.DEFAULT_CACHED_BLOB_SIZE)); + assertThat(cachedBlob.length(), equalTo(BlobStoreCacheService.DEFAULT_CACHED_BLOB_SIZE)); numberOfCachedBlobs += 1; } } diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryStatsTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryStatsTests.java index b6fb060b937cc..0fea6616c6c2e 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryStatsTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryStatsTests.java @@ -202,7 +202,7 @@ public void testDirectBytesReadsWithCache() { final IndexInputStats inputStats = directory.getStats(fileName); // account for internal buffered reads - final long bufferSize = BaseSearchableSnapshotIndexInput.DEFAULT_BUFFER_SIZE; + final long bufferSize = BufferedIndexInput.bufferSize(ioContext); final long remaining = input.length() % bufferSize; final long expectedTotal = input.length(); final long expectedCount = input.length() / bufferSize + (remaining > 0L ? 1L : 0L); @@ -319,7 +319,7 @@ public void testReadBytesContiguously() { final IndexInputStats inputStats = cacheDirectory.getStats(fileName); // account for the CacheBufferedIndexInput internal buffer - final long bufferSize = BaseSearchableSnapshotIndexInput.DEFAULT_BUFFER_SIZE; + final long bufferSize = BufferedIndexInput.bufferSize(ioContext); final long remaining = input.length() % bufferSize; final long expectedTotal = input.length(); final long expectedCount = input.length() / bufferSize + (remaining > 0L ? 1L : 0L); @@ -384,7 +384,7 @@ public void testReadBytesNonContiguously() { input.readBytes(readBuffer, 0, size); // BufferedIndexInput tries to read as much bytes as possible - final long bytesRead = Math.min(BaseSearchableSnapshotIndexInput.DEFAULT_BUFFER_SIZE, input.length() - randomPosition); + final long bytesRead = Math.min(BufferedIndexInput.bufferSize(ioContext), input.length() - randomPosition); lastReadPosition = randomPosition + bytesRead; totalBytesRead += bytesRead; minBytesRead = (bytesRead < minBytesRead) ? bytesRead : minBytesRead; From 4b01c9a01fad72570499103a843554bf2a2bf360 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 18 Aug 2020 16:52:05 +0100 Subject: [PATCH 27/63] Better assertion --- .../blobstore/cache/BlobStoreCacheService.java | 1 - .../SearchableSnapshotsIntegTests.java | 11 +++++------ 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index bc19ec32f37d7..06411a7939953 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -34,7 +34,6 @@ import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.IndexNotFoundException; -import org.elasticsearch.index.store.BaseSearchableSnapshotIndexInput; import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java index 84306a929495f..fe55601badf67 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java @@ -792,12 +792,11 @@ private void assertSearchableSnapshotStats(String indexName, boolean cacheEnable equalTo(0L) ); } else if (nodeIdsWithLargeEnoughCache.contains(stats.getShardRouting().currentNodeId())) { - // not necessarily, it may have been entirely in blob cache TODO improve stats to handle this - // assertThat( - // "Expected at least 1 cache read or write for " + fileName + " of shard " + shardRouting, - // Math.max(indexInputStats.getCachedBytesRead().getCount(), indexInputStats.getCachedBytesWritten().getCount()), - // greaterThan(0L) - // ); + assertThat( + "Expected no bytes requested from blob store for " + fileName + " of shard " + shardRouting, + indexInputStats.getBlobStoreBytesRequested().getCount(), + equalTo(0L) + ); assertThat( "Expected no optimized read for " + fileName + " of shard " + shardRouting, indexInputStats.getOptimizedBytesRead().getCount(), From d3d58cd731d97ec1326f643cb8aacd18ee1d4008 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 18 Aug 2020 17:12:44 +0100 Subject: [PATCH 28/63] Distinguish definite cache miss from cache-not-ready --- .../cache/BlobStoreCacheService.java | 6 ++-- .../blobstore/cache/CachedBlob.java | 11 ++++++ .../store/SearchableSnapshotDirectory.java | 4 +-- .../cache/CachedBlobContainerIndexInput.java | 34 +++++++++++-------- 4 files changed, 35 insertions(+), 20 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index 06411a7939953..047db7b50a527 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -243,7 +243,7 @@ protected void getAsync(String repository, String name, String path, long offset // TODO TBD can we just execute the GET request and let it fail if the index isn't ready yet? // We might get lucky and hit a started shard anyway. logger.debug("not ready : [{}]", CachedBlob.generateId(repository, name, path, offset)); - listener.onResponse(null); + listener.onResponse(CachedBlob.CACHE_NOT_READY); return; } try { @@ -260,7 +260,7 @@ public void onResponse(GetResponse response) { listener.onResponse(cachedBlob); } else { logger.debug("cache miss: [{}]", request.id()); - listener.onResponse(null); + listener.onResponse(CachedBlob.CACHE_MISS); } } @@ -271,7 +271,7 @@ public void onFailure(Exception e) { // Failing here might bubble up the exception and fail the searchable snapshot shard which is potentially // recovering. logger.debug(() -> new ParameterizedMessage("failed to retrieve cached blob from system index [{}]", index), e); - listener.onResponse(null); + listener.onResponse(CachedBlob.CACHE_NOT_READY); } else { logger.warn(() -> new ParameterizedMessage("failed to retrieve cached blob from system index [{}]", index), e); listener.onFailure(e); diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CachedBlob.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CachedBlob.java index 111cd335d0496..9b22b8162aea1 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CachedBlob.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CachedBlob.java @@ -9,6 +9,7 @@ import org.elasticsearch.Version; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.cache.Cache; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -19,6 +20,16 @@ public class CachedBlob implements ToXContent { + /** + * Sentinel {@link CachedBlob} indicating that searching the cache index returned an error. + */ + public static final CachedBlob CACHE_NOT_READY = new CachedBlob(null, null, null, "CACHE_NOT_READY", null, BytesArray.EMPTY, 0L, 0L); + + /** + * Sentinel {@link CachedBlob} indicating that the cache index definitely did not contain the requested data. + */ + public static final CachedBlob CACHE_MISS = new CachedBlob(null, null, null, "CACHE_MISS", null, BytesArray.EMPTY, 0L, 0L); + private static final String TYPE = "blob"; private final Instant creationTime; diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java index 7d1599b40bc53..15fb5530ae3c2 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java @@ -616,8 +616,8 @@ public static SearchableSnapshotDirectory unwrapDirectory(Directory dir) { public CachedBlob getCachedBlob(String name, long offset, int length) { final CachedBlob cachedBlob = blobStoreCacheService.get(repository, name, blobStoreCachePath, offset); - assert cachedBlob == null || cachedBlob.from() <= offset; - assert cachedBlob == null || offset + length <= cachedBlob.to(); + assert cachedBlob == CachedBlob.CACHE_MISS || cachedBlob == CachedBlob.CACHE_NOT_READY || cachedBlob.from() <= offset; + assert cachedBlob == CachedBlob.CACHE_MISS || cachedBlob == CachedBlob.CACHE_NOT_READY || offset + length <= cachedBlob.to(); return cachedBlob; } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index aa37ae3927877..e5e06ff6d19dc 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -172,7 +172,22 @@ protected void readInternal(ByteBuffer b) throws IOException { if (canBeFullyCached || isStartOfFile) { final CachedBlob cachedBlob = directory.getCachedBlob(fileInfo.physicalName(), 0L, length); - if (cachedBlob != null) { + + if (cachedBlob == CachedBlob.CACHE_MISS) { + // 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 regions of the file we would like to have the next time. The regions are expressed as tuples of + // {start, end} ranges 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 = Tuple.tuple(0L, fileInfo.length()); + } else { + // the index input is too large to fully cache, so just cache the initial range + indexCacheMiss = Tuple.tuple(0L, (long) BlobStoreCacheService.DEFAULT_CACHED_BLOB_SIZE); + } + } else if (cachedBlob == CachedBlob.CACHE_NOT_READY) { + // No point in trying to populate the cache if it wasn't even ready yet + indexCacheMiss = null; + } else { logger.trace( "reading [{}] bytes of file [{}] at position [{}] using index cache", length, @@ -226,23 +241,12 @@ protected void readInternal(ByteBuffer b) throws IOException { return; } - - // Cache miss may be that the cache is completely unavailable (no point in populating it) or that the blob is - // definitely absent. TODO only bother populating the cache in the latter case. - } - - // 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 regions of the file we would like to have the next time. The regions are expressed as tuples of - // {start, end} ranges 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 = Tuple.tuple(0L, fileInfo.length()); - } else if (isStartOfFile) { - indexCacheMiss = Tuple.tuple(0L, (long) BlobStoreCacheService.DEFAULT_CACHED_BLOB_SIZE); } else { + // requested range is not eligible for caching indexCacheMiss = null; } - logger.trace("recovery cache miss for [{}], falling through with cache miss [{}]", this, indexCacheMiss); + + logger.trace("recovery cache miss for [{}], falling through with [{}]", this, indexCacheMiss); } else { indexCacheMiss = null; } From 0bcabe126b44146670d7945fc586ef8bd0eb43f2 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 18 Aug 2020 17:14:57 +0100 Subject: [PATCH 29/63] Precommit --- .../main/java/org/elasticsearch/blobstore/cache/CachedBlob.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CachedBlob.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CachedBlob.java index 9b22b8162aea1..acc0c7cbe9260 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CachedBlob.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/CachedBlob.java @@ -9,7 +9,6 @@ import org.elasticsearch.Version; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.cache.Cache; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; From 43f281ae86ec0d731480c5093eb8db079d06e084 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 18 Aug 2020 17:24:36 +0100 Subject: [PATCH 30/63] Test bug --- .../java/org/elasticsearch/index/store/cache/TestUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/TestUtils.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/TestUtils.java index c8114d50cbeb6..e7d96f7b62474 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/TestUtils.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/TestUtils.java @@ -224,7 +224,7 @@ public NoopBlobStoreCacheService() { @Override protected void getAsync(String repository, String name, String path, long offset, ActionListener listener) { - listener.onResponse(null); + listener.onResponse(CachedBlob.CACHE_NOT_READY); } @Override From 78680bebde572afcd0a77b976ef1489c773afc3d Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 18 Aug 2020 17:41:46 +0100 Subject: [PATCH 31/63] Track reads from index cache too --- .../SearchableSnapshotShardStats.java | 25 +++++++++++++++---- .../SearchableSnapshotShardStatsTests.java | 2 +- .../index/store/IndexInputStats.java | 9 +++++++ .../cache/CachedBlobContainerIndexInput.java | 1 + ...ansportSearchableSnapshotsStatsAction.java | 1 + .../SearchableSnapshotsIntegTests.java | 12 ++++++--- ...SearchableSnapshotsStatsResponseTests.java | 1 + 7 files changed, 42 insertions(+), 9 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java index 02b295d7534c3..9ddccc69414be 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java @@ -135,6 +135,7 @@ public static class CacheIndexInputStats implements Writeable, ToXContentObject private final Counter contiguousReads; private final Counter nonContiguousReads; private final Counter cachedBytesRead; + private final Counter indexCacheBytesRead; private final TimedCounter cachedBytesWritten; private final TimedCounter directBytesRead; private final TimedCounter optimizedBytesRead; @@ -145,8 +146,8 @@ public CacheIndexInputStats(String fileName, long fileLength, long openCount, lo Counter forwardSmallSeeks, Counter backwardSmallSeeks, Counter forwardLargeSeeks, Counter backwardLargeSeeks, Counter contiguousReads, Counter nonContiguousReads, - Counter cachedBytesRead, TimedCounter cachedBytesWritten, - TimedCounter directBytesRead, TimedCounter optimizedBytesRead, + Counter cachedBytesRead, Counter indexCacheBytesRead, + TimedCounter cachedBytesWritten, TimedCounter directBytesRead, TimedCounter optimizedBytesRead, Counter blobStoreBytesRequested, long currentIndexCacheFills) { this.fileName = fileName; this.fileLength = fileLength; @@ -159,6 +160,7 @@ public CacheIndexInputStats(String fileName, long fileLength, long openCount, lo this.contiguousReads = contiguousReads; this.nonContiguousReads = nonContiguousReads; this.cachedBytesRead = cachedBytesRead; + this.indexCacheBytesRead = indexCacheBytesRead; this.cachedBytesWritten = cachedBytesWritten; this.directBytesRead = directBytesRead; this.optimizedBytesRead = optimizedBytesRead; @@ -178,6 +180,11 @@ public CacheIndexInputStats(String fileName, long fileLength, long openCount, lo this.contiguousReads = new Counter(in); this.nonContiguousReads = new Counter(in); this.cachedBytesRead = new Counter(in); + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + this.indexCacheBytesRead = new Counter(in); + } else { + this.indexCacheBytesRead = new Counter(0, 0, 0, 0); + } this.cachedBytesWritten = new TimedCounter(in); this.directBytesRead = new TimedCounter(in); this.optimizedBytesRead = new TimedCounter(in); @@ -187,7 +194,6 @@ public CacheIndexInputStats(String fileName, long fileLength, long openCount, lo } else { this.blobStoreBytesRequested = new Counter(0, 0, 0, 0); this.currentIndexCacheFills = 0; - } } @@ -205,6 +211,9 @@ public void writeTo(StreamOutput out) throws IOException { contiguousReads.writeTo(out); nonContiguousReads.writeTo(out); cachedBytesRead.writeTo(out); + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + indexCacheBytesRead.writeTo(out); + } cachedBytesWritten.writeTo(out); directBytesRead.writeTo(out); optimizedBytesRead.writeTo(out); @@ -258,6 +267,10 @@ public Counter getCachedBytesRead() { return cachedBytesRead; } + public Counter getIndexCacheBytesRead() { + return indexCacheBytesRead; + } + public TimedCounter getCachedBytesWritten() { return cachedBytesWritten; } @@ -289,6 +302,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field("contiguous_bytes_read", getContiguousReads()); builder.field("non_contiguous_bytes_read", getNonContiguousReads()); builder.field("cached_bytes_read", getCachedBytesRead()); + builder.field("index_cache_bytes_read", getIndexCacheBytesRead()); builder.field("cached_bytes_written", getCachedBytesWritten()); builder.field("direct_bytes_read", getDirectBytesRead()); builder.field("optimized_bytes_read", getOptimizedBytesRead()); @@ -330,6 +344,7 @@ public boolean equals(Object other) { && Objects.equals(contiguousReads, stats.contiguousReads) && Objects.equals(nonContiguousReads, stats.nonContiguousReads) && Objects.equals(cachedBytesRead, stats.cachedBytesRead) + && Objects.equals(indexCacheBytesRead, stats.indexCacheBytesRead) && Objects.equals(cachedBytesWritten, stats.cachedBytesWritten) && Objects.equals(directBytesRead, stats.directBytesRead) && Objects.equals(optimizedBytesRead, stats.optimizedBytesRead) @@ -343,8 +358,8 @@ public int hashCode() { forwardSmallSeeks, backwardSmallSeeks, forwardLargeSeeks, backwardLargeSeeks, contiguousReads, nonContiguousReads, - cachedBytesRead, cachedBytesWritten, - directBytesRead, optimizedBytesRead, + cachedBytesRead, indexCacheBytesRead, + cachedBytesWritten, directBytesRead, optimizedBytesRead, blobStoreBytesRequested, currentIndexCacheFills); } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java index 70eeab11c086e..d57d0746db417 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java @@ -46,7 +46,7 @@ private CacheIndexInputStats randomCacheIndexInputStats() { randomCounter(), randomCounter(), randomCounter(), randomCounter(), randomCounter(), randomCounter(), - randomCounter(), randomTimedCounter(), + randomCounter(), randomCounter(), randomTimedCounter(), randomTimedCounter(), randomTimedCounter(), randomCounter(), randomNonNegativeLong()); } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java index e087268e21f0e..0f652d604ba5d 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java @@ -44,6 +44,7 @@ public class IndexInputStats { private final TimedCounter optimizedBytesRead = new TimedCounter(); private final Counter cachedBytesRead = new Counter(); + private final Counter indexCacheBytesRead = new Counter(); private final TimedCounter cachedBytesWritten = new TimedCounter(); private final Counter blobStoreBytesRequested = new Counter(); @@ -78,6 +79,10 @@ public void addCachedBytesRead(int bytesRead) { cachedBytesRead.add(bytesRead); } + public void addIndexCacheBytesRead(int bytesRead) { + indexCacheBytesRead.add(bytesRead); + } + public void addCachedBytesWritten(long bytesWritten, long nanoseconds) { cachedBytesWritten.add(bytesWritten, nanoseconds); } @@ -177,6 +182,10 @@ public Counter getCachedBytesRead() { return cachedBytesRead; } + public Counter getIndexCacheBytesRead() { + return indexCacheBytesRead; + } + public TimedCounter getCachedBytesWritten() { return cachedBytesWritten; } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index e5e06ff6d19dc..40005c40e12de 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -194,6 +194,7 @@ protected void readInternal(ByteBuffer b) throws IOException { fileInfo.physicalName(), position ); + stats.addIndexCacheBytesRead(cachedBlob.length()); b.put(BytesReference.toBytes(cachedBlob.bytes().slice(Math.toIntExact(position), length))); try { diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java index fd8bc8c559d02..8d0c3eac5fe02 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java @@ -108,6 +108,7 @@ private static CacheIndexInputStats toCacheIndexInputStats(final String fileName toCounter(inputStats.getContiguousReads()), toCounter(inputStats.getNonContiguousReads()), toCounter(inputStats.getCachedBytesRead()), + toCounter(inputStats.getIndexCacheBytesRead()), toTimedCounter(inputStats.getCachedBytesWritten()), toTimedCounter(inputStats.getDirectBytesRead()), toTimedCounter(inputStats.getOptimizedBytesRead()), diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java index fe55601badf67..0e9d185ac6a02 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java @@ -793,9 +793,15 @@ private void assertSearchableSnapshotStats(String indexName, boolean cacheEnable ); } else if (nodeIdsWithLargeEnoughCache.contains(stats.getShardRouting().currentNodeId())) { assertThat( - "Expected no bytes requested from blob store for " + fileName + " of shard " + shardRouting, - indexInputStats.getBlobStoreBytesRequested().getCount(), - equalTo(0L) + "Expected at least 1 cache read or write for " + fileName + " of shard " + shardRouting, + Math.max( + Math.max( + indexInputStats.getCachedBytesRead().getCount(), + indexInputStats.getCachedBytesWritten().getCount() + ), + indexInputStats.getIndexCacheBytesRead().getCount() + ), + greaterThan(0L) ); assertThat( "Expected no optimized read for " + fileName + " of shard " + shardRouting, diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java index 723cba70b7cf5..ad57c607d1722 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java @@ -107,6 +107,7 @@ private static SearchableSnapshotShardStats.CacheIndexInputStats randomCacheInde randomCounter(), randomCounter(), randomCounter(), + randomCounter(), randomTimedCounter(), randomTimedCounter(), randomTimedCounter(), From c50dfa6dbc431bb169b3113d2ba41892d4daf832 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 19 Aug 2020 08:03:12 +0100 Subject: [PATCH 32/63] Retry on INFE --- .../SearchableSnapshotsBlobStoreCacheIntegTests.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java index da3c5c20ba8b3..cb9eb62ca10a8 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java @@ -336,9 +336,13 @@ private Client systemClient() { } private void refreshSystemIndex() { - final RefreshResponse refreshResponse = systemClient().admin().indices().prepareRefresh(SNAPSHOT_BLOB_CACHE_INDEX).get(); - assertThat(refreshResponse.getSuccessfulShards(), greaterThan(0)); - assertThat(refreshResponse.getFailedShards(), equalTo(0)); + try { + final RefreshResponse refreshResponse = systemClient().admin().indices().prepareRefresh(SNAPSHOT_BLOB_CACHE_INDEX).get(); + assertThat(refreshResponse.getSuccessfulShards(), greaterThan(0)); + assertThat(refreshResponse.getFailedShards(), equalTo(0)); + } catch (IndexNotFoundException indexNotFoundException) { + throw new AssertionError("unexpected", indexNotFoundException); + } } /** From d6ef9cd76017046a60a2513d784733b866a681ef Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 20 Aug 2020 12:47:09 +0100 Subject: [PATCH 33/63] Always put a missing cache entry even if cache not ready --- .../index/store/cache/CachedBlobContainerIndexInput.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index 40005c40e12de..66a86ff378451 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -173,7 +173,7 @@ protected void readInternal(ByteBuffer b) throws IOException { if (canBeFullyCached || isStartOfFile) { final CachedBlob cachedBlob = directory.getCachedBlob(fileInfo.physicalName(), 0L, length); - if (cachedBlob == CachedBlob.CACHE_MISS) { + if (cachedBlob == CachedBlob.CACHE_MISS || cachedBlob == CachedBlob.CACHE_NOT_READY) { // 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 regions of the file we would like to have the next time. The regions are expressed as tuples of // {start, end} ranges where positions are relative to the whole file. @@ -184,9 +184,9 @@ protected void readInternal(ByteBuffer b) throws IOException { // the index input is too large to fully cache, so just cache the initial range indexCacheMiss = Tuple.tuple(0L, (long) BlobStoreCacheService.DEFAULT_CACHED_BLOB_SIZE); } - } else if (cachedBlob == CachedBlob.CACHE_NOT_READY) { - // No point in trying to populate the cache if it wasn't even ready yet - indexCacheMiss = null; + + // 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. } else { logger.trace( "reading [{}] bytes of file [{}] at position [{}] using index cache", From 4f9584c07fcc840e4f66505f6173d092de4c2b23 Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 20 Aug 2020 12:52:55 +0100 Subject: [PATCH 34/63] Let's keep indexing things for now --- .../blobstore/cache/BlobStoreCacheService.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index 047db7b50a527..983a07c84e0c5 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -150,26 +150,22 @@ private static XContentBuilder mappings() throws IOException { { builder.startObject("type"); builder.field("type", "keyword"); - builder.field("index", "false"); builder.endObject(); } { builder.startObject("creation_time"); builder.field("type", "date"); builder.field("format", "epoch_millis"); - builder.field("index", "false"); builder.endObject(); } { builder.startObject("version"); builder.field("type", "integer"); - builder.field("index", "false"); builder.endObject(); } { builder.startObject("repository"); builder.field("type", "keyword"); - builder.field("index", "false"); builder.endObject(); } { @@ -180,11 +176,9 @@ private static XContentBuilder mappings() throws IOException { { builder.startObject("name"); builder.field("type", "keyword"); - builder.field("index", "false"); builder.endObject(); builder.startObject("path"); builder.field("type", "keyword"); - builder.field("index", "false"); builder.endObject(); } builder.endObject(); @@ -204,19 +198,16 @@ private static XContentBuilder mappings() throws IOException { { builder.startObject("length"); builder.field("type", "long"); - builder.field("index", "false"); builder.endObject(); } { builder.startObject("from"); builder.field("type", "long"); - builder.field("index", "false"); builder.endObject(); } { builder.startObject("to"); builder.field("type", "long"); - builder.field("index", "false"); builder.endObject(); } builder.endObject(); From cd77a43bce70607c961bbb9fd8e3ce6240da2304 Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 20 Aug 2020 16:05:25 +0100 Subject: [PATCH 35/63] Unnecessary repo --- .../SearchableSnapshotsBlobStoreCacheIntegTests.java | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java index cb9eb62ca10a8..8ff601b0c6cc2 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java @@ -31,7 +31,6 @@ import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; import org.elasticsearch.plugins.ClusterPlugin; import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.junit.annotations.TestLogging; @@ -132,16 +131,6 @@ public void testBlobStoreCache() throws Exception { final Map blobsInSnapshot = blobsInSnapshot(repositoryLocation, snapshot.getUUID()); assertThat("Failed to load all shard snapshot metadata files", blobsInSnapshot.size(), equalTo(numberOfShards.numPrimaries)); - // register a new repository that can track blob read operations - assertAcked(client().admin().cluster().prepareDeleteRepository(repositoryName)); - createRepository( - repositoryName, - "fs", - Settings.builder().put(FsRepository.LOCATION_SETTING.getKey(), repositoryLocation).build(), - false - ); - assertBusy(this::ensureClusterStateConsistency); - expectThrows( IndexNotFoundException.class, ".snapshot-blob-cache system index should not be created yet", From 1c38aaa0e052ca06d7aeda7e7e073ffd353be0b8 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 24 Aug 2020 09:52:29 +0100 Subject: [PATCH 36/63] Less allocation --- .../ChecksumBlobContainerIndexInput.java | 21 ++++++++----------- 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/checksum/ChecksumBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/checksum/ChecksumBlobContainerIndexInput.java index af9c6e69813fc..d0b7b1ebb2ef3 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/checksum/ChecksumBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/checksum/ChecksumBlobContainerIndexInput.java @@ -7,11 +7,9 @@ package org.elasticsearch.index.store.checksum; import org.apache.lucene.codecs.CodecUtil; -import org.apache.lucene.store.ByteBuffersDataOutput; -import org.apache.lucene.store.ByteBuffersIndexOutput; +import org.apache.lucene.store.ByteArrayDataOutput; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; -import org.apache.lucene.store.IndexOutput; import org.elasticsearch.index.store.Store; import java.io.EOFException; @@ -135,14 +133,13 @@ public static ChecksumBlobContainerIndexInput create(String name, long length, S } public static byte[] checksumToBytesArray(String checksum) throws IOException { - final ByteBuffersDataOutput out = new ByteBuffersDataOutput(); - try (IndexOutput output = new ByteBuffersIndexOutput(out, "footerChecksumToBytesArray", "tmp")) { - // reverse CodecUtil.writeFooter() - output.writeInt(CodecUtil.FOOTER_MAGIC); - output.writeInt(0); - output.writeLong(Long.parseLong(checksum, Character.MAX_RADIX)); - output.close(); - return out.toArrayCopy(); - } + final byte[] result = new byte[CodecUtil.footerLength()]; + final ByteArrayDataOutput output = new ByteArrayDataOutput(result); + // reverse CodecUtil.writeFooter() + output.writeInt(CodecUtil.FOOTER_MAGIC); + output.writeInt(0); + output.writeLong(Long.parseLong(checksum, Character.MAX_RADIX)); + assert output.getPosition() == result.length; + return result; } } From 0fbb71f14c73e5120420329519fe4eecfa814638 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 24 Aug 2020 10:22:39 +0100 Subject: [PATCH 37/63] Use TransportActions.isShardNotAvailableException --- .../elasticsearch/blobstore/cache/BlobStoreCacheService.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index 983a07c84e0c5..1518b65eb7f72 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -20,6 +20,7 @@ import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.client.Client; import org.elasticsearch.client.OriginSettingClient; import org.elasticsearch.cluster.ClusterChangedEvent; @@ -257,7 +258,7 @@ public void onResponse(GetResponse response) { @Override public void onFailure(Exception e) { - if (e instanceof IndexNotFoundException || e instanceof NoShardAvailableActionException) { + if (TransportActions.isShardNotAvailableException(e)) { // In case the blob cache system index got unavailable, we pretend we didn't find a cache entry and we move on. // Failing here might bubble up the exception and fail the searchable snapshot shard which is potentially // recovering. From 7e6eb5323617b9c19e3855a1f92e032d5cadcfad Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 24 Aug 2020 10:23:04 +0100 Subject: [PATCH 38/63] Always notify listener --- .../cache/BlobStoreCacheService.java | 35 ++++++++++--------- 1 file changed, 19 insertions(+), 16 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index 1518b65eb7f72..1dcac6b47001f 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -13,7 +13,6 @@ import org.elasticsearch.ResourceAlreadyExistsException; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.NoShardAvailableActionException; import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; @@ -34,7 +33,6 @@ import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; @@ -279,6 +277,7 @@ public void putAsync(String repository, String name, String path, long offset, B createIndexIfNecessary(new ActionListener<>() { @Override public void onResponse(String s) { + final IndexRequest request; try { final CachedBlob cachedBlob = new CachedBlob( Instant.ofEpochMilli(threadPool.absoluteTimeInMillis()), @@ -289,34 +288,38 @@ public void onResponse(String s) { content, offset ); - final IndexRequest request = new IndexRequest(index).id(cachedBlob.generatedId()); + request = new IndexRequest(index).id(cachedBlob.generatedId()); try (XContentBuilder builder = jsonBuilder()) { request.source(cachedBlob.toXContent(builder, ToXContent.EMPTY_PARAMS)); } - client.index(request, new ActionListener<>() { - @Override - public void onResponse(IndexResponse indexResponse) { - logger.trace("cache fill ({}): [{}]", indexResponse.status(), request.id()); - listener.onResponse(null); - } - - @Override - public void onFailure(Exception e) { - logger.debug(new ParameterizedMessage("failure in cache fill: [{}]", request.id()), e); - listener.onFailure(e); - } - }); } catch (IOException e) { logger.warn( new ParameterizedMessage("cache fill failure: [{}]", CachedBlob.generateId(repository, name, path, offset)), e ); + listener.onFailure(e); + return; } + + client.index(request, new ActionListener<>() { + @Override + public void onResponse(IndexResponse indexResponse) { + logger.trace("cache fill ({}): [{}]", indexResponse.status(), request.id()); + listener.onResponse(null); + } + + @Override + public void onFailure(Exception e) { + logger.debug(new ParameterizedMessage("failure in cache fill: [{}]", request.id()), e); + listener.onFailure(e); + } + }); } @Override public void onFailure(Exception e) { logger.error(() -> new ParameterizedMessage("failed to create blob cache system index [{}]", index), e); + listener.onFailure(e); } }); } From 6391d8fa8b77a0746da33cb0e51993bfaf0ba9ed Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 24 Aug 2020 11:40:53 +0100 Subject: [PATCH 39/63] Propagate direct exception directly to caller --- .../cache/BlobStoreCacheService.java | 58 +++++++++---------- 1 file changed, 27 insertions(+), 31 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index 1dcac6b47001f..da5b715ecd47f 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -236,41 +236,37 @@ protected void getAsync(String repository, String name, String path, long offset listener.onResponse(CachedBlob.CACHE_NOT_READY); return; } - try { - final GetRequest request = new GetRequest(index).id(CachedBlob.generateId(repository, name, path, offset)); - client.get(request, new ActionListener<>() { - @Override - public void onResponse(GetResponse response) { - if (response.isExists()) { - logger.debug("cache hit : [{}]", request.id()); - assert response.isSourceEmpty() == false; + final GetRequest request = new GetRequest(index).id(CachedBlob.generateId(repository, name, path, offset)); + client.get(request, new ActionListener<>() { + @Override + public void onResponse(GetResponse response) { + if (response.isExists()) { + logger.debug("cache hit : [{}]", request.id()); + assert response.isSourceEmpty() == false; - final CachedBlob cachedBlob = CachedBlob.fromSource(response.getSource()); - assert response.getId().equals(cachedBlob.generatedId()); - listener.onResponse(cachedBlob); - } else { - logger.debug("cache miss: [{}]", request.id()); - listener.onResponse(CachedBlob.CACHE_MISS); - } + final CachedBlob cachedBlob = CachedBlob.fromSource(response.getSource()); + assert response.getId().equals(cachedBlob.generatedId()); + listener.onResponse(cachedBlob); + } else { + logger.debug("cache miss: [{}]", request.id()); + listener.onResponse(CachedBlob.CACHE_MISS); } + } - @Override - public void onFailure(Exception e) { - if (TransportActions.isShardNotAvailableException(e)) { - // In case the blob cache system index got unavailable, we pretend we didn't find a cache entry and we move on. - // Failing here might bubble up the exception and fail the searchable snapshot shard which is potentially - // recovering. - logger.debug(() -> new ParameterizedMessage("failed to retrieve cached blob from system index [{}]", index), e); - listener.onResponse(CachedBlob.CACHE_NOT_READY); - } else { - logger.warn(() -> new ParameterizedMessage("failed to retrieve cached blob from system index [{}]", index), e); - listener.onFailure(e); - } + @Override + public void onFailure(Exception e) { + if (TransportActions.isShardNotAvailableException(e)) { + // In case the blob cache system index got unavailable, we pretend we didn't find a cache entry and we move on. + // Failing here might bubble up the exception and fail the searchable snapshot shard which is potentially + // recovering. + logger.debug(() -> new ParameterizedMessage("failed to retrieve cached blob from system index [{}]", index), e); + listener.onResponse(CachedBlob.CACHE_NOT_READY); + } else { + logger.warn(() -> new ParameterizedMessage("failed to retrieve cached blob from system index [{}]", index), e); + listener.onFailure(e); } - }); - } catch (Exception e) { - listener.onFailure(e); - } + } + }); } public void putAsync(String repository, String name, String path, long offset, BytesReference content, ActionListener listener) { From 7bd0e47cd9e08d9d614cab9f8dda5e6230bfa29e Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 24 Aug 2020 11:43:14 +0100 Subject: [PATCH 40/63] Also permit disconnected/not-connected/node-closed exceptions, and assert nothing else happens --- .../cache/BlobStoreCacheService.java | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index da5b715ecd47f..22343d2fcc712 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -33,7 +33,9 @@ import org.elasticsearch.common.unit.ByteSizeUnit; 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 java.io.IOException; import java.time.Instant; @@ -255,20 +257,25 @@ public void onResponse(GetResponse response) { @Override public void onFailure(Exception e) { - if (TransportActions.isShardNotAvailableException(e)) { - // In case the blob cache system index got unavailable, we pretend we didn't find a cache entry and we move on. - // Failing here might bubble up the exception and fail the searchable snapshot shard which is potentially - // recovering. + // In case the blob cache system index is unavailable, we indicate it's not ready and move on. We do not fail the request: + // a failure here is not fatal since the data exists in the blob store, so we can simply indicate the cache is not ready. + if (isExpectedCacheGetException(e)) { logger.debug(() -> new ParameterizedMessage("failed to retrieve cached blob from system index [{}]", index), e); - listener.onResponse(CachedBlob.CACHE_NOT_READY); } else { logger.warn(() -> new ParameterizedMessage("failed to retrieve cached blob from system index [{}]", index), e); - listener.onFailure(e); + assert false : e; } + listener.onResponse(CachedBlob.CACHE_NOT_READY); } }); } + private static boolean isExpectedCacheGetException(Exception e) { + return TransportActions.isShardNotAvailableException(e) + || e instanceof ConnectTransportException + || ExceptionsHelper.unwrapCause(e) instanceof NodeClosedException; + } + public void putAsync(String repository, String name, String path, long offset, BytesReference content, ActionListener listener) { createIndexIfNecessary(new ActionListener<>() { @Override From 60ceabb49f9f59fb06ef36219e5ab490f17a55b9 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 24 Aug 2020 11:56:32 +0100 Subject: [PATCH 41/63] Precommit --- .../elasticsearch/blobstore/cache/BlobStoreCacheService.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index 22343d2fcc712..ea6ff6d22bac8 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -272,8 +272,8 @@ public void onFailure(Exception e) { private static boolean isExpectedCacheGetException(Exception e) { return TransportActions.isShardNotAvailableException(e) - || e instanceof ConnectTransportException - || ExceptionsHelper.unwrapCause(e) instanceof NodeClosedException; + || e instanceof ConnectTransportException + || ExceptionsHelper.unwrapCause(e) instanceof NodeClosedException; } public void putAsync(String repository, String name, String path, long offset, BytesReference content, ActionListener listener) { From b50d1589f983176a3ad597dafb1f05d3c82f9825 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 24 Aug 2020 13:26:11 +0100 Subject: [PATCH 42/63] Check the cache index even after recovery complete --- .../cache/CachedBlobContainerIndexInput.java | 152 ++++++++++-------- 1 file changed, 88 insertions(+), 64 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index 66a86ff378451..f5727e25f6530 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -24,6 +24,7 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.Channels; import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo; import org.elasticsearch.index.snapshots.blobstore.SlicedInputStream; import org.elasticsearch.index.store.BaseSearchableSnapshotIndexInput; @@ -160,46 +161,77 @@ protected void readInternal(ByteBuffer b) throws IOException { logger.trace("readInternal: read [{}-{}] ([{}] bytes) from [{}]", position, position + length, length, this); - final Tuple indexCacheMiss; // null if not a miss - - // We prefer to use the index cache if the recovery is not done yet - if (directory.isRecoveryDone() == false) { - // We try to use the snapshot blob cache if: - // - the file is small enough to be fully cached in the blob cache - 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 (cachedBlob == CachedBlob.CACHE_MISS || cachedBlob == CachedBlob.CACHE_NOT_READY) { - // 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 regions of the file we would like to have the next time. The regions are expressed as tuples of - // {start, end} ranges 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 = Tuple.tuple(0L, fileInfo.length()); + try { + final CacheFile cacheFile = getCacheFileSafe(); + try (Releasable ignored = cacheFile.fileLock()) { + + // Can we serve the read directly from disk? If so, do so and don't worry about anything else. + + if (cacheFile.getAbsentRangeWithin(position, position + length) == null) { + final Tuple targetRange = Tuple.tuple(position, position + length); + cacheFile.populateAndRead(targetRange, targetRange, channel -> { + final int read = readCacheFile(channel, position, b); + assert read == length : read + " vs " + length; + return read; + }, (channel, from, to, progressUpdater) -> { + final String message = "range [" + + from + + "-" + + to + + "] ([" + + (to - from) + + "] bytes) unexpectedly missing when reading [" + + position + + "-" + + (position + length) + + "] from " + + CachedBlobContainerIndexInput.this; + assert false : message; + throw new IllegalStateException(message); + }, EsExecutors.newDirectExecutorService()); // never used + return; + } + + // Requested data is not on disk, so try the cache index next. + + final Tuple indexCacheMiss; // null if not a miss + + // We try to use the cache index if: + // - the file is small enough to be fully cached + 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 (cachedBlob == CachedBlob.CACHE_MISS || cachedBlob == CachedBlob.CACHE_NOT_READY) { + // 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 = Tuple.tuple(0L, fileInfo.length()); + } else { + // the index input is too large to fully cache, so just cache the initial range + indexCacheMiss = Tuple.tuple(0L, (long) BlobStoreCacheService.DEFAULT_CACHED_BLOB_SIZE); + } + + // 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. } else { - // the index input is too large to fully cache, so just cache the initial range - indexCacheMiss = Tuple.tuple(0L, (long) BlobStoreCacheService.DEFAULT_CACHED_BLOB_SIZE); - } + logger.trace( + "reading [{}] bytes of file [{}] at position [{}] using cache index", + length, + fileInfo.physicalName(), + position + ); + stats.addIndexCacheBytesRead(cachedBlob.length()); - // 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. - } else { - logger.trace( - "reading [{}] bytes of file [{}] at position [{}] using index cache", - length, - fileInfo.physicalName(), - position - ); - stats.addIndexCacheBytesRead(cachedBlob.length()); - b.put(BytesReference.toBytes(cachedBlob.bytes().slice(Math.toIntExact(position), length))); + // TODO don't call toBytes here + b.put(BytesReference.toBytes(cachedBlob.bytes().slice(Math.toIntExact(position), length))); - try { - final CacheFile cacheFile = getCacheFileSafe(); - try (Releasable ignored = cacheFile.fileLock()) { + try { final Tuple cachedRange = Tuple.tuple(cachedBlob.from(), cachedBlob.to()); cacheFile.populateAndRead( cachedRange, @@ -222,41 +254,33 @@ protected void readInternal(ByteBuffer b) throws IOException { assert writePosition == to : writePosition + " vs " + to; final long endTimeNanos = stats.currentTimeNanos(); stats.addCachedBytesWritten(to - from, endTimeNanos - startTimeNanos); - logger.trace("copied bytes [{}-{}] of file [{}] from index cache to node cache", from, to, fileInfo); + logger.trace("copied bytes [{}-{}] of file [{}] from cache index to disk", from, to, fileInfo); }, directory.cacheFetchAsyncExecutor() ); + } catch (Exception e) { + logger.debug( + new ParameterizedMessage( + "failed to store bytes [{}-{}] of file [{}] obtained from index cache", + cachedBlob.from(), + cachedBlob.to(), + fileInfo + ), + e + ); + // oh well, no big deal, at least we can return them to the caller. } - } catch (Exception e) { - logger.debug( - new ParameterizedMessage( - "failed to store bytes [{}-{}] of file [{}] obtained from index cache", - cachedBlob.from(), - cachedBlob.to(), - fileInfo - ), - e - ); - // oh well, no big deal, at least we can return them to the caller. - } - return; + return; + } + } else { + // requested range is not eligible for caching + indexCacheMiss = null; } - } else { - // requested range is not eligible for caching - indexCacheMiss = null; - } - - logger.trace("recovery cache miss for [{}], falling through with [{}]", this, indexCacheMiss); - } else { - indexCacheMiss = null; - } - try { - final CacheFile cacheFile = getCacheFileSafe(); - try (Releasable ignored = cacheFile.fileLock()) { + // Requested data is also not in the cache index, so we must visit the blob store to satisfy both the target range and any + // miss in the cache index. - // Read all target ranges in one go, including any cache misses identified above. final Tuple startRangeToWrite = computeRange(position); final Tuple endRangeToWrite = computeRange(position + length - 1); assert startRangeToWrite.v2() <= endRangeToWrite.v2() : startRangeToWrite + " vs " + endRangeToWrite; From d53443e753bc24bb787298fab97665874e89a384 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 24 Aug 2020 14:08:53 +0100 Subject: [PATCH 43/63] Collect timings for cache index reads --- .../SearchableSnapshotShardStats.java | 10 +++++----- .../SearchableSnapshotShardStatsTests.java | 2 +- .../org/elasticsearch/index/store/IndexInputStats.java | 8 ++++---- .../store/cache/CachedBlobContainerIndexInput.java | 5 ++++- .../TransportSearchableSnapshotsStatsAction.java | 2 +- .../action/SearchableSnapshotsStatsResponseTests.java | 2 +- 6 files changed, 16 insertions(+), 13 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java index 9ddccc69414be..0154a21bb894f 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java @@ -135,7 +135,7 @@ public static class CacheIndexInputStats implements Writeable, ToXContentObject private final Counter contiguousReads; private final Counter nonContiguousReads; private final Counter cachedBytesRead; - private final Counter indexCacheBytesRead; + private final TimedCounter indexCacheBytesRead; private final TimedCounter cachedBytesWritten; private final TimedCounter directBytesRead; private final TimedCounter optimizedBytesRead; @@ -146,7 +146,7 @@ public CacheIndexInputStats(String fileName, long fileLength, long openCount, lo Counter forwardSmallSeeks, Counter backwardSmallSeeks, Counter forwardLargeSeeks, Counter backwardLargeSeeks, Counter contiguousReads, Counter nonContiguousReads, - Counter cachedBytesRead, Counter indexCacheBytesRead, + Counter cachedBytesRead, TimedCounter indexCacheBytesRead, TimedCounter cachedBytesWritten, TimedCounter directBytesRead, TimedCounter optimizedBytesRead, Counter blobStoreBytesRequested, long currentIndexCacheFills) { this.fileName = fileName; @@ -181,9 +181,9 @@ public CacheIndexInputStats(String fileName, long fileLength, long openCount, lo this.nonContiguousReads = new Counter(in); this.cachedBytesRead = new Counter(in); if (in.getVersion().onOrAfter(Version.V_8_0_0)) { - this.indexCacheBytesRead = new Counter(in); + this.indexCacheBytesRead = new TimedCounter(in); } else { - this.indexCacheBytesRead = new Counter(0, 0, 0, 0); + this.indexCacheBytesRead = new TimedCounter(0, 0, 0, 0, 0); } this.cachedBytesWritten = new TimedCounter(in); this.directBytesRead = new TimedCounter(in); @@ -267,7 +267,7 @@ public Counter getCachedBytesRead() { return cachedBytesRead; } - public Counter getIndexCacheBytesRead() { + public TimedCounter getIndexCacheBytesRead() { return indexCacheBytesRead; } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java index d57d0746db417..626b2e9168a44 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java @@ -46,7 +46,7 @@ private CacheIndexInputStats randomCacheIndexInputStats() { randomCounter(), randomCounter(), randomCounter(), randomCounter(), randomCounter(), randomCounter(), - randomCounter(), randomCounter(), randomTimedCounter(), + randomCounter(), randomTimedCounter(), randomTimedCounter(), randomTimedCounter(), randomTimedCounter(), randomCounter(), randomNonNegativeLong()); } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java index 0f652d604ba5d..5cf7e7c0e758d 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java @@ -44,7 +44,7 @@ public class IndexInputStats { private final TimedCounter optimizedBytesRead = new TimedCounter(); private final Counter cachedBytesRead = new Counter(); - private final Counter indexCacheBytesRead = new Counter(); + private final TimedCounter indexCacheBytesRead = new TimedCounter(); private final TimedCounter cachedBytesWritten = new TimedCounter(); private final Counter blobStoreBytesRequested = new Counter(); @@ -79,8 +79,8 @@ public void addCachedBytesRead(int bytesRead) { cachedBytesRead.add(bytesRead); } - public void addIndexCacheBytesRead(int bytesRead) { - indexCacheBytesRead.add(bytesRead); + public void addIndexCacheBytesRead(int bytesRead, long nanoseconds) { + indexCacheBytesRead.add(bytesRead, nanoseconds); } public void addCachedBytesWritten(long bytesWritten, long nanoseconds) { @@ -182,7 +182,7 @@ public Counter getCachedBytesRead() { return cachedBytesRead; } - public Counter getIndexCacheBytesRead() { + public TimedCounter getIndexCacheBytesRead() { return indexCacheBytesRead; } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index f5727e25f6530..a216400b09a5b 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -203,12 +203,15 @@ protected void readInternal(ByteBuffer b) throws IOException { final boolean isStartOfFile = (position + length <= BlobStoreCacheService.DEFAULT_CACHED_BLOB_SIZE); if (canBeFullyCached || isStartOfFile) { + final long cacheReadStartTime = stats.currentTimeNanos(); final CachedBlob cachedBlob = directory.getCachedBlob(fileInfo.physicalName(), 0L, length); if (cachedBlob == CachedBlob.CACHE_MISS || cachedBlob == CachedBlob.CACHE_NOT_READY) { // 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. + stats.addIndexCacheBytesRead(0, stats.currentTimeNanos() - cacheReadStartTime); + if (canBeFullyCached) { // if the index input is smaller than twice the size of the blob cache, it will be fully indexed indexCacheMiss = Tuple.tuple(0L, fileInfo.length()); @@ -226,7 +229,7 @@ protected void readInternal(ByteBuffer b) throws IOException { fileInfo.physicalName(), position ); - stats.addIndexCacheBytesRead(cachedBlob.length()); + stats.addIndexCacheBytesRead(cachedBlob.length(), stats.currentTimeNanos() - cacheReadStartTime); // TODO don't call toBytes here b.put(BytesReference.toBytes(cachedBlob.bytes().slice(Math.toIntExact(position), length))); diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java index 8d0c3eac5fe02..9343f54e84314 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java @@ -108,7 +108,7 @@ private static CacheIndexInputStats toCacheIndexInputStats(final String fileName toCounter(inputStats.getContiguousReads()), toCounter(inputStats.getNonContiguousReads()), toCounter(inputStats.getCachedBytesRead()), - toCounter(inputStats.getIndexCacheBytesRead()), + toTimedCounter(inputStats.getIndexCacheBytesRead()), toTimedCounter(inputStats.getCachedBytesWritten()), toTimedCounter(inputStats.getDirectBytesRead()), toTimedCounter(inputStats.getOptimizedBytesRead()), diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java index ad57c607d1722..9e7a5a1732fc3 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java @@ -107,7 +107,7 @@ private static SearchableSnapshotShardStats.CacheIndexInputStats randomCacheInde randomCounter(), randomCounter(), randomCounter(), - randomCounter(), + randomTimedCounter(), randomTimedCounter(), randomTimedCounter(), randomTimedCounter(), From 299443fe3e901c7fb5a10d15c48a04d4ab36865d Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 24 Aug 2020 14:20:45 +0100 Subject: [PATCH 44/63] Assert enough space --- .../index/store/checksum/ChecksumBlobContainerIndexInput.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/checksum/ChecksumBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/checksum/ChecksumBlobContainerIndexInput.java index d0b7b1ebb2ef3..bb72e354b2dfc 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/checksum/ChecksumBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/checksum/ChecksumBlobContainerIndexInput.java @@ -134,6 +134,7 @@ public static ChecksumBlobContainerIndexInput create(String name, long length, S public static byte[] checksumToBytesArray(String checksum) throws IOException { final byte[] result = new byte[CodecUtil.footerLength()]; + assert result.length >= Integer.BYTES + Integer.BYTES + Long.BYTES; // ensure that nobody changed the file format under us final ByteArrayDataOutput output = new ByteArrayDataOutput(result); // reverse CodecUtil.writeFooter() output.writeInt(CodecUtil.FOOTER_MAGIC); From 8b607b04ae0febff52635ae213f37a5ad7ee8d5c Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 25 Aug 2020 10:31:21 +0100 Subject: [PATCH 45/63] Fix bytes-read stats --- .../index/store/cache/CachedBlobContainerIndexInput.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index a216400b09a5b..a65bf4ff48e65 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -189,6 +189,9 @@ protected void readInternal(ByteBuffer b) throws IOException { assert false : message; throw new IllegalStateException(message); }, EsExecutors.newDirectExecutorService()); // never used + + readComplete(position, length); + return; } @@ -274,6 +277,8 @@ protected void readInternal(ByteBuffer b) throws IOException { // oh well, no big deal, at least we can return them to the caller. } + readComplete(position, length); + return; } } else { @@ -370,6 +375,10 @@ public void onFailure(Exception e1) { // already a rare case caused by an overfull/undersized cache. } + readComplete(position, length); + } + + private void readComplete(long position, int length) { stats.incrementBytesRead(lastReadPosition, position, length); lastReadPosition = position + length; lastSeekPosition = lastReadPosition; From dbf6f2f2d327946d5f78f9e9e7095eeed4a4a06f Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 25 Aug 2020 10:31:34 +0100 Subject: [PATCH 46/63] Revert back to untimed counter --- .../SearchableSnapshotShardStats.java | 10 +++++----- .../org/elasticsearch/index/store/IndexInputStats.java | 8 ++++---- .../store/cache/CachedBlobContainerIndexInput.java | 4 +--- .../TransportSearchableSnapshotsStatsAction.java | 2 +- .../action/SearchableSnapshotsStatsResponseTests.java | 2 +- 5 files changed, 12 insertions(+), 14 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java index 0154a21bb894f..9ddccc69414be 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java @@ -135,7 +135,7 @@ public static class CacheIndexInputStats implements Writeable, ToXContentObject private final Counter contiguousReads; private final Counter nonContiguousReads; private final Counter cachedBytesRead; - private final TimedCounter indexCacheBytesRead; + private final Counter indexCacheBytesRead; private final TimedCounter cachedBytesWritten; private final TimedCounter directBytesRead; private final TimedCounter optimizedBytesRead; @@ -146,7 +146,7 @@ public CacheIndexInputStats(String fileName, long fileLength, long openCount, lo Counter forwardSmallSeeks, Counter backwardSmallSeeks, Counter forwardLargeSeeks, Counter backwardLargeSeeks, Counter contiguousReads, Counter nonContiguousReads, - Counter cachedBytesRead, TimedCounter indexCacheBytesRead, + Counter cachedBytesRead, Counter indexCacheBytesRead, TimedCounter cachedBytesWritten, TimedCounter directBytesRead, TimedCounter optimizedBytesRead, Counter blobStoreBytesRequested, long currentIndexCacheFills) { this.fileName = fileName; @@ -181,9 +181,9 @@ public CacheIndexInputStats(String fileName, long fileLength, long openCount, lo this.nonContiguousReads = new Counter(in); this.cachedBytesRead = new Counter(in); if (in.getVersion().onOrAfter(Version.V_8_0_0)) { - this.indexCacheBytesRead = new TimedCounter(in); + this.indexCacheBytesRead = new Counter(in); } else { - this.indexCacheBytesRead = new TimedCounter(0, 0, 0, 0, 0); + this.indexCacheBytesRead = new Counter(0, 0, 0, 0); } this.cachedBytesWritten = new TimedCounter(in); this.directBytesRead = new TimedCounter(in); @@ -267,7 +267,7 @@ public Counter getCachedBytesRead() { return cachedBytesRead; } - public TimedCounter getIndexCacheBytesRead() { + public Counter getIndexCacheBytesRead() { return indexCacheBytesRead; } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java index 5cf7e7c0e758d..0f652d604ba5d 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java @@ -44,7 +44,7 @@ public class IndexInputStats { private final TimedCounter optimizedBytesRead = new TimedCounter(); private final Counter cachedBytesRead = new Counter(); - private final TimedCounter indexCacheBytesRead = new TimedCounter(); + private final Counter indexCacheBytesRead = new Counter(); private final TimedCounter cachedBytesWritten = new TimedCounter(); private final Counter blobStoreBytesRequested = new Counter(); @@ -79,8 +79,8 @@ public void addCachedBytesRead(int bytesRead) { cachedBytesRead.add(bytesRead); } - public void addIndexCacheBytesRead(int bytesRead, long nanoseconds) { - indexCacheBytesRead.add(bytesRead, nanoseconds); + public void addIndexCacheBytesRead(int bytesRead) { + indexCacheBytesRead.add(bytesRead); } public void addCachedBytesWritten(long bytesWritten, long nanoseconds) { @@ -182,7 +182,7 @@ public Counter getCachedBytesRead() { return cachedBytesRead; } - public TimedCounter getIndexCacheBytesRead() { + public Counter getIndexCacheBytesRead() { return indexCacheBytesRead; } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index a65bf4ff48e65..b2b7910c62fba 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -206,14 +206,12 @@ protected void readInternal(ByteBuffer b) throws IOException { final boolean isStartOfFile = (position + length <= BlobStoreCacheService.DEFAULT_CACHED_BLOB_SIZE); if (canBeFullyCached || isStartOfFile) { - final long cacheReadStartTime = stats.currentTimeNanos(); final CachedBlob cachedBlob = directory.getCachedBlob(fileInfo.physicalName(), 0L, length); if (cachedBlob == CachedBlob.CACHE_MISS || cachedBlob == CachedBlob.CACHE_NOT_READY) { // 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. - stats.addIndexCacheBytesRead(0, stats.currentTimeNanos() - cacheReadStartTime); if (canBeFullyCached) { // if the index input is smaller than twice the size of the blob cache, it will be fully indexed @@ -232,7 +230,7 @@ protected void readInternal(ByteBuffer b) throws IOException { fileInfo.physicalName(), position ); - stats.addIndexCacheBytesRead(cachedBlob.length(), stats.currentTimeNanos() - cacheReadStartTime); + stats.addIndexCacheBytesRead(cachedBlob.length()); // TODO don't call toBytes here b.put(BytesReference.toBytes(cachedBlob.bytes().slice(Math.toIntExact(position), length))); diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java index 9343f54e84314..8d0c3eac5fe02 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/TransportSearchableSnapshotsStatsAction.java @@ -108,7 +108,7 @@ private static CacheIndexInputStats toCacheIndexInputStats(final String fileName toCounter(inputStats.getContiguousReads()), toCounter(inputStats.getNonContiguousReads()), toCounter(inputStats.getCachedBytesRead()), - toTimedCounter(inputStats.getIndexCacheBytesRead()), + toCounter(inputStats.getIndexCacheBytesRead()), toTimedCounter(inputStats.getCachedBytesWritten()), toTimedCounter(inputStats.getDirectBytesRead()), toTimedCounter(inputStats.getOptimizedBytesRead()), diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java index 9e7a5a1732fc3..ad57c607d1722 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java @@ -107,7 +107,7 @@ private static SearchableSnapshotShardStats.CacheIndexInputStats randomCacheInde randomCounter(), randomCounter(), randomCounter(), - randomTimedCounter(), + randomCounter(), randomTimedCounter(), randomTimedCounter(), randomTimedCounter(), From e8bc5616874e388fcc098e40b6ac5934c3901e38 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 25 Aug 2020 10:57:17 +0100 Subject: [PATCH 47/63] Another revert to counter --- .../searchablesnapshots/SearchableSnapshotShardStatsTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java index 626b2e9168a44..d57d0746db417 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java @@ -46,7 +46,7 @@ private CacheIndexInputStats randomCacheIndexInputStats() { randomCounter(), randomCounter(), randomCounter(), randomCounter(), randomCounter(), randomCounter(), - randomCounter(), randomTimedCounter(), randomTimedCounter(), + randomCounter(), randomCounter(), randomTimedCounter(), randomTimedCounter(), randomTimedCounter(), randomCounter(), randomNonNegativeLong()); } From 1172e68e3b14291e4e56e73368106e390e2372a6 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 25 Aug 2020 11:40:40 +0100 Subject: [PATCH 48/63] Avoid allocation reading cached blob --- .../index/store/cache/CachedBlobContainerIndexInput.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index b2b7910c62fba..36be6e123d89a 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -232,8 +232,12 @@ protected void readInternal(ByteBuffer b) throws IOException { ); stats.addIndexCacheBytesRead(cachedBlob.length()); - // TODO don't call toBytes here - b.put(BytesReference.toBytes(cachedBlob.bytes().slice(Math.toIntExact(position), length))); + final BytesRefIterator cachedBytesIterator = cachedBlob.bytes().slice(Math.toIntExact(position), length).iterator(); + BytesRef bytesRef; + while ((bytesRef = cachedBytesIterator.next()) != null) { + b.put(bytesRef.bytes, bytesRef.offset, bytesRef.length); + } + assert b.position() == length : "copied " + b.position() + " but expected " + length; try { final Tuple cachedRange = Tuple.tuple(cachedBlob.from(), cachedBlob.to()); From 181b1f516029fa4cbe168cc55fe58139425a29d3 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 25 Aug 2020 12:41:04 +0100 Subject: [PATCH 49/63] Allow waiting for a cache file range without committing to filling any gaps --- .../index/store/cache/CacheFile.java | 35 +++++++ .../cache/CachedBlobContainerIndexInput.java | 48 +++------- .../index/store/cache/SparseFileTracker.java | 91 ++++++++++++++++++- .../store/cache/SparseFileTrackerTests.java | 79 ++++++++++------ 4 files changed, 188 insertions(+), 65 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CacheFile.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CacheFile.java index f71c48cba89f0..e4bc6ac8bb613 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CacheFile.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CacheFile.java @@ -339,6 +339,41 @@ public void onFailure(Exception e) { return future; } + /** + * Notifies the {@link RangeAvailableHandler} when {@code rangeToRead} is available to read from the file. If {@code rangeToRead} is + * already available then the {@link RangeAvailableHandler} is called synchronously by this method; if not, but it is pending, then the + * {@link RangeAvailableHandler} is notified when the pending ranges have completed. If it contains gaps that are not currently pending + * then no listeners are registered and this method returns {@code null}. + * + * @return a future which returns the result of the {@link RangeAvailableHandler} once it has completed, or {@code null} if the + * target range is neither available nor pending. + */ + @Nullable + CompletableFuture readIfAvailableOrPending(final Tuple rangeToRead, final RangeAvailableHandler reader) { + final CompletableFuture future = new CompletableFuture<>(); + try { + ensureOpen(); + if (tracker.waitForRangeIfPending(rangeToRead, ActionListener.wrap(success -> { + final int read = reader.onRangeAvailable(channel); + assert read == rangeToRead.v2() - rangeToRead.v1() : "partial read [" + + read + + "] does not match the range to read [" + + rangeToRead.v2() + + '-' + + rangeToRead.v1() + + ']'; + future.complete(read); + }, future::completeExceptionally))) { + return future; + } else { + return null; + } + } catch (Exception e) { + future.completeExceptionally(e); + return future; + } + } + public Tuple getAbsentRangeWithin(long start, long end) { ensureOpen(); return tracker.getAbsentRangeWithin(start, end); diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java index 36be6e123d89a..4a4eebfce8d46 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/CachedBlobContainerIndexInput.java @@ -24,7 +24,6 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.Channels; import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo; import org.elasticsearch.index.snapshots.blobstore.SlicedInputStream; import org.elasticsearch.index.store.BaseSearchableSnapshotIndexInput; @@ -167,31 +166,19 @@ protected void readInternal(ByteBuffer b) throws IOException { // Can we serve the read directly from disk? If so, do so and don't worry about anything else. - if (cacheFile.getAbsentRangeWithin(position, position + length) == null) { - final Tuple targetRange = Tuple.tuple(position, position + length); - cacheFile.populateAndRead(targetRange, targetRange, channel -> { + final CompletableFuture waitingForRead = cacheFile.readIfAvailableOrPending( + Tuple.tuple(position, position + length), + channel -> { final int read = readCacheFile(channel, position, b); assert read == length : read + " vs " + length; return read; - }, (channel, from, to, progressUpdater) -> { - final String message = "range [" - + from - + "-" - + to - + "] ([" - + (to - from) - + "] bytes) unexpectedly missing when reading [" - + position - + "-" - + (position + length) - + "] from " - + CachedBlobContainerIndexInput.this; - assert false : message; - throw new IllegalStateException(message); - }, EsExecutors.newDirectExecutorService()); // never used + } + ); + if (waitingForRead != null) { + final Integer read = waitingForRead.get(); + assert read == length; readComplete(position, length); - return; } @@ -323,7 +310,7 @@ protected void readInternal(ByteBuffer b) throws IOException { if (indexCacheMiss != null) { final Releasable onCacheFillComplete = stats.addIndexCacheFill(); - cacheFile.populateAndRead(indexCacheMiss, indexCacheMiss, channel -> { + final CompletableFuture readFuture = cacheFile.readIfAvailableOrPending(indexCacheMiss, channel -> { final int indexCacheMissLength = Math.toIntExact(indexCacheMiss.v2() - indexCacheMiss.v1()); // We assume that we only cache small portions of blobs so that we do not need to: @@ -349,19 +336,14 @@ public void onFailure(Exception e1) { } }); return indexCacheMissLength; - }, (channel, from, to, progressUpdater) -> { + }); + + if (readFuture == null) { // Normally doesn't happen, we're already obtaining a range covering all cache misses above, but theoretically // possible in the case that the real populateAndRead call already failed to obtain this range of the file. In that - // case, try and fill just the cache miss from the blob store because there may be other reads waiting on this - // range. - logger.debug( - "directly filling index cache miss [{}-{}] of {} due to earlier failure", - from, - to, - CachedBlobContainerIndexInput.this - ); - writeCacheFile(channel, from, to, progressUpdater); - }, directory.cacheFetchAsyncExecutor()); + // case, simply move on. + onCacheFillComplete.close(); + } } final int bytesRead = populateCacheFuture.get(); diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/SparseFileTracker.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/SparseFileTracker.java index feeb55152e314..9e9217688b66a 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/SparseFileTracker.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/SparseFileTracker.java @@ -178,7 +178,7 @@ public List waitForRange(final Tuple range, final Tuple null), - Math.min(requiredRange.end, subRange != null ? subRange.v2() : Long.MAX_VALUE) + Math.min(requiredRange.end, subRange.v2()) ); break; default: @@ -187,10 +187,7 @@ public List waitForRange(final Tuple range, final Tuple r.completionListener.addListener( - groupedActionListener, - Math.min(r.end, subRange != null ? subRange.v2() : Long.MAX_VALUE) - ) + r -> r.completionListener.addListener(groupedActionListener, Math.min(r.end, subRange.v2())) ); } @@ -203,6 +200,90 @@ public List waitForRange(final Tuple range, final Tuple range, final ActionListener listener) { + final long start = range.v1(); + final long end = range.v2(); + if (end < start || start < 0L || length < end) { + throw new IllegalArgumentException("invalid range [start=" + start + ", end=" + end + ", length=" + length + "]"); + } + + synchronized (mutex) { + assert invariant(); + + final List pendingRanges = new ArrayList<>(); + + final Range targetRange = new Range(start, end, null); + final SortedSet earlierRanges = ranges.headSet(targetRange, false); // ranges with strictly earlier starts + if (earlierRanges.isEmpty() == false) { + final Range lastEarlierRange = earlierRanges.last(); + if (start < lastEarlierRange.end) { + if (lastEarlierRange.isPending()) { + pendingRanges.add(lastEarlierRange); + } + targetRange.start = Math.min(end, lastEarlierRange.end); + } + } + + while (targetRange.start < end) { + assert 0 <= targetRange.start : targetRange; + assert invariant(); + + final SortedSet existingRanges = ranges.tailSet(targetRange); + if (existingRanges.isEmpty()) { + return false; + } else { + final Range firstExistingRange = existingRanges.first(); + assert targetRange.start <= firstExistingRange.start : targetRange + " vs " + firstExistingRange; + + if (targetRange.start == firstExistingRange.start) { + if (firstExistingRange.isPending()) { + pendingRanges.add(firstExistingRange); + } + targetRange.start = Math.min(end, firstExistingRange.end); + } else { + return false; + } + } + } + assert targetRange.start == targetRange.end : targetRange; + assert targetRange.start == end : targetRange; + assert invariant(); + + switch (pendingRanges.size()) { + case 0: + break; + case 1: + final Range pendingRange = pendingRanges.get(0); + pendingRange.completionListener.addListener( + ActionListener.map(listener, progress -> null), + Math.min(pendingRange.end, end) + ); + return true; + default: + final GroupedActionListener groupedActionListener = new GroupedActionListener<>( + ActionListener.map(listener, progress -> null), + pendingRanges.size() + ); + pendingRanges.forEach(r -> r.completionListener.addListener(groupedActionListener, Math.min(r.end, end))); + return true; + } + } + + listener.onResponse(null); + return true; + } + /** * Returns a range that contains all bytes of the target range which are absent (possibly pending). The returned range may include * some ranges of present bytes. It tries to return the smallest possible range, but does so on a best-effort basis. This method does diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/SparseFileTrackerTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/SparseFileTrackerTests.java index e64bea4412ff5..822c07037fa4b 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/SparseFileTrackerTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/SparseFileTrackerTests.java @@ -220,17 +220,21 @@ public void testCallsListenerWhenRangeIsAvailable() { if (pending == false) { final AtomicBoolean wasNotified = new AtomicBoolean(); - final List gaps = sparseFileTracker.waitForRange( - range, - subRange, - ActionListener.wrap(ignored -> assertTrue(wasNotified.compareAndSet(false, true)), e -> { throw new AssertionError(e); }) + final ActionListener listener = ActionListener.wrap( + ignored -> assertTrue(wasNotified.compareAndSet(false, true)), + e -> { throw new AssertionError(e); } ); + final List gaps = sparseFileTracker.waitForRange(range, subRange, listener); assertTrue( "All bytes of the sub range " + subRange + " are available, listener must be executed immediately", wasNotified.get() ); + wasNotified.set(false); + assertTrue(sparseFileTracker.waitForRangeIfPending(subRange, listener)); + assertTrue(wasNotified.get()); + for (final SparseFileTracker.Gap gap : gaps) { assertThat(gap.start(), greaterThanOrEqualTo(range.v1())); assertThat(gap.end(), lessThanOrEqualTo(range.v2())); @@ -238,13 +242,19 @@ public void testCallsListenerWhenRangeIsAvailable() { for (long i = gap.start(); i < gap.end(); i++) { assertThat(fileContents[Math.toIntExact(i)], equalTo(UNAVAILABLE)); fileContents[Math.toIntExact(i)] = AVAILABLE; - assertTrue(wasNotified.get()); gap.onProgress(i + 1L); } gap.onCompletion(); } } else { + final AtomicBoolean waitIfPendingWasNotified = new AtomicBoolean(); + final ActionListener waitIfPendingListener = ActionListener.wrap( + ignored -> assertTrue(waitIfPendingWasNotified.compareAndSet(false, true)), + e -> { throw new AssertionError(e); } + ); + assertFalse(sparseFileTracker.waitForRangeIfPending(subRange, waitIfPendingListener)); + final AtomicBoolean wasNotified = new AtomicBoolean(); final AtomicBoolean expectNotification = new AtomicBoolean(); final List gaps = sparseFileTracker.waitForRange(range, subRange, ActionListener.wrap(ignored -> { @@ -254,6 +264,9 @@ public void testCallsListenerWhenRangeIsAvailable() { assertFalse("Listener should not have been executed yet", wasNotified.get()); + assertTrue(sparseFileTracker.waitForRangeIfPending(subRange, waitIfPendingListener)); + assertFalse(waitIfPendingWasNotified.get()); + long triggeringProgress = -1L; for (long i = subRange.v1(); i < subRange.v2(); i++) { if (fileContents[Math.toIntExact(i)] == UNAVAILABLE) { @@ -278,7 +291,7 @@ public void testCallsListenerWhenRangeIsAvailable() { + "] is reached, but it was triggered after progress got updated to [" + i + ']', - wasNotified.get(), + wasNotified.get() && waitIfPendingWasNotified.get(), equalTo(triggeringProgress < i) ); @@ -290,7 +303,7 @@ public void testCallsListenerWhenRangeIsAvailable() { + "] is reached, but it was triggered after progress got updated to [" + i + ']', - wasNotified.get(), + wasNotified.get() && waitIfPendingWasNotified.get(), equalTo(triggeringProgress < i + 1L) ); } @@ -305,8 +318,10 @@ public void testCallsListenerWhenRangeIsAvailable() { wasNotified.get(), equalTo(triggeringProgress < gap.end()) ); + assertThat(waitIfPendingWasNotified.get(), equalTo(triggeringProgress < gap.end())); } assertTrue(wasNotified.get()); + assertTrue(waitIfPendingWasNotified.get()); } final AtomicBoolean wasNotified = new AtomicBoolean(); @@ -430,34 +445,44 @@ private static void waitForRandomRange( final AtomicBoolean listenerCalled = new AtomicBoolean(); listenerCalledConsumer.accept(listenerCalled); - final boolean useSubRange = randomBoolean(); + final boolean fillInGaps = randomBoolean(); + final boolean useSubRange = fillInGaps && randomBoolean(); final long subRangeStart = useSubRange ? randomLongBetween(rangeStart, rangeEnd) : rangeStart; final long subRangeEnd = useSubRange ? randomLongBetween(subRangeStart, rangeEnd) : rangeEnd; - final List gaps = sparseFileTracker.waitForRange( - Tuple.tuple(rangeStart, rangeEnd), - Tuple.tuple(subRangeStart, subRangeEnd), - new ActionListener<>() { - @Override - public void onResponse(Void aVoid) { - for (long i = subRangeStart; i < subRangeEnd; i++) { - assertThat(fileContents[Math.toIntExact(i)], equalTo(AVAILABLE)); - } - assertTrue(listenerCalled.compareAndSet(false, true)); + final ActionListener actionListener = new ActionListener<>() { + @Override + public void onResponse(Void aVoid) { + for (long i = subRangeStart; i < subRangeEnd; i++) { + assertThat(fileContents[Math.toIntExact(i)], equalTo(AVAILABLE)); } + assertTrue(listenerCalled.compareAndSet(false, true)); + } - @Override - public void onFailure(Exception e) { - assertTrue(listenerCalled.compareAndSet(false, true)); - } + @Override + public void onFailure(Exception e) { + assertTrue(listenerCalled.compareAndSet(false, true)); } - ); + }; - 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)); + if (randomBoolean()) { + final List gaps = sparseFileTracker.waitForRange( + Tuple.tuple(rangeStart, rangeEnd), + Tuple.tuple(subRangeStart, subRangeEnd), + actionListener + ); + + 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)); + } + gapConsumer.accept(gap); + } + } else { + final boolean listenerRegistered = sparseFileTracker.waitForRangeIfPending(Tuple.tuple(rangeStart, rangeEnd), actionListener); + if (listenerRegistered == false) { + assertTrue(listenerCalled.compareAndSet(false, true)); } - gapConsumer.accept(gap); } } From abdbf3cecdf4548732ef0653a65b105f84d71cab Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 25 Aug 2020 13:14:40 +0100 Subject: [PATCH 50/63] No longer need to track whether recovery is done --- .../index/store/SearchableSnapshotDirectory.java | 14 +------------- 1 file changed, 1 insertion(+), 13 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java index 15fb5530ae3c2..cf43474817d38 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java @@ -124,7 +124,6 @@ public class SearchableSnapshotDirectory extends BaseDirectory { private final long uncachedChunkSize; // if negative use BlobContainer#readBlobPreferredLength, see #getUncachedChunkSize() private final Path cacheDir; private final ShardPath shardPath; - private final AtomicBoolean recovered; private final AtomicBoolean closed; // volatile fields are updated once under `this` lock, all together, iff loaded is not true. @@ -161,7 +160,6 @@ public SearchableSnapshotDirectory( this.cacheService = Objects.requireNonNull(cacheService); this.cacheDir = Objects.requireNonNull(cacheDir); this.shardPath = Objects.requireNonNull(shardPath); - this.recovered = new AtomicBoolean(false); this.closed = new AtomicBoolean(false); this.useCache = SNAPSHOT_CACHE_ENABLED_SETTING.get(indexSettings); this.prewarmCache = useCache ? SNAPSHOT_CACHE_PREWARM_ENABLED_SETTING.get(indexSettings) : false; @@ -198,6 +196,7 @@ public boolean loadSnapshot(RecoveryState recoveryState) { assert recoveryState != null; assert recoveryState instanceof SearchableSnapshotRecoveryState; assert assertCurrentThreadMayLoadSnapshot(); + // noinspection ConstantConditions in case assertions are disabled if (recoveryState instanceof SearchableSnapshotRecoveryState == false) { throw new IllegalArgumentException("A SearchableSnapshotRecoveryState instance was expected"); } @@ -233,17 +232,6 @@ public BlobStoreIndexShardSnapshot snapshot() { return snapshot; } - public boolean isRecoveryDone() { - if (recovered.get()) { - return true; - } - final RecoveryState recoveryState = this.recoveryState; - if (recoveryState != null && recoveryState.getStage() == RecoveryState.Stage.DONE) { - return recovered.compareAndSet(false, true); - } - return false; - } - private List files() { if (loaded == false) { return List.of(); From 38bf6976d402dfd7abaaf5dd35375c732c2f2d85 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 25 Aug 2020 13:27:00 +0100 Subject: [PATCH 51/63] No need to wait for fetch threads to be idle --- ...ableSnapshotsBlobStoreCacheIntegTests.java | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java index 8ff601b0c6cc2..775f089d2dba0 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java @@ -34,7 +34,6 @@ import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.junit.annotations.TestLogging; -import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.ClientHelper; import org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotShardStats; import org.elasticsearch.xpack.searchablesnapshots.BaseSearchableSnapshotsIntegTestCase; @@ -56,13 +55,10 @@ import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.concurrent.ThreadPoolExecutor; import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.INDEX_SHARD_SNAPSHOT_FORMAT; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; -import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.CACHE_FETCH_ASYNC_THREAD_POOL_NAME; -import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.CACHE_PREWARMING_THREAD_POOL_NAME; import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsConstants.SNAPSHOT_BLOB_CACHE_INDEX; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; @@ -148,7 +144,6 @@ public void testBlobStoreCache() throws Exception { .build() ); ensureGreen(restoredIndex); - ensureExecutorsAreIdle(); // wait for all async cache fills to complete assertBusy(() -> { @@ -215,7 +210,6 @@ public void testBlobStoreCache() throws Exception { .build() ); ensureGreen(restoredAgainIndex); - ensureExecutorsAreIdle(); logger.info("--> verifying shards of [{}] were started without using the blob store more than necessary", restoredAgainIndex); for (final SearchableSnapshotShardStats shardStats : client().execute( @@ -279,7 +273,6 @@ public Settings onNodeStopped(String nodeName) throws Exception { } }); ensureGreen(restoredAgainIndex); - ensureExecutorsAreIdle(); logger.info("--> verifying documents in index [{}]", restoredAgainIndex); assertHitCount(client().prepareSearch(restoredAgainIndex).setSize(0).setTrackTotalHits(true).get(), numberOfDocs); @@ -360,18 +353,6 @@ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IO return Map.copyOf(blobsPerShard); } - private void ensureExecutorsAreIdle() throws Exception { - assertBusy(() -> { - for (ThreadPool threadPool : internalCluster().getDataNodeInstances(ThreadPool.class)) { - for (String threadPoolName : List.of(CACHE_FETCH_ASYNC_THREAD_POOL_NAME, CACHE_PREWARMING_THREAD_POOL_NAME)) { - final ThreadPoolExecutor executor = (ThreadPoolExecutor) threadPool.executor(threadPoolName); - assertThat(threadPoolName, executor.getQueue().size(), equalTo(0)); - assertThat(threadPoolName, executor.getActiveCount(), equalTo(0)); - } - } - }); - } - private void assertCachedBlobsInSystemIndex(final String repositoryName, final Map blobsInSnapshot) throws Exception { assertBusy(() -> { From 9fbf8ce44185797e090fcd17e808891878ba4ca7 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 25 Aug 2020 18:43:35 +0100 Subject: [PATCH 52/63] Delete restored index first to make sure it doesn't re-create the cache index --- .../searchablesnapshots/SearchableSnapshotsIntegTests.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java index 0e9d185ac6a02..95150506013ea 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java @@ -506,6 +506,8 @@ public void testMaxRestoreBytesPerSecIsUsed() throws Exception { ); } } + + assertAcked(client().admin().indices().prepareDelete(restoredIndexName)); } private Map getMaxShardSizeByNodeInBytes(String indexName) { From ec9cd242c602ce850a5bab8efac252283509ae44 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 25 Aug 2020 18:43:41 +0100 Subject: [PATCH 53/63] Remove test logging --- .../cache/SearchableSnapshotsBlobStoreCacheIntegTests.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java index 775f089d2dba0..55a34197248ce 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java @@ -33,7 +33,6 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.InternalTestCluster; -import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.xpack.core.ClientHelper; import org.elasticsearch.xpack.core.searchablesnapshots.SearchableSnapshotShardStats; import org.elasticsearch.xpack.searchablesnapshots.BaseSearchableSnapshotsIntegTestCase; @@ -64,8 +63,6 @@ import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.is; -@TestLogging(reason = "debugging", value = "org.elasticsearch.index.store.cache.CachedBlobContainerIndexInput:TRACE," - + "org.elasticsearch.blobstore.cache.BlobStoreCacheService:TRACE") // TODO remove this before merge public class SearchableSnapshotsBlobStoreCacheIntegTests extends BaseSearchableSnapshotsIntegTestCase { @Override From 1535e58f69f6284f8d44ffc98795e10b1453e650 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 25 Aug 2020 18:54:16 +0100 Subject: [PATCH 54/63] And another one --- .../SearchableSnapshotRecoveryStateIntegrationTests.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotRecoveryStateIntegrationTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotRecoveryStateIntegrationTests.java index a95103488a1df..cd4e5e3ce68c0 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotRecoveryStateIntegrationTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotRecoveryStateIntegrationTests.java @@ -122,6 +122,8 @@ public void testRecoveryStateRecoveredBytesMatchPhysicalCacheState() throws Exce assertThat("Physical cache size doesn't match with recovery state data", physicalCacheSize, equalTo(recoveredBytes)); assertThat("Expected to recover 100% of files", recoveryState.getIndex().recoveredBytesPercent(), equalTo(100.0f)); + + assertAcked(client().admin().indices().prepareDelete(restoredIndexName)); } @SuppressForbidden(reason = "Uses FileSystem APIs") From eb1a08ec99bde821f3a131ef6e684cd1140b15e7 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 26 Aug 2020 12:09:52 +0100 Subject: [PATCH 55/63] Time out cache get after 5s --- .../blobstore/cache/BlobStoreCacheService.java | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index ea6ff6d22bac8..a8cc669137a2c 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -9,6 +9,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.ElasticsearchTimeoutException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ResourceAlreadyExistsException; import org.elasticsearch.Version; @@ -39,6 +40,7 @@ import java.io.IOException; import java.time.Instant; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; @@ -227,7 +229,18 @@ private static XContentBuilder mappings() throws IOException { public CachedBlob get(String repository, String name, String path, long offset) { final PlainActionFuture future = PlainActionFuture.newFuture(); getAsync(repository, name, path, offset, future); - return future.actionGet(); + try { + return future.actionGet(5, TimeUnit.SECONDS); + } catch (ElasticsearchTimeoutException e) { + if (logger.isDebugEnabled()) { + logger.warn(() -> new ParameterizedMessage( + "get from cache index timed out after [5s], retrieving from blob store instead [id={}]", + CachedBlob.generateId(repository, name, path, offset)), e); + } else { + logger.warn("get from cache index timed out after [5s], retrieving from blob store instead"); + } + return CachedBlob.CACHE_NOT_READY; + } } protected void getAsync(String repository, String name, String path, long offset, ActionListener listener) { From deda3054dbb45980c1d8e332ba54a16e352afe4c Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 26 Aug 2020 12:20:45 +0100 Subject: [PATCH 56/63] Assert not blocking a SYSTEM_READ thread --- .../elasticsearch/blobstore/cache/BlobStoreCacheService.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index a8cc669137a2c..c77f192dac292 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -227,6 +227,9 @@ private static XContentBuilder mappings() throws IOException { } public CachedBlob get(String repository, String name, String path, long offset) { + assert Thread.currentThread().getName().contains(ThreadPool.Names.SYSTEM_READ) == false + : "must not block [" + Thread.currentThread().getName() + "] for a cache read"; + final PlainActionFuture future = PlainActionFuture.newFuture(); getAsync(repository, name, path, offset, future); try { From dad5dfd471dbeb82bcb54fcf45ce719cd64ff7f6 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 26 Aug 2020 12:22:14 +0100 Subject: [PATCH 57/63] Spotless --- .../blobstore/cache/BlobStoreCacheService.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index c77f192dac292..817b716d50e54 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -227,8 +227,9 @@ private static XContentBuilder mappings() throws IOException { } public CachedBlob get(String repository, String name, String path, long offset) { - assert Thread.currentThread().getName().contains(ThreadPool.Names.SYSTEM_READ) == false - : "must not block [" + Thread.currentThread().getName() + "] for a cache read"; + assert Thread.currentThread().getName().contains(ThreadPool.Names.SYSTEM_READ) == false : "must not block [" + + Thread.currentThread().getName() + + "] for a cache read"; final PlainActionFuture future = PlainActionFuture.newFuture(); getAsync(repository, name, path, offset, future); @@ -236,9 +237,13 @@ public CachedBlob get(String repository, String name, String path, long offset) return future.actionGet(5, TimeUnit.SECONDS); } catch (ElasticsearchTimeoutException e) { if (logger.isDebugEnabled()) { - logger.warn(() -> new ParameterizedMessage( + logger.warn( + () -> new ParameterizedMessage( "get from cache index timed out after [5s], retrieving from blob store instead [id={}]", - CachedBlob.generateId(repository, name, path, offset)), e); + CachedBlob.generateId(repository, name, path, offset) + ), + e + ); } else { logger.warn("get from cache index timed out after [5s], retrieving from blob store instead"); } From 99045e06e7052a892a8faa88ad6c63c19fed0ae4 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 26 Aug 2020 12:34:03 +0100 Subject: [PATCH 58/63] Delimiters --- .../elasticsearch/blobstore/cache/BlobStoreCacheService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index 817b716d50e54..1f3d4be72e028 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -227,7 +227,7 @@ private static XContentBuilder mappings() throws IOException { } public CachedBlob get(String repository, String name, String path, long offset) { - assert Thread.currentThread().getName().contains(ThreadPool.Names.SYSTEM_READ) == false : "must not block [" + assert Thread.currentThread().getName().contains('[' + ThreadPool.Names.SYSTEM_READ + ']') == false : "must not block [" + Thread.currentThread().getName() + "] for a cache read"; From 2003ec07b8a22409ba94675ce942530eb7faad95 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 26 Aug 2020 12:48:27 +0100 Subject: [PATCH 59/63] Add stats to tests --- .../src/test/resources/rest-api-spec/test/stats.yml | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/x-pack/plugin/searchable-snapshots/qa/rest/src/test/resources/rest-api-spec/test/stats.yml b/x-pack/plugin/searchable-snapshots/qa/rest/src/test/resources/rest-api-spec/test/stats.yml index 6f3cd6271eefb..a0256f8b483aa 100644 --- a/x-pack/plugin/searchable-snapshots/qa/rest/src/test/resources/rest-api-spec/test/stats.yml +++ b/x-pack/plugin/searchable-snapshots/qa/rest/src/test/resources/rest-api-spec/test/stats.yml @@ -164,6 +164,11 @@ teardown: - gte: { indices.docs.shards.0.0.files.0.cached_bytes_read.min: 0 } - gte: { indices.docs.shards.0.0.files.0.cached_bytes_read.max: 0 } + - gte: { indices.docs.shards.0.0.files.0.index_cache_bytes_read.count: 0 } + - gte: { indices.docs.shards.0.0.files.0.index_cache_bytes_read.sum: 0 } + - gte: { indices.docs.shards.0.0.files.0.index_cache_bytes_read.min: 0 } + - gte: { indices.docs.shards.0.0.files.0.index_cache_bytes_read.max: 0 } + - gte: { indices.docs.shards.0.0.files.0.cached_bytes_written.count: 0 } - gte: { indices.docs.shards.0.0.files.0.cached_bytes_written.sum: 0 } - gte: { indices.docs.shards.0.0.files.0.cached_bytes_written.min: 0 } @@ -203,6 +208,13 @@ teardown: - gte: { indices.docs.shards.0.0.files.0.backward_seeks.large.min: 0 } - gte: { indices.docs.shards.0.0.files.0.backward_seeks.large.max: 0 } + - gte: { indices.docs.shards.0.0.files.0.blob_store_bytes_requested.count: 0 } + - gte: { indices.docs.shards.0.0.files.0.blob_store_bytes_requested.sum: 0 } + - gte: { indices.docs.shards.0.0.files.0.blob_store_bytes_requested.min: 0 } + - gte: { indices.docs.shards.0.0.files.0.blob_store_bytes_requested.max: 0 } + + - gte: { indices.docs.shards.0.0.files.0.current_index_cache_fills: 0 } + - do: searchable_snapshots.stats: index: "d*" From 62fe090c99be6cc15b3f0def26763c10bf3beabe Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 26 Aug 2020 12:51:55 +0100 Subject: [PATCH 60/63] Catch exception during indexing --- .../cache/BlobStoreCacheService.java | 31 +++++++++---------- 1 file changed, 15 insertions(+), 16 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java index 1f3d4be72e028..f000f06e5c1b4 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/blobstore/cache/BlobStoreCacheService.java @@ -316,28 +316,27 @@ public void onResponse(String s) { try (XContentBuilder builder = jsonBuilder()) { request.source(cachedBlob.toXContent(builder, ToXContent.EMPTY_PARAMS)); } - } catch (IOException e) { + + client.index(request, new ActionListener<>() { + @Override + public void onResponse(IndexResponse indexResponse) { + logger.trace("cache fill ({}): [{}]", indexResponse.status(), request.id()); + listener.onResponse(null); + } + + @Override + public void onFailure(Exception e) { + logger.debug(new ParameterizedMessage("failure in cache fill: [{}]", request.id()), e); + listener.onFailure(e); + } + }); + } catch (Exception e) { logger.warn( new ParameterizedMessage("cache fill failure: [{}]", CachedBlob.generateId(repository, name, path, offset)), e ); listener.onFailure(e); - return; } - - client.index(request, new ActionListener<>() { - @Override - public void onResponse(IndexResponse indexResponse) { - logger.trace("cache fill ({}): [{}]", indexResponse.status(), request.id()); - listener.onResponse(null); - } - - @Override - public void onFailure(Exception e) { - logger.debug(new ParameterizedMessage("failure in cache fill: [{}]", request.id()), e); - listener.onFailure(e); - } - }); } @Override From 2c2a7c0c6d7174a0818a1baa15c30d58daacbdca Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 26 Aug 2020 12:54:58 +0100 Subject: [PATCH 61/63] Remove duplicate assertion --- .../cache/SearchableSnapshotsBlobStoreCacheIntegTests.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java index 55a34197248ce..2b22fef67e3ef 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/blobstore/cache/SearchableSnapshotsBlobStoreCacheIntegTests.java @@ -273,7 +273,6 @@ public Settings onNodeStopped(String nodeName) throws Exception { logger.info("--> verifying documents in index [{}]", restoredAgainIndex); assertHitCount(client().prepareSearch(restoredAgainIndex).setSize(0).setTrackTotalHits(true).get(), numberOfDocs); - assertHitCount(client().prepareSearch(restoredAgainIndex).setSize(0).setTrackTotalHits(true).get(), numberOfDocs); assertHitCount( client().prepareSearch(restoredAgainIndex) .setQuery(QueryBuilders.rangeQuery("num").lte(numberOfDocs)) From bb6d4abd97cdfa1db803683d1cf30324e5f7fd6a Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 26 Aug 2020 13:02:12 +0100 Subject: [PATCH 62/63] Another test needing cleanup --- .../searchablesnapshots/SearchableSnapshotsIntegTests.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java index 95150506013ea..e0da6edc25c23 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java @@ -424,6 +424,8 @@ public void testCanMountSnapshotTakenWhileConcurrentlyIndexing() throws Exceptio final RestoreSnapshotResponse restoreSnapshotResponse = client().execute(MountSearchableSnapshotAction.INSTANCE, req).get(); assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), equalTo(0)); ensureGreen(restoredIndexName); + + assertAcked(client().admin().indices().prepareDelete(restoredIndexName)); } public void testMaxRestoreBytesPerSecIsUsed() throws Exception { From 9e228c15a5075a562243bca9b5518b5ae32170cc Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 26 Aug 2020 15:49:57 +0100 Subject: [PATCH 63/63] Account for 'any kind' reads from index cache --- .../SearchableSnapshotsIntegTests.java | 31 +++++++++---------- 1 file changed, 15 insertions(+), 16 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java index e0da6edc25c23..474729be2e610 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java @@ -787,22 +787,20 @@ private void assertSearchableSnapshotStats(String indexName, boolean cacheEnable if (cacheEnabled == false || nonCachedExtensions.contains(IndexFileNames.getExtension(fileName))) { assertThat( "Expected at least 1 optimized or direct read for " + fileName + " of shard " + shardRouting, - Math.max(indexInputStats.getOptimizedBytesRead().getCount(), indexInputStats.getDirectBytesRead().getCount()), + max(indexInputStats.getOptimizedBytesRead().getCount(), indexInputStats.getDirectBytesRead().getCount()), greaterThan(0L) ); assertThat( "Expected no cache read or write for " + fileName + " of shard " + shardRouting, - Math.max(indexInputStats.getCachedBytesRead().getCount(), indexInputStats.getCachedBytesWritten().getCount()), + max(indexInputStats.getCachedBytesRead().getCount(), indexInputStats.getCachedBytesWritten().getCount()), equalTo(0L) ); } else if (nodeIdsWithLargeEnoughCache.contains(stats.getShardRouting().currentNodeId())) { assertThat( "Expected at least 1 cache read or write for " + fileName + " of shard " + shardRouting, - Math.max( - Math.max( - indexInputStats.getCachedBytesRead().getCount(), - indexInputStats.getCachedBytesWritten().getCount() - ), + max( + indexInputStats.getCachedBytesRead().getCount(), + indexInputStats.getCachedBytesWritten().getCount(), indexInputStats.getIndexCacheBytesRead().getCount() ), greaterThan(0L) @@ -820,15 +818,12 @@ private void assertSearchableSnapshotStats(String indexName, boolean cacheEnable } else { assertThat( "Expected at least 1 read or write of any kind for " + fileName + " of shard " + shardRouting, - Math.max( - Math.max( - indexInputStats.getCachedBytesRead().getCount(), - indexInputStats.getCachedBytesWritten().getCount() - ), - Math.max( - indexInputStats.getOptimizedBytesRead().getCount(), - indexInputStats.getDirectBytesRead().getCount() - ) + max( + indexInputStats.getCachedBytesRead().getCount(), + indexInputStats.getCachedBytesWritten().getCount(), + indexInputStats.getOptimizedBytesRead().getCount(), + indexInputStats.getDirectBytesRead().getCount(), + indexInputStats.getIndexCacheBytesRead().getCount() ), greaterThan(0L) ); @@ -838,6 +833,10 @@ private void assertSearchableSnapshotStats(String indexName, boolean cacheEnable } } + private static long max(long... values) { + return Arrays.stream(values).max().orElseThrow(() -> new AssertionError("no values")); + } + private ByteSizeValue getCacheSizeForNode(String nodeName) { return CacheService.SNAPSHOT_CACHE_SIZE_SETTING.get(internalCluster().getInstance(Environment.class, nodeName).settings()); }