Skip to content

Commit 1e99bc6

Browse files
authored
Make peer recovery work with archive data (#81522)
Adapts peer recovery so that it properly integrates with the hook to convert old indices. Relates #81210
1 parent 0410fbf commit 1e99bc6

File tree

7 files changed

+65
-52
lines changed

7 files changed

+65
-52
lines changed

server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java

Lines changed: 8 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -53,12 +53,13 @@
5353

5454
import static org.elasticsearch.common.lucene.Lucene.indexWriterConfigWithNoMerging;
5555
import static org.elasticsearch.core.TimeValue.timeValueMillis;
56+
import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.isSearchableSnapshotStore;
5657

5758
/**
5859
* This package private utility class encapsulates the logic to recover an index shard from either an existing index on
5960
* disk or from a snapshot in a repository.
6061
*/
61-
final class StoreRecovery {
62+
public final class StoreRecovery {
6263

6364
private final Logger logger;
6465
private final ShardId shardId;
@@ -549,14 +550,17 @@ private void restore(
549550
}
550551
}
551552

552-
private void bootstrap(final IndexShard indexShard, final Store store) throws IOException {
553-
store.bootstrapNewHistory();
553+
public static void bootstrap(final IndexShard indexShard, final Store store) throws IOException {
554+
if (isSearchableSnapshotStore(indexShard.indexSettings().getSettings()) == false) {
555+
// not bootstrapping new history for searchable snapshots (which are read-only) allows sequence-number based peer recoveries
556+
store.bootstrapNewHistory();
557+
}
554558
final SegmentInfos segmentInfos = store.readLastCommittedSegmentsInfo();
555559
final long localCheckpoint = Long.parseLong(segmentInfos.userData.get(SequenceNumbers.LOCAL_CHECKPOINT_KEY));
556560
final String translogUUID = Translog.createEmptyTranslog(
557561
indexShard.shardPath().resolveTranslog(),
558562
localCheckpoint,
559-
shardId,
563+
indexShard.shardId(),
560564
indexShard.getPendingPrimaryTerm()
561565
);
562566
store.associateIndexWithNewTranslog(translogUUID);

server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,7 @@
4444
import org.elasticsearch.index.shard.ShardId;
4545
import org.elasticsearch.index.shard.ShardLongFieldRange;
4646
import org.elasticsearch.index.shard.ShardNotFoundException;
47+
import org.elasticsearch.index.shard.StoreRecovery;
4748
import org.elasticsearch.index.store.Store;
4849
import org.elasticsearch.index.translog.Translog;
4950
import org.elasticsearch.index.translog.TranslogCorruptedException;
@@ -65,6 +66,7 @@
6566

6667
import static org.elasticsearch.core.TimeValue.timeValueMillis;
6768
import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO;
69+
import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.isSearchableSnapshotStore;
6870

6971
/**
7072
* The recovery target handles recoveries of peer shards of the shard+node to recover to.
@@ -226,6 +228,17 @@ private void doRecovery(final long recoveryId, final StartRecoveryRequest preExi
226228
assert recoveryTarget.sourceNode() != null : "can not do a recovery without a source node";
227229
logger.trace("{} preparing shard for peer recovery", recoveryTarget.shardId());
228230
indexShard.prepareForIndexRecovery();
231+
if (isSearchableSnapshotStore(indexShard.indexSettings().getSettings())) {
232+
// for searchable snapshots, peer recovery is treated similarly to recovery from snapshot
233+
indexShard.getIndexEventListener().afterFilesRestoredFromRepository(indexShard);
234+
final Store store = indexShard.store();
235+
store.incRef();
236+
try {
237+
StoreRecovery.bootstrap(indexShard, store);
238+
} finally {
239+
store.decRef();
240+
}
241+
}
229242
final long startingSeqNo = indexShard.recoverLocallyUpToGlobalCheckpoint();
230243
assert startingSeqNo == UNASSIGNED_SEQ_NO || recoveryTarget.state().getStage() == RecoveryState.Stage.TRANSLOG
231244
: "unexpected recovery stage [" + recoveryTarget.state().getStage() + "] starting seqno [ " + startingSeqNo + "]";

server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java

Lines changed: 14 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -2674,30 +2674,23 @@ public void snapshotShard(SnapshotShardContext context) {
26742674
long indexIncrementalSize = 0;
26752675
long indexTotalFileSize = 0;
26762676
final BlockingQueue<BlobStoreIndexShardSnapshot.FileInfo> filesToSnapshot = new LinkedBlockingQueue<>();
2677-
// If we did not find a set of files that is equal to the current commit we determine the files to upload by comparing files
2678-
// in the commit with files already in the repository
2679-
if (filesFromSegmentInfos == null) {
2677+
2678+
if (isSearchableSnapshotStore(store.indexSettings().getSettings())) {
2679+
indexCommitPointFiles = Collections.emptyList();
2680+
} else if (filesFromSegmentInfos == null) {
2681+
// If we did not find a set of files that is equal to the current commit we determine the files to upload by comparing files
2682+
// in the commit with files already in the repository
26802683
indexCommitPointFiles = new ArrayList<>();
26812684
final Collection<String> fileNames;
26822685
final Store.MetadataSnapshot metadataFromStore;
2683-
if (isSearchableSnapshotStore(store.indexSettings().getSettings())) {
2684-
fileNames = Collections.emptyList();
2685-
metadataFromStore = Store.MetadataSnapshot.EMPTY;
2686-
} else {
2687-
try (Releasable ignored = incrementStoreRef(store, snapshotStatus, shardId)) {
2688-
// TODO apparently we don't use the MetadataSnapshot#.recoveryDiff(...) here but we should
2689-
try {
2690-
logger.trace(
2691-
"[{}] [{}] Loading store metadata using index commit [{}]",
2692-
shardId,
2693-
snapshotId,
2694-
snapshotIndexCommit
2695-
);
2696-
metadataFromStore = store.getMetadata(snapshotIndexCommit);
2697-
fileNames = snapshotIndexCommit.getFileNames();
2698-
} catch (IOException e) {
2699-
throw new IndexShardSnapshotFailedException(shardId, "Failed to get store file metadata", e);
2700-
}
2686+
try (Releasable ignored = incrementStoreRef(store, snapshotStatus, shardId)) {
2687+
// TODO apparently we don't use the MetadataSnapshot#.recoveryDiff(...) here but we should
2688+
try {
2689+
logger.trace("[{}] [{}] Loading store metadata using index commit [{}]", shardId, snapshotId, snapshotIndexCommit);
2690+
metadataFromStore = store.getMetadata(snapshotIndexCommit);
2691+
fileNames = snapshotIndexCommit.getFileNames();
2692+
} catch (IOException e) {
2693+
throw new IndexShardSnapshotFailedException(shardId, "Failed to get store file metadata", e);
27012694
}
27022695
}
27032696
for (String fileName : fileNames) {

x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/codecs/BWCCodec.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@
3131
import org.apache.lucene.index.Terms;
3232
import org.apache.lucene.store.Directory;
3333
import org.apache.lucene.store.IOContext;
34+
import org.elasticsearch.index.mapper.SeqNoFieldMapper;
3435
import org.elasticsearch.xpack.lucene.bwc.codecs.lucene70.BWCLucene70Codec;
3536

3637
import java.io.IOException;
@@ -169,6 +170,10 @@ public void write(Directory directory, SegmentInfo segmentInfo, String segmentSu
169170
private static FieldInfos filterFields(FieldInfos fieldInfos) {
170171
List<FieldInfo> fieldInfoCopy = new ArrayList<>(fieldInfos.size());
171172
for (FieldInfo fieldInfo : fieldInfos) {
173+
// omit sequence number field so that it doesn't interfere with peer recovery
174+
if (fieldInfo.name.equals(SeqNoFieldMapper.NAME)) {
175+
continue;
176+
}
172177
fieldInfoCopy.add(
173178
new FieldInfo(
174179
fieldInfo.name,

x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotIndexEventListener.java

Lines changed: 0 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -9,7 +9,6 @@
99
import org.apache.logging.log4j.LogManager;
1010
import org.apache.logging.log4j.Logger;
1111
import org.apache.logging.log4j.message.ParameterizedMessage;
12-
import org.apache.lucene.index.SegmentInfos;
1312
import org.elasticsearch.action.StepListener;
1413
import org.elasticsearch.cluster.node.DiscoveryNode;
1514
import org.elasticsearch.cluster.routing.RecoverySource;
@@ -18,20 +17,15 @@
1817
import org.elasticsearch.core.Nullable;
1918
import org.elasticsearch.index.IndexService;
2019
import org.elasticsearch.index.IndexSettings;
21-
import org.elasticsearch.index.seqno.SequenceNumbers;
2220
import org.elasticsearch.index.shard.IndexEventListener;
2321
import org.elasticsearch.index.shard.IndexShard;
2422
import org.elasticsearch.index.shard.ShardId;
25-
import org.elasticsearch.index.translog.Translog;
26-
import org.elasticsearch.index.translog.TranslogException;
2723
import org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason;
2824
import org.elasticsearch.threadpool.ThreadPool;
2925
import org.elasticsearch.xpack.searchablesnapshots.cache.full.CacheService;
3026
import org.elasticsearch.xpack.searchablesnapshots.cache.shared.FrozenCacheService;
3127
import org.elasticsearch.xpack.searchablesnapshots.store.SearchableSnapshotDirectory;
3228

33-
import java.nio.file.Path;
34-
3529
import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.isSearchableSnapshotStore;
3630
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_INDEX_NAME_SETTING;
3731
import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_SNAPSHOT_ID_SETTING;
@@ -65,7 +59,6 @@ public SearchableSnapshotIndexEventListener(
6559
public void beforeIndexShardRecovery(IndexShard indexShard, IndexSettings indexSettings) {
6660
assert Thread.currentThread().getName().contains(ThreadPool.Names.GENERIC);
6761
ensureSnapshotIsLoaded(indexShard);
68-
associateNewEmptyTranslogWithIndex(indexShard);
6962
}
7063

7164
private static void ensureSnapshotIsLoaded(IndexShard indexShard) {
@@ -93,26 +86,6 @@ private static void ensureSnapshotIsLoaded(IndexShard indexShard) {
9386
: "loading snapshot must not be called twice unless we are retrying a peer recovery";
9487
}
9588

96-
private static void associateNewEmptyTranslogWithIndex(IndexShard indexShard) {
97-
final ShardId shardId = indexShard.shardId();
98-
assert isSearchableSnapshotStore(indexShard.indexSettings().getSettings()) : "Expected a searchable snapshot shard " + shardId;
99-
if (indexShard.routingEntry().primary()
100-
&& indexShard.routingEntry().recoverySource().getType().equals(RecoverySource.Type.SNAPSHOT)) {
101-
// translog initialization is done later in the restore step
102-
return;
103-
}
104-
try {
105-
final SegmentInfos segmentInfos = indexShard.store().readLastCommittedSegmentsInfo();
106-
final long localCheckpoint = Long.parseLong(segmentInfos.userData.get(SequenceNumbers.LOCAL_CHECKPOINT_KEY));
107-
final long primaryTerm = indexShard.getPendingPrimaryTerm();
108-
final String translogUUID = segmentInfos.userData.get(Translog.TRANSLOG_UUID_KEY);
109-
final Path translogLocation = indexShard.shardPath().resolveTranslog();
110-
Translog.createEmptyTranslog(translogLocation, shardId, localCheckpoint, primaryTerm, translogUUID, null);
111-
} catch (Exception e) {
112-
throw new TranslogException(shardId, "failed to associate a new translog", e);
113-
}
114-
}
115-
11689
@Override
11790
public void beforeIndexRemoved(IndexService indexService, IndexRemovalReason reason) {
11891
if (shouldEvictCacheFiles(reason)) {

x-pack/qa/repository-old-versions/build.gradle

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -68,6 +68,7 @@ if (Os.isFamily(Os.FAMILY_WINDOWS)) {
6868

6969
def testClusterProvider = testClusters.register(clusterName) {
7070
testDistribution = 'DEFAULT'
71+
numberOfNodes = 2
7172

7273
setting 'path.repo', repoLocation
7374
setting 'xpack.license.self_generated.type', 'trial'

x-pack/qa/repository-old-versions/src/test/java/org/elasticsearch/oldrepos/OldRepositoryAccessIT.java

Lines changed: 24 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@
1010
import org.apache.http.HttpHost;
1111
import org.elasticsearch.Build;
1212
import org.elasticsearch.Version;
13+
import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest;
1314
import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryRequest;
1415
import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsRequest;
1516
import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotRequest;
@@ -26,6 +27,8 @@
2627
import org.elasticsearch.client.indices.CloseIndexRequest;
2728
import org.elasticsearch.client.searchable_snapshots.MountSnapshotRequest;
2829
import org.elasticsearch.cluster.SnapshotsInProgress;
30+
import org.elasticsearch.cluster.health.ClusterHealthStatus;
31+
import org.elasticsearch.cluster.metadata.IndexMetadata;
2932
import org.elasticsearch.common.settings.SecureString;
3033
import org.elasticsearch.common.settings.Settings;
3134
import org.elasticsearch.common.util.concurrent.ThreadContext;
@@ -211,6 +214,16 @@ private void restoreMountAndVerify(int numDocs, Set<String> expectedIds, RestHig
211214
assertEquals(numberOfShards, restoreSnapshotResponse.getRestoreInfo().totalShards());
212215
assertEquals(numberOfShards, restoreSnapshotResponse.getRestoreInfo().successfulShards());
213216

217+
assertEquals(
218+
ClusterHealthStatus.GREEN,
219+
client.cluster()
220+
.health(
221+
new ClusterHealthRequest("restored_test").waitForGreenStatus().waitForNoRelocatingShards(true),
222+
RequestOptions.DEFAULT
223+
)
224+
.getStatus()
225+
);
226+
214227
// run a search against the index
215228
assertDocs("restored_test", numDocs, expectedIds, client);
216229

@@ -219,13 +232,24 @@ private void restoreMountAndVerify(int numDocs, Set<String> expectedIds, RestHig
219232
.mountSnapshot(
220233
new MountSnapshotRequest("testrepo", "snap1", "test").storage(MountSnapshotRequest.Storage.FULL_COPY)
221234
.renamedIndex("mounted_full_copy_test")
235+
.indexSettings(Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).build())
222236
.waitForCompletion(true),
223237
RequestOptions.DEFAULT
224238
);
225239
assertNotNull(mountSnapshotResponse.getRestoreInfo());
226240
assertEquals(numberOfShards, mountSnapshotResponse.getRestoreInfo().totalShards());
227241
assertEquals(numberOfShards, mountSnapshotResponse.getRestoreInfo().successfulShards());
228242

243+
assertEquals(
244+
ClusterHealthStatus.GREEN,
245+
client.cluster()
246+
.health(
247+
new ClusterHealthRequest("mounted_full_copy_test").waitForGreenStatus().waitForNoRelocatingShards(true),
248+
RequestOptions.DEFAULT
249+
)
250+
.getStatus()
251+
);
252+
229253
// run a search against the index
230254
assertDocs("mounted_full_copy_test", numDocs, expectedIds, client);
231255

0 commit comments

Comments
 (0)