浏览代码

Simplify Some Code in Snapshotting (#41316)

* Just a few obvious simplifications and dead code removals
Armin Braun 6 年之前
父节点
当前提交
a0c4eabe8d

+ 4 - 6
server/src/main/java/org/elasticsearch/index/snapshots/IndexShardSnapshotStatus.java

@@ -71,8 +71,7 @@ public class IndexShardSnapshotStatus {
 
 
     private IndexShardSnapshotStatus(final Stage stage, final long startTime, final long totalTime,
     private IndexShardSnapshotStatus(final Stage stage, final long startTime, final long totalTime,
                                      final int incrementalFileCount, final int totalFileCount, final int processedFileCount,
                                      final int incrementalFileCount, final int totalFileCount, final int processedFileCount,
-                                     final long incrementalSize, final long totalSize, final long processedSize,
-                                     final long indexVersion, final String failure) {
+                                     final long incrementalSize, final long totalSize, final long processedSize, final String failure) {
         this.stage = new AtomicReference<>(Objects.requireNonNull(stage));
         this.stage = new AtomicReference<>(Objects.requireNonNull(stage));
         this.startTime = startTime;
         this.startTime = startTime;
         this.totalTime = totalTime;
         this.totalTime = totalTime;
@@ -82,7 +81,6 @@ public class IndexShardSnapshotStatus {
         this.totalSize = totalSize;
         this.totalSize = totalSize;
         this.processedSize = processedSize;
         this.processedSize = processedSize;
         this.incrementalSize = incrementalSize;
         this.incrementalSize = incrementalSize;
-        this.indexVersion = indexVersion;
         this.failure = failure;
         this.failure = failure;
     }
     }
 
 
@@ -161,7 +159,7 @@ public class IndexShardSnapshotStatus {
     }
     }
 
 
     public static IndexShardSnapshotStatus newInitializing() {
     public static IndexShardSnapshotStatus newInitializing() {
-        return new IndexShardSnapshotStatus(Stage.INIT, 0L, 0L, 0, 0, 0, 0, 0, 0, 0, null);
+        return new IndexShardSnapshotStatus(Stage.INIT, 0L, 0L, 0, 0, 0, 0, 0, 0, null);
     }
     }
 
 
     public static IndexShardSnapshotStatus newFailed(final String failure) {
     public static IndexShardSnapshotStatus newFailed(final String failure) {
@@ -169,7 +167,7 @@ public class IndexShardSnapshotStatus {
         if (failure == null) {
         if (failure == null) {
             throw new IllegalArgumentException("A failure description is required for a failed IndexShardSnapshotStatus");
             throw new IllegalArgumentException("A failure description is required for a failed IndexShardSnapshotStatus");
         }
         }
-        return new IndexShardSnapshotStatus(Stage.FAILURE, 0L, 0L, 0, 0, 0, 0, 0, 0, 0, failure);
+        return new IndexShardSnapshotStatus(Stage.FAILURE, 0L, 0L, 0, 0, 0, 0, 0, 0, failure);
     }
     }
 
 
     public static IndexShardSnapshotStatus newDone(final long startTime, final long totalTime,
     public static IndexShardSnapshotStatus newDone(final long startTime, final long totalTime,
@@ -177,7 +175,7 @@ public class IndexShardSnapshotStatus {
                                                    final long incrementalSize, final long size) {
                                                    final long incrementalSize, final long size) {
         // The snapshot is done which means the number of processed files is the same as total
         // The snapshot is done which means the number of processed files is the same as total
         return new IndexShardSnapshotStatus(Stage.DONE, startTime, totalTime, incrementalFileCount, fileCount, incrementalFileCount,
         return new IndexShardSnapshotStatus(Stage.DONE, startTime, totalTime, incrementalFileCount, fileCount, incrementalFileCount,
-            incrementalSize, size, incrementalSize, 0, null);
+            incrementalSize, size, incrementalSize, null);
     }
     }
 
 
     /**
     /**

+ 3 - 14
server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java

@@ -26,7 +26,6 @@ import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.lucene.Lucene;
 import org.elasticsearch.common.lucene.Lucene;
 import org.elasticsearch.common.unit.ByteSizeValue;
 import org.elasticsearch.common.unit.ByteSizeValue;
-import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.ToXContentFragment;
 import org.elasticsearch.common.xcontent.ToXContentFragment;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.XContentParser;
@@ -247,9 +246,8 @@ public class BlobStoreIndexShardSnapshot implements ToXContentFragment {
          *
          *
          * @param file    file info
          * @param file    file info
          * @param builder XContent builder
          * @param builder XContent builder
-         * @param params  parameters
          */
          */
-        public static void toXContent(FileInfo file, XContentBuilder builder, ToXContent.Params params) throws IOException {
+        public static void toXContent(FileInfo file, XContentBuilder builder) throws IOException {
             builder.startObject();
             builder.startObject();
             builder.field(NAME, file.name);
             builder.field(NAME, file.name);
             builder.field(PHYSICAL_NAME, file.metadata.name());
             builder.field(PHYSICAL_NAME, file.metadata.name());
@@ -382,22 +380,13 @@ public class BlobStoreIndexShardSnapshot implements ToXContentFragment {
         assert indexVersion >= 0;
         assert indexVersion >= 0;
         this.snapshot = snapshot;
         this.snapshot = snapshot;
         this.indexVersion = indexVersion;
         this.indexVersion = indexVersion;
-        this.indexFiles = Collections.unmodifiableList(new ArrayList<>(indexFiles));
+        this.indexFiles = List.copyOf(indexFiles);
         this.startTime = startTime;
         this.startTime = startTime;
         this.time = time;
         this.time = time;
         this.incrementalFileCount = incrementalFileCount;
         this.incrementalFileCount = incrementalFileCount;
         this.incrementalSize = incrementalSize;
         this.incrementalSize = incrementalSize;
     }
     }
 
 
-    /**
-     * Returns index version
-     *
-     * @return index version
-     */
-    public long indexVersion() {
-        return indexVersion;
-    }
-
     /**
     /**
      * Returns snapshot id
      * Returns snapshot id
      *
      *
@@ -493,7 +482,7 @@ public class BlobStoreIndexShardSnapshot implements ToXContentFragment {
         builder.field(INCREMENTAL_SIZE, incrementalSize);
         builder.field(INCREMENTAL_SIZE, incrementalSize);
         builder.startArray(FILES);
         builder.startArray(FILES);
         for (FileInfo fileInfo : indexFiles) {
         for (FileInfo fileInfo : indexFiles) {
-            FileInfo.toXContent(fileInfo, builder, params);
+            FileInfo.toXContent(fileInfo, builder);
         }
         }
         builder.endArray();
         builder.endArray();
         return builder;
         return builder;

+ 6 - 16
server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java

@@ -49,7 +49,7 @@ public class BlobStoreIndexShardSnapshots implements Iterable<SnapshotFiles>, To
     private final Map<String, List<FileInfo>> physicalFiles;
     private final Map<String, List<FileInfo>> physicalFiles;
 
 
     public BlobStoreIndexShardSnapshots(List<SnapshotFiles> shardSnapshots) {
     public BlobStoreIndexShardSnapshots(List<SnapshotFiles> shardSnapshots) {
-        this.shardSnapshots = Collections.unmodifiableList(new ArrayList<>(shardSnapshots));
+        this.shardSnapshots = List.copyOf(shardSnapshots);
         // Map between blob names and file info
         // Map between blob names and file info
         Map<String, FileInfo> newFiles = new HashMap<>();
         Map<String, FileInfo> newFiles = new HashMap<>();
         // Map between original physical names and file info
         // Map between original physical names and file info
@@ -65,17 +65,12 @@ public class BlobStoreIndexShardSnapshots implements Iterable<SnapshotFiles>, To
             // the first loop de-duplicates fileInfo objects that were loaded from different snapshots but refer to
             // the first loop de-duplicates fileInfo objects that were loaded from different snapshots but refer to
             // the same blob
             // the same blob
             for (FileInfo fileInfo : snapshot.indexFiles()) {
             for (FileInfo fileInfo : snapshot.indexFiles()) {
-                List<FileInfo> physicalFileList = physicalFiles.get(fileInfo.physicalName());
-                if (physicalFileList == null) {
-                    physicalFileList = new ArrayList<>();
-                    physicalFiles.put(fileInfo.physicalName(), physicalFileList);
-                }
-                physicalFileList.add(newFiles.get(fileInfo.name()));
+                physicalFiles.computeIfAbsent(fileInfo.physicalName(), k -> new ArrayList<>()).add(newFiles.get(fileInfo.name()));
             }
             }
         }
         }
         Map<String, List<FileInfo>> mapBuilder = new HashMap<>();
         Map<String, List<FileInfo>> mapBuilder = new HashMap<>();
         for (Map.Entry<String, List<FileInfo>> entry : physicalFiles.entrySet()) {
         for (Map.Entry<String, List<FileInfo>> entry : physicalFiles.entrySet()) {
-            mapBuilder.put(entry.getKey(), Collections.unmodifiableList(new ArrayList<>(entry.getValue())));
+            mapBuilder.put(entry.getKey(), List.copyOf(entry.getValue()));
         }
         }
         this.physicalFiles = unmodifiableMap(mapBuilder);
         this.physicalFiles = unmodifiableMap(mapBuilder);
         this.files = unmodifiableMap(newFiles);
         this.files = unmodifiableMap(newFiles);
@@ -87,17 +82,12 @@ public class BlobStoreIndexShardSnapshots implements Iterable<SnapshotFiles>, To
         Map<String, List<FileInfo>> physicalFiles = new HashMap<>();
         Map<String, List<FileInfo>> physicalFiles = new HashMap<>();
         for (SnapshotFiles snapshot : shardSnapshots) {
         for (SnapshotFiles snapshot : shardSnapshots) {
             for (FileInfo fileInfo : snapshot.indexFiles()) {
             for (FileInfo fileInfo : snapshot.indexFiles()) {
-                List<FileInfo> physicalFileList = physicalFiles.get(fileInfo.physicalName());
-                if (physicalFileList == null) {
-                    physicalFileList = new ArrayList<>();
-                    physicalFiles.put(fileInfo.physicalName(), physicalFileList);
-                }
-                physicalFileList.add(files.get(fileInfo.name()));
+                physicalFiles.computeIfAbsent(fileInfo.physicalName(), k -> new ArrayList<>()).add(files.get(fileInfo.name()));
             }
             }
         }
         }
         Map<String, List<FileInfo>> mapBuilder = new HashMap<>();
         Map<String, List<FileInfo>> mapBuilder = new HashMap<>();
         for (Map.Entry<String, List<FileInfo>> entry : physicalFiles.entrySet()) {
         for (Map.Entry<String, List<FileInfo>> entry : physicalFiles.entrySet()) {
-            mapBuilder.put(entry.getKey(), Collections.unmodifiableList(new ArrayList<>(entry.getValue())));
+            mapBuilder.put(entry.getKey(), List.copyOf(entry.getValue()));
         }
         }
         this.physicalFiles = unmodifiableMap(mapBuilder);
         this.physicalFiles = unmodifiableMap(mapBuilder);
     }
     }
@@ -203,7 +193,7 @@ public class BlobStoreIndexShardSnapshots implements Iterable<SnapshotFiles>, To
         // First we list all blobs with their file infos:
         // First we list all blobs with their file infos:
         builder.startArray(Fields.FILES);
         builder.startArray(Fields.FILES);
         for (Map.Entry<String, FileInfo> entry : files.entrySet()) {
         for (Map.Entry<String, FileInfo> entry : files.entrySet()) {
-            FileInfo.toXContent(entry.getValue(), builder, params);
+            FileInfo.toXContent(entry.getValue(), builder);
         }
         }
         builder.endArray();
         builder.endArray();
         // Then we list all snapshots with list of all blobs that are used by the snapshot
         // Then we list all snapshots with list of all blobs that are used by the snapshot

+ 1 - 1
server/src/main/java/org/elasticsearch/index/snapshots/blobstore/SnapshotFiles.java

@@ -66,7 +66,7 @@ public class SnapshotFiles {
      * @param physicalName the original file name
      * @param physicalName the original file name
      * @return information about this file
      * @return information about this file
      */
      */
-    public FileInfo findPhysicalIndexFile(String physicalName) {
+    private FileInfo findPhysicalIndexFile(String physicalName) {
         if (physicalFiles == null) {
         if (physicalFiles == null) {
             Map<String, FileInfo> files = new HashMap<>();
             Map<String, FileInfo> files = new HashMap<>();
             for(FileInfo fileInfo : indexFiles) {
             for(FileInfo fileInfo : indexFiles) {

+ 2 - 2
server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java

@@ -119,7 +119,7 @@ public class RepositoriesService implements ClusterStateApplier {
         }
         }
 
 
         clusterService.submitStateUpdateTask("put_repository [" + request.name() + "]",
         clusterService.submitStateUpdateTask("put_repository [" + request.name() + "]",
-            new AckedClusterStateUpdateTask<ClusterStateUpdateResponse>(request, registrationListener) {
+            new AckedClusterStateUpdateTask<>(request, registrationListener) {
                 @Override
                 @Override
                 protected ClusterStateUpdateResponse newResponse(boolean acknowledged) {
                 protected ClusterStateUpdateResponse newResponse(boolean acknowledged) {
                     return new ClusterStateUpdateResponse(acknowledged);
                     return new ClusterStateUpdateResponse(acknowledged);
@@ -186,7 +186,7 @@ public class RepositoriesService implements ClusterStateApplier {
      */
      */
     public void unregisterRepository(final DeleteRepositoryRequest request, final ActionListener<ClusterStateUpdateResponse> listener) {
     public void unregisterRepository(final DeleteRepositoryRequest request, final ActionListener<ClusterStateUpdateResponse> listener) {
         clusterService.submitStateUpdateTask("delete_repository [" + request.name() + "]",
         clusterService.submitStateUpdateTask("delete_repository [" + request.name() + "]",
-            new AckedClusterStateUpdateTask<ClusterStateUpdateResponse>(request, listener) {
+            new AckedClusterStateUpdateTask<>(request, listener) {
                 @Override
                 @Override
                 protected ClusterStateUpdateResponse newResponse(boolean acknowledged) {
                 protected ClusterStateUpdateResponse newResponse(boolean acknowledged) {
                     return new ClusterStateUpdateResponse(acknowledged);
                     return new ClusterStateUpdateResponse(acknowledged);

+ 4 - 17
server/src/main/java/org/elasticsearch/repositories/RepositoryData.java

@@ -169,18 +169,7 @@ public final class RepositoryData {
         newSnapshotStates.put(snapshotId.getUUID(), snapshotState);
         newSnapshotStates.put(snapshotId.getUUID(), snapshotState);
         Map<IndexId, Set<SnapshotId>> allIndexSnapshots = new HashMap<>(indexSnapshots);
         Map<IndexId, Set<SnapshotId>> allIndexSnapshots = new HashMap<>(indexSnapshots);
         for (final IndexId indexId : snapshottedIndices) {
         for (final IndexId indexId : snapshottedIndices) {
-            if (allIndexSnapshots.containsKey(indexId)) {
-                Set<SnapshotId> ids = allIndexSnapshots.get(indexId);
-                if (ids == null) {
-                    ids = new LinkedHashSet<>();
-                    allIndexSnapshots.put(indexId, ids);
-                }
-                ids.add(snapshotId);
-            } else {
-                Set<SnapshotId> ids = new LinkedHashSet<>();
-                ids.add(snapshotId);
-                allIndexSnapshots.put(indexId, ids);
-            }
+            allIndexSnapshots.computeIfAbsent(indexId, k -> new LinkedHashSet<>()).add(snapshotId);
         }
         }
         return new RepositoryData(genId, snapshots, newSnapshotStates, allIndexSnapshots, incompatibleSnapshotIds);
         return new RepositoryData(genId, snapshots, newSnapshotStates, allIndexSnapshots, incompatibleSnapshotIds);
     }
     }
@@ -308,7 +297,7 @@ public final class RepositoryData {
      * Writes the snapshots metadata and the related indices metadata to x-content, omitting the
      * Writes the snapshots metadata and the related indices metadata to x-content, omitting the
      * incompatible snapshots.
      * incompatible snapshots.
      */
      */
-    public XContentBuilder snapshotsToXContent(final XContentBuilder builder, final ToXContent.Params params) throws IOException {
+    public XContentBuilder snapshotsToXContent(final XContentBuilder builder) throws IOException {
         builder.startObject();
         builder.startObject();
         // write the snapshots list
         // write the snapshots list
         builder.startArray(SNAPSHOTS);
         builder.startArray(SNAPSHOTS);
@@ -453,18 +442,16 @@ public final class RepositoryData {
     /**
     /**
      * Writes the incompatible snapshot ids to x-content.
      * Writes the incompatible snapshot ids to x-content.
      */
      */
-    public XContentBuilder incompatibleSnapshotsToXContent(final XContentBuilder builder, final ToXContent.Params params)
-        throws IOException {
+    public void incompatibleSnapshotsToXContent(final XContentBuilder builder) throws IOException {
 
 
         builder.startObject();
         builder.startObject();
         // write the incompatible snapshots list
         // write the incompatible snapshots list
         builder.startArray(INCOMPATIBLE_SNAPSHOTS);
         builder.startArray(INCOMPATIBLE_SNAPSHOTS);
         for (final SnapshotId snapshot : getIncompatibleSnapshotIds()) {
         for (final SnapshotId snapshot : getIncompatibleSnapshotIds()) {
-            snapshot.toXContent(builder, params);
+            snapshot.toXContent(builder, ToXContent.EMPTY_PARAMS);
         }
         }
         builder.endArray();
         builder.endArray();
         builder.endObject();
         builder.endObject();
-        return builder;
     }
     }
 
 
     /**
     /**

+ 1 - 19
server/src/main/java/org/elasticsearch/repositories/VerificationFailure.java

@@ -31,23 +31,11 @@ public class VerificationFailure implements Streamable {
 
 
     private Exception cause;
     private Exception cause;
 
 
-    VerificationFailure() {
-
-    }
-
-    public VerificationFailure(String nodeId, Exception cause) {
+    VerificationFailure(String nodeId, Exception cause) {
         this.nodeId = nodeId;
         this.nodeId = nodeId;
         this.cause = cause;
         this.cause = cause;
     }
     }
 
 
-    public String nodeId() {
-        return nodeId;
-    }
-
-    public Throwable cause() {
-        return cause;
-    }
-
     @Override
     @Override
     public void readFrom(StreamInput in) throws IOException {
     public void readFrom(StreamInput in) throws IOException {
         nodeId = in.readOptionalString();
         nodeId = in.readOptionalString();
@@ -60,12 +48,6 @@ public class VerificationFailure implements Streamable {
         out.writeException(cause);
         out.writeException(cause);
     }
     }
 
 
-    public static VerificationFailure readNode(StreamInput in) throws IOException {
-        VerificationFailure failure = new VerificationFailure();
-        failure.readFrom(in);
-        return failure;
-    }
-
     @Override
     @Override
     public String toString() {
     public String toString() {
         return "[" + nodeId + ", '" + cause + "']";
         return "[" + nodeId + ", '" + cause + "']";

+ 1 - 2
server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreFormat.java

@@ -111,8 +111,7 @@ public abstract class BlobStoreFormat<T extends ToXContent> {
     protected T read(BytesReference bytes) throws IOException {
     protected T read(BytesReference bytes) throws IOException {
         try (XContentParser parser = XContentHelper
         try (XContentParser parser = XContentHelper
                 .createParser(namedXContentRegistry, LoggingDeprecationHandler.INSTANCE, bytes)) {
                 .createParser(namedXContentRegistry, LoggingDeprecationHandler.INSTANCE, bytes)) {
-            T obj = reader.apply(parser);
-            return obj;
+            return reader.apply(parser);
         }
         }
     }
     }
 
 

+ 5 - 10
server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java

@@ -65,7 +65,6 @@ import org.elasticsearch.common.unit.ByteSizeValue;
 import org.elasticsearch.common.util.set.Sets;
 import org.elasticsearch.common.util.set.Sets;
 import org.elasticsearch.common.xcontent.LoggingDeprecationHandler;
 import org.elasticsearch.common.xcontent.LoggingDeprecationHandler;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
-import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.common.xcontent.XContentHelper;
 import org.elasticsearch.common.xcontent.XContentHelper;
@@ -476,7 +475,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
         }
         }
         final ActionListener<Void> groupedListener = new GroupedActionListener<>(ActionListener.map(listener, v -> null), indices.size());
         final ActionListener<Void> groupedListener = new GroupedActionListener<>(ActionListener.map(listener, v -> null), indices.size());
         for (IndexId indexId: indices) {
         for (IndexId indexId: indices) {
-            threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(new ActionRunnable<Void>(groupedListener) {
+            threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(new ActionRunnable<>(groupedListener) {
 
 
                 @Override
                 @Override
                 protected void doRun() {
                 protected void doRun() {
@@ -516,9 +515,6 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
         }
         }
     }
     }
 
 
-    /**
-     * {@inheritDoc}
-     */
     @Override
     @Override
     public SnapshotInfo finalizeSnapshot(final SnapshotId snapshotId,
     public SnapshotInfo finalizeSnapshot(final SnapshotId snapshotId,
                                          final List<IndexId> indices,
                                          final List<IndexId> indices,
@@ -718,7 +714,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
         try (BytesStreamOutput bStream = new BytesStreamOutput()) {
         try (BytesStreamOutput bStream = new BytesStreamOutput()) {
             try (StreamOutput stream = new OutputStreamStreamOutput(bStream)) {
             try (StreamOutput stream = new OutputStreamStreamOutput(bStream)) {
                 XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON, stream);
                 XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON, stream);
-                repositoryData.snapshotsToXContent(builder, ToXContent.EMPTY_PARAMS);
+                repositoryData.snapshotsToXContent(builder);
                 builder.close();
                 builder.close();
             }
             }
             snapshotsBytes = bStream.bytes();
             snapshotsBytes = bStream.bytes();
@@ -752,10 +748,9 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
         assert isReadOnly() == false; // can not write to a read only repository
         assert isReadOnly() == false; // can not write to a read only repository
         final BytesReference bytes;
         final BytesReference bytes;
         try (BytesStreamOutput bStream = new BytesStreamOutput()) {
         try (BytesStreamOutput bStream = new BytesStreamOutput()) {
-            try (StreamOutput stream = new OutputStreamStreamOutput(bStream)) {
-                XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON, stream);
-                repositoryData.incompatibleSnapshotsToXContent(builder, ToXContent.EMPTY_PARAMS);
-                builder.close();
+            try (StreamOutput stream = new OutputStreamStreamOutput(bStream);
+                 XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON, stream)) {
+                repositoryData.incompatibleSnapshotsToXContent(builder);
             }
             }
             bytes = bStream.bytes();
             bytes = bStream.bytes();
         }
         }

+ 1 - 1
server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java

@@ -163,7 +163,7 @@ public class ChecksumBlobStoreFormat<T extends ToXContent> extends BlobStoreForm
                 CodecUtil.writeHeader(indexOutput, codec, VERSION);
                 CodecUtil.writeHeader(indexOutput, codec, VERSION);
                 try (OutputStream indexOutputOutputStream = new IndexOutputOutputStream(indexOutput) {
                 try (OutputStream indexOutputOutputStream = new IndexOutputOutputStream(indexOutput) {
                     @Override
                     @Override
-                    public void close() throws IOException {
+                    public void close() {
                         // this is important since some of the XContentBuilders write bytes on close.
                         // this is important since some of the XContentBuilders write bytes on close.
                         // in order to write the footer we need to prevent closing the actual index input.
                         // in order to write the footer we need to prevent closing the actual index input.
                     }
                     }

+ 1 - 1
server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java

@@ -489,7 +489,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements
     }
     }
 
 
     /** Updates the shard snapshot status by sending a {@link UpdateIndexShardSnapshotStatusRequest} to the master node */
     /** Updates the shard snapshot status by sending a {@link UpdateIndexShardSnapshotStatusRequest} to the master node */
-    void sendSnapshotShardUpdate(final Snapshot snapshot, final ShardId shardId, final ShardSnapshotStatus status) {
+    private void sendSnapshotShardUpdate(final Snapshot snapshot, final ShardId shardId, final ShardSnapshotStatus status) {
         remoteFailedRequestDeduplicator.executeOnce(
         remoteFailedRequestDeduplicator.executeOnce(
             new UpdateIndexShardSnapshotStatusRequest(snapshot, shardId, status),
             new UpdateIndexShardSnapshotStatusRequest(snapshot, shardId, status),
             new ActionListener<>() {
             new ActionListener<>() {

+ 1 - 2
server/src/test/java/org/elasticsearch/index/snapshots/blobstore/FileInfoTests.java

@@ -23,7 +23,6 @@ import org.apache.lucene.util.Version;
 import org.elasticsearch.ElasticsearchParseException;
 import org.elasticsearch.ElasticsearchParseException;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.unit.ByteSizeValue;
 import org.elasticsearch.common.unit.ByteSizeValue;
-import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.XContentParser;
@@ -56,7 +55,7 @@ public class FileInfoTests extends ESTestCase {
             ByteSizeValue size = new ByteSizeValue(Math.abs(randomLong()));
             ByteSizeValue size = new ByteSizeValue(Math.abs(randomLong()));
             BlobStoreIndexShardSnapshot.FileInfo info = new BlobStoreIndexShardSnapshot.FileInfo("_foobar", meta, size);
             BlobStoreIndexShardSnapshot.FileInfo info = new BlobStoreIndexShardSnapshot.FileInfo("_foobar", meta, size);
             XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON).prettyPrint();
             XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON).prettyPrint();
-            BlobStoreIndexShardSnapshot.FileInfo.toXContent(info, builder, ToXContent.EMPTY_PARAMS);
+            BlobStoreIndexShardSnapshot.FileInfo.toXContent(info, builder);
             byte[] xcontent = BytesReference.toBytes(BytesReference.bytes(shuffleXContent(builder)));
             byte[] xcontent = BytesReference.toBytes(BytesReference.bytes(shuffleXContent(builder)));
 
 
             final BlobStoreIndexShardSnapshot.FileInfo parsedInfo;
             final BlobStoreIndexShardSnapshot.FileInfo parsedInfo;

+ 3 - 4
server/src/test/java/org/elasticsearch/repositories/RepositoryDataTests.java

@@ -22,7 +22,6 @@ package org.elasticsearch.repositories;
 import org.elasticsearch.ElasticsearchParseException;
 import org.elasticsearch.ElasticsearchParseException;
 import org.elasticsearch.common.UUIDs;
 import org.elasticsearch.common.UUIDs;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.bytes.BytesReference;
-import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContent;
 import org.elasticsearch.common.xcontent.XContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.XContentParser;
@@ -61,7 +60,7 @@ public class RepositoryDataTests extends ESTestCase {
     public void testXContent() throws IOException {
     public void testXContent() throws IOException {
         RepositoryData repositoryData = generateRandomRepoData();
         RepositoryData repositoryData = generateRandomRepoData();
         XContentBuilder builder = JsonXContent.contentBuilder();
         XContentBuilder builder = JsonXContent.contentBuilder();
-        repositoryData.snapshotsToXContent(builder, ToXContent.EMPTY_PARAMS);
+        repositoryData.snapshotsToXContent(builder);
         try (XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder))) {
         try (XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder))) {
             long gen = (long) randomIntBetween(0, 500);
             long gen = (long) randomIntBetween(0, 500);
             RepositoryData fromXContent = RepositoryData.snapshotsFromXContent(parser, gen);
             RepositoryData fromXContent = RepositoryData.snapshotsFromXContent(parser, gen);
@@ -166,7 +165,7 @@ public class RepositoryDataTests extends ESTestCase {
         final RepositoryData repositoryData = generateRandomRepoData();
         final RepositoryData repositoryData = generateRandomRepoData();
 
 
         XContentBuilder builder = XContentBuilder.builder(xContent);
         XContentBuilder builder = XContentBuilder.builder(xContent);
-        repositoryData.snapshotsToXContent(builder, ToXContent.EMPTY_PARAMS);
+        repositoryData.snapshotsToXContent(builder);
         RepositoryData parsedRepositoryData;
         RepositoryData parsedRepositoryData;
         try (XContentParser xParser = createParser(builder)) {
         try (XContentParser xParser = createParser(builder)) {
             parsedRepositoryData = RepositoryData.snapshotsFromXContent(xParser, repositoryData.getGenId());
             parsedRepositoryData = RepositoryData.snapshotsFromXContent(xParser, repositoryData.getGenId());
@@ -197,7 +196,7 @@ public class RepositoryDataTests extends ESTestCase {
             indexSnapshots, new ArrayList<>(parsedRepositoryData.getIncompatibleSnapshotIds()));
             indexSnapshots, new ArrayList<>(parsedRepositoryData.getIncompatibleSnapshotIds()));
 
 
         final XContentBuilder corruptedBuilder = XContentBuilder.builder(xContent);
         final XContentBuilder corruptedBuilder = XContentBuilder.builder(xContent);
-        corruptedRepositoryData.snapshotsToXContent(corruptedBuilder, ToXContent.EMPTY_PARAMS);
+        corruptedRepositoryData.snapshotsToXContent(corruptedBuilder);
 
 
         try (XContentParser xParser = createParser(corruptedBuilder)) {
         try (XContentParser xParser = createParser(corruptedBuilder)) {
             ElasticsearchParseException e = expectThrows(ElasticsearchParseException.class, () ->
             ElasticsearchParseException e = expectThrows(ElasticsearchParseException.class, () ->