Remove Redundant Version Param from Repository APIs (#48231) (#48298)

This parameter isn't used by any implementation
This commit is contained in:
Armin Braun 2019-10-21 16:20:45 +02:00 committed by GitHub
parent cdfac9dfb0
commit dc08feadc6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 26 additions and 58 deletions

View File

@ -466,8 +466,8 @@ final class StoreRecovery {
} }
final IndexId indexId = repository.getRepositoryData().resolveIndexId(indexName); final IndexId indexId = repository.getRepositoryData().resolveIndexId(indexName);
assert indexShard.getEngineOrNull() == null; assert indexShard.getEngineOrNull() == null;
repository.restoreShard(indexShard, indexShard.store(), restoreSource.snapshot().getSnapshotId(), repository.restoreShard(indexShard.store(), restoreSource.snapshot().getSnapshotId(), indexId, snapshotShardId,
restoreSource.version(), indexId, snapshotShardId, indexShard.recoveryState()); indexShard.recoveryState());
final Store store = indexShard.store(); final Store store = indexShard.store();
bootstrap(indexShard, store); bootstrap(indexShard, store);
assert indexShard.shardRouting.primary() : "only primary shards can recover from store"; assert indexShard.shardRouting.primary() : "only primary shards can recover from store";

View File

@ -19,7 +19,6 @@
package org.elasticsearch.repositories; package org.elasticsearch.repositories;
import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexCommit;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaData;
@ -127,14 +126,13 @@ public class FilterRepository implements Repository {
in.snapshotShard(store, mapperService, snapshotId, indexId, snapshotIndexCommit, snapshotStatus, listener); in.snapshotShard(store, mapperService, snapshotId, indexId, snapshotIndexCommit, snapshotStatus, listener);
} }
@Override @Override
public void restoreShard(Store store, SnapshotId snapshotId, public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) {
Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { in.restoreShard(store, snapshotId, indexId, snapshotShardId, recoveryState);
in.restoreShard(store, snapshotId, version, indexId, snapshotShardId, recoveryState);
} }
@Override @Override
public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Version version, IndexId indexId, ShardId shardId) { public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, IndexId indexId, ShardId shardId) {
return in.getShardSnapshotStatus(snapshotId, version, indexId, shardId); return in.getShardSnapshotStatus(snapshotId, indexId, shardId);
} }
@Override @Override

View File

@ -19,7 +19,6 @@
package org.elasticsearch.repositories; package org.elasticsearch.repositories;
import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexCommit;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaData;
@ -27,7 +26,6 @@ import org.elasticsearch.cluster.metadata.RepositoryMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.component.LifecycleComponent; import org.elasticsearch.common.component.LifecycleComponent;
import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus;
import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.Store;
@ -216,45 +214,23 @@ public interface Repository extends LifecycleComponent {
* Restores snapshot of the shard. * Restores snapshot of the shard.
* <p> * <p>
* The index can be renamed on restore, hence different {@code shardId} and {@code snapshotShardId} are supplied. * The index can be renamed on restore, hence different {@code shardId} and {@code snapshotShardId} are supplied.
* @param shard the shard to restore the index into
* @param store the store to restore the index into * @param store the store to restore the index into
* @param snapshotId snapshot id * @param snapshotId snapshot id
* @param version version of elasticsearch that created this snapshot
* @param indexId id of the index in the repository from which the restore is occurring
* @param snapshotShardId shard id (in the snapshot)
* @param recoveryState recovery state
* @deprecated use {@link #restoreShard(Store, SnapshotId, Version, IndexId, ShardId, RecoveryState)} instead
*/
@Deprecated
default void restoreShard(IndexShard shard, Store store, SnapshotId snapshotId, Version version, IndexId indexId,
ShardId snapshotShardId, RecoveryState recoveryState) {
restoreShard(store, snapshotId, version, indexId, snapshotShardId, recoveryState);
}
/**
* Restores snapshot of the shard.
* <p>
* The index can be renamed on restore, hence different {@code shardId} and {@code snapshotShardId} are supplied.
* @param store the store to restore the index into
* @param snapshotId snapshot id
* @param version version of elasticsearch that created this snapshot
* @param indexId id of the index in the repository from which the restore is occurring * @param indexId id of the index in the repository from which the restore is occurring
* @param snapshotShardId shard id (in the snapshot) * @param snapshotShardId shard id (in the snapshot)
* @param recoveryState recovery state * @param recoveryState recovery state
*/ */
void restoreShard(Store store, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState);
RecoveryState recoveryState);
/** /**
* Retrieve shard snapshot status for the stored snapshot * Retrieve shard snapshot status for the stored snapshot
* *
* @param snapshotId snapshot id * @param snapshotId snapshot id
* @param version version of elasticsearch that created this snapshot
* @param indexId the snapshotted index id for the shard to get status for * @param indexId the snapshotted index id for the shard to get status for
* @param shardId shard id * @param shardId shard id
* @return snapshot status * @return snapshot status
*/ */
IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Version version, IndexId indexId, ShardId shardId); IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, IndexId indexId, ShardId shardId);
} }

View File

@ -28,7 +28,6 @@ import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.RateLimiter; import org.apache.lucene.store.RateLimiter;
import org.apache.lucene.util.SetOnce; import org.apache.lucene.util.SetOnce;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.ActionRunnable;
import org.elasticsearch.action.StepListener; import org.elasticsearch.action.StepListener;
@ -1186,7 +1185,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
} }
@Override @Override
public void restoreShard(Store store, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId,
RecoveryState recoveryState) { RecoveryState recoveryState) {
ShardId shardId = store.shardId(); ShardId shardId = store.shardId();
try { try {
@ -1212,7 +1211,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
} }
@Override @Override
public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Version version, IndexId indexId, ShardId shardId) { public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, IndexId indexId, ShardId shardId) {
BlobStoreIndexShardSnapshot snapshot = loadShardSnapshot(shardContainer(indexId, shardId), snapshotId); BlobStoreIndexShardSnapshot snapshot = loadShardSnapshot(shardContainer(indexId, shardId), snapshotId);
return IndexShardSnapshotStatus.newDone(snapshot.startTime(), snapshot.time(), return IndexShardSnapshotStatus.newDone(snapshot.startTime(), snapshot.time(),
snapshot.incrementalFileCount(), snapshot.totalFileCount(), snapshot.incrementalFileCount(), snapshot.totalFileCount(),

View File

@ -701,7 +701,6 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus
} else { } else {
shardSnapshotStatus = repository.getShardSnapshotStatus( shardSnapshotStatus = repository.getShardSnapshotStatus(
snapshotInfo.snapshotId(), snapshotInfo.snapshotId(),
snapshotInfo.version(),
indexId, indexId,
shardId); shardId);
} }

View File

@ -2350,8 +2350,8 @@ public class IndexShardTests extends IndexShardTestCase {
target.markAsRecovering("store", new RecoveryState(routing, localNode, null)); target.markAsRecovering("store", new RecoveryState(routing, localNode, null));
assertTrue(target.restoreFromRepository(new RestoreOnlyRepository("test") { assertTrue(target.restoreFromRepository(new RestoreOnlyRepository("test") {
@Override @Override
public void restoreShard(Store store, SnapshotId snapshotId, public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId,
Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { RecoveryState recoveryState) {
try { try {
cleanLuceneIndex(targetStore.directory()); cleanLuceneIndex(targetStore.directory());
for (String file : sourceStore.directory().listAll()) { for (String file : sourceStore.directory().listAll()) {

View File

@ -20,7 +20,6 @@
package org.elasticsearch.repositories; package org.elasticsearch.repositories;
import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexCommit;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryRequest; import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryRequest;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -209,13 +208,13 @@ public class RepositoriesServiceTests extends ESTestCase {
} }
@Override @Override
public void restoreShard(Store store, SnapshotId snapshotId, public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId,
Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { RecoveryState recoveryState) {
} }
@Override @Override
public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Version version, IndexId indexId, ShardId shardId) { public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, IndexId indexId, ShardId shardId) {
return null; return null;
} }

View File

@ -118,7 +118,7 @@ public class FsRepositoryTests extends ESTestCase {
routing = ShardRoutingHelper.initialize(routing, localNode.getId(), 0); routing = ShardRoutingHelper.initialize(routing, localNode.getId(), 0);
RecoveryState state = new RecoveryState(routing, localNode, null); RecoveryState state = new RecoveryState(routing, localNode, null);
runGeneric(threadPool, () -> runGeneric(threadPool, () ->
repository.restoreShard(store, snapshotId, Version.CURRENT, indexId, shardId, state)); repository.restoreShard(store, snapshotId, indexId, shardId, state));
assertTrue(state.getIndex().recoveredBytes() > 0); assertTrue(state.getIndex().recoveredBytes() > 0);
assertEquals(0, state.getIndex().reusedFileCount()); assertEquals(0, state.getIndex().reusedFileCount());
assertEquals(indexCommit.getFileNames().size(), state.getIndex().recoveredFileCount()); assertEquals(indexCommit.getFileNames().size(), state.getIndex().recoveredFileCount());
@ -140,13 +140,13 @@ public class FsRepositoryTests extends ESTestCase {
// roll back to the first snap and then incrementally restore // roll back to the first snap and then incrementally restore
RecoveryState firstState = new RecoveryState(routing, localNode, null); RecoveryState firstState = new RecoveryState(routing, localNode, null);
runGeneric(threadPool, () -> runGeneric(threadPool, () ->
repository.restoreShard(store, snapshotId, Version.CURRENT, indexId, shardId, firstState)); repository.restoreShard(store, snapshotId, indexId, shardId, firstState));
assertEquals("should reuse everything except of .liv and .si", assertEquals("should reuse everything except of .liv and .si",
commitFileNames.size()-2, firstState.getIndex().reusedFileCount()); commitFileNames.size()-2, firstState.getIndex().reusedFileCount());
RecoveryState secondState = new RecoveryState(routing, localNode, null); RecoveryState secondState = new RecoveryState(routing, localNode, null);
runGeneric(threadPool, () -> runGeneric(threadPool, () ->
repository.restoreShard(store, incSnapshotId, Version.CURRENT, indexId, shardId, secondState)); repository.restoreShard(store, incSnapshotId, indexId, shardId, secondState));
assertEquals(secondState.getIndex().reusedFileCount(), commitFileNames.size()-2); assertEquals(secondState.getIndex().reusedFileCount(), commitFileNames.size()-2);
assertEquals(secondState.getIndex().recoveredFileCount(), 2); assertEquals(secondState.getIndex().recoveredFileCount(), 2);
List<RecoveryState.File> recoveredFiles = List<RecoveryState.File> recoveredFiles =

View File

@ -806,7 +806,7 @@ public abstract class IndexShardTestCase extends ESTestCase {
final ShardRouting shardRouting = newShardRouting(shardId, node.getId(), true, ShardRoutingState.INITIALIZING, recoverySource); final ShardRouting shardRouting = newShardRouting(shardId, node.getId(), true, ShardRoutingState.INITIALIZING, recoverySource);
shard.markAsRecovering("from snapshot", new RecoveryState(shardRouting, node, null)); shard.markAsRecovering("from snapshot", new RecoveryState(shardRouting, node, null));
repository.restoreShard(shard.store(), repository.restoreShard(shard.store(),
snapshot.getSnapshotId(), version, snapshot.getSnapshotId(),
indexId, indexId,
shard.shardId(), shard.shardId(),
shard.recoveryState()); shard.recoveryState());

View File

@ -19,7 +19,6 @@
package org.elasticsearch.index.shard; package org.elasticsearch.index.shard;
import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexCommit;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaData;
@ -140,7 +139,7 @@ public abstract class RestoreOnlyRepository extends AbstractLifecycleComponent i
} }
@Override @Override
public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Version version, IndexId indexId, ShardId shardId) { public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, IndexId indexId, ShardId shardId) {
return null; return null;
} }

View File

@ -15,7 +15,6 @@ import org.apache.lucene.index.IndexCommit;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchSecurityException; import org.elasticsearch.ElasticsearchSecurityException;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
@ -301,8 +300,7 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit
} }
@Override @Override
public void restoreShard(Store store, SnapshotId snapshotId, public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) {
Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) {
// TODO: Add timeouts to network calls / the restore process. // TODO: Add timeouts to network calls / the restore process.
createEmptyStore(store); createEmptyStore(store);
ShardId shardId = store.shardId(); ShardId shardId = store.shardId();
@ -413,7 +411,7 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit
} }
@Override @Override
public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Version version, IndexId indexId, ShardId leaderShardId) { public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, IndexId indexId, ShardId leaderShardId) {
throw new UnsupportedOperationException("Unsupported for repository of type: " + TYPE); throw new UnsupportedOperationException("Unsupported for repository of type: " + TYPE);
} }

View File

@ -453,8 +453,8 @@ public class ShardFollowTaskReplicationTests extends ESIndexLevelReplicationTest
primary.markAsRecovering("remote recovery from leader", new RecoveryState(routing, localNode, null)); primary.markAsRecovering("remote recovery from leader", new RecoveryState(routing, localNode, null));
primary.restoreFromRepository(new RestoreOnlyRepository(index.getName()) { primary.restoreFromRepository(new RestoreOnlyRepository(index.getName()) {
@Override @Override
public void restoreShard(Store store, SnapshotId snapshotId, public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId,
Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { RecoveryState recoveryState) {
try { try {
IndexShard leader = leaderGroup.getPrimary(); IndexShard leader = leaderGroup.getPrimary();
Lucene.cleanLuceneIndex(primary.store().directory()); Lucene.cleanLuceneIndex(primary.store().directory());

View File

@ -127,8 +127,8 @@ public class FollowEngineIndexShardTests extends IndexShardTestCase {
target.markAsRecovering("store", new RecoveryState(routing, localNode, null)); target.markAsRecovering("store", new RecoveryState(routing, localNode, null));
assertTrue(target.restoreFromRepository(new RestoreOnlyRepository("test") { assertTrue(target.restoreFromRepository(new RestoreOnlyRepository("test") {
@Override @Override
public void restoreShard(Store store, SnapshotId snapshotId, public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId,
Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { RecoveryState recoveryState) {
try { try {
cleanLuceneIndex(targetStore.directory()); cleanLuceneIndex(targetStore.directory());
for (String file : sourceStore.directory().listAll()) { for (String file : sourceStore.directory().listAll()) {