Browse Source

Simplify Some Code in Snapshotting (#41316)

* Just a few obvious simplifications and dead code removals
Armin Braun 6 years ago
parent
commit
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,
                                      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.startTime = startTime;
         this.totalTime = totalTime;
@@ -82,7 +81,6 @@ public class IndexShardSnapshotStatus {
         this.totalSize = totalSize;
         this.processedSize = processedSize;
         this.incrementalSize = incrementalSize;
-        this.indexVersion = indexVersion;
         this.failure = failure;
     }
 
@@ -161,7 +159,7 @@ public class IndexShardSnapshotStatus {
     }
 
     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) {
@@ -169,7 +167,7 @@ public class IndexShardSnapshotStatus {
         if (failure == null) {
             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,
@@ -177,7 +175,7 @@ public class IndexShardSnapshotStatus {
                                                    final long incrementalSize, final long size) {
         // 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,
-            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.lucene.Lucene;
 import org.elasticsearch.common.unit.ByteSizeValue;
-import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.ToXContentFragment;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
@@ -247,9 +246,8 @@ public class BlobStoreIndexShardSnapshot implements ToXContentFragment {
          *
          * @param file    file info
          * @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.field(NAME, file.name);
             builder.field(PHYSICAL_NAME, file.metadata.name());
@@ -382,22 +380,13 @@ public class BlobStoreIndexShardSnapshot implements ToXContentFragment {
         assert indexVersion >= 0;
         this.snapshot = snapshot;
         this.indexVersion = indexVersion;
-        this.indexFiles = Collections.unmodifiableList(new ArrayList<>(indexFiles));
+        this.indexFiles = List.copyOf(indexFiles);
         this.startTime = startTime;
         this.time = time;
         this.incrementalFileCount = incrementalFileCount;
         this.incrementalSize = incrementalSize;
     }
 
-    /**
-     * Returns index version
-     *
-     * @return index version
-     */
-    public long indexVersion() {
-        return indexVersion;
-    }
-
     /**
      * Returns snapshot id
      *
@@ -493,7 +482,7 @@ public class BlobStoreIndexShardSnapshot implements ToXContentFragment {
         builder.field(INCREMENTAL_SIZE, incrementalSize);
         builder.startArray(FILES);
         for (FileInfo fileInfo : indexFiles) {
-            FileInfo.toXContent(fileInfo, builder, params);
+            FileInfo.toXContent(fileInfo, builder);
         }
         builder.endArray();
         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;
 
     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<String, FileInfo> newFiles = new HashMap<>();
         // 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 same blob
             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<>();
         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.files = unmodifiableMap(newFiles);
@@ -87,17 +82,12 @@ public class BlobStoreIndexShardSnapshots implements Iterable<SnapshotFiles>, To
         Map<String, List<FileInfo>> physicalFiles = new HashMap<>();
         for (SnapshotFiles snapshot : shardSnapshots) {
             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<>();
         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);
     }
@@ -203,7 +193,7 @@ public class BlobStoreIndexShardSnapshots implements Iterable<SnapshotFiles>, To
         // First we list all blobs with their file infos:
         builder.startArray(Fields.FILES);
         for (Map.Entry<String, FileInfo> entry : files.entrySet()) {
-            FileInfo.toXContent(entry.getValue(), builder, params);
+            FileInfo.toXContent(entry.getValue(), builder);
         }
         builder.endArray();
         // 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
      * @return information about this file
      */
-    public FileInfo findPhysicalIndexFile(String physicalName) {
+    private FileInfo findPhysicalIndexFile(String physicalName) {
         if (physicalFiles == null) {
             Map<String, FileInfo> files = new HashMap<>();
             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() + "]",
-            new AckedClusterStateUpdateTask<ClusterStateUpdateResponse>(request, registrationListener) {
+            new AckedClusterStateUpdateTask<>(request, registrationListener) {
                 @Override
                 protected ClusterStateUpdateResponse newResponse(boolean acknowledged) {
                     return new ClusterStateUpdateResponse(acknowledged);
@@ -186,7 +186,7 @@ public class RepositoriesService implements ClusterStateApplier {
      */
     public void unregisterRepository(final DeleteRepositoryRequest request, final ActionListener<ClusterStateUpdateResponse> listener) {
         clusterService.submitStateUpdateTask("delete_repository [" + request.name() + "]",
-            new AckedClusterStateUpdateTask<ClusterStateUpdateResponse>(request, listener) {
+            new AckedClusterStateUpdateTask<>(request, listener) {
                 @Override
                 protected ClusterStateUpdateResponse newResponse(boolean 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);
         Map<IndexId, Set<SnapshotId>> allIndexSnapshots = new HashMap<>(indexSnapshots);
         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);
     }
@@ -308,7 +297,7 @@ public final class RepositoryData {
      * Writes the snapshots metadata and the related indices metadata to x-content, omitting the
      * incompatible snapshots.
      */
-    public XContentBuilder snapshotsToXContent(final XContentBuilder builder, final ToXContent.Params params) throws IOException {
+    public XContentBuilder snapshotsToXContent(final XContentBuilder builder) throws IOException {
         builder.startObject();
         // write the snapshots list
         builder.startArray(SNAPSHOTS);
@@ -453,18 +442,16 @@ public final class RepositoryData {
     /**
      * 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();
         // write the incompatible snapshots list
         builder.startArray(INCOMPATIBLE_SNAPSHOTS);
         for (final SnapshotId snapshot : getIncompatibleSnapshotIds()) {
-            snapshot.toXContent(builder, params);
+            snapshot.toXContent(builder, ToXContent.EMPTY_PARAMS);
         }
         builder.endArray();
         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;
 
-    VerificationFailure() {
-
-    }
-
-    public VerificationFailure(String nodeId, Exception cause) {
+    VerificationFailure(String nodeId, Exception cause) {
         this.nodeId = nodeId;
         this.cause = cause;
     }
 
-    public String nodeId() {
-        return nodeId;
-    }
-
-    public Throwable cause() {
-        return cause;
-    }
-
     @Override
     public void readFrom(StreamInput in) throws IOException {
         nodeId = in.readOptionalString();
@@ -60,12 +48,6 @@ public class VerificationFailure implements Streamable {
         out.writeException(cause);
     }
 
-    public static VerificationFailure readNode(StreamInput in) throws IOException {
-        VerificationFailure failure = new VerificationFailure();
-        failure.readFrom(in);
-        return failure;
-    }
-
     @Override
     public String toString() {
         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 {
         try (XContentParser parser = XContentHelper
                 .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.xcontent.LoggingDeprecationHandler;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
-import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentFactory;
 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());
         for (IndexId indexId: indices) {
-            threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(new ActionRunnable<Void>(groupedListener) {
+            threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(new ActionRunnable<>(groupedListener) {
 
                 @Override
                 protected void doRun() {
@@ -516,9 +515,6 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
         }
     }
 
-    /**
-     * {@inheritDoc}
-     */
     @Override
     public SnapshotInfo finalizeSnapshot(final SnapshotId snapshotId,
                                          final List<IndexId> indices,
@@ -718,7 +714,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
         try (BytesStreamOutput bStream = new BytesStreamOutput()) {
             try (StreamOutput stream = new OutputStreamStreamOutput(bStream)) {
                 XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON, stream);
-                repositoryData.snapshotsToXContent(builder, ToXContent.EMPTY_PARAMS);
+                repositoryData.snapshotsToXContent(builder);
                 builder.close();
             }
             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
         final BytesReference bytes;
         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();
         }

+ 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);
                 try (OutputStream indexOutputOutputStream = new IndexOutputOutputStream(indexOutput) {
                     @Override
-                    public void close() throws IOException {
+                    public void 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.
                     }

+ 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 */
-    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(
             new UpdateIndexShardSnapshotStatusRequest(snapshot, shardId, status),
             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.common.bytes.BytesReference;
 import org.elasticsearch.common.unit.ByteSizeValue;
-import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.common.xcontent.XContentParser;
@@ -56,7 +55,7 @@ public class FileInfoTests extends ESTestCase {
             ByteSizeValue size = new ByteSizeValue(Math.abs(randomLong()));
             BlobStoreIndexShardSnapshot.FileInfo info = new BlobStoreIndexShardSnapshot.FileInfo("_foobar", meta, size);
             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)));
 
             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.common.UUIDs;
 import org.elasticsearch.common.bytes.BytesReference;
-import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
@@ -61,7 +60,7 @@ public class RepositoryDataTests extends ESTestCase {
     public void testXContent() throws IOException {
         RepositoryData repositoryData = generateRandomRepoData();
         XContentBuilder builder = JsonXContent.contentBuilder();
-        repositoryData.snapshotsToXContent(builder, ToXContent.EMPTY_PARAMS);
+        repositoryData.snapshotsToXContent(builder);
         try (XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder))) {
             long gen = (long) randomIntBetween(0, 500);
             RepositoryData fromXContent = RepositoryData.snapshotsFromXContent(parser, gen);
@@ -166,7 +165,7 @@ public class RepositoryDataTests extends ESTestCase {
         final RepositoryData repositoryData = generateRandomRepoData();
 
         XContentBuilder builder = XContentBuilder.builder(xContent);
-        repositoryData.snapshotsToXContent(builder, ToXContent.EMPTY_PARAMS);
+        repositoryData.snapshotsToXContent(builder);
         RepositoryData parsedRepositoryData;
         try (XContentParser xParser = createParser(builder)) {
             parsedRepositoryData = RepositoryData.snapshotsFromXContent(xParser, repositoryData.getGenId());
@@ -197,7 +196,7 @@ public class RepositoryDataTests extends ESTestCase {
             indexSnapshots, new ArrayList<>(parsedRepositoryData.getIncompatibleSnapshotIds()));
 
         final XContentBuilder corruptedBuilder = XContentBuilder.builder(xContent);
-        corruptedRepositoryData.snapshotsToXContent(corruptedBuilder, ToXContent.EMPTY_PARAMS);
+        corruptedRepositoryData.snapshotsToXContent(corruptedBuilder);
 
         try (XContentParser xParser = createParser(corruptedBuilder)) {
             ElasticsearchParseException e = expectThrows(ElasticsearchParseException.class, () ->