Browse Source

Use Pooled Byte Arrays in BlobStoreRepository Serialization (#63461)

Many of the metadata blobs we handle in the changed spots can grow
up in size up to `O(1M)`. Not using recycled bytes when working with
them causes significant spikes in memory use for larger repositories.
Armin Braun 5 years ago
parent
commit
c419cd3251
43 changed files with 277 additions and 141 deletions
  1. 4 2
      modules/repository-url/src/main/java/org/elasticsearch/plugin/repository/url/URLRepositoryPlugin.java
  2. 3 2
      modules/repository-url/src/main/java/org/elasticsearch/repositories/url/URLRepository.java
  3. 2 0
      modules/repository-url/src/test/java/org/elasticsearch/repositories/url/URLRepositoryTests.java
  4. 3 0
      plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepository.java
  5. 4 2
      plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepositoryPlugin.java
  6. 2 0
      plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureRepositorySettingsTests.java
  7. 5 2
      plugins/repository-gcs/src/internalClusterTest/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStoreRepositoryTests.java
  8. 4 2
      plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStoragePlugin.java
  9. 4 1
      plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageRepository.java
  10. 4 2
      plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsPlugin.java
  11. 3 2
      plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java
  12. 3 2
      plugins/repository-s3/src/internalClusterTest/java/org/elasticsearch/repositories/s3/S3BlobStoreRepositoryTests.java
  13. 3 0
      plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java
  14. 6 3
      plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3RepositoryPlugin.java
  15. 3 2
      plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/RepositoryCredentialsTests.java
  16. 3 1
      plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RepositoryTests.java
  17. 2 1
      server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java
  18. 6 4
      server/src/internalClusterTest/java/org/elasticsearch/snapshots/MetadataLoadingDuringSnapshotRestoreIT.java
  19. 4 2
      server/src/internalClusterTest/java/org/elasticsearch/snapshots/RepositoryFilterUserMetadataIT.java
  20. 22 0
      server/src/main/java/org/elasticsearch/common/io/Streams.java
  21. 1 1
      server/src/main/java/org/elasticsearch/node/Node.java
  22. 3 1
      server/src/main/java/org/elasticsearch/plugins/RepositoryPlugin.java
  23. 5 5
      server/src/main/java/org/elasticsearch/repositories/RepositoriesModule.java
  24. 74 48
      server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java
  25. 21 11
      server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java
  26. 3 1
      server/src/main/java/org/elasticsearch/repositories/blobstore/MeteredBlobStoreRepository.java
  27. 3 2
      server/src/main/java/org/elasticsearch/repositories/fs/FsRepository.java
  28. 16 13
      server/src/test/java/org/elasticsearch/repositories/RepositoriesModuleTests.java
  29. 3 0
      server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java
  30. 2 0
      server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java
  31. 4 2
      server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java
  32. 2 1
      server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java
  33. 14 9
      server/src/test/java/org/elasticsearch/snapshots/BlobStoreFormatTests.java
  34. 4 2
      server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java
  35. 3 1
      server/src/test/java/org/elasticsearch/snapshots/mockstore/MockEventuallyConsistentRepository.java
  36. 6 4
      test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java
  37. 3 1
      x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java
  38. 3 1
      x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java
  39. 2 0
      x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java
  40. 4 3
      x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/LocalStateCompositeXPackPlugin.java
  41. 3 1
      x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPrewarmingIntegTests.java
  42. 2 0
      x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryTests.java
  43. 6 4
      x-pack/plugin/voting-only-node/src/internalClusterTest/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java

+ 4 - 2
modules/repository-url/src/main/java/org/elasticsearch/plugin/repository/url/URLRepositoryPlugin.java

@@ -21,6 +21,7 @@ package org.elasticsearch.plugin.repository.url;
 
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.settings.Setting;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.indices.recovery.RecoverySettings;
@@ -47,8 +48,9 @@ public class URLRepositoryPlugin extends Plugin implements RepositoryPlugin {
 
     @Override
     public Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry,
-                                                           ClusterService clusterService, RecoverySettings recoverySettings) {
+                                                           ClusterService clusterService, BigArrays bigArrays,
+                                                           RecoverySettings recoverySettings) {
         return Collections.singletonMap(URLRepository.TYPE,
-            metadata -> new URLRepository(metadata, env, namedXContentRegistry, clusterService, recoverySettings));
+            metadata -> new URLRepository(metadata, env, namedXContentRegistry, clusterService, bigArrays, recoverySettings));
     }
 }

+ 3 - 2
modules/repository-url/src/main/java/org/elasticsearch/repositories/url/URLRepository.java

@@ -29,6 +29,7 @@ import org.elasticsearch.common.blobstore.BlobStore;
 import org.elasticsearch.common.blobstore.url.URLBlobStore;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Setting.Property;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.URIPattern;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
@@ -82,9 +83,9 @@ public class URLRepository extends BlobStoreRepository {
      * Constructs a read-only URL-based repository
      */
     public URLRepository(RepositoryMetadata metadata, Environment environment,
-                         NamedXContentRegistry namedXContentRegistry, ClusterService clusterService,
+                         NamedXContentRegistry namedXContentRegistry, ClusterService clusterService, BigArrays bigArrays,
                          RecoverySettings recoverySettings) {
-        super(metadata, namedXContentRegistry, clusterService, recoverySettings, BlobPath.cleanPath());
+        super(metadata, namedXContentRegistry, clusterService, bigArrays, recoverySettings, BlobPath.cleanPath());
 
         if (URL_SETTING.exists(metadata.settings()) == false && REPOSITORIES_URL_SETTING.exists(environment.settings()) ==  false) {
             throw new RepositoryException(metadata.name(), "missing url");

+ 2 - 0
modules/repository-url/src/test/java/org/elasticsearch/repositories/url/URLRepositoryTests.java

@@ -22,6 +22,7 @@ package org.elasticsearch.repositories.url;
 import org.elasticsearch.cluster.metadata.RepositoryMetadata;
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.MockBigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.TestEnvironment;
@@ -43,6 +44,7 @@ public class URLRepositoryTests extends ESTestCase {
     private URLRepository createRepository(Settings baseSettings, RepositoryMetadata repositoryMetadata) {
         return new URLRepository(repositoryMetadata, TestEnvironment.newEnvironment(baseSettings),
             new NamedXContentRegistry(Collections.emptyList()), BlobStoreTestUtil.mockClusterService(),
+            MockBigArrays.NON_RECYCLING_INSTANCE,
             new RecoverySettings(baseSettings, new ClusterSettings(baseSettings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))) {
             @Override
             protected void assertSnapshotOrGenericThread() {

+ 3 - 0
plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepository.java

@@ -31,6 +31,7 @@ import org.elasticsearch.common.blobstore.BlobStore;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Setting.Property;
 import org.elasticsearch.common.unit.ByteSizeValue;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.indices.recovery.RecoverySettings;
 import org.elasticsearch.repositories.blobstore.MeteredBlobStoreRepository;
@@ -82,10 +83,12 @@ public class AzureRepository extends MeteredBlobStoreRepository {
         final NamedXContentRegistry namedXContentRegistry,
         final AzureStorageService storageService,
         final ClusterService clusterService,
+        final BigArrays bigArrays,
         final RecoverySettings recoverySettings) {
         super(metadata,
             namedXContentRegistry,
             clusterService,
+            bigArrays,
             recoverySettings,
             buildBasePath(metadata),
             buildLocation(metadata));

+ 4 - 2
plugins/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureRepositoryPlugin.java

@@ -24,6 +24,7 @@ import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.settings.SettingsException;
 import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.indices.recovery.RecoverySettings;
@@ -61,9 +62,10 @@ public class AzureRepositoryPlugin extends Plugin implements RepositoryPlugin, R
 
     @Override
     public Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry,
-                                                           ClusterService clusterService, RecoverySettings recoverySettings) {
+                                                           ClusterService clusterService, BigArrays bigArrays,
+                                                           RecoverySettings recoverySettings) {
         return Collections.singletonMap(AzureRepository.TYPE,
-                (metadata) -> new AzureRepository(metadata, namedXContentRegistry, azureStoreService, clusterService,
+                (metadata) -> new AzureRepository(metadata, namedXContentRegistry, azureStoreService, clusterService, bigArrays,
                     recoverySettings));
     }
 

+ 2 - 0
plugins/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureRepositorySettingsTests.java

@@ -25,6 +25,7 @@ import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.unit.ByteSizeValue;
+import org.elasticsearch.common.util.MockBigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.indices.recovery.RecoverySettings;
@@ -45,6 +46,7 @@ public class AzureRepositorySettingsTests extends ESTestCase {
             .build();
         final AzureRepository azureRepository = new AzureRepository(new RepositoryMetadata("foo", "azure", internalSettings),
             NamedXContentRegistry.EMPTY, mock(AzureStorageService.class), BlobStoreTestUtil.mockClusterService(),
+            MockBigArrays.NON_RECYCLING_INSTANCE,
             new RecoverySettings(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)));
         assertThat(azureRepository.getBlobStore(), is(nullValue()));
         return azureRepository;

+ 5 - 2
plugins/repository-gcs/src/internalClusterTest/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageBlobStoreRepositoryTests.java

@@ -44,6 +44,7 @@ import org.elasticsearch.common.settings.MockSecureSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.unit.ByteSizeValue;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.indices.recovery.RecoverySettings;
@@ -224,9 +225,11 @@ public class GoogleCloudStorageBlobStoreRepositoryTests extends ESMockAPIBasedRe
 
         @Override
         public Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry registry,
-                                                               ClusterService clusterService, RecoverySettings recoverySettings) {
+                                                               ClusterService clusterService, BigArrays bigArrays,
+                                                               RecoverySettings recoverySettings) {
             return Collections.singletonMap(GoogleCloudStorageRepository.TYPE,
-                metadata -> new GoogleCloudStorageRepository(metadata, registry, this.storageService, clusterService, recoverySettings) {
+                metadata -> new GoogleCloudStorageRepository(metadata, registry, this.storageService, clusterService,
+                        bigArrays, recoverySettings) {
                     @Override
                     protected GoogleCloudStorageBlobStore createBlobStore() {
                         return new GoogleCloudStorageBlobStore(

+ 4 - 2
plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStoragePlugin.java

@@ -22,6 +22,7 @@ package org.elasticsearch.repositories.gcs;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.indices.recovery.RecoverySettings;
@@ -53,9 +54,10 @@ public class GoogleCloudStoragePlugin extends Plugin implements RepositoryPlugin
 
     @Override
     public Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry,
-                                                           ClusterService clusterService, RecoverySettings recoverySettings) {
+                                                           ClusterService clusterService, BigArrays bigArrays,
+                                                           RecoverySettings recoverySettings) {
         return Collections.singletonMap(GoogleCloudStorageRepository.TYPE,
-            metadata -> new GoogleCloudStorageRepository(metadata, namedXContentRegistry, this.storageService, clusterService,
+            metadata -> new GoogleCloudStorageRepository(metadata, namedXContentRegistry, this.storageService, clusterService, bigArrays,
                 recoverySettings));
     }
 

+ 4 - 1
plugins/repository-gcs/src/main/java/org/elasticsearch/repositories/gcs/GoogleCloudStorageRepository.java

@@ -28,6 +28,7 @@ import org.elasticsearch.common.blobstore.BlobPath;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.unit.ByteSizeValue;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.indices.recovery.RecoverySettings;
 import org.elasticsearch.repositories.RepositoryException;
@@ -72,8 +73,10 @@ class GoogleCloudStorageRepository extends MeteredBlobStoreRepository {
         final NamedXContentRegistry namedXContentRegistry,
         final GoogleCloudStorageService storageService,
         final ClusterService clusterService,
+        final BigArrays bigArrays,
         final RecoverySettings recoverySettings) {
-        super(metadata, namedXContentRegistry, clusterService, recoverySettings, buildBasePath(metadata), buildLocation(metadata));
+        super(metadata, namedXContentRegistry, clusterService, bigArrays, recoverySettings, buildBasePath(metadata),
+                buildLocation(metadata));
         this.storageService = storageService;
 
         this.chunkSize = getSetting(CHUNK_SIZE, metadata);

+ 4 - 2
plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsPlugin.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.elasticsearch.SpecialPermission;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.SuppressForbidden;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.indices.recovery.RecoverySettings;
@@ -113,8 +114,9 @@ public final class HdfsPlugin extends Plugin implements RepositoryPlugin {
 
     @Override
     public Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry,
-                                                           ClusterService clusterService, RecoverySettings recoverySettings) {
+                                                           ClusterService clusterService, BigArrays bigArrays,
+                                                           RecoverySettings recoverySettings) {
         return Collections.singletonMap("hdfs", (metadata) -> new HdfsRepository(metadata, env, namedXContentRegistry, clusterService,
-            recoverySettings));
+            bigArrays, recoverySettings));
     }
 }

+ 3 - 2
plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java

@@ -37,6 +37,7 @@ import org.elasticsearch.common.SuppressForbidden;
 import org.elasticsearch.common.blobstore.BlobPath;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeValue;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.indices.recovery.RecoverySettings;
@@ -63,9 +64,9 @@ public final class HdfsRepository extends BlobStoreRepository {
     private final String pathSetting;
 
     public HdfsRepository(RepositoryMetadata metadata, Environment environment,
-                          NamedXContentRegistry namedXContentRegistry, ClusterService clusterService,
+                          NamedXContentRegistry namedXContentRegistry, ClusterService clusterService, BigArrays bigArrays,
                           RecoverySettings recoverySettings) {
-        super(metadata, namedXContentRegistry, clusterService, recoverySettings, BlobPath.cleanPath());
+        super(metadata, namedXContentRegistry, clusterService, bigArrays, recoverySettings, BlobPath.cleanPath());
 
         this.environment = environment;
         this.chunkSize = metadata.settings().getAsBytesSize("chunk_size", null);

+ 3 - 2
plugins/repository-s3/src/internalClusterTest/java/org/elasticsearch/repositories/s3/S3BlobStoreRepositoryTests.java

@@ -38,6 +38,7 @@ import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.indices.recovery.RecoverySettings;
@@ -196,8 +197,8 @@ public class S3BlobStoreRepositoryTests extends ESMockAPIBasedRepositoryIntegTes
 
         @Override
         protected S3Repository createRepository(RepositoryMetadata metadata, NamedXContentRegistry registry,
-                                                ClusterService clusterService, RecoverySettings recoverySettings) {
-            return new S3Repository(metadata, registry, service, clusterService, recoverySettings) {
+                                                ClusterService clusterService, BigArrays bigArrays, RecoverySettings recoverySettings) {
+            return new S3Repository(metadata, registry, service, clusterService, bigArrays, recoverySettings) {
 
                 @Override
                 public BlobStore blobStore() {

+ 3 - 0
plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java

@@ -35,6 +35,7 @@ import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.unit.ByteSizeValue;
 import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.indices.recovery.RecoverySettings;
 import org.elasticsearch.monitor.jvm.JvmInfo;
@@ -194,10 +195,12 @@ class S3Repository extends MeteredBlobStoreRepository {
         final NamedXContentRegistry namedXContentRegistry,
         final S3Service service,
         final ClusterService clusterService,
+        final BigArrays bigArrays,
         final RecoverySettings recoverySettings) {
         super(metadata,
             namedXContentRegistry,
             clusterService,
+            bigArrays,
             recoverySettings,
             buildBasePath(metadata),
             buildLocation(metadata));

+ 6 - 3
plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3RepositoryPlugin.java

@@ -25,6 +25,7 @@ import org.elasticsearch.cluster.metadata.RepositoryMetadata;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.indices.recovery.RecoverySettings;
@@ -81,14 +82,16 @@ public class S3RepositoryPlugin extends Plugin implements RepositoryPlugin, Relo
         final RepositoryMetadata metadata,
         final NamedXContentRegistry registry,
         final ClusterService clusterService,
+        final BigArrays bigArrays,
         final RecoverySettings recoverySettings) {
-        return new S3Repository(metadata, registry, service, clusterService, recoverySettings);
+        return new S3Repository(metadata, registry, service, clusterService, bigArrays, recoverySettings);
     }
 
     @Override
     public Map<String, Repository.Factory> getRepositories(final Environment env, final NamedXContentRegistry registry,
-                                                           final ClusterService clusterService, final RecoverySettings recoverySettings) {
-        return Collections.singletonMap(S3Repository.TYPE, metadata -> createRepository(metadata, registry, clusterService,
+                                                           final ClusterService clusterService, final BigArrays bigArrays,
+                                                           final RecoverySettings recoverySettings) {
+        return Collections.singletonMap(S3Repository.TYPE, metadata -> createRepository(metadata, registry, clusterService, bigArrays,
             recoverySettings));
     }
 

+ 3 - 2
plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/RepositoryCredentialsTests.java

@@ -28,6 +28,7 @@ import org.elasticsearch.cluster.metadata.RepositoryMetadata;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.settings.MockSecureSettings;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.indices.recovery.RecoverySettings;
 import org.elasticsearch.plugins.Plugin;
@@ -147,9 +148,9 @@ public class RepositoryCredentialsTests extends ESSingleNodeTestCase {
 
         @Override
         protected S3Repository createRepository(RepositoryMetadata metadata,
-                                                NamedXContentRegistry registry, ClusterService clusterService,
+                                                NamedXContentRegistry registry, ClusterService clusterService, BigArrays bigArrays,
                                                 RecoverySettings recoverySettings) {
-            return new S3Repository(metadata, registry, service, clusterService, recoverySettings) {
+            return new S3Repository(metadata, registry, service, clusterService, bigArrays, recoverySettings) {
                 @Override
                 protected void assertSnapshotOrGenericThread() {
                     // eliminate thread name check as we create repo manually on test/main threads

+ 3 - 1
plugins/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RepositoryTests.java

@@ -25,6 +25,7 @@ import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.unit.ByteSizeValue;
+import org.elasticsearch.common.util.MockBigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.indices.recovery.RecoverySettings;
 import org.elasticsearch.repositories.RepositoryException;
@@ -122,7 +123,8 @@ public class S3RepositoryTests extends ESTestCase {
 
     private S3Repository createS3Repo(RepositoryMetadata metadata) {
         return new S3Repository(metadata, NamedXContentRegistry.EMPTY, new DummyS3Service(), BlobStoreTestUtil.mockClusterService(),
-            new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))) {
+                MockBigArrays.NON_RECYCLING_INSTANCE,
+                new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))) {
             @Override
             protected void assertSnapshotOrGenericThread() {
                 // eliminate thread name check as we create repo manually on test/main threads

+ 2 - 1
server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java

@@ -25,6 +25,7 @@ import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotStatus;
 import org.elasticsearch.action.support.master.AcknowledgedResponse;
 import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.util.MockBigArrays;
 import org.elasticsearch.index.IndexNotFoundException;
 import org.elasticsearch.repositories.RepositoriesService;
 import org.elasticsearch.repositories.RepositoryData;
@@ -509,7 +510,7 @@ public class CloneSnapshotIT extends AbstractSnapshotIntegTestCase {
                                                                     String generation) {
         return PlainActionFuture.get(f -> repository.threadPool().generic().execute(ActionRunnable.supply(f,
                 () -> BlobStoreRepository.INDEX_SHARD_SNAPSHOTS_FORMAT.read(repository.shardContainer(repositoryShardId.index(),
-                        repositoryShardId.shardId()), generation, NamedXContentRegistry.EMPTY))));
+                        repositoryShardId.shardId()), generation, NamedXContentRegistry.EMPTY, MockBigArrays.NON_RECYCLING_INSTANCE))));
     }
 
     private static BlobStoreIndexShardSnapshot readShardSnapshot(BlobStoreRepository repository, RepositoryShardId repositoryShardId,

+ 6 - 4
server/src/internalClusterTest/java/org/elasticsearch/snapshots/MetadataLoadingDuringSnapshotRestoreIT.java

@@ -28,6 +28,7 @@ import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.metadata.RepositoryMetadata;
 import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.indices.recovery.RecoverySettings;
@@ -186,8 +187,8 @@ public class MetadataLoadingDuringSnapshotRestoreIT extends AbstractSnapshotInte
         public CountingMockRepository(final RepositoryMetadata metadata,
                                       final Environment environment,
                                       final NamedXContentRegistry namedXContentRegistry, ClusterService clusterService,
-                                      RecoverySettings recoverySettings) {
-            super(metadata, environment, namedXContentRegistry, clusterService, recoverySettings);
+                                      BigArrays bigArrays, RecoverySettings recoverySettings) {
+            super(metadata, environment, namedXContentRegistry, clusterService, bigArrays, recoverySettings);
         }
 
         @Override
@@ -211,9 +212,10 @@ public class MetadataLoadingDuringSnapshotRestoreIT extends AbstractSnapshotInte
 
         @Override
         public Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry,
-                                                               ClusterService clusterService, RecoverySettings recoverySettings) {
+                                                               ClusterService clusterService, BigArrays bigArrays,
+                                                               RecoverySettings recoverySettings) {
             return Collections.singletonMap(TYPE,
-                metadata -> new CountingMockRepository(metadata, env, namedXContentRegistry, clusterService, recoverySettings));
+                metadata -> new CountingMockRepository(metadata, env, namedXContentRegistry, clusterService, bigArrays, recoverySettings));
         }
     }
 }

+ 4 - 2
server/src/internalClusterTest/java/org/elasticsearch/snapshots/RepositoryFilterUserMetadataIT.java

@@ -25,6 +25,7 @@ import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.index.mapper.MapperService;
@@ -78,9 +79,10 @@ public class RepositoryFilterUserMetadataIT extends ESIntegTestCase {
 
         @Override
         public Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry,
-                                                               ClusterService clusterService, RecoverySettings recoverySettings) {
+                                                               ClusterService clusterService, BigArrays bigArrays,
+                                                               RecoverySettings recoverySettings) {
             return Collections.singletonMap("mock_meta_filtering", metadata ->
-                new FsRepository(metadata, env, namedXContentRegistry, clusterService, recoverySettings) {
+                new FsRepository(metadata, env, namedXContentRegistry, clusterService, bigArrays, recoverySettings) {
 
                     // Storing the initially expected metadata value here to verify that #filterUserMetadata is only called once on the
                     // initial master node starting the snapshot

+ 22 - 0
server/src/main/java/org/elasticsearch/common/io/Streams.java

@@ -26,6 +26,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
 
 import java.io.BufferedReader;
 import java.io.FilterInputStream;
+import java.io.FilterOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
@@ -216,6 +217,27 @@ public abstract class Streams {
         };
     }
 
+    /**
+     * Wraps an {@link OutputStream} such that it's {@code close} method becomes a noop
+     *
+     * @param stream {@code OutputStream} to wrap
+     * @return wrapped {@code OutputStream}
+     */
+    public static OutputStream noCloseStream(OutputStream stream) {
+        return new FilterOutputStream(stream) {
+
+            @Override
+            public void write(byte[] b, int off, int len) throws IOException {
+                out.write(b, off, len);
+            }
+
+            @Override
+            public void close() {
+                // noop
+            }
+        };
+    }
+
     /**
      * Wraps the given {@link BytesStream} in a {@link StreamOutput} that simply flushes when
      * close is called.

+ 1 - 1
server/src/main/java/org/elasticsearch/node/Node.java

@@ -563,7 +563,7 @@ public class Node implements Closeable {
 
             final RecoverySettings recoverySettings = new RecoverySettings(settings, settingsModule.getClusterSettings());
             RepositoriesModule repositoriesModule = new RepositoriesModule(this.environment,
-                pluginsService.filterPlugins(RepositoryPlugin.class), transportService, clusterService, threadPool, xContentRegistry,
+                pluginsService.filterPlugins(RepositoryPlugin.class), transportService, clusterService, bigArrays, xContentRegistry,
                 recoverySettings);
             RepositoriesService repositoryService = repositoriesModule.getRepositoryService();
             repositoriesServiceReference.set(repositoryService);

+ 3 - 1
server/src/main/java/org/elasticsearch/plugins/RepositoryPlugin.java

@@ -20,6 +20,7 @@
 package org.elasticsearch.plugins;
 
 import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.indices.recovery.RecoverySettings;
@@ -42,7 +43,8 @@ public interface RepositoryPlugin {
      * the value is a factory to construct the {@link Repository} interface.
      */
     default Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry,
-                                                            ClusterService clusterService, RecoverySettings recoverySettings) {
+                                                            ClusterService clusterService, BigArrays bigArrays,
+                                                            RecoverySettings recoverySettings) {
         return Collections.emptyMap();
     }
 

+ 5 - 5
server/src/main/java/org/elasticsearch/repositories/RepositoriesModule.java

@@ -21,12 +21,12 @@ package org.elasticsearch.repositories;
 
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.indices.recovery.RecoverySettings;
 import org.elasticsearch.plugins.RepositoryPlugin;
 import org.elasticsearch.repositories.fs.FsRepository;
-import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.TransportService;
 
 import java.util.Collections;
@@ -42,15 +42,15 @@ public final class RepositoriesModule {
     private final RepositoriesService repositoriesService;
 
     public RepositoriesModule(Environment env, List<RepositoryPlugin> repoPlugins, TransportService transportService,
-                              ClusterService clusterService, ThreadPool threadPool, NamedXContentRegistry namedXContentRegistry,
+                              ClusterService clusterService, BigArrays bigArrays, NamedXContentRegistry namedXContentRegistry,
                               RecoverySettings recoverySettings) {
         Map<String, Repository.Factory> factories = new HashMap<>();
-        factories.put(FsRepository.TYPE, metadata -> new FsRepository(metadata, env, namedXContentRegistry, clusterService,
+        factories.put(FsRepository.TYPE, metadata -> new FsRepository(metadata, env, namedXContentRegistry, clusterService, bigArrays,
             recoverySettings));
 
         for (RepositoryPlugin repoPlugin : repoPlugins) {
             Map<String, Repository.Factory> newRepoTypes = repoPlugin.getRepositories(env, namedXContentRegistry, clusterService,
-                recoverySettings);
+                    bigArrays, recoverySettings);
             for (Map.Entry<String, Repository.Factory> entry : newRepoTypes.entrySet()) {
                 if (factories.put(entry.getKey(), entry.getValue()) != null) {
                     throw new IllegalArgumentException("Repository type [" + entry.getKey() + "] is already registered");
@@ -77,7 +77,7 @@ public final class RepositoriesModule {
         Map<String, Repository.Factory> repositoryTypes = Collections.unmodifiableMap(factories);
         Map<String, Repository.Factory> internalRepositoryTypes = Collections.unmodifiableMap(internalFactories);
         repositoriesService = new RepositoriesService(settings, clusterService, transportService, repositoryTypes,
-            internalRepositoryTypes, threadPool);
+            internalRepositoryTypes, transportService.getThreadPool());
     }
 
     public RepositoriesService getRepositoryService() {

+ 74 - 48
server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java

@@ -68,6 +68,7 @@ import org.elasticsearch.common.component.AbstractLifecycleComponent;
 import org.elasticsearch.common.compress.CompressorFactory;
 import org.elasticsearch.common.compress.NotXContentException;
 import org.elasticsearch.common.io.Streams;
+import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
 import org.elasticsearch.common.lease.Releasable;
 import org.elasticsearch.common.lucene.Lucene;
 import org.elasticsearch.common.lucene.store.InputStreamIndexInput;
@@ -77,10 +78,12 @@ import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.unit.ByteSizeValue;
 import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.concurrent.AbstractRunnable;
 import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
 import org.elasticsearch.common.xcontent.LoggingDeprecationHandler;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
+import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.XContentType;
@@ -261,6 +264,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
 
     private final NamedXContentRegistry namedXContentRegistry;
 
+    private final BigArrays bigArrays;
+
     /**
      * Flag that is set to {@code true} if this instance is started with {@link #metadata} that has a higher value for
      * {@link RepositoryMetadata#pendingGeneration()} than for {@link RepositoryMetadata#generation()} indicating a full cluster restart
@@ -300,11 +305,13 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
         final RepositoryMetadata metadata,
         final NamedXContentRegistry namedXContentRegistry,
         final ClusterService clusterService,
+        final BigArrays bigArrays,
         final RecoverySettings recoverySettings,
         final BlobPath basePath) {
         this.metadata = metadata;
         this.threadPool = clusterService.getClusterApplierService().threadPool();
         this.clusterService = clusterService;
+        this.bigArrays = bigArrays;
         this.recoverySettings = recoverySettings;
         this.compress = COMPRESS_SETTING.get(metadata.settings());
         this.supportURLRepo = SUPPORT_URL_REPO.get(metadata.settings());
@@ -453,9 +460,9 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
             logger.trace("[{}] [{}] writing shard snapshot file for clone", shardId, target);
             INDEX_SHARD_SNAPSHOT_FORMAT.write(sourceMeta.asClone(target.getName(), startTime,
                     threadPool.absoluteTimeInMillis() - startTime),
-                    shardContainer, target.getUUID(), compress);
+                    shardContainer, target.getUUID(), compress, bigArrays);
             INDEX_SHARD_SNAPSHOTS_FORMAT.write(existingSnapshots.withClone(source.getName(), target.getName()), shardContainer, newGen,
-                    compress);
+                    compress, bigArrays);
             return newGen;
         }));
     }
@@ -798,7 +805,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
             for (String indexMetaGeneration : indexMetaGenerations) {
                 executor.execute(ActionRunnable.supply(allShardCountsListener, () -> {
                     try {
-                        return INDEX_METADATA_FORMAT.read(indexContainer, indexMetaGeneration, namedXContentRegistry).getNumberOfShards();
+                        return INDEX_METADATA_FORMAT.read(indexContainer, indexMetaGeneration, namedXContentRegistry,
+                                bigArrays).getNumberOfShards();
                     } catch (Exception ex) {
                         logger.warn(() -> new ParameterizedMessage(
                                 "[{}] [{}] failed to read metadata for index", indexMetaGeneration, indexId.getName()), ex);
@@ -1112,7 +1120,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
 
             // Write Global MetaData
             executor.execute(ActionRunnable.run(allMetaListener,
-                () -> GLOBAL_METADATA_FORMAT.write(clusterMetadata, blobContainer(), snapshotId.getUUID(), compress)));
+                () -> GLOBAL_METADATA_FORMAT.write(clusterMetadata, blobContainer(), snapshotId.getUUID(), compress, bigArrays)));
 
             // write the index metadata for each index in the snapshot
             for (IndexId index : indices) {
@@ -1124,19 +1132,19 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
                             if (metaUUID == null) {
                                 // We don't yet have this version of the metadata so we write it
                                 metaUUID = UUIDs.base64UUID();
-                                INDEX_METADATA_FORMAT.write(indexMetaData, indexContainer(index), metaUUID, compress);
+                                INDEX_METADATA_FORMAT.write(indexMetaData, indexContainer(index), metaUUID, compress, bigArrays);
                                 indexMetaIdentifiers.put(identifiers, metaUUID);
                             }
                             indexMetas.put(index, identifiers);
                         } else {
                             INDEX_METADATA_FORMAT.write(
-                                clusterMetadata.index(index.getName()), indexContainer(index), snapshotId.getUUID(), compress);
+                                clusterMetadata.index(index.getName()), indexContainer(index), snapshotId.getUUID(), compress, bigArrays);
                         }
                     }
                 ));
             }
             executor.execute(ActionRunnable.run(allMetaListener,
-                () -> SNAPSHOT_FORMAT.write(snapshotInfo, blobContainer(), snapshotId.getUUID(), compress)));
+                () -> SNAPSHOT_FORMAT.write(snapshotInfo, blobContainer(), snapshotId.getUUID(), compress, bigArrays)));
         }, onUpdateFailure);
     }
 
@@ -1157,7 +1165,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
     @Override
     public SnapshotInfo getSnapshotInfo(final SnapshotId snapshotId) {
         try {
-            return SNAPSHOT_FORMAT.read(blobContainer(), snapshotId.getUUID(), namedXContentRegistry);
+            return SNAPSHOT_FORMAT.read(blobContainer(), snapshotId.getUUID(), namedXContentRegistry, bigArrays);
         } catch (NoSuchFileException ex) {
             throw new SnapshotMissingException(metadata.name(), snapshotId, ex);
         } catch (IOException | NotXContentException ex) {
@@ -1168,7 +1176,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
     @Override
     public Metadata getSnapshotGlobalMetadata(final SnapshotId snapshotId) {
         try {
-            return GLOBAL_METADATA_FORMAT.read(blobContainer(), snapshotId.getUUID(), namedXContentRegistry);
+            return GLOBAL_METADATA_FORMAT.read(blobContainer(), snapshotId.getUUID(), namedXContentRegistry, bigArrays);
         } catch (NoSuchFileException ex) {
             throw new SnapshotMissingException(metadata.name(), snapshotId, ex);
         } catch (IOException ex) {
@@ -1180,7 +1188,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
     public IndexMetadata getSnapshotIndexMetaData(RepositoryData repositoryData, SnapshotId snapshotId, IndexId index) throws IOException {
         try {
             return INDEX_METADATA_FORMAT.read(indexContainer(index),
-                repositoryData.indexMetaDataGenerations().indexMetaBlobId(snapshotId, index), namedXContentRegistry);
+                repositoryData.indexMetaDataGenerations().indexMetaBlobId(snapshotId, index), namedXContentRegistry, bigArrays);
         } catch (NoSuchFileException e) {
             throw new SnapshotMissingException(metadata.name(), snapshotId, e);
         }
@@ -1340,8 +1348,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
                     // We can cache serialized in the most recent version here without regard to the actual repository metadata version
                     // since we're only caching the information that we just wrote and thus won't accidentally cache any information that
                     // isn't safe
-                    cacheRepositoryData(
-                            BytesReference.bytes(loaded.snapshotsToXContent(XContentFactory.jsonBuilder(), Version.CURRENT)), genToLoad);
+                    cacheRepositoryData(compressRepoDataForCache(BytesReference.bytes(
+                            loaded.snapshotsToXContent(XContentFactory.jsonBuilder(), Version.CURRENT))), genToLoad);
                 }
                 listener.onResponse(loaded);
                 return;
@@ -1376,38 +1384,48 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
      * modification can lead to moving from a higher {@code N} to a lower {@code N} value which mean we can't safely assume that a given
      * generation will always contain the same {@link RepositoryData}.
      *
-     * @param updated    serialized RepositoryData to cache if newer than the cache contents
+     * @param serialized serialized RepositoryData to cache if newer than the cache contents or null if no data should be cached for the
+     *                   given generation
      * @param generation repository generation of the given repository data
      */
-    private void cacheRepositoryData(BytesReference updated, long generation) {
-        if (cacheRepositoryData && bestEffortConsistency == false) {
-            final BytesReference serialized;
-            try {
-                serialized = CompressorFactory.COMPRESSOR.compress(updated);
-                final int len = serialized.length();
-                if (len > ByteSizeUnit.KB.toBytes(500)) {
-                    logger.debug("Not caching repository data of size [{}] for repository [{}] because it is larger than 500KB in" +
+    private void cacheRepositoryData(@Nullable BytesReference serialized, long generation) {
+        latestKnownRepositoryData.updateAndGet(known -> {
+            if (known != null && known.v1() > generation) {
+                return known;
+            }
+            return serialized == null ? null : new Tuple<>(generation, serialized);
+        });
+    }
+
+    /**
+     * Creates a compressed version of serialized {@link RepositoryData} that can be used with {@link #cacheRepositoryData} if possible.
+     *
+     * @param uncompressed uncompressed, serialized {@link RepositoryData}
+     * @return compressed repository data to cache or {@code null} if caching is disabled or the data is too large to cache
+     */
+    @Nullable
+    private BytesReference compressRepoDataForCache(BytesReference uncompressed) {
+        if (cacheRepositoryData == false || bestEffortConsistency) {
+            return null;
+        }
+        try {
+            final BytesReference serialized = CompressorFactory.COMPRESSOR.compress(uncompressed);
+            final int len = serialized.length();
+            if (len > ByteSizeUnit.KB.toBytes(500)) {
+                logger.debug("Not caching repository data of size [{}] for repository [{}] because it is larger than 500KB in" +
                         " serialized size", len, metadata.name());
-                    if (len > ByteSizeUnit.MB.toBytes(5)) {
-                        logger.warn("Your repository metadata blob for repository [{}] is larger than 5MB. Consider moving to a fresh" +
+                if (len > ByteSizeUnit.MB.toBytes(5)) {
+                    logger.warn("Your repository metadata blob for repository [{}] is larger than 5MB. Consider moving to a fresh" +
                             " repository for new snapshots or deleting unneeded snapshots from your repository to ensure stable" +
                             " repository behavior going forward.", metadata.name());
-                    }
-                    // Set empty repository data to not waste heap for an outdated cached value
-                    latestKnownRepositoryData.set(null);
-                    return;
                 }
-            } catch (IOException e) {
-                assert false : new AssertionError("Impossible, no IO happens here", e);
-                logger.warn("Failed to serialize repository data", e);
-                return;
+                return null;
             }
-            latestKnownRepositoryData.updateAndGet(known -> {
-                if (known != null && known.v1() > generation) {
-                    return known;
-                }
-                return new Tuple<>(generation, serialized);
-            });
+            return serialized;
+        } catch (IOException e) {
+            assert false : new AssertionError("Impossible, no IO happens here", e);
+            logger.warn("Failed to serialize repository data", e);
+            return null;
         }
     }
 
@@ -1630,9 +1648,15 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
             }
             final String indexBlob = INDEX_FILE_PREFIX + Long.toString(newGen);
             logger.debug("Repository [{}] writing new index generational blob [{}]", metadata.name(), indexBlob);
-            final BytesReference serializedRepoData =
-                    BytesReference.bytes(newRepositoryData.snapshotsToXContent(XContentFactory.jsonBuilder(), version));
-            writeAtomic(blobContainer(), indexBlob, serializedRepoData, true);
+            final BytesReference repoDataToCache;
+            try (ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(bigArrays)) {
+                try (XContentBuilder xContentBuilder = XContentFactory.jsonBuilder(Streams.noCloseStream(out))) {
+                    newRepositoryData.snapshotsToXContent(xContentBuilder, version);
+                }
+                final BytesReference serializedRepoData = out.bytes();
+                writeAtomic(blobContainer(), indexBlob, serializedRepoData, true);
+                repoDataToCache = compressRepoDataForCache(serializedRepoData);
+            }
             maybeWriteIndexLatest(newGen);
 
             // Step 3: Update CS to reflect new repository generation.
@@ -1664,7 +1688,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
 
                     @Override
                     public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
-                        cacheRepositoryData(serializedRepoData, newGen);
+                        cacheRepositoryData(repoDataToCache, newGen);
                         threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(ActionRunnable.supply(listener, () -> {
                             // Delete all now outdated index files up to 1000 blobs back from the new generation.
                             // If there are more than 1000 dangling index-N cleanup functionality on repo delete will take care of them.
@@ -1987,7 +2011,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
                 // reference a generation that has not had all its files fully upload.
                 indexGeneration = UUIDs.randomBase64UUID();
                 try {
-                    INDEX_SHARD_SNAPSHOTS_FORMAT.write(updatedBlobStoreIndexShardSnapshots, shardContainer, indexGeneration, compress);
+                    INDEX_SHARD_SNAPSHOTS_FORMAT.write(updatedBlobStoreIndexShardSnapshots, shardContainer, indexGeneration, compress,
+                            bigArrays);
                 } catch (IOException e) {
                     throw new IndexShardSnapshotFailedException(shardId,
                             "Failed to write shard level snapshot metadata for [" + snapshotId + "] to ["
@@ -2039,7 +2064,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
                             threadPool.absoluteTimeInMillis() - lastSnapshotStatus.getStartTime(),
                             lastSnapshotStatus.getIncrementalFileCount(),
                             lastSnapshotStatus.getIncrementalSize()
-                    ), shardContainer, snapshotId.getUUID(), compress);
+                    ), shardContainer, snapshotId.getUUID(), compress, bigArrays);
                 } catch (IOException e) {
                     throw new IndexShardSnapshotFailedException(shardId, "Failed to write commit point", e);
                 }
@@ -2314,7 +2339,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
                 final BlobStoreIndexShardSnapshots updatedSnapshots = new BlobStoreIndexShardSnapshots(newSnapshotsList);
                 if (indexGeneration < 0L) {
                     writtenGeneration = UUIDs.randomBase64UUID();
-                    INDEX_SHARD_SNAPSHOTS_FORMAT.write(updatedSnapshots, shardContainer, writtenGeneration, compress);
+                    INDEX_SHARD_SNAPSHOTS_FORMAT.write(updatedSnapshots, shardContainer, writtenGeneration, compress, bigArrays);
                 } else {
                     writtenGeneration = String.valueOf(indexGeneration);
                     writeShardIndexBlobAtomic(shardContainer, indexGeneration, updatedSnapshots);
@@ -2339,7 +2364,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
         logger.trace(() -> new ParameterizedMessage("[{}] Writing shard index [{}] to [{}]", metadata.name(),
                 indexGeneration, shardContainer.path()));
         final String blobName = INDEX_SHARD_SNAPSHOTS_FORMAT.blobName(String.valueOf(indexGeneration));
-        writeAtomic(shardContainer, blobName, INDEX_SHARD_SNAPSHOTS_FORMAT.serialize(updatedSnapshots, blobName, compress), true);
+        INDEX_SHARD_SNAPSHOTS_FORMAT.serialize(updatedSnapshots, blobName, compress, bigArrays,
+                bytes -> writeAtomic(shardContainer, blobName, bytes, true));
     }
 
     // Unused blobs are all previous index-, data- and meta-blobs and that are not referenced by the new index- as well as all
@@ -2360,7 +2386,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
      */
     public BlobStoreIndexShardSnapshot loadShardSnapshot(BlobContainer shardContainer, SnapshotId snapshotId) {
         try {
-            return INDEX_SHARD_SNAPSHOT_FORMAT.read(shardContainer, snapshotId.getUUID(), namedXContentRegistry);
+            return INDEX_SHARD_SNAPSHOT_FORMAT.read(shardContainer, snapshotId.getUUID(), namedXContentRegistry, bigArrays);
         } catch (NoSuchFileException ex) {
             throw new SnapshotMissingException(metadata.name(), snapshotId, ex);
         } catch (IOException ex) {
@@ -2386,7 +2412,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
             if (generation.equals(ShardGenerations.NEW_SHARD_GEN)) {
                 return new Tuple<>(BlobStoreIndexShardSnapshots.EMPTY, ShardGenerations.NEW_SHARD_GEN);
             }
-            return new Tuple<>(INDEX_SHARD_SNAPSHOTS_FORMAT.read(shardContainer, generation, namedXContentRegistry), generation);
+            return new Tuple<>(INDEX_SHARD_SNAPSHOTS_FORMAT.read(shardContainer, generation, namedXContentRegistry, bigArrays), generation);
         }
         final Tuple<BlobStoreIndexShardSnapshots, Long> legacyIndex = buildBlobStoreIndexShardSnapshots(blobs, shardContainer);
         return new Tuple<>(legacyIndex.v1(), String.valueOf(legacyIndex.v2()));
@@ -2403,7 +2429,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
         long latest = latestGeneration(blobs);
         if (latest >= 0) {
             final BlobStoreIndexShardSnapshots shardSnapshots =
-                    INDEX_SHARD_SNAPSHOTS_FORMAT.read(shardContainer, Long.toString(latest), namedXContentRegistry);
+                    INDEX_SHARD_SNAPSHOTS_FORMAT.read(shardContainer, Long.toString(latest), namedXContentRegistry, bigArrays);
             return new Tuple<>(shardSnapshots, latest);
         } else if (blobs.stream().anyMatch(b -> b.startsWith(SNAPSHOT_PREFIX) || b.startsWith(INDEX_FILE_PREFIX)
                                                                               || b.startsWith(UPLOADED_DATA_BLOB_PREFIX))) {

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

@@ -28,14 +28,15 @@ import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.OutputStreamIndexOutput;
 import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.cluster.metadata.Metadata;
+import org.elasticsearch.common.CheckedConsumer;
 import org.elasticsearch.common.CheckedFunction;
 import org.elasticsearch.common.blobstore.BlobContainer;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.compress.CompressorFactory;
-import org.elasticsearch.common.io.Streams;
-import org.elasticsearch.common.io.stream.BytesStreamOutput;
+import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
 import org.elasticsearch.common.lucene.store.ByteArrayIndexInput;
 import org.elasticsearch.common.lucene.store.IndexOutputOutputStream;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.LoggingDeprecationHandler;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.common.xcontent.ToXContent;
@@ -44,10 +45,12 @@ import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.common.xcontent.XContentHelper;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.XContentType;
+import org.elasticsearch.core.internal.io.Streams;
 import org.elasticsearch.gateway.CorruptStateException;
 import org.elasticsearch.snapshots.SnapshotInfo;
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -101,9 +104,14 @@ public final class ChecksumBlobStoreFormat<T extends ToXContent> {
      * @param name          name to be translated into
      * @return parsed blob object
      */
-    public T read(BlobContainer blobContainer, String name, NamedXContentRegistry namedXContentRegistry) throws IOException {
+    public T read(BlobContainer blobContainer, String name, NamedXContentRegistry namedXContentRegistry,
+                  BigArrays bigArrays) throws IOException {
         String blobName = blobName(name);
-        return deserialize(blobName, namedXContentRegistry, Streams.readFully(blobContainer.readBlob(blobName)));
+        try (ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(bigArrays);
+             InputStream in = blobContainer.readBlob(blobName)) {
+            Streams.copy(in, out, false);
+            return deserialize(blobName, namedXContentRegistry, out.bytes());
+        }
     }
 
     public String blobName(String name) {
@@ -140,16 +148,18 @@ public final class ChecksumBlobStoreFormat<T extends ToXContent> {
      * @param name                blob name
      * @param compress            whether to use compression
      */
-    public void write(T obj, BlobContainer blobContainer, String name, boolean compress) throws IOException {
+    public void write(T obj, BlobContainer blobContainer, String name, boolean compress, BigArrays bigArrays) throws IOException {
         final String blobName = blobName(name);
-        final BytesReference bytes = serialize(obj, blobName, compress);
-        blobContainer.writeBlob(blobName, bytes.streamInput(), bytes.length(), false);
+        serialize(obj, blobName, compress, bigArrays, bytes -> blobContainer.writeBlob(blobName, bytes.streamInput(), bytes.length(),
+                false));
     }
 
-    public BytesReference serialize(final T obj, final String blobName, final boolean compress) throws IOException {
-        try (BytesStreamOutput outputStream = new BytesStreamOutput()) {
+    public void serialize(final T obj, final String blobName, final boolean compress, BigArrays bigArrays,
+                          CheckedConsumer<BytesReference, IOException> consumer) throws IOException {
+        try (ReleasableBytesStreamOutput outputStream = new ReleasableBytesStreamOutput(bigArrays)) {
             try (OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput(
-                    "ChecksumBlobStoreFormat.writeBlob(blob=\"" + blobName + "\")", blobName, outputStream, BUFFER_SIZE)) {
+                    "ChecksumBlobStoreFormat.writeBlob(blob=\"" + blobName + "\")", blobName,
+                    org.elasticsearch.common.io.Streams.noCloseStream(outputStream), BUFFER_SIZE)) {
                 CodecUtil.writeHeader(indexOutput, codec, VERSION);
                 try (OutputStream indexOutputOutputStream = new IndexOutputOutputStream(indexOutput) {
                     @Override
@@ -166,7 +176,7 @@ public final class ChecksumBlobStoreFormat<T extends ToXContent> {
                 }
                 CodecUtil.writeFooter(indexOutput);
             }
-            return outputStream.bytes();
+            consumer.accept(outputStream.bytes());
         }
     }
 }

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

@@ -23,6 +23,7 @@ import org.elasticsearch.cluster.metadata.RepositoryMetadata;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.UUIDs;
 import org.elasticsearch.common.blobstore.BlobPath;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.indices.recovery.RecoverySettings;
 import org.elasticsearch.repositories.RepositoryInfo;
@@ -37,10 +38,11 @@ public abstract class MeteredBlobStoreRepository extends BlobStoreRepository {
     public MeteredBlobStoreRepository(RepositoryMetadata metadata,
                                       NamedXContentRegistry namedXContentRegistry,
                                       ClusterService clusterService,
+                                      BigArrays bigArrays,
                                       RecoverySettings recoverySettings,
                                       BlobPath basePath,
                                       Map<String, String> location) {
-        super(metadata, namedXContentRegistry, clusterService, recoverySettings, basePath);
+        super(metadata, namedXContentRegistry, clusterService, bigArrays, recoverySettings, basePath);
         ThreadPool threadPool = clusterService.getClusterApplierService().threadPool();
         this.repositoryInfo = new RepositoryInfo(UUIDs.randomBase64UUID(),
             metadata.name(),

+ 3 - 2
server/src/main/java/org/elasticsearch/repositories/fs/FsRepository.java

@@ -29,6 +29,7 @@ import org.elasticsearch.common.blobstore.fs.FsBlobStore;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Setting.Property;
 import org.elasticsearch.common.unit.ByteSizeValue;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.indices.recovery.RecoverySettings;
@@ -71,8 +72,8 @@ public class FsRepository extends BlobStoreRepository {
      * Constructs a shared file system repository.
      */
     public FsRepository(RepositoryMetadata metadata, Environment environment, NamedXContentRegistry namedXContentRegistry,
-                        ClusterService clusterService, RecoverySettings recoverySettings) {
-        super(metadata, namedXContentRegistry, clusterService, recoverySettings, BlobPath.cleanPath());
+                        ClusterService clusterService, BigArrays bigArrays, RecoverySettings recoverySettings) {
+        super(metadata, namedXContentRegistry, clusterService, bigArrays, recoverySettings, BlobPath.cleanPath());
         this.environment = environment;
         String location = REPOSITORIES_LOCATION_SETTING.get(metadata.settings());
         if (location.isEmpty()) {

+ 16 - 13
server/src/test/java/org/elasticsearch/repositories/RepositoriesModuleTests.java

@@ -21,6 +21,7 @@ package org.elasticsearch.repositories;
 
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.MockBigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.indices.recovery.RecoverySettings;
@@ -44,7 +45,7 @@ public class RepositoriesModuleTests extends ESTestCase {
     private RepositoryPlugin plugin1;
     private RepositoryPlugin plugin2;
     private Repository.Factory factory;
-    private ThreadPool threadPool;
+    private TransportService transportService;
     private ClusterService clusterService;
     private RecoverySettings recoverySettings;
 
@@ -53,7 +54,9 @@ public class RepositoriesModuleTests extends ESTestCase {
         super.setUp();
         environment = mock(Environment.class);
         contentRegistry = mock(NamedXContentRegistry.class);
-        threadPool = mock(ThreadPool.class);
+        ThreadPool threadPool = mock(ThreadPool.class);
+        transportService = mock(TransportService.class);
+        when(transportService.getThreadPool()).thenReturn(threadPool);
         clusterService = mock(ClusterService.class);
         recoverySettings = mock(RecoverySettings.class);
         plugin1 = mock(RepositoryPlugin.class);
@@ -65,26 +68,26 @@ public class RepositoriesModuleTests extends ESTestCase {
     }
 
     public void testCanRegisterTwoRepositoriesWithDifferentTypes() {
-        when(plugin1.getRepositories(environment, contentRegistry, clusterService, recoverySettings))
+        when(plugin1.getRepositories(environment, contentRegistry, clusterService, MockBigArrays.NON_RECYCLING_INSTANCE, recoverySettings))
             .thenReturn(Collections.singletonMap("type1", factory));
-        when(plugin2.getRepositories(environment, contentRegistry, clusterService, recoverySettings))
+        when(plugin2.getRepositories(environment, contentRegistry, clusterService, MockBigArrays.NON_RECYCLING_INSTANCE, recoverySettings))
             .thenReturn(Collections.singletonMap("type2", factory));
 
         // Would throw
         new RepositoriesModule(
-            environment, repoPlugins, mock(TransportService.class), mock(ClusterService.class), threadPool, contentRegistry,
+            environment, repoPlugins, transportService, mock(ClusterService.class), MockBigArrays.NON_RECYCLING_INSTANCE, contentRegistry,
             recoverySettings);
     }
 
     public void testCannotRegisterTwoRepositoriesWithSameTypes() {
-        when(plugin1.getRepositories(environment, contentRegistry, clusterService, recoverySettings))
+        when(plugin1.getRepositories(environment, contentRegistry, clusterService, MockBigArrays.NON_RECYCLING_INSTANCE, recoverySettings))
             .thenReturn(Collections.singletonMap("type1", factory));
-        when(plugin2.getRepositories(environment, contentRegistry, clusterService, recoverySettings))
+        when(plugin2.getRepositories(environment, contentRegistry, clusterService, MockBigArrays.NON_RECYCLING_INSTANCE, recoverySettings))
             .thenReturn(Collections.singletonMap("type1", factory));
 
         IllegalArgumentException ex = expectThrows(IllegalArgumentException.class,
-            () -> new RepositoriesModule(environment, repoPlugins, mock(TransportService.class), clusterService,
-                threadPool, contentRegistry, recoverySettings));
+            () -> new RepositoriesModule(environment, repoPlugins, transportService, clusterService,
+                    MockBigArrays.NON_RECYCLING_INSTANCE, contentRegistry, recoverySettings));
 
         assertEquals("Repository type [type1] is already registered", ex.getMessage());
     }
@@ -97,20 +100,20 @@ public class RepositoriesModuleTests extends ESTestCase {
 
         IllegalArgumentException ex = expectThrows(IllegalArgumentException.class,
             () -> new RepositoriesModule(environment, repoPlugins, mock(TransportService.class), clusterService,
-                threadPool, contentRegistry, recoverySettings));
+                MockBigArrays.NON_RECYCLING_INSTANCE, contentRegistry, recoverySettings));
 
         assertEquals("Internal repository type [type1] is already registered", ex.getMessage());
     }
 
     public void testCannotRegisterNormalAndInternalRepositoriesWithSameTypes() {
-        when(plugin1.getRepositories(environment, contentRegistry, clusterService, recoverySettings))
+        when(plugin1.getRepositories(environment, contentRegistry, clusterService, MockBigArrays.NON_RECYCLING_INSTANCE, recoverySettings))
             .thenReturn(Collections.singletonMap("type1", factory));
         when(plugin2.getInternalRepositories(environment, contentRegistry, clusterService, recoverySettings))
             .thenReturn(Collections.singletonMap("type1", factory));
 
         IllegalArgumentException ex = expectThrows(IllegalArgumentException.class,
-            () -> new RepositoriesModule(environment, repoPlugins, mock(TransportService.class), clusterService, threadPool,
-                contentRegistry, recoverySettings));
+            () -> new RepositoriesModule(environment, repoPlugins, mock(TransportService.class), clusterService,
+                    MockBigArrays.NON_RECYCLING_INSTANCE, contentRegistry, recoverySettings));
 
         assertEquals("Internal repository type [type1] is already registered as a non-internal repository", ex.getMessage());
     }

+ 3 - 0
server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java

@@ -41,6 +41,7 @@ import org.elasticsearch.common.blobstore.BlobStore;
 import org.elasticsearch.common.component.Lifecycle;
 import org.elasticsearch.common.component.LifecycleListener;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.MockBigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.index.mapper.MapperService;
 import org.elasticsearch.index.shard.ShardId;
@@ -332,6 +333,7 @@ public class RepositoriesServiceTests extends ESTestCase {
             super(metadata,
                 mock(NamedXContentRegistry.class),
                 clusterService,
+                MockBigArrays.NON_RECYCLING_INSTANCE,
                 mock(RecoverySettings.class),
                 BlobPath.cleanPath(),
                 Map.of("bucket", "bucket-a"));
@@ -356,6 +358,7 @@ public class RepositoriesServiceTests extends ESTestCase {
             super(metadata,
                 mock(NamedXContentRegistry.class),
                 clusterService,
+                MockBigArrays.NON_RECYCLING_INSTANCE,
                 mock(RecoverySettings.class),
                 BlobPath.cleanPath(),
                 Map.of("bucket", "bucket-b"));

+ 2 - 0
server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java

@@ -32,6 +32,7 @@ import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.UUIDs;
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.MockBigArrays;
 import org.elasticsearch.core.internal.io.IOUtils;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.TestEnvironment;
@@ -205,6 +206,7 @@ public class BlobStoreRepositoryRestoreTests extends IndexShardTestCase {
         RepositoryMetadata repositoryMetadata = new RepositoryMetadata(randomAlphaOfLength(10), FsRepository.TYPE, settings);
         final ClusterService clusterService = BlobStoreTestUtil.mockClusterService(repositoryMetadata);
         final FsRepository repository = new FsRepository(repositoryMetadata, createEnvironment(), xContentRegistry(), clusterService,
+            MockBigArrays.NON_RECYCLING_INSTANCE,
             new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))) {
             @Override
             protected void assertSnapshotOrGenericThread() {

+ 4 - 2
server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryTests.java

@@ -28,6 +28,7 @@ import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.UUIDs;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeUnit;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.indices.recovery.RecoverySettings;
@@ -74,9 +75,10 @@ public class BlobStoreRepositoryTests extends ESSingleNodeTestCase {
 
         @Override
         public Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry,
-                                                               ClusterService clusterService, RecoverySettings recoverySettings) {
+                                                               ClusterService clusterService, BigArrays bigArrays,
+                                                               RecoverySettings recoverySettings) {
             return Collections.singletonMap(REPO_TYPE,
-                (metadata) -> new FsRepository(metadata, env, namedXContentRegistry, clusterService, recoverySettings) {
+                (metadata) -> new FsRepository(metadata, env, namedXContentRegistry, clusterService, bigArrays, recoverySettings) {
                     @Override
                     protected void assertSnapshotOrGenericThread() {
                         // eliminate thread name check as we access blobStore on test/main threads

+ 2 - 1
server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java

@@ -47,6 +47,7 @@ import org.elasticsearch.common.lucene.Lucene;
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeUnit;
+import org.elasticsearch.common.util.MockBigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.index.IndexSettings;
@@ -94,7 +95,7 @@ public class FsRepositoryTests extends ESTestCase {
             int numDocs = indexDocs(directory);
             RepositoryMetadata metadata = new RepositoryMetadata("test", "fs", settings);
             FsRepository repository = new FsRepository(metadata, new Environment(settings, null), NamedXContentRegistry.EMPTY,
-                BlobStoreTestUtil.mockClusterService(), new RecoverySettings(settings,
+                BlobStoreTestUtil.mockClusterService(), MockBigArrays.NON_RECYCLING_INSTANCE, new RecoverySettings(settings,
                 new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)));
             repository.start();
             final Settings indexSettings = Settings.builder().put(IndexMetadata.SETTING_INDEX_UUID, "myindexUUID").build();

+ 14 - 9
server/src/test/java/org/elasticsearch/snapshots/BlobStoreFormatTests.java

@@ -30,6 +30,7 @@ import org.elasticsearch.common.bytes.BytesArray;
 import org.elasticsearch.common.io.Streams;
 import org.elasticsearch.common.io.stream.BytesStreamOutput;
 import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.util.MockBigArrays;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.ToXContentFragment;
 import org.elasticsearch.common.xcontent.XContentBuilder;
@@ -104,12 +105,15 @@ public class BlobStoreFormatTests extends ESTestCase {
         ChecksumBlobStoreFormat<BlobObj> checksumSMILE = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj::fromXContent);
 
         // Write blobs in different formats
-        checksumSMILE.write(new BlobObj("checksum smile"), blobContainer, "check-smile", false);
-        checksumSMILE.write(new BlobObj("checksum smile compressed"), blobContainer, "check-smile-comp", true);
+        checksumSMILE.write(new BlobObj("checksum smile"), blobContainer, "check-smile", false, MockBigArrays.NON_RECYCLING_INSTANCE);
+        checksumSMILE.write(new BlobObj("checksum smile compressed"), blobContainer, "check-smile-comp", true,
+                MockBigArrays.NON_RECYCLING_INSTANCE);
 
         // Assert that all checksum blobs can be read
-        assertEquals(checksumSMILE.read(blobContainer, "check-smile", xContentRegistry()).getText(), "checksum smile");
-        assertEquals(checksumSMILE.read(blobContainer, "check-smile-comp", xContentRegistry()).getText(), "checksum smile compressed");
+        assertEquals(checksumSMILE.read(blobContainer, "check-smile", xContentRegistry(), MockBigArrays.NON_RECYCLING_INSTANCE).getText(),
+                "checksum smile");
+        assertEquals(checksumSMILE.read(blobContainer, "check-smile-comp", xContentRegistry(),
+                MockBigArrays.NON_RECYCLING_INSTANCE).getText(), "checksum smile compressed");
     }
 
     public void testCompressionIsApplied() throws IOException {
@@ -121,8 +125,8 @@ public class BlobStoreFormatTests extends ESTestCase {
         }
         ChecksumBlobStoreFormat<BlobObj> checksumFormat = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj::fromXContent);
         BlobObj blobObj = new BlobObj(veryRedundantText.toString());
-        checksumFormat.write(blobObj, blobContainer, "blob-comp", true);
-        checksumFormat.write(blobObj, blobContainer, "blob-not-comp", false);
+        checksumFormat.write(blobObj, blobContainer, "blob-comp", true, MockBigArrays.NON_RECYCLING_INSTANCE);
+        checksumFormat.write(blobObj, blobContainer, "blob-not-comp", false, MockBigArrays.NON_RECYCLING_INSTANCE);
         Map<String, BlobMetadata> blobs = blobContainer.listBlobsByPrefix("blob-");
         assertEquals(blobs.size(), 2);
         assertThat(blobs.get("blob-not-comp").length(), greaterThan(blobs.get("blob-comp").length()));
@@ -134,11 +138,12 @@ public class BlobStoreFormatTests extends ESTestCase {
         String testString = randomAlphaOfLength(randomInt(10000));
         BlobObj blobObj = new BlobObj(testString);
         ChecksumBlobStoreFormat<BlobObj> checksumFormat = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj::fromXContent);
-        checksumFormat.write(blobObj, blobContainer, "test-path", randomBoolean());
-        assertEquals(checksumFormat.read(blobContainer, "test-path", xContentRegistry()).getText(), testString);
+        checksumFormat.write(blobObj, blobContainer, "test-path", randomBoolean(), MockBigArrays.NON_RECYCLING_INSTANCE);
+        assertEquals(checksumFormat.read(blobContainer, "test-path", xContentRegistry(), MockBigArrays.NON_RECYCLING_INSTANCE).getText(),
+                testString);
         randomCorruption(blobContainer, "test-path");
         try {
-            checksumFormat.read(blobContainer, "test-path", xContentRegistry());
+            checksumFormat.read(blobContainer, "test-path", xContentRegistry(), MockBigArrays.NON_RECYCLING_INSTANCE);
             fail("Should have failed due to corruption");
         } catch (ElasticsearchCorruptionException ex) {
             assertThat(ex.getMessage(), containsString("test-path"));

+ 4 - 2
server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java

@@ -1399,6 +1399,8 @@ public class SnapshotResiliencyTests extends ESTestCase {
 
             private final ThreadPool threadPool;
 
+            private final BigArrays bigArrays;
+
             private Coordinator coordinator;
 
             TestClusterNode(DiscoveryNode node) throws IOException {
@@ -1500,7 +1502,7 @@ public class SnapshotResiliencyTests extends ESTestCase {
                 rerouteServiceSetOnce.set(rerouteService);
                 final IndexScopedSettings indexScopedSettings =
                     new IndexScopedSettings(settings, IndexScopedSettings.BUILT_IN_INDEX_SETTINGS);
-                final BigArrays bigArrays = new BigArrays(new PageCacheRecycler(settings), null, "test");
+                bigArrays = new BigArrays(new PageCacheRecycler(settings), null, "test");
                 final MapperRegistry mapperRegistry = new IndicesModule(Collections.emptyList()).getMapperRegistry();
                 indicesService = new IndicesService(
                     settings,
@@ -1667,7 +1669,7 @@ public class SnapshotResiliencyTests extends ESTestCase {
             private Repository.Factory getRepoFactory(Environment environment) {
                 // Run half the tests with the eventually consistent repository
                 if (blobStoreContext == null) {
-                    return metadata -> new FsRepository(metadata, environment, xContentRegistry(), clusterService,
+                    return metadata -> new FsRepository(metadata, environment, xContentRegistry(), clusterService, bigArrays,
                         recoverySettings) {
                         @Override
                         protected void assertSnapshotOrGenericThread() {

+ 3 - 1
server/src/test/java/org/elasticsearch/snapshots/mockstore/MockEventuallyConsistentRepository.java

@@ -31,6 +31,7 @@ import org.elasticsearch.common.blobstore.support.PlainBlobMetadata;
 import org.elasticsearch.common.bytes.BytesArray;
 import org.elasticsearch.common.io.Streams;
 import org.elasticsearch.common.util.Maps;
+import org.elasticsearch.common.util.MockBigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.indices.recovery.RecoverySettings;
 import org.elasticsearch.repositories.blobstore.BlobStoreRepository;
@@ -76,7 +77,8 @@ public class MockEventuallyConsistentRepository extends BlobStoreRepository {
         final RecoverySettings recoverySettings,
         final Context context,
         final Random random) {
-        super(metadata, namedXContentRegistry, clusterService, recoverySettings, BlobPath.cleanPath());
+        super(metadata, namedXContentRegistry, clusterService, MockBigArrays.NON_RECYCLING_INSTANCE, recoverySettings,
+                BlobPath.cleanPath());
         this.context = context;
         this.namedXContentRegistry = namedXContentRegistry;
         this.random = random;

+ 6 - 4
test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java

@@ -37,6 +37,7 @@ import org.elasticsearch.common.io.PathUtils;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Setting.Property;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.indices.recovery.RecoverySettings;
@@ -74,9 +75,10 @@ public class MockRepository extends FsRepository {
 
         @Override
         public Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry,
-                                                               ClusterService clusterService, RecoverySettings recoverySettings) {
+                                                               ClusterService clusterService, BigArrays bigArrays,
+                                                               RecoverySettings recoverySettings) {
             return Collections.singletonMap("mock", (metadata) ->
-                new MockRepository(metadata, env, namedXContentRegistry, clusterService, recoverySettings));
+                new MockRepository(metadata, env, namedXContentRegistry, clusterService, bigArrays, recoverySettings));
         }
 
         @Override
@@ -136,9 +138,9 @@ public class MockRepository extends FsRepository {
     private volatile boolean blocked = false;
 
     public MockRepository(RepositoryMetadata metadata, Environment environment,
-                          NamedXContentRegistry namedXContentRegistry, ClusterService clusterService,
+                          NamedXContentRegistry namedXContentRegistry, ClusterService clusterService, BigArrays bigArrays,
                           RecoverySettings recoverySettings) {
-        super(overrideSettings(metadata, environment), environment, namedXContentRegistry, clusterService, recoverySettings);
+        super(overrideSettings(metadata, environment), environment, namedXContentRegistry, clusterService, bigArrays, recoverySettings);
         randomControlIOExceptionRate = metadata.settings().getAsDouble("random_control_io_exception_rate", 0.0);
         randomDataFileIOExceptionRate = metadata.settings().getAsDouble("random_data_file_io_exception_rate", 0.0);
         useLuceneCorruptionException = metadata.settings().getAsBoolean("use_lucene_corruption", false);

+ 3 - 1
x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java

@@ -20,6 +20,7 @@ import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.env.Environment;
@@ -71,7 +72,8 @@ public class SourceOnlySnapshotIT extends ESIntegTestCase {
     public static final class MyPlugin extends Plugin implements RepositoryPlugin, EnginePlugin {
         @Override
         public Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry,
-                                                               ClusterService clusterService, RecoverySettings recoverySettings) {
+                                                               ClusterService clusterService, BigArrays bigArrays,
+                                                               RecoverySettings recoverySettings) {
             return Collections.singletonMap("source", SourceOnlySnapshotRepository.newRepositoryFactory());
         }
         @Override

+ 3 - 1
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackPlugin.java

@@ -31,6 +31,7 @@ import org.elasticsearch.common.settings.IndexScopedSettings;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.settings.SettingsFilter;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.NodeEnvironment;
@@ -354,7 +355,8 @@ public class XPackPlugin extends XPackClientPlugin implements ExtensiblePlugin,
 
     @Override
     public Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry,
-                                                           ClusterService clusterService, RecoverySettings recoverySettings) {
+                                                           ClusterService clusterService, BigArrays bigArrays,
+                                                           RecoverySettings recoverySettings) {
         return Collections.singletonMap("source", SourceOnlySnapshotRepository.newRepositoryFactory());
     }
 

+ 2 - 0
x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java

@@ -38,6 +38,7 @@ import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.lucene.uid.Versions;
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.MockBigArrays;
 import org.elasticsearch.common.xcontent.XContentHelper;
 import org.elasticsearch.core.internal.io.IOUtils;
 import org.elasticsearch.env.Environment;
@@ -368,6 +369,7 @@ public class SourceOnlySnapshotShardTests extends IndexShardTestCase {
         RepositoryMetadata repositoryMetadata = new RepositoryMetadata(randomAlphaOfLength(10), FsRepository.TYPE, settings);
         final ClusterService clusterService = BlobStoreTestUtil.mockClusterService(repositoryMetadata);
         final Repository repository = new FsRepository(repositoryMetadata, createEnvironment(), xContentRegistry(), clusterService,
+            MockBigArrays.NON_RECYCLING_INSTANCE,
             new RecoverySettings(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)));
         clusterService.addStateApplier(e -> repository.updateState(e.state()));
         // Apply state once to initialize repo properly like RepositoriesService would

+ 4 - 3
x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/LocalStateCompositeXPackPlugin.java

@@ -441,11 +441,12 @@ public class LocalStateCompositeXPackPlugin extends XPackPlugin implements Scrip
 
     @Override
     public Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry,
-                                                           ClusterService clusterService, RecoverySettings recoverySettings) {
+                                                           ClusterService clusterService, BigArrays bigArrays,
+                                                           RecoverySettings recoverySettings) {
         HashMap<String, Repository.Factory> repositories =
-            new HashMap<>(super.getRepositories(env, namedXContentRegistry, clusterService, recoverySettings));
+            new HashMap<>(super.getRepositories(env, namedXContentRegistry, clusterService, bigArrays, recoverySettings));
         filterPlugins(RepositoryPlugin.class).forEach(
-            r -> repositories.putAll(r.getRepositories(env, namedXContentRegistry, clusterService, recoverySettings)));
+            r -> repositories.putAll(r.getRepositories(env, namedXContentRegistry, clusterService, bigArrays, recoverySettings)));
         return repositories;
     }
 

+ 3 - 1
x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPrewarmingIntegTests.java

@@ -25,6 +25,7 @@ import org.elasticsearch.common.blobstore.BlobStore;
 import org.elasticsearch.common.blobstore.support.FilterBlobContainer;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeUnit;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.concurrent.AtomicArray;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.env.Environment;
@@ -327,11 +328,12 @@ public class SearchableSnapshotsPrewarmingIntegTests extends ESSingleNodeTestCas
             Environment env,
             NamedXContentRegistry namedXContentRegistry,
             ClusterService clusterService,
+            BigArrays bigArrays,
             RecoverySettings recoverySettings
         ) {
             return Collections.singletonMap(
                 "tracking",
-                (metadata) -> new FsRepository(metadata, env, namedXContentRegistry, clusterService, recoverySettings) {
+                (metadata) -> new FsRepository(metadata, env, namedXContentRegistry, clusterService, bigArrays, recoverySettings) {
 
                     @Override
                     protected void assertSnapshotOrGenericThread() {

+ 2 - 0
x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/SearchableSnapshotDirectoryTests.java

@@ -64,6 +64,7 @@ import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.unit.ByteSizeValue;
+import org.elasticsearch.common.util.MockBigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.core.internal.io.IOUtils;
 import org.elasticsearch.env.Environment;
@@ -548,6 +549,7 @@ public class SearchableSnapshotDirectoryTests extends ESTestCase {
                     ),
                     NamedXContentRegistry.EMPTY,
                     BlobStoreTestUtil.mockClusterService(repositoryMetadata),
+                    MockBigArrays.NON_RECYCLING_INSTANCE,
                     new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS))
                 ) {
 

+ 6 - 4
x-pack/plugin/voting-only-node/src/internalClusterTest/java/org/elasticsearch/cluster/coordination/VotingOnlyNodePluginTests.java

@@ -17,6 +17,7 @@ import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.Priority;
 import org.elasticsearch.common.blobstore.BlobStore;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.discovery.MasterNotDiscoveredException;
 import org.elasticsearch.env.Environment;
@@ -197,9 +198,10 @@ public class VotingOnlyNodePluginTests extends ESIntegTestCase {
 
         @Override
         public Map<String, Repository.Factory> getRepositories(Environment env, NamedXContentRegistry namedXContentRegistry,
-                                                               ClusterService clusterService, RecoverySettings recoverySettings) {
+                                                               ClusterService clusterService, BigArrays bigArrays,
+                                                               RecoverySettings recoverySettings) {
             return Collections.singletonMap("verifyaccess-fs", (metadata) ->
-                new AccessVerifyingRepo(metadata, env, namedXContentRegistry, clusterService, recoverySettings));
+                new AccessVerifyingRepo(metadata, env, namedXContentRegistry, clusterService, bigArrays, recoverySettings));
         }
 
         private static class AccessVerifyingRepo extends FsRepository {
@@ -207,8 +209,8 @@ public class VotingOnlyNodePluginTests extends ESIntegTestCase {
             private final ClusterService clusterService;
 
             private AccessVerifyingRepo(RepositoryMetadata metadata, Environment environment, NamedXContentRegistry namedXContentRegistry,
-                                       ClusterService clusterService, RecoverySettings recoverySettings) {
-                super(metadata, environment, namedXContentRegistry, clusterService, recoverySettings);
+                                       ClusterService clusterService, BigArrays bigArrays, RecoverySettings recoverySettings) {
+                super(metadata, environment, namedXContentRegistry, clusterService, bigArrays, recoverySettings);
                 this.clusterService = clusterService;
             }