Remove Redundant Loading of RepositoryData during Restore (#51977) (#52108)

We can just put the `IndexId` instead of just the index name into the recovery soruce and
save one load of `RepositoryData` on each shard restore that way.
This commit is contained in:
Armin Braun 2020-02-09 21:44:18 +01:00 committed by GitHub
parent 3e7f939f63
commit 90eb6a020d
17 changed files with 88 additions and 40 deletions

View File

@ -21,12 +21,14 @@ package org.elasticsearch.cluster.routing;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.RestoreInProgress;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.repositories.IndexId;
import org.elasticsearch.snapshots.Snapshot;
import java.io.IOException;
@ -220,14 +222,14 @@ public abstract class RecoverySource implements Writeable, ToXContentObject {
public static class SnapshotRecoverySource extends RecoverySource {
private final String restoreUUID;
private final Snapshot snapshot;
private final String index;
private final IndexId index;
private final Version version;
public SnapshotRecoverySource(String restoreUUID, Snapshot snapshot, Version version, String index) {
public SnapshotRecoverySource(String restoreUUID, Snapshot snapshot, Version version, IndexId indexId) {
this.restoreUUID = restoreUUID;
this.snapshot = Objects.requireNonNull(snapshot);
this.version = Objects.requireNonNull(version);
this.index = Objects.requireNonNull(index);
this.index = Objects.requireNonNull(indexId);
}
SnapshotRecoverySource(StreamInput in) throws IOException {
@ -238,7 +240,11 @@ public abstract class RecoverySource implements Writeable, ToXContentObject {
}
snapshot = new Snapshot(in);
version = Version.readVersion(in);
index = in.readString();
if (in.getVersion().onOrAfter(Version.V_7_7_0)) {
index = new IndexId(in);
} else {
index = new IndexId(in.readString(), IndexMetaData.INDEX_UUID_NA_VALUE);
}
}
public String restoreUUID() {
@ -249,7 +255,13 @@ public abstract class RecoverySource implements Writeable, ToXContentObject {
return snapshot;
}
public String index() {
/**
* Gets the {@link IndexId} of the recovery source. May contain {@link IndexMetaData#INDEX_UUID_NA_VALUE} as the index uuid if it
* was created by an older version master in a mixed version cluster.
*
* @return IndexId
*/
public IndexId index() {
return index;
}
@ -264,7 +276,11 @@ public abstract class RecoverySource implements Writeable, ToXContentObject {
}
snapshot.writeTo(out);
Version.writeVersion(version, out);
out.writeString(index);
if (out.getVersion().onOrAfter(Version.V_7_7_0)) {
index.writeTo(out);
} else {
out.writeString(index.getName());
}
}
@Override
@ -277,7 +293,7 @@ public abstract class RecoverySource implements Writeable, ToXContentObject {
builder.field("repository", snapshot.getRepository())
.field("snapshot", snapshot.getSnapshotId().getName())
.field("version", version.toString())
.field("index", index)
.field("index", index.getName())
.field("restoreUUID", restoreUUID);
}

View File

@ -33,6 +33,7 @@ import org.apache.lucene.store.IndexInput;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.StepListener;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.routing.RecoverySource;
@ -484,20 +485,24 @@ final class StoreRecovery {
translogState.totalOperationsOnStart(0);
indexShard.prepareForIndexRecovery();
final ShardId snapshotShardId;
final String indexName = restoreSource.index();
if (!shardId.getIndexName().equals(indexName)) {
snapshotShardId = new ShardId(indexName, IndexMetaData.INDEX_UUID_NA_VALUE, shardId.id());
} else {
final IndexId indexId = restoreSource.index();
if (shardId.getIndexName().equals(indexId.getName())) {
snapshotShardId = shardId;
} else {
snapshotShardId = new ShardId(indexId.getName(), IndexMetaData.INDEX_UUID_NA_VALUE, shardId.id());
}
repository.getRepositoryData(ActionListener.wrap(
repositoryData -> {
final IndexId indexId = repositoryData.resolveIndexId(indexName);
assert indexShard.getEngineOrNull() == null;
repository.restoreShard(indexShard.store(), restoreSource.snapshot().getSnapshotId(), indexId, snapshotShardId,
indexShard.recoveryState(), restoreListener);
}, restoreListener::onFailure
));
final StepListener<IndexId> indexIdListener = new StepListener<>();
// If the index UUID was not found in the recovery source we will have to load RepositoryData and resolve it by index name
if (indexId.getId().equals(IndexMetaData.INDEX_UUID_NA_VALUE)) {
// BwC path, running against an old version master that did not add the IndexId to the recovery source
repository.getRepositoryData(ActionListener.map(
indexIdListener, repositoryData -> repositoryData.resolveIndexId(indexId.getName())));
} else {
indexIdListener.onResponse(indexId);
}
assert indexShard.getEngineOrNull() == null;
indexIdListener.whenComplete(idx -> repository.restoreShard(indexShard.store(), restoreSource.snapshot().getSnapshotId(),
idx, snapshotShardId, indexShard.recoveryState(), restoreListener), restoreListener::onFailure);
} catch (Exception e) {
restoreListener.onFailure(e);
}

View File

@ -258,8 +258,8 @@ public class RestoreService implements ClusterStateApplier {
for (Map.Entry<String, String> indexEntry : indices.entrySet()) {
String index = indexEntry.getValue();
boolean partial = checkPartial(index);
SnapshotRecoverySource recoverySource =
new SnapshotRecoverySource(restoreUUID, snapshot, snapshotInfo.version(), index);
SnapshotRecoverySource recoverySource = new SnapshotRecoverySource(restoreUUID, snapshot,
snapshotInfo.version(), repositoryData.resolveIndexId(index));
String renamedIndexName = indexEntry.getKey();
IndexMetaData snapshotIndexMetaData = metaData.index(index);
snapshotIndexMetaData = updateIndexSettings(snapshotIndexMetaData,

View File

@ -23,6 +23,7 @@ import org.elasticsearch.Version;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.repositories.IndexId;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.snapshots.Snapshot;
@ -165,7 +166,8 @@ public class ShardRoutingTests extends ESTestCase {
otherRouting = new ShardRouting(otherRouting.shardId(), otherRouting.currentNodeId(),
otherRouting.relocatingNodeId(), otherRouting.primary(), otherRouting.state(),
new RecoverySource.SnapshotRecoverySource(UUIDs.randomBase64UUID(), new Snapshot("test",
new SnapshotId("s1", UUIDs.randomBase64UUID())), Version.CURRENT, "test"),
new SnapshotId("s1", UUIDs.randomBase64UUID())), Version.CURRENT, new IndexId("test",
UUIDs.randomBase64UUID(random()))),
otherRouting.unassignedInfo(), otherRouting.allocationId(), otherRouting.getExpectedShardSize());
}
break;

View File

@ -39,6 +39,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.Index;
import org.elasticsearch.repositories.IndexId;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.test.VersionUtils;
@ -175,7 +176,8 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
.metaData(metaData)
.routingTable(RoutingTable.builder().addAsNewRestore(metaData.index("test"), new SnapshotRecoverySource(
UUIDs.randomBase64UUID(),
new Snapshot("rep1", new SnapshotId("snp1", UUIDs.randomBase64UUID())), Version.CURRENT, "test"),
new Snapshot("rep1", new SnapshotId("snp1", UUIDs.randomBase64UUID())), Version.CURRENT,
new IndexId("test", UUIDs.randomBase64UUID(random()))),
new IntHashSet()).build()).build();
for (ShardRouting shard : clusterState.getRoutingNodes().shardsWithState(UNASSIGNED)) {
assertThat(shard.unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.NEW_INDEX_RESTORED));
@ -192,7 +194,8 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
.routingTable(RoutingTable.builder().addAsRestore(metaData.index("test"),
new SnapshotRecoverySource(
UUIDs.randomBase64UUID(), new Snapshot("rep1",
new SnapshotId("snp1", UUIDs.randomBase64UUID())), Version.CURRENT, "test")).build()).build();
new SnapshotId("snp1", UUIDs.randomBase64UUID())), Version.CURRENT,
new IndexId("test", UUIDs.randomBase64UUID(random())))).build()).build();
for (ShardRouting shard : clusterState.getRoutingNodes().shardsWithState(UNASSIGNED)) {
assertThat(shard.unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.EXISTING_INDEX_RESTORED));
}

View File

@ -53,6 +53,7 @@ import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.repositories.IndexId;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.test.VersionUtils;
@ -366,7 +367,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
new SnapshotRecoverySource(
UUIDs.randomBase64UUID(),
new Snapshot("rep1", new SnapshotId("snp1", UUIDs.randomBase64UUID())),
Version.CURRENT, "test")).build())
Version.CURRENT, new IndexId("test", UUIDs.randomBase64UUID(random())))).build())
.nodes(DiscoveryNodes.builder().add(newNode).add(oldNode1).add(oldNode2)).build();
AllocationDeciders allocationDeciders = new AllocationDeciders(Arrays.asList(
new ReplicaAfterPrimaryActiveAllocationDecider(),
@ -480,14 +481,15 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
assertThat(decision.getExplanation(), is("cannot relocate primary shard from a node with version [" +
newNode.node().getVersion() + "] to a node with older version [" + oldNode.node().getVersion() + "]"));
final IndexId indexId = new IndexId("test", UUIDs.randomBase64UUID(random()));
final SnapshotRecoverySource newVersionSnapshot = new SnapshotRecoverySource(
UUIDs.randomBase64UUID(),
new Snapshot("rep1", new SnapshotId("snp1", UUIDs.randomBase64UUID())),
newNode.node().getVersion(), "test");
newNode.node().getVersion(), indexId);
final SnapshotRecoverySource oldVersionSnapshot = new SnapshotRecoverySource(
UUIDs.randomBase64UUID(),
new Snapshot("rep1", new SnapshotId("snp1", UUIDs.randomBase64UUID())),
oldNode.node().getVersion(), "test");
oldNode.node().getVersion(), indexId);
decision = allocationDecider.canAllocate(ShardRoutingHelper.newWithRestoreSource(primaryShard, newVersionSnapshot),
oldNode, routingAllocation);

View File

@ -47,6 +47,7 @@ import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.repositories.IndexId;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.test.gateway.TestGatewayAllocator;
@ -360,13 +361,15 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase {
snapshotIndices.add(index.getName());
routingTableBuilder.addAsNewRestore(indexMetaData,
new SnapshotRecoverySource(
restoreUUID, snapshot, Version.CURRENT, indexMetaData.getIndex().getName()), new IntHashSet());
restoreUUID, snapshot, Version.CURRENT,
new IndexId(indexMetaData.getIndex().getName(), UUIDs.randomBase64UUID(random()))), new IntHashSet());
break;
case 4:
snapshotIndices.add(index.getName());
routingTableBuilder.addAsRestore(indexMetaData,
new SnapshotRecoverySource(
restoreUUID, snapshot, Version.CURRENT, indexMetaData.getIndex().getName()));
restoreUUID, snapshot, Version.CURRENT,
new IndexId(indexMetaData.getIndex().getName(), UUIDs.randomBase64UUID(random()))));
break;
case 5:
routingTableBuilder.addAsNew(indexMetaData);

View File

@ -41,6 +41,7 @@ import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.repositories.IndexId;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.snapshots.SnapshotId;
@ -205,6 +206,7 @@ public class RestoreInProgressAllocationDeciderTests extends ESAllocationTestCas
private RecoverySource.SnapshotRecoverySource createSnapshotRecoverySource(final String snapshotName) {
Snapshot snapshot = new Snapshot("_repository", new SnapshotId(snapshotName, "_uuid"));
return new RecoverySource.SnapshotRecoverySource(UUIDs.randomBase64UUID(), snapshot, Version.CURRENT, "test");
return new RecoverySource.SnapshotRecoverySource(UUIDs.randomBase64UUID(), snapshot, Version.CURRENT,
new IndexId("test", UUIDs.randomBase64UUID(random())));
}
}

View File

@ -48,6 +48,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.env.ShardLockObtainFailedException;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.repositories.IndexId;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.snapshots.SnapshotId;
import org.junit.Before;
@ -392,7 +393,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
final Snapshot snapshot = new Snapshot("test", new SnapshotId("test", UUIDs.randomBase64UUID()));
RoutingTable routingTable = RoutingTable.builder()
.addAsRestore(metaData.index(shardId.getIndex()),
new SnapshotRecoverySource(UUIDs.randomBase64UUID(), snapshot, Version.CURRENT, shardId.getIndexName()))
new SnapshotRecoverySource(UUIDs.randomBase64UUID(), snapshot, Version.CURRENT,
new IndexId(shardId.getIndexName(), UUIDs.randomBase64UUID(random()))))
.build();
ClusterState state = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY))
.metaData(metaData)

View File

@ -2342,7 +2342,8 @@ public class IndexShardTests extends IndexShardTestCase {
RecoverySource.ExistingStoreRecoverySource.INSTANCE);
final Snapshot snapshot = new Snapshot("foo", new SnapshotId("bar", UUIDs.randomBase64UUID()));
routing = ShardRoutingHelper.newWithRestoreSource(routing,
new RecoverySource.SnapshotRecoverySource(UUIDs.randomBase64UUID(), snapshot, Version.CURRENT, "test"));
new RecoverySource.SnapshotRecoverySource(UUIDs.randomBase64UUID(), snapshot, Version.CURRENT,
new IndexId("test", UUIDs.randomBase64UUID(random()))));
target = reinitShard(target, routing);
Store sourceStore = source.store();
Store targetStore = target.store();

View File

@ -22,6 +22,7 @@ package org.elasticsearch.indices.recovery;
import org.apache.lucene.analysis.TokenStream;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionRunnable;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
@ -84,6 +85,9 @@ import org.elasticsearch.node.RecoverySettingsChunkSizePlugin;
import org.elasticsearch.plugins.AnalysisPlugin;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.plugins.PluginsService;
import org.elasticsearch.repositories.RepositoriesService;
import org.elasticsearch.repositories.Repository;
import org.elasticsearch.repositories.RepositoryData;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.snapshots.SnapshotState;
import org.elasticsearch.test.BackgroundIndexer;
@ -96,6 +100,7 @@ import org.elasticsearch.test.engine.MockEngineSupport;
import org.elasticsearch.test.store.MockFSIndexStore;
import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.test.transport.StubbableTransport;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.ConnectTransportException;
import org.elasticsearch.transport.Transport;
import org.elasticsearch.transport.TransportRequest;
@ -644,6 +649,10 @@ public class IndexRecoveryIT extends ESIntegTestCase {
logger.info("--> request recoveries");
RecoveryResponse response = client().admin().indices().prepareRecoveries(INDEX_NAME).execute().actionGet();
ThreadPool threadPool = internalCluster().getMasterNodeInstance(ThreadPool.class);
Repository repository = internalCluster().getMasterNodeInstance(RepositoriesService.class).repository(REPO_NAME);
final RepositoryData repositoryData = PlainActionFuture.get(f ->
threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(ActionRunnable.wrap(f, repository::getRepositoryData)));
for (Map.Entry<String, List<RecoveryState>> indexRecoveryStates : response.shardRecoveryStates().entrySet()) {
assertThat(indexRecoveryStates.getKey(), equalTo(INDEX_NAME));
@ -654,7 +663,7 @@ public class IndexRecoveryIT extends ESIntegTestCase {
SnapshotRecoverySource recoverySource = new SnapshotRecoverySource(
((SnapshotRecoverySource)recoveryState.getRecoverySource()).restoreUUID(),
new Snapshot(REPO_NAME, createSnapshotResponse.getSnapshotInfo().snapshotId()),
Version.CURRENT, INDEX_NAME);
Version.CURRENT, repositoryData.resolveIndexId(INDEX_NAME));
assertRecoveryState(recoveryState, 0, recoverySource, true, Stage.DONE, null, nodeA);
validateIndexRecoveryState(recoveryState.getIndex());
}

View File

@ -116,7 +116,7 @@ public class FsRepositoryTests extends ESTestCase {
expectThrows(org.apache.lucene.index.IndexNotFoundException.class, () -> Lucene.readSegmentInfos(directory));
DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
ShardRouting routing = ShardRouting.newUnassigned(shardId, true, new RecoverySource.SnapshotRecoverySource("test",
new Snapshot("foo", snapshotId), Version.CURRENT, "myindex"),
new Snapshot("foo", snapshotId), Version.CURRENT, indexId),
new UnassignedInfo(UnassignedInfo.Reason.EXISTING_INDEX_RESTORED, ""));
routing = ShardRoutingHelper.initialize(routing, localNode.getId(), 0);
RecoveryState state = new RecoveryState(routing, localNode, null);

View File

@ -23,10 +23,12 @@ import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.repositories.IndexId;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.test.ESTestCase;
import static org.apache.lucene.util.LuceneTestCase.random;
import static org.elasticsearch.test.ESTestCase.randomAlphaOfLength;
/**
@ -147,6 +149,6 @@ public class TestShardRouting {
UUIDs.randomBase64UUID(),
new Snapshot("repo", new SnapshotId(randomAlphaOfLength(8), UUIDs.randomBase64UUID())),
Version.CURRENT,
"some_index"));
new IndexId("some_index", UUIDs.randomBase64UUID(random()))));
}
}

View File

@ -806,11 +806,10 @@ public abstract class IndexShardTestCase extends ESTestCase {
final Repository repository) {
final Version version = Version.CURRENT;
final ShardId shardId = shard.shardId();
final String index = shardId.getIndexName();
final IndexId indexId = new IndexId(shardId.getIndex().getName(), shardId.getIndex().getUUID());
final DiscoveryNode node = getFakeDiscoNode(shard.routingEntry().currentNodeId());
final RecoverySource.SnapshotRecoverySource recoverySource =
new RecoverySource.SnapshotRecoverySource(UUIDs.randomBase64UUID(), snapshot, version, index);
new RecoverySource.SnapshotRecoverySource(UUIDs.randomBase64UUID(), snapshot, version, indexId);
final ShardRouting shardRouting = newShardRouting(shardId, node.getId(), true, ShardRoutingState.INITIALIZING, recoverySource);
shard.markAsRecovering("from snapshot", new RecoveryState(shardRouting, node, null));
final PlainActionFuture<Void> future = PlainActionFuture.newFuture();

View File

@ -451,7 +451,8 @@ public class ShardFollowTaskReplicationTests extends ESIndexLevelReplicationTest
DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
Snapshot snapshot = new Snapshot("foo", new SnapshotId("bar", UUIDs.randomBase64UUID()));
ShardRouting routing = ShardRoutingHelper.newWithRestoreSource(primary.routingEntry(),
new RecoverySource.SnapshotRecoverySource(UUIDs.randomBase64UUID(), snapshot, Version.CURRENT, "test"));
new RecoverySource.SnapshotRecoverySource(UUIDs.randomBase64UUID(), snapshot, Version.CURRENT,
new IndexId("test", UUIDs.randomBase64UUID(random()))));
primary.markAsRecovering("remote recovery from leader", new RecoveryState(routing, localNode, null));
final PlainActionFuture<Boolean> future = PlainActionFuture.newFuture();
primary.restoreFromRepository(new RestoreOnlyRepository(index.getName()) {

View File

@ -121,7 +121,8 @@ public class FollowEngineIndexShardTests extends IndexShardTestCase {
RecoverySource.ExistingStoreRecoverySource.INSTANCE);
final Snapshot snapshot = new Snapshot("foo", new SnapshotId("bar", UUIDs.randomBase64UUID()));
routing = ShardRoutingHelper.newWithRestoreSource(routing,
new RecoverySource.SnapshotRecoverySource(UUIDs.randomBase64UUID(), snapshot, Version.CURRENT, "test"));
new RecoverySource.SnapshotRecoverySource(UUIDs.randomBase64UUID(), snapshot, Version.CURRENT,
new IndexId("test", UUIDs.randomBase64UUID(random()))));
target = reinitShard(target, routing);
Store sourceStore = source.store();
Store targetStore = target.store();

View File

@ -229,7 +229,7 @@ public class SourceOnlySnapshotShardTests extends IndexShardTestCase {
ShardRouting shardRouting = TestShardRouting.newShardRouting(new ShardId("index", "_na_", 0), randomAlphaOfLength(10), true,
ShardRoutingState.INITIALIZING,
new RecoverySource.SnapshotRecoverySource(
UUIDs.randomBase64UUID(), new Snapshot("src_only", snapshotId), Version.CURRENT, indexId.getName()));
UUIDs.randomBase64UUID(), new Snapshot("src_only", snapshotId), Version.CURRENT, indexId));
IndexMetaData metaData = runAsSnapshot(threadPool, () -> repository.getSnapshotIndexMetaData(snapshotId, indexId));
IndexShard restoredShard = newShard(
shardRouting, metaData, null, SourceOnlySnapshotRepository.getEngineFactory(), () -> {}, RetentionLeaseSyncer.EMPTY);