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:
*
* - Master calls {@link #initializeSnapshot(SnapshotId, List, org.elasticsearch.cluster.metadata.MetaData)}
* with list of indices that will be included into the snapshot
- * - Data nodes call {@link Repository#snapshot(IndexShard, SnapshotId, IndexCommit, IndexShardSnapshotStatus)}
+ *
- Data nodes call {@link Repository#snapshotShard(IndexShard, SnapshotId, IndexCommit, IndexShardSnapshotStatus)}
* for each shard
* - When all shard calls return master calls {@link #finalizeSnapshot} with possible list of failures
*
@@ -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;
}
}