diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/RecoverySource.java b/server/src/main/java/org/elasticsearch/cluster/routing/RecoverySource.java index 90ae4ec3b2f8d..2bfc76e447195 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/RecoverySource.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/RecoverySource.java @@ -20,12 +20,14 @@ package org.elasticsearch.cluster.routing; import org.elasticsearch.Version; +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; @@ -213,21 +215,25 @@ public String toString() { 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 { restoreUUID = in.readString(); snapshot = new Snapshot(in); version = Version.readVersion(in); - index = in.readString(); + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + index = new IndexId(in); + } else { + index = new IndexId(in.readString(), IndexMetaData.INDEX_UUID_NA_VALUE); + } } public String restoreUUID() { @@ -238,7 +244,13 @@ public Snapshot snapshot() { 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; } @@ -251,7 +263,11 @@ protected void writeAdditionalFields(StreamOutput out) throws IOException { out.writeString(restoreUUID); snapshot.writeTo(out); Version.writeVersion(version, out); - out.writeString(index); + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + index.writeTo(out); + } else { + out.writeString(index.getName()); + } } @Override @@ -264,7 +280,7 @@ public void addAdditionalFields(XContentBuilder builder, ToXContent.Params param builder.field("repository", snapshot.getRepository()) .field("snapshot", snapshot.getSnapshotId().getName()) .field("version", version.toString()) - .field("index", index) + .field("index", index.getName()) .field("restoreUUID", restoreUUID); } diff --git a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java index c3948507fc7fb..b84b6aa4c1e49 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java @@ -31,6 +31,7 @@ import org.apache.lucene.store.IndexInput; import org.elasticsearch.ExceptionsHelper; 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; @@ -473,20 +474,24 @@ private void restore(IndexShard indexShard, Repository repository, SnapshotRecov 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()); + } + final StepListener 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); } - 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 - )); + 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); } diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java index a334479671b18..be4f02fb5a114 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -250,8 +250,8 @@ public ClusterState execute(ClusterState currentState) { for (Map.Entry 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, diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingTests.java index 5bcac4a1e2618..1dd5e562db8ca 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingTests.java @@ -23,6 +23,7 @@ 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 void testEqualsIgnoringVersion() { 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; diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index eb5b5c10c843c..da872fffb6b1b 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -38,6 +38,7 @@ 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; @@ -151,7 +152,8 @@ public void testNewIndexRestored() { .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)); @@ -168,7 +170,8 @@ public void testExistingIndexRestored() { .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)); } diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java index e4b215cf2e25c..cdd442822be0a 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java @@ -53,6 +53,7 @@ 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 void testRestoreDoesNotAllocateSnapshotOnOlderNodes() { 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 void testMessages() { 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); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ThrottlingAllocationTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ThrottlingAllocationTests.java index 6a4b68c16f6ba..eaef71329627f 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ThrottlingAllocationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ThrottlingAllocationTests.java @@ -47,6 +47,7 @@ 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 @@ private ClusterState createRecoveryStateAndInitalizeAllocations(MetaData metaDat 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); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java index fbedd5e1799c6..817cb29955e69 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/RestoreInProgressAllocationDeciderTests.java @@ -41,6 +41,7 @@ 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; @@ -207,6 +208,7 @@ private Decision executeAllocation(final ClusterState clusterState, final ShardR 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()))); } } diff --git a/server/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java b/server/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java index 3be4f1c1d809e..135d18d11214f 100644 --- a/server/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java +++ b/server/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java @@ -48,6 +48,7 @@ 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 @@ private RoutingAllocation getRestoreRoutingAllocation(AllocationDeciders allocat 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) diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 39f120b2fa3a3..2f2eb987d990d 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -2327,7 +2327,8 @@ public void testRestoreShard() throws IOException { 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(); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java index a8b3d82a02989..3a1b9dc570965 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -22,6 +22,7 @@ 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; @@ -85,6 +86,9 @@ 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; @@ -97,6 +101,7 @@ 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; @@ -651,6 +656,10 @@ public void testSnapshotRecovery() throws Exception { 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> indexRecoveryStates : response.shardRecoveryStates().entrySet()) { assertThat(indexRecoveryStates.getKey(), equalTo(INDEX_NAME)); @@ -661,7 +670,7 @@ public void testSnapshotRecovery() throws Exception { 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()); } diff --git a/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java index 735afc8c8b17d..42d3ab7f4efc3 100644 --- a/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java @@ -116,7 +116,7 @@ public void testSnapshotAndRestore() throws IOException, InterruptedException { 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); diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/routing/TestShardRouting.java b/test/framework/src/main/java/org/elasticsearch/cluster/routing/TestShardRouting.java index 9d892d192a25a..d052fe6021b44 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/routing/TestShardRouting.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/routing/TestShardRouting.java @@ -23,10 +23,12 @@ 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 static RecoverySource randomRecoverySource() { UUIDs.randomBase64UUID(), new Snapshot("repo", new SnapshotId(randomAlphaOfLength(8), UUIDs.randomBase64UUID())), Version.CURRENT, - "some_index")); + new IndexId("some_index", UUIDs.randomBase64UUID(random())))); } } diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index c398245d8ab5e..995be1a1bdbb9 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -805,11 +805,10 @@ protected void recoverShardFromSnapshot(final IndexShard shard, 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 future = PlainActionFuture.newFuture(); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskReplicationTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskReplicationTests.java index 67c46b60dddf6..f7356dc3844d0 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskReplicationTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskReplicationTests.java @@ -447,7 +447,8 @@ protected synchronized void recoverPrimary(IndexShard primary) { 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 future = PlainActionFuture.newFuture(); primary.restoreFromRepository(new RestoreOnlyRepository(index.getName()) { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowEngineIndexShardTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowEngineIndexShardTests.java index 74a331064dce8..80f6c2290d837 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowEngineIndexShardTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowEngineIndexShardTests.java @@ -113,7 +113,8 @@ public void testRestoreShard() throws IOException { 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(); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java index 65d30267e2bfb..1ef11cdc6e5e8 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java @@ -228,7 +228,7 @@ public void testRestoreMinmal() throws IOException { 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);