Bläddra i källkod

make units required for byte size values

Michael McCandless 10 år sedan
förälder
incheckning
6723c72716

+ 2 - 2
src/main/java/org/elasticsearch/common/settings/Settings.java

@@ -56,7 +56,7 @@ import static org.elasticsearch.common.unit.SizeValue.parseSizeValue;
 import static org.elasticsearch.common.unit.TimeValue.parseTimeValue;
 
 /**
- * An immutable {@code Settings} implementation.
+ * An immutable settings implementation.
  */
 public final class Settings implements ToXContent {
 
@@ -986,7 +986,7 @@ public final class Settings implements ToXContent {
          * @return The builder
          */
         public Builder put(String setting, long value, ByteSizeUnit sizeUnit) {
-            put(setting, sizeUnit.toBytes(value));
+            put(setting, sizeUnit.toBytes(value) + "b");
             return this;
         }
 

+ 8 - 4
src/main/java/org/elasticsearch/common/unit/ByteSizeValue.java

@@ -30,9 +30,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Streamable;
 
-/**
- *
- */
 public class ByteSizeValue implements Serializable, Streamable {
 
     private long size;
@@ -206,8 +203,15 @@ public class ByteSizeValue implements Serializable, Streamable {
                 bytes = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 2)) * ByteSizeUnit.C5);
             } else if (lastTwoChars.endsWith("b")) {
                 bytes = Long.parseLong(sValue.substring(0, sValue.length() - 1));
+            } else if (sValue.equals("-1")) {
+                // Allow this special value to be unit-less:
+                bytes = -1;
+            } else if (sValue.equals("0")) {
+                // Allow this special value to be unit-less:
+                bytes = 0;
             } else {
-                bytes = Long.parseLong(sValue);
+                // Missing units:
+                throw new ElasticsearchParseException("Failed to parse setting [" + settingName + "] with value [" + sValue + "] as a size in bytes: unit is missing or unrecognized") ;
             }
         } catch (NumberFormatException e) {
             throw new ElasticsearchParseException("Failed to parse [" + sValue + "]", e);

+ 0 - 2
src/main/java/org/elasticsearch/common/unit/TimeValue.java

@@ -255,8 +255,6 @@ public class TimeValue implements Serializable, Streamable {
                 millis = 0;
             } else {
                 // Missing units:
-                // nocommit
-                System.out.println("MKM: Failed to parse setting [" + settingName + "] with value [" + sValue + "] as a time value: unit is missing or unrecognized");
                 throw new ElasticsearchParseException("Failed to parse setting [" + settingName + "] with value [" + sValue + "] as a time value: unit is missing or unrecognized");
             }
             return new TimeValue(millis, TimeUnit.MILLISECONDS);

+ 4 - 3
src/test/java/org/elasticsearch/cluster/settings/ClusterSettingsTests.java

@@ -24,6 +24,7 @@ import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResp
 import org.elasticsearch.cluster.metadata.MetaData;
 import org.elasticsearch.cluster.routing.allocation.decider.DisableAllocationDecider;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.discovery.DiscoverySettings;
 import org.elasticsearch.indices.store.IndicesStore;
 import org.elasticsearch.test.ElasticsearchIntegrationTest;
@@ -62,7 +63,7 @@ public class ClusterSettingsTests extends ElasticsearchIntegrationTest {
         String key2 = DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION;
         boolean value2 = true;
 
-        Settings transientSettings1 = Settings.builder().put(key1, value1).build();
+        Settings transientSettings1 = Settings.builder().put(key1, value1, ByteSizeUnit.BYTES).build();
         Settings persistentSettings1 = Settings.builder().put(key2, value2).build();
 
         ClusterUpdateSettingsResponse response1 = client().admin().cluster()
@@ -78,7 +79,7 @@ public class ClusterSettingsTests extends ElasticsearchIntegrationTest {
         assertThat(response1.getPersistentSettings().get(key1), nullValue());
         assertThat(response1.getPersistentSettings().get(key2), notNullValue());
 
-        Settings transientSettings2 = Settings.builder().put(key1, value1).put(key2, value2).build();
+        Settings transientSettings2 = Settings.builder().put(key1, value1, ByteSizeUnit.BYTES).put(key2, value2).build();
         Settings persistentSettings2 = Settings.EMPTY;
 
         ClusterUpdateSettingsResponse response2 = client().admin().cluster()
@@ -95,7 +96,7 @@ public class ClusterSettingsTests extends ElasticsearchIntegrationTest {
         assertThat(response2.getPersistentSettings().get(key2), nullValue());
 
         Settings transientSettings3 = Settings.EMPTY;
-        Settings persistentSettings3 = Settings.builder().put(key1, value1).put(key2, value2).build();
+        Settings persistentSettings3 = Settings.builder().put(key1, value1, ByteSizeUnit.BYTES).put(key2, value2).build();
 
         ClusterUpdateSettingsResponse response3 = client().admin().cluster()
                 .prepareUpdateSettings()

+ 3 - 2
src/test/java/org/elasticsearch/cluster/settings/SettingsValidatorTests.java

@@ -84,7 +84,8 @@ public class SettingsValidatorTests extends ElasticsearchTestCase {
 
         assertThat(Validator.PERCENTAGE.validate("", "asdasd"), notNullValue());
         assertThat(Validator.PERCENTAGE.validate("", "-1"), notNullValue());
-        assertThat(Validator.PERCENTAGE.validate("", "20"), notNullValue()); // we expect 20%
+        // nocommit require % too:
+        assertThat(Validator.PERCENTAGE.validate("", "20"), notNullValue());
         assertThat(Validator.PERCENTAGE.validate("", "-1%"), notNullValue());
         assertThat(Validator.PERCENTAGE.validate("", "101%"), notNullValue());
         assertThat(Validator.PERCENTAGE.validate("", "100%"), nullValue());
@@ -92,7 +93,7 @@ public class SettingsValidatorTests extends ElasticsearchTestCase {
         assertThat(Validator.PERCENTAGE.validate("", "0%"), nullValue());
 
         assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "asdasd"), notNullValue());
-        assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "20"), nullValue());
+        assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "20"), notNullValue());
         assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "20mb"), nullValue());
         assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "-1%"), notNullValue());
         assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "101%"), notNullValue());

+ 10 - 1
src/test/java/org/elasticsearch/common/unit/ByteSizeValueTests.java

@@ -83,7 +83,16 @@ public class ByteSizeValueTests extends ElasticsearchTestCase {
         assertThat(ByteSizeValue.parseBytesSizeValue("1b", "testParsing").toString(), is("1b"));
         assertThat(ByteSizeValue.parseBytesSizeValue("23kb", "testParsing").toString(), is("23kb"));
         assertThat(ByteSizeValue.parseBytesSizeValue("23k", "testParsing").toString(), is("23kb"));
-        assertThat(ByteSizeValue.parseBytesSizeValue("23", "testParsing").toString(), is("23b"));
+    }
+
+    @Test(expected = ElasticsearchParseException.class)
+    public void testFailOnMissingUnits() {
+        ByteSizeValue.parseBytesSizeValue("23", "test");
+    }
+
+    @Test(expected = ElasticsearchParseException.class)
+    public void testFailOnUnknownUnits() {
+        ByteSizeValue.parseBytesSizeValue("23jw", "test");
     }
 
     @Test(expected = ElasticsearchParseException.class)

+ 3 - 2
src/test/java/org/elasticsearch/common/util/BigArraysTests.java

@@ -24,6 +24,7 @@ import org.elasticsearch.cache.recycler.PageCacheRecycler;
 import org.elasticsearch.common.breaker.CircuitBreaker;
 import org.elasticsearch.common.breaker.CircuitBreakingException;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
 import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
 import org.elasticsearch.node.settings.NodeSettingsService;
@@ -336,7 +337,7 @@ public class BigArraysTests extends ElasticsearchSingleNodeTest {
         for (String type : Arrays.asList("Byte", "Int", "Long", "Float", "Double", "Object")) {
             HierarchyCircuitBreakerService hcbs = new HierarchyCircuitBreakerService(
                     Settings.builder()
-                            .put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, size - 1)
+                            .put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, size - 1, ByteSizeUnit.BYTES)
                             .build(),
                     new NodeSettingsService(Settings.EMPTY));
             BigArrays bigArrays = new BigArrays(null, hcbs).withCircuitBreaking();
@@ -356,7 +357,7 @@ public class BigArraysTests extends ElasticsearchSingleNodeTest {
             final long maxSize = randomIntBetween(1 << 10, 1 << 22);
             HierarchyCircuitBreakerService hcbs = new HierarchyCircuitBreakerService(
                     Settings.builder()
-                            .put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, maxSize)
+                            .put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, maxSize, ByteSizeUnit.BYTES)
                             .build(),
                     new NodeSettingsService(Settings.EMPTY));
             BigArrays bigArrays = new BigArrays(null, hcbs).withCircuitBreaking();

+ 2 - 1
src/test/java/org/elasticsearch/index/store/CorruptedFileTest.java

@@ -48,6 +48,7 @@ import org.elasticsearch.common.io.PathUtils;
 import org.elasticsearch.common.io.stream.BytesStreamOutput;
 import org.elasticsearch.common.lucene.Lucene;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.discovery.Discovery;
 import org.elasticsearch.gateway.GatewayAllocator;
 import org.elasticsearch.index.merge.policy.MergePolicyModule;
@@ -489,7 +490,7 @@ public class CorruptedFileTest extends ElasticsearchIntegrationTest {
                 .setType("fs").setSettings(settingsBuilder()
                         .put("location", randomRepoPath().toAbsolutePath())
                         .put("compress", randomBoolean())
-                        .put("chunk_size", randomIntBetween(100, 1000))));
+                        .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
         logger.info("--> snapshot");
         CreateSnapshotResponse createSnapshotResponse = client().admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test").get();
         assertThat(createSnapshotResponse.getSnapshotInfo().state(), equalTo(SnapshotState.PARTIAL));

+ 2 - 1
src/test/java/org/elasticsearch/index/translog/BufferedTranslogTests.java

@@ -20,6 +20,7 @@
 package org.elasticsearch.index.translog;
 
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.util.BigArrays;
 
 import java.io.IOException;
@@ -34,7 +35,7 @@ public class BufferedTranslogTests extends TranslogTests {
     protected Translog create(Path path) throws IOException {
         Settings build = Settings.settingsBuilder()
                 .put("index.translog.fs.type", TranslogWriter.Type.BUFFERED.name())
-                .put("index.translog.fs.buffer_size", 10 + randomInt(128 * 1024))
+                .put("index.translog.fs.buffer_size", 10 + randomInt(128 * 1024), ByteSizeUnit.BYTES)
                 .build();
         TranslogConfig translogConfig = new TranslogConfig(shardId, path, build, Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, null);
         return new Translog(translogConfig);

+ 4 - 3
src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryTests.java

@@ -38,6 +38,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand;
 import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDecider;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.unit.ByteSizeValue;
 import org.elasticsearch.discovery.DiscoveryService;
 import org.elasticsearch.index.recovery.RecoveryStats;
@@ -47,8 +48,8 @@ import org.elasticsearch.indices.IndicesService;
 import org.elasticsearch.indices.recovery.RecoveryState.Stage;
 import org.elasticsearch.indices.recovery.RecoveryState.Type;
 import org.elasticsearch.snapshots.SnapshotState;
-import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
+import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.elasticsearch.test.junit.annotations.TestLogging;
 import org.elasticsearch.test.store.MockFSDirectoryService;
 import org.elasticsearch.test.transport.MockTransportService;
@@ -126,8 +127,8 @@ public class IndexRecoveryTests extends ElasticsearchIntegrationTest {
         assertTrue(client().admin().cluster().prepareUpdateSettings()
                 .setTransientSettings(Settings.builder()
                                 // one chunk per sec..
-                                .put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC, chunkSize)
-                                .put(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, chunkSize)
+                                .put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC, chunkSize, ByteSizeUnit.BYTES)
+                                .put(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, chunkSize, ByteSizeUnit.BYTES)
                 )
                 .get().isAcknowledged());
     }

+ 8 - 2
src/test/java/org/elasticsearch/recovery/RecoverySettingsTest.java

@@ -19,6 +19,7 @@
 package org.elasticsearch.recovery;
 
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.indices.recovery.RecoverySettings;
 import org.elasticsearch.test.ElasticsearchSingleNodeTest;
 import org.junit.Test;
@@ -34,7 +35,7 @@ public class RecoverySettingsTest extends ElasticsearchSingleNodeTest {
 
     @Test
     public void testAllSettingsAreDynamicallyUpdatable() {
-        innerTestSettings(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, randomIntBetween(1, 200), new Validator() {
+        innerTestSettings(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, randomIntBetween(1, 200), ByteSizeUnit.BYTES, new Validator() {
             @Override
             public void validate(RecoverySettings recoverySettings, int expectedValue) {
                 assertEquals(expectedValue, recoverySettings.fileChunkSize().bytesAsInt());
@@ -46,7 +47,7 @@ public class RecoverySettingsTest extends ElasticsearchSingleNodeTest {
                 assertEquals(expectedValue, recoverySettings.translogOps());
             }
         });
-        innerTestSettings(RecoverySettings.INDICES_RECOVERY_TRANSLOG_SIZE, randomIntBetween(1, 200), new Validator() {
+        innerTestSettings(RecoverySettings.INDICES_RECOVERY_TRANSLOG_SIZE, randomIntBetween(1, 200), ByteSizeUnit.BYTES, new Validator() {
             @Override
             public void validate(RecoverySettings recoverySettings, int expectedValue) {
                 assertEquals(expectedValue, recoverySettings.translogSize().bytesAsInt());
@@ -127,6 +128,11 @@ public class RecoverySettingsTest extends ElasticsearchSingleNodeTest {
         validator.validate(getInstanceFromNode(RecoverySettings.class), newValue);
     }
 
+    private void innerTestSettings(String key, int newValue, ByteSizeUnit byteSizeUnit, Validator validator) {
+        client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder().put(key, newValue, byteSizeUnit)).get();
+        validator.validate(getInstanceFromNode(RecoverySettings.class), newValue);
+    }
+
     private void innerTestSettings(String key, boolean newValue, Validator validator) {
         client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder().put(key, newValue)).get();
         validator.validate(getInstanceFromNode(RecoverySettings.class), newValue);

+ 4 - 3
src/test/java/org/elasticsearch/recovery/RelocationTests.java

@@ -43,6 +43,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDeci
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.Priority;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.discovery.DiscoveryService;
 import org.elasticsearch.env.NodeEnvironment;
@@ -56,8 +57,8 @@ import org.elasticsearch.indices.recovery.RecoveryTarget;
 import org.elasticsearch.search.SearchHit;
 import org.elasticsearch.search.SearchHits;
 import org.elasticsearch.test.BackgroundIndexer;
-import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
+import org.elasticsearch.test.ElasticsearchIntegrationTest;
 import org.elasticsearch.test.junit.annotations.TestLogging;
 import org.elasticsearch.test.transport.MockTransportService;
 import org.elasticsearch.transport.*;
@@ -397,8 +398,8 @@ public class RelocationTests extends ElasticsearchIntegrationTest {
         assertTrue(client().admin().cluster().prepareUpdateSettings()
                 .setTransientSettings(Settings.builder()
                                 // one chunk per sec..
-                                .put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC, chunkSize)
-                                .put(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, chunkSize)
+                                .put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC, chunkSize, ByteSizeUnit.BYTES)
+                                .put(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, chunkSize, ByteSizeUnit.BYTES)
                 )
                 .get().isAcknowledged());
 

+ 1 - 1
src/test/java/org/elasticsearch/search/aggregations/bucket/TermsDocCountErrorTests.java

@@ -47,7 +47,7 @@ import static org.hamcrest.Matchers.lessThanOrEqualTo;
 import static org.hamcrest.core.IsNull.notNullValue;
 
 @ElasticsearchIntegrationTest.SuiteScopeTest
-public class TermsDocCountErrorTests extends ElasticsearchIntegrationTest{
+public class TermsDocCountErrorTests extends ElasticsearchIntegrationTest {
 
     private static final String STRING_FIELD_NAME = "s_value";
     private static final String LONG_FIELD_NAME = "l_value";

+ 5 - 4
src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreTests.java

@@ -35,19 +35,20 @@ import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotStatus;
 import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotsStatusResponse;
 import org.elasticsearch.action.admin.indices.recovery.ShardRecoveryResponse;
 import org.elasticsearch.client.Client;
+import org.elasticsearch.cluster.AbstractDiffable;
 import org.elasticsearch.cluster.ClusterService;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.ProcessedClusterStateUpdateTask;
-import org.elasticsearch.cluster.AbstractDiffable;
-import org.elasticsearch.cluster.metadata.MetaData;
 import org.elasticsearch.cluster.metadata.MetaData.Custom;
-import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
+import org.elasticsearch.cluster.metadata.MetaData;
 import org.elasticsearch.cluster.metadata.MetaDataIndexStateService;
+import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.Priority;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
@@ -638,7 +639,7 @@ public class DedicatedClusterSnapshotRestoreTests extends AbstractSnapshotTests
                 .setType("fs").setSettings(Settings.settingsBuilder()
                         .put("location", randomRepoPath())
                         .put("compress", randomBoolean())
-                        .put("chunk_size", randomIntBetween(100, 1000))));
+                        .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
 
         int initialIndices = between(1, 3);
         logger.info("--> create {} indices", initialIndices);

+ 3 - 2
src/test/java/org/elasticsearch/snapshots/RepositoriesTests.java

@@ -31,6 +31,7 @@ import org.elasticsearch.cluster.metadata.RepositoriesMetaData;
 import org.elasticsearch.cluster.metadata.RepositoryMetaData;
 import org.elasticsearch.common.io.FileSystemUtils;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.repositories.RepositoryException;
 import org.elasticsearch.repositories.RepositoryVerificationException;
 import org.elasticsearch.snapshots.mockstore.MockRepositoryModule;
@@ -155,7 +156,7 @@ public class RepositoriesTests extends AbstractSnapshotTests {
                 .setType("fs").setSettings(Settings.settingsBuilder()
                                 .put("location", randomRepoPath())
                                 .put("compress", randomBoolean())
-                                .put("chunk_size", randomIntBetween(5, 100))
+                                .put("chunk_size", randomIntBetween(5, 100), ByteSizeUnit.BYTES)
                 )
                 .setTimeout("0s").get();
         assertThat(putRepositoryResponse.isAcknowledged(), equalTo(false));
@@ -165,7 +166,7 @@ public class RepositoriesTests extends AbstractSnapshotTests {
                 .setType("fs").setSettings(Settings.settingsBuilder()
                                 .put("location", randomRepoPath())
                                 .put("compress", randomBoolean())
-                                .put("chunk_size", randomIntBetween(5, 100))
+                                .put("chunk_size", randomIntBetween(5, 100), ByteSizeUnit.BYTES)
                 ).get();
         assertThat(putRepositoryResponse.isAcknowledged(), equalTo(true));
 

+ 16 - 15
src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreTests.java

@@ -50,6 +50,7 @@ import org.elasticsearch.cluster.service.PendingClusterTask;
 import org.elasticsearch.common.Priority;
 import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.index.store.IndexStore;
@@ -91,7 +92,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
                 .setType("fs").setSettings(Settings.settingsBuilder()
                         .put("location", randomRepoPath())
                         .put("compress", randomBoolean())
-                        .put("chunk_size", randomIntBetween(100, 1000))));
+                        .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
 
         createIndex("test-idx-1", "test-idx-2", "test-idx-3");
         ensureGreen();
@@ -232,7 +233,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
                 .setType("fs").setSettings(Settings.settingsBuilder()
                         .put("location", randomRepoPath())
                         .put("compress", randomBoolean())
-                        .put("chunk_size", randomIntBetween(100, 1000))));
+                        .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
 
         createIndex("test");
         String originalIndexUUID = client().admin().indices().prepareGetSettings("test").get().getSetting("test", IndexMetaData.SETTING_UUID);
@@ -281,7 +282,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
                 .setType("fs").setSettings(Settings.settingsBuilder()
                         .put("location", randomRepoPath())
                         .put("compress", randomBoolean())
-                        .put("chunk_size", randomIntBetween(100, 1000))));
+                        .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
 
         logger.info("--> create index with foo type");
         assertAcked(prepareCreate("test-idx", 2, Settings.builder()
@@ -761,7 +762,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
                 .setType("fs").setSettings(Settings.settingsBuilder()
                         .put("location", repo)
                         .put("compress", false)
-                        .put("chunk_size", randomIntBetween(100, 1000))));
+                        .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
 
         createIndex("test-idx");
         ensureGreen();
@@ -818,7 +819,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
                 .setType("fs").setSettings(Settings.settingsBuilder()
                         .put("location", repo)
                         .put("compress", false)
-                        .put("chunk_size", randomIntBetween(100, 1000))));
+                        .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
 
         createIndex("test-idx-1", "test-idx-2");
         ensureYellow();
@@ -857,7 +858,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
                 .setType("fs").setSettings(Settings.settingsBuilder()
                         .put("location", repo)
                         .put("compress", false)
-                        .put("chunk_size", randomIntBetween(100, 1000))));
+                        .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
 
         createIndex("test-idx-1", "test-idx-2");
         ensureYellow();
@@ -892,7 +893,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
                 .setType("fs").setSettings(Settings.settingsBuilder()
                         .put("location", repo)
                         .put("compress", false)
-                        .put("chunk_size", randomIntBetween(100, 1000))));
+                        .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
 
         createIndex("test-idx-1", "test-idx-2");
         ensureYellow();
@@ -1247,7 +1248,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
                 .setType("fs").setSettings(Settings.settingsBuilder()
                         .put("location", repositoryLocation)
                         .put("compress", randomBoolean())
-                        .put("chunk_size", randomIntBetween(100, 1000))));
+                        .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
 
         createIndex("test-idx");
         ensureGreen();
@@ -1307,7 +1308,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
                 .setType("fs").setSettings(Settings.settingsBuilder()
                         .put("location", repositoryLocation)
                         .put("compress", randomBoolean())
-                        .put("chunk_size", randomIntBetween(1000, 10000))
+                        .put("chunk_size", randomIntBetween(1000, 10000), ByteSizeUnit.BYTES)
                         .put("max_restore_bytes_per_sec", throttleRestore ? "0.5k" : "0")
                         .put("max_snapshot_bytes_per_sec", throttleSnapshot ? "0.5k" : "0")));
 
@@ -1463,7 +1464,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
                 .setType("fs").setSettings(Settings.settingsBuilder()
                         .put("location", randomRepoPath())
                         .put("compress", randomBoolean())
-                        .put("chunk_size", randomIntBetween(100, 1000))));
+                        .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
 
         // Create index on 1 nodes and make sure each node has a primary by setting no replicas
         assertAcked(prepareCreate("test-idx", 1, Settings.builder().put("number_of_replicas", 0)));
@@ -1511,7 +1512,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
                 .setType("fs").setSettings(Settings.settingsBuilder()
                         .put("location", randomRepoPath())
                         .put("compress", randomBoolean())
-                        .put("chunk_size", randomIntBetween(100, 1000))));
+                        .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
 
         // only one shard
         assertAcked(prepareCreate("test").setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)));
@@ -1574,7 +1575,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
                 .setType("fs").setSettings(Settings.settingsBuilder()
                         .put("location", randomRepoPath())
                         .put("compress", randomBoolean())
-                        .put("chunk_size", randomIntBetween(100, 1000))));
+                        .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
 
         logger.info("--> create test index with synonyms search analyzer");
 
@@ -1691,7 +1692,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
                 .setType(MockRepositoryModule.class.getCanonicalName()).setSettings(Settings.settingsBuilder()
                         .put("location", randomRepoPath())
                         .put("compress", randomBoolean())
-                        .put("chunk_size", randomIntBetween(100, 1000))
+                        .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)
                         .put("block_on_init", true)
                 ));
 
@@ -1743,7 +1744,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
                 .setType(MockRepositoryModule.class.getCanonicalName()).setSettings(Settings.settingsBuilder()
                                 .put("location", randomRepoPath())
                                 .put("compress", randomBoolean())
-                                .put("chunk_size", randomIntBetween(100, 1000))
+                                .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)
                 ));
 
         createIndex("test-idx");
@@ -1823,7 +1824,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
                 .setType("fs").setSettings(Settings.settingsBuilder()
                         .put("location", randomRepoPath())
                         .put("compress", randomBoolean())
-                        .put("chunk_size", randomIntBetween(100, 1000))));
+                        .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
 
         assertAcked(prepareCreate("test-idx", 0, settingsBuilder().put("number_of_shards", between(1, 20))
                 .put("number_of_replicas", 0)));

+ 3 - 2
src/test/java/org/elasticsearch/snapshots/SnapshotBackwardsCompatibilityTest.java

@@ -32,6 +32,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
 import org.elasticsearch.common.io.FileSystemUtils;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.test.ElasticsearchBackwardsCompatIntegrationTest;
 import org.junit.Test;
 
@@ -58,7 +59,7 @@ public class SnapshotBackwardsCompatibilityTest extends ElasticsearchBackwardsCo
                 .setType("fs").setSettings(Settings.settingsBuilder()
                         .put("location", randomRepoPath().toAbsolutePath())
                         .put("compress", randomBoolean())
-                        .put("chunk_size", randomIntBetween(100, 1000))));
+                        .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
         String[] indicesBefore = new String[randomIntBetween(2,5)];
         String[] indicesAfter = new String[randomIntBetween(2,5)];
         for (int i = 0; i < indicesBefore.length; i++) {
@@ -168,7 +169,7 @@ public class SnapshotBackwardsCompatibilityTest extends ElasticsearchBackwardsCo
                 .setType("fs").setSettings(Settings.settingsBuilder()
                         .put("location", tempDir)
                         .put("compress", randomBoolean())
-                        .put("chunk_size", randomIntBetween(100, 1000))));
+                        .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
 
         // only one shard
         assertAcked(prepareCreate("test").setSettings(Settings.builder()