diff --git a/core/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java b/core/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java index c1016f0f4e3..ff2d1d298a5 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java +++ b/core/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java @@ -397,7 +397,7 @@ final class StoreRecovery { if (!shardId.getIndexName().equals(restoreSource.index())) { snapshotShardId = new ShardId(restoreSource.index(), IndexMetaData.INDEX_UUID_NA_VALUE, shardId.id()); } - repository.restore(indexShard, restoreSource.snapshot().getSnapshotId(), restoreSource.version(), snapshotShardId, indexShard.recoveryState()); + repository.restoreShard(indexShard, restoreSource.snapshot().getSnapshotId(), restoreSource.version(), snapshotShardId, indexShard.recoveryState()); indexShard.skipTranslogRecovery(); indexShard.finalizeRecovery(); indexShard.postRecovery("restore done"); diff --git a/core/src/main/java/org/elasticsearch/repositories/Repository.java b/core/src/main/java/org/elasticsearch/repositories/Repository.java index 6c4cf93a7a4..2230f046eba 100644 --- a/core/src/main/java/org/elasticsearch/repositories/Repository.java +++ b/core/src/main/java/org/elasticsearch/repositories/Repository.java @@ -35,15 +35,16 @@ import java.io.IOException; import java.util.List; /** - * Snapshot repository interface. + * An interface for interacting with a repository in snapshot and restore. *

- * Responsible for index and cluster and shard level operations. + * Implementations are responsible for reading and writing both metadata and actual shard data to and from + * a repository backend. *

- * Typical snapshot usage pattern: + * To perform a snapshot: *

@@ -56,7 +57,7 @@ public interface Repository extends LifecycleComponent { * @param snapshotId snapshot id * @return information about snapshot */ - SnapshotInfo readSnapshot(SnapshotId snapshotId); + SnapshotInfo getSnapshotInfo(SnapshotId snapshotId); /** * Returns global metadata associate with the snapshot. @@ -67,7 +68,7 @@ public interface Repository extends LifecycleComponent { * @param indices list of indices * @return information about snapshot */ - MetaData readSnapshotMetaData(SnapshotInfo snapshot, List indices) throws IOException; + MetaData getSnapshotMetaData(SnapshotInfo snapshot, List indices) throws IOException; /** * Returns the list of snapshots currently stored in the repository that match the given predicate on the snapshot name. @@ -75,7 +76,7 @@ public interface Repository extends LifecycleComponent { * * @return snapshot list */ - List snapshots(); + List getSnapshots(); /** * Starts snapshotting process @@ -109,12 +110,12 @@ public interface Repository extends LifecycleComponent { /** * Returns snapshot throttle time in nanoseconds */ - long snapshotThrottleTimeInNanos(); + long getSnapshotThrottleTimeInNanos(); /** * Returns restore throttle time in nanoseconds */ - long restoreThrottleTimeInNanos(); + long getRestoreThrottleTimeInNanos(); /** @@ -136,11 +137,18 @@ public interface Repository extends LifecycleComponent { */ void endVerification(String verificationToken); + /** + * Verifies repository settings on data node. + * @param verificationToken value returned by {@link org.elasticsearch.repositories.Repository#startVerification()} + * @param localNode the local node information, for inclusion in verification errors + */ + void verify(String verificationToken, DiscoveryNode localNode); + /** * Returns true if the repository supports only read operations * @return true if the repository is read/only */ - boolean readOnly(); + boolean isReadOnly(); /** * Creates a snapshot of the shard based on the index commit point. @@ -156,7 +164,7 @@ public interface Repository extends LifecycleComponent { * @param snapshotIndexCommit commit point * @param snapshotStatus snapshot status */ - void snapshot(IndexShard shard, SnapshotId snapshotId, IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus); + void snapshotShard(IndexShard shard, SnapshotId snapshotId, IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus); /** * Restores snapshot of the shard. @@ -169,22 +177,17 @@ public interface Repository extends LifecycleComponent { * @param snapshotShardId shard id (in the snapshot) * @param recoveryState recovery state */ - void restore(IndexShard shard, SnapshotId snapshotId, Version version, ShardId snapshotShardId, RecoveryState recoveryState); + void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, ShardId snapshotShardId, RecoveryState recoveryState); /** * Retrieve shard snapshot status for the stored snapshot * * @param snapshotId snapshot id - * @param version version of elasticsearch that created this snapshot + * @param version version of elasticsearch that created this snapshot * @param shardId shard id * @return snapshot status */ - IndexShardSnapshotStatus snapshotStatus(SnapshotId snapshotId, Version version, ShardId shardId); + IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Version version, ShardId shardId); + - /** - * Verifies repository settings on data node. - * @param verificationToken value returned by {@link org.elasticsearch.repositories.Repository#startVerification()} - * @param localNode the local node information, for inclusion in verification errors - */ - void verify(String verificationToken, DiscoveryNode localNode); } diff --git a/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index 4c328872742..173937ce69c 100644 --- a/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -300,13 +300,13 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp @Override public void initializeSnapshot(SnapshotId snapshotId, List indices, MetaData metaData) { - if (readOnly()) { + if (isReadOnly()) { throw new RepositoryException(this.repositoryName, "cannot create snapshot in a readonly repository"); } try { final String snapshotName = snapshotId.getName(); // check if the snapshot name already exists in the repository - if (snapshots().stream().anyMatch(s -> s.getName().equals(snapshotName))) { + if (getSnapshots().stream().anyMatch(s -> s.getName().equals(snapshotName))) { throw new SnapshotCreationException(repositoryName, snapshotId, "snapshot with the same name already exists"); } if (snapshotFormat.exists(snapshotsBlobContainer, blobId(snapshotId)) || @@ -328,13 +328,13 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp @Override public void deleteSnapshot(SnapshotId snapshotId) { - if (readOnly()) { + if (isReadOnly()) { throw new RepositoryException(this.repositoryName, "cannot delete snapshot from a readonly repository"); } List indices = Collections.emptyList(); SnapshotInfo snapshot = null; try { - snapshot = readSnapshot(snapshotId); + snapshot = getSnapshotInfo(snapshotId); indices = snapshot.indices(); } catch (SnapshotMissingException ex) { throw ex; @@ -365,7 +365,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp globalMetaDataFormat.delete(snapshotsBlobContainer, snapshotName); } // Delete snapshot from the snapshot list - List snapshotIds = snapshots().stream().filter(id -> snapshotId.equals(id) == false).collect(Collectors.toList()); + List snapshotIds = getSnapshots().stream().filter(id -> snapshotId.equals(id) == false).collect(Collectors.toList()); writeSnapshotsToIndexGen(snapshotIds); // Now delete all indices @@ -411,7 +411,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp totalShards, shardFailures); snapshotFormat.write(blobStoreSnapshot, snapshotsBlobContainer, blobId(snapshotId)); - List snapshotIds = snapshots(); + List snapshotIds = getSnapshots(); if (!snapshotIds.contains(snapshotId)) { snapshotIds = new ArrayList<>(snapshotIds); snapshotIds.add(snapshotId); @@ -425,7 +425,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp } @Override - public List snapshots() { + public List getSnapshots() { try { return Collections.unmodifiableList(readSnapshotsFromIndex()); } catch (NoSuchFileException | FileNotFoundException e) { @@ -437,12 +437,12 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp } @Override - public MetaData readSnapshotMetaData(SnapshotInfo snapshot, List indices) throws IOException { + public MetaData getSnapshotMetaData(SnapshotInfo snapshot, List indices) throws IOException { return readSnapshotMetaData(snapshot.snapshotId(), snapshot.version(), indices, false); } @Override - public SnapshotInfo readSnapshot(final SnapshotId snapshotId) { + public SnapshotInfo getSnapshotInfo(final SnapshotId snapshotId) { try { return snapshotFormat.read(snapshotsBlobContainer, blobId(snapshotId)); } catch (FileNotFoundException | NoSuchFileException ex) { @@ -561,19 +561,19 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp private static final String UUID = "uuid"; @Override - public long snapshotThrottleTimeInNanos() { + public long getSnapshotThrottleTimeInNanos() { return snapshotRateLimitingTimeInNanos.count(); } @Override - public long restoreThrottleTimeInNanos() { + public long getRestoreThrottleTimeInNanos() { return restoreRateLimitingTimeInNanos.count(); } @Override public String startVerification() { try { - if (readOnly()) { + if (isReadOnly()) { // It's readonly - so there is not much we can do here to verify it return null; } else { @@ -593,7 +593,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp @Override public void endVerification(String seed) { - if (readOnly()) { + if (isReadOnly()) { throw new UnsupportedOperationException("shouldn't be called"); } try { @@ -608,7 +608,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp } @Override - public boolean readOnly() { + public boolean isReadOnly() { return readOnly; } @@ -618,7 +618,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp } protected void writeSnapshotsToIndexGen(final List snapshots) throws IOException { - assert readOnly() == false; // can not write to a read only repository + assert isReadOnly() == false; // can not write to a read only repository final BytesReference snapshotsBytes; try (BytesStreamOutput bStream = new BytesStreamOutput()) { try (StreamOutput stream = new OutputStreamStreamOutput(bStream)) { @@ -641,7 +641,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp // write the index file writeAtomic(SNAPSHOTS_FILE_PREFIX + Long.toString(gen), snapshotsBytes); // delete the N-2 index file if it exists, keep the previous one around as a backup - if (readOnly() == false && gen - 2 >= 0) { + if (isReadOnly() == false && gen - 2 >= 0) { final String oldSnapshotIndexFile = SNAPSHOTS_FILE_PREFIX + Long.toString(gen - 2); if (snapshotsBlobContainer.blobExists(oldSnapshotIndexFile)) { snapshotsBlobContainer.deleteBlob(oldSnapshotIndexFile); @@ -797,7 +797,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp } @Override - public void snapshot(IndexShard shard, SnapshotId snapshotId, IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) { + public void snapshotShard(IndexShard shard, SnapshotId snapshotId, IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) { SnapshotContext snapshotContext = new SnapshotContext(shard, snapshotId, snapshotStatus); snapshotStatus.startTime(System.currentTimeMillis()); @@ -818,7 +818,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp } @Override - public void restore(IndexShard shard, SnapshotId snapshotId, Version version, ShardId snapshotShardId, RecoveryState recoveryState) { + public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, ShardId snapshotShardId, RecoveryState recoveryState) { final RestoreContext snapshotContext = new RestoreContext(shard, snapshotId, version, snapshotShardId, recoveryState); try { snapshotContext.restore(); @@ -828,7 +828,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp } @Override - public IndexShardSnapshotStatus snapshotStatus(SnapshotId snapshotId, Version version, ShardId shardId) { + public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Version version, ShardId shardId) { Context context = new Context(snapshotId, version, shardId); BlobStoreIndexShardSnapshot snapshot = context.loadSnapshot(); IndexShardSnapshotStatus status = new IndexShardSnapshotStatus(); diff --git a/core/src/main/java/org/elasticsearch/repositories/uri/URLRepository.java b/core/src/main/java/org/elasticsearch/repositories/uri/URLRepository.java index f08faa2c0d6..4058baa63ef 100644 --- a/core/src/main/java/org/elasticsearch/repositories/uri/URLRepository.java +++ b/core/src/main/java/org/elasticsearch/repositories/uri/URLRepository.java @@ -141,7 +141,7 @@ public class URLRepository extends BlobStoreRepository { } @Override - public boolean readOnly() { + public boolean isReadOnly() { return true; } diff --git a/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java index 67a5ae361bb..dedcc6d8d21 100644 --- a/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -185,16 +185,16 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis try { // Read snapshot info and metadata from the repository Repository repository = repositoriesService.repository(request.repositoryName); - final Optional matchingSnapshotId = repository.snapshots().stream() + final Optional matchingSnapshotId = repository.getSnapshots().stream() .filter(s -> request.snapshotName.equals(s.getName())).findFirst(); if (matchingSnapshotId.isPresent() == false) { throw new SnapshotRestoreException(request.repositoryName, request.snapshotName, "snapshot does not exist"); } final SnapshotId snapshotId = matchingSnapshotId.get(); - final SnapshotInfo snapshotInfo = repository.readSnapshot(snapshotId); + final SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotId); final Snapshot snapshot = new Snapshot(request.repositoryName, snapshotId); List filteredIndices = SnapshotUtils.filterIndices(snapshotInfo.indices(), request.indices(), request.indicesOptions()); - MetaData metaDataIn = repository.readSnapshotMetaData(snapshotInfo, filteredIndices); + MetaData metaDataIn = repository.getSnapshotMetaData(snapshotInfo, filteredIndices); final MetaData metaData; if (snapshotInfo.version().before(Version.V_2_0_0_beta1)) { diff --git a/core/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/core/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index b24627bb634..7741ef1c0e6 100644 --- a/core/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/core/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -340,7 +340,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements // we flush first to make sure we get the latest writes snapshotted IndexCommit snapshotIndexCommit = indexShard.snapshotIndex(true); try { - repository.snapshot(indexShard, snapshot.getSnapshotId(), snapshotIndexCommit, snapshotStatus); + repository.snapshotShard(indexShard, snapshot.getSnapshotId(), snapshotIndexCommit, snapshotStatus); if (logger.isDebugEnabled()) { StringBuilder sb = new StringBuilder(); sb.append(" index : version [").append(snapshotStatus.indexVersion()).append("], number_of_files [").append(snapshotStatus.numberOfFiles()).append("] with total_size [").append(new ByteSizeValue(snapshotStatus.totalSize())).append("]\n"); diff --git a/core/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/core/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 8d2407dd2e4..df56f2a24a6 100644 --- a/core/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/core/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -132,7 +132,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus public List snapshotIds(final String repositoryName) { Repository repository = repositoriesService.repository(repositoryName); assert repository != null; // should only be called once we've validated the repository exists - return repository.snapshots(); + return repository.getSnapshots(); } /** @@ -148,7 +148,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus if (!entries.isEmpty()) { return inProgressSnapshot(entries.iterator().next()); } - return repositoriesService.repository(repositoryName).readSnapshot(snapshotId); + return repositoriesService.repository(repositoryName).getSnapshotInfo(snapshotId); } /** @@ -174,7 +174,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus final Repository repository = repositoriesService.repository(repositoryName); for (SnapshotId snapshotId : snapshotIdsToIterate) { try { - snapshotSet.add(repository.readSnapshot(snapshotId)); + snapshotSet.add(repository.getSnapshotInfo(snapshotId)); } catch (Exception ex) { if (ignoreUnavailable) { logger.warn("failed to get snapshot [{}]", ex, snapshotId); @@ -546,7 +546,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus final SnapshotInfo snapshotInfo) throws IOException { Map shardStatus = new HashMap<>(); Repository repository = repositoriesService.repository(repositoryName); - MetaData metaData = repository.readSnapshotMetaData(snapshotInfo, snapshotInfo.indices()); + MetaData metaData = repository.getSnapshotMetaData(snapshotInfo, snapshotInfo.indices()); for (String index : snapshotInfo.indices()) { IndexMetaData indexMetaData = metaData.indices().get(index); if (indexMetaData != null) { @@ -561,7 +561,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus shardStatus.put(shardId, shardSnapshotStatus); } else { IndexShardSnapshotStatus shardSnapshotStatus = - repository.snapshotStatus(snapshotInfo.snapshotId(), snapshotInfo.version(), shardId); + repository.getShardSnapshotStatus(snapshotInfo.snapshotId(), snapshotInfo.version(), shardId); shardStatus.put(shardId, shardSnapshotStatus); } } @@ -953,7 +953,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus public void deleteSnapshot(final String repositoryName, final String snapshotName, final DeleteSnapshotListener listener) { // First, look for the snapshot in the repository final Repository repository = repositoriesService.repository(repositoryName); - Optional matchedEntry = repository.snapshots().stream().filter(s -> s.getName().equals(snapshotName)).findFirst(); + Optional matchedEntry = repository.getSnapshots().stream().filter(s -> s.getName().equals(snapshotName)).findFirst(); // if nothing found by the same name, then look in the cluster state for current in progress snapshots if (matchedEntry.isPresent() == false) { matchedEntry = currentSnapshots(repositoryName, Collections.emptyList()).stream() diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index acec53da24b..e51af79df99 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -1184,7 +1184,7 @@ public class IndexShardTests extends ESSingleNodeTestCase { test_target_shard.markAsRecovering("store", new RecoveryState(routing.shardId(), routing.primary(), RecoveryState.Type.SNAPSHOT, routing.restoreSource(), localNode)); assertTrue(test_target_shard.restoreFromRepository(new RestoreOnlyRepository() { @Override - public void restore(IndexShard shard, SnapshotId snapshotId, Version version, ShardId snapshotShardId, RecoveryState recoveryState) { + public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, ShardId snapshotShardId, RecoveryState recoveryState) { try { cleanLuceneIndex(targetStore.directory()); for (String file : sourceStore.directory().listAll()) { @@ -1653,15 +1653,15 @@ public class IndexShardTests extends ESSingleNodeTestCase { @Override protected void doClose() {} @Override - public SnapshotInfo readSnapshot(SnapshotId snapshotId) { + public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) { return null; } @Override - public MetaData readSnapshotMetaData(SnapshotInfo snapshot, List indices) throws IOException { + public MetaData getSnapshotMetaData(SnapshotInfo snapshot, List indices) throws IOException { return null; } @Override - public List snapshots() { + public List getSnapshots() { return null; } @Override @@ -1673,11 +1673,11 @@ public class IndexShardTests extends ESSingleNodeTestCase { @Override public void deleteSnapshot(SnapshotId snapshotId) {} @Override - public long snapshotThrottleTimeInNanos() { + public long getSnapshotThrottleTimeInNanos() { return 0; } @Override - public long restoreThrottleTimeInNanos() { + public long getRestoreThrottleTimeInNanos() { return 0; } @Override @@ -1687,13 +1687,13 @@ public class IndexShardTests extends ESSingleNodeTestCase { @Override public void endVerification(String verificationToken) {} @Override - public boolean readOnly() { + public boolean isReadOnly() { return false; } @Override - public void snapshot(IndexShard shard, SnapshotId snapshotId, IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) {} + public void snapshotShard(IndexShard shard, SnapshotId snapshotId, IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) {} @Override - public IndexShardSnapshotStatus snapshotStatus(SnapshotId snapshotId, Version version, ShardId shardId) { + public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Version version, ShardId shardId) { return null; } @Override diff --git a/core/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java b/core/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java index 17e2481b7d0..c9d8ff81aa2 100644 --- a/core/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java +++ b/core/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java @@ -100,7 +100,7 @@ public class BlobStoreRepositoryTests extends ESSingleNodeTestCase { (BlobStoreRepository) repositoriesService.repository(repositoryName); final List originalSnapshots = Arrays.asList(snapshotId1, snapshotId2); - List snapshotIds = repository.snapshots().stream() + List snapshotIds = repository.getSnapshots().stream() .sorted((s1, s2) -> s1.getName().compareTo(s2.getName())) .collect(Collectors.toList()); assertThat(snapshotIds, equalTo(originalSnapshots)); @@ -110,9 +110,9 @@ public class BlobStoreRepositoryTests extends ESSingleNodeTestCase { final BlobStoreRepository repository = setupRepo(); // write to and read from a snapshot file with no entries - assertThat(repository.snapshots().size(), equalTo(0)); + assertThat(repository.getSnapshots().size(), equalTo(0)); repository.writeSnapshotsToIndexGen(Collections.emptyList()); - assertThat(repository.snapshots().size(), equalTo(0)); + assertThat(repository.getSnapshots().size(), equalTo(0)); // write to and read from a snapshot file with a random number of entries final int numSnapshots = randomIntBetween(1, 1000); @@ -121,7 +121,7 @@ public class BlobStoreRepositoryTests extends ESSingleNodeTestCase { snapshotIds.add(new SnapshotId(randomAsciiOfLength(8), UUIDs.randomBase64UUID())); } repository.writeSnapshotsToIndexGen(snapshotIds); - assertThat(repository.snapshots(), equalTo(snapshotIds)); + assertThat(repository.getSnapshots(), equalTo(snapshotIds)); } public void testIndexGenerationalFiles() throws Exception { @@ -165,7 +165,7 @@ public class BlobStoreRepositoryTests extends ESSingleNodeTestCase { snapshotIds.add(new SnapshotId(randomAsciiOfLength(8), SnapshotId.UNASSIGNED_UUID)); } writeOldFormat(repository, snapshotIds.stream().map(SnapshotId::getName).collect(Collectors.toList())); - assertThat(Sets.newHashSet(repository.snapshots()), equalTo(Sets.newHashSet(snapshotIds))); + assertThat(Sets.newHashSet(repository.getSnapshots()), equalTo(Sets.newHashSet(snapshotIds))); // write to and read from a snapshot file with a random number of new entries added final int numSnapshots = randomIntBetween(1, 1000); @@ -173,7 +173,7 @@ public class BlobStoreRepositoryTests extends ESSingleNodeTestCase { snapshotIds.add(new SnapshotId(randomAsciiOfLength(8), UUIDs.randomBase64UUID())); } repository.writeSnapshotsToIndexGen(snapshotIds); - assertThat(Sets.newHashSet(repository.snapshots()), equalTo(Sets.newHashSet(snapshotIds))); + assertThat(Sets.newHashSet(repository.getSnapshots()), equalTo(Sets.newHashSet(snapshotIds))); } public void testBlobId() { diff --git a/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java b/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java index f6049002852..fb52d0f634b 100644 --- a/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -1397,8 +1397,8 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas long snapshotPause = 0L; long restorePause = 0L; for (RepositoriesService repositoriesService : internalCluster().getDataNodeInstances(RepositoriesService.class)) { - snapshotPause += repositoriesService.repository("test-repo").snapshotThrottleTimeInNanos(); - restorePause += repositoriesService.repository("test-repo").restoreThrottleTimeInNanos(); + snapshotPause += repositoriesService.repository("test-repo").getSnapshotThrottleTimeInNanos(); + restorePause += repositoriesService.repository("test-repo").getRestoreThrottleTimeInNanos(); } if (throttleSnapshot) { diff --git a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepository.java b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepository.java index 47ec4128c7c..248c87ed604 100644 --- a/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepository.java +++ b/plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepository.java @@ -185,7 +185,7 @@ public class AzureRepository extends BlobStoreRepository { } @Override - public boolean readOnly() { + public boolean isReadOnly() { return readonly; } }