require setting name to Memory/ByteSizeValue parsers
This commit is contained in:
parent
b03d7584b4
commit
464b15447f
|
@ -564,10 +564,11 @@ public class DiskThresholdDecider extends AllocationDecider {
|
||||||
* a ByteSizeValue of 0 bytes if the value cannot be parsed.
|
* a ByteSizeValue of 0 bytes if the value cannot be parsed.
|
||||||
*/
|
*/
|
||||||
public ByteSizeValue thresholdBytesFromWatermark(String watermark) {
|
public ByteSizeValue thresholdBytesFromWatermark(String watermark) {
|
||||||
|
// nocommit: why be lenient here?
|
||||||
try {
|
try {
|
||||||
return ByteSizeValue.parseBytesSizeValue(watermark);
|
return ByteSizeValue.parseBytesSizeValue(watermark, "DiskThresholdDecider watermark");
|
||||||
} catch (ElasticsearchParseException ex) {
|
} catch (ElasticsearchParseException ex) {
|
||||||
return ByteSizeValue.parseBytesSizeValue("0b");
|
return ByteSizeValue.parseBytesSizeValue("0b", "DiskThresholdDecider watermark");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -581,7 +582,7 @@ public class DiskThresholdDecider extends AllocationDecider {
|
||||||
return true;
|
return true;
|
||||||
} catch (ElasticsearchParseException e) {
|
} catch (ElasticsearchParseException e) {
|
||||||
try {
|
try {
|
||||||
ByteSizeValue.parseBytesSizeValue(watermark);
|
ByteSizeValue.parseBytesSizeValue(watermark, "DiskThresholdDecider watermark");
|
||||||
return true;
|
return true;
|
||||||
} catch (ElasticsearchParseException ex) {
|
} catch (ElasticsearchParseException ex) {
|
||||||
return false;
|
return false;
|
||||||
|
|
|
@ -197,7 +197,7 @@ public interface Validator {
|
||||||
@Override
|
@Override
|
||||||
public String validate(String setting, String value) {
|
public String validate(String setting, String value) {
|
||||||
try {
|
try {
|
||||||
parseBytesSizeValue(value);
|
parseBytesSizeValue(value, setting);
|
||||||
} catch (ElasticsearchParseException ex) {
|
} catch (ElasticsearchParseException ex) {
|
||||||
return ex.getMessage();
|
return ex.getMessage();
|
||||||
}
|
}
|
||||||
|
@ -247,7 +247,7 @@ public interface Validator {
|
||||||
@Override
|
@Override
|
||||||
public String validate(String setting, String value) {
|
public String validate(String setting, String value) {
|
||||||
try {
|
try {
|
||||||
parseBytesSizeValueOrHeapRatio(value);
|
parseBytesSizeValueOrHeapRatio(value, setting);
|
||||||
} catch (ElasticsearchParseException ex) {
|
} catch (ElasticsearchParseException ex) {
|
||||||
return ex.getMessage();
|
return ex.getMessage();
|
||||||
}
|
}
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -440,7 +440,7 @@ public final class Settings implements ToXContent {
|
||||||
* returns the default value provided.
|
* returns the default value provided.
|
||||||
*/
|
*/
|
||||||
public ByteSizeValue getAsBytesSize(String setting, ByteSizeValue defaultValue) throws SettingsException {
|
public ByteSizeValue getAsBytesSize(String setting, ByteSizeValue defaultValue) throws SettingsException {
|
||||||
return parseBytesSizeValue(get(setting), defaultValue);
|
return parseBytesSizeValue(get(setting), defaultValue, setting);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -448,7 +448,14 @@ public final class Settings implements ToXContent {
|
||||||
* returns the default value provided.
|
* returns the default value provided.
|
||||||
*/
|
*/
|
||||||
public ByteSizeValue getAsBytesSize(String[] settings, ByteSizeValue defaultValue) throws SettingsException {
|
public ByteSizeValue getAsBytesSize(String[] settings, ByteSizeValue defaultValue) throws SettingsException {
|
||||||
return parseBytesSizeValue(get(settings), defaultValue);
|
// NOTE: duplicated from get(String[]) so we can pass which setting name was actually used to parseBytesSizeValue
|
||||||
|
for (String setting : settings) {
|
||||||
|
String retVal = get(setting);
|
||||||
|
if (retVal != null) {
|
||||||
|
parseBytesSizeValue(get(settings), defaultValue, setting);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return defaultValue;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -457,7 +464,7 @@ public final class Settings implements ToXContent {
|
||||||
* (eg. 12%). If it does not exists, parses the default value provided.
|
* (eg. 12%). If it does not exists, parses the default value provided.
|
||||||
*/
|
*/
|
||||||
public ByteSizeValue getAsMemory(String setting, String defaultValue) throws SettingsException {
|
public ByteSizeValue getAsMemory(String setting, String defaultValue) throws SettingsException {
|
||||||
return MemorySizeValue.parseBytesSizeValueOrHeapRatio(get(setting, defaultValue));
|
return MemorySizeValue.parseBytesSizeValueOrHeapRatio(get(setting, defaultValue), setting);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -466,7 +473,14 @@ public final class Settings implements ToXContent {
|
||||||
* (eg. 12%). If it does not exists, parses the default value provided.
|
* (eg. 12%). If it does not exists, parses the default value provided.
|
||||||
*/
|
*/
|
||||||
public ByteSizeValue getAsMemory(String[] settings, String defaultValue) throws SettingsException {
|
public ByteSizeValue getAsMemory(String[] settings, String defaultValue) throws SettingsException {
|
||||||
return MemorySizeValue.parseBytesSizeValueOrHeapRatio(get(settings, defaultValue));
|
// NOTE: duplicated from get(String[]) so we can pass which setting name was actually used to parseBytesSizeValueOrHeapRatio
|
||||||
|
for (String setting : settings) {
|
||||||
|
String retVal = get(setting);
|
||||||
|
if (retVal != null) {
|
||||||
|
return MemorySizeValue.parseBytesSizeValueOrHeapRatio(retVal, setting);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return MemorySizeValue.parseBytesSizeValueOrHeapRatio(defaultValue, settings[0]);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -19,16 +19,17 @@
|
||||||
|
|
||||||
package org.elasticsearch.common.unit;
|
package org.elasticsearch.common.unit;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.Serializable;
|
||||||
|
import java.util.Locale;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
import org.elasticsearch.ElasticsearchParseException;
|
import org.elasticsearch.ElasticsearchParseException;
|
||||||
import org.elasticsearch.common.Strings;
|
import org.elasticsearch.common.Strings;
|
||||||
import org.elasticsearch.common.io.stream.StreamInput;
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
import org.elasticsearch.common.io.stream.Streamable;
|
import org.elasticsearch.common.io.stream.Streamable;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.io.Serializable;
|
|
||||||
import java.util.Locale;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
|
@ -171,11 +172,12 @@ public class ByteSizeValue implements Serializable, Streamable {
|
||||||
return Strings.format1Decimals(value, suffix);
|
return Strings.format1Decimals(value, suffix);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static ByteSizeValue parseBytesSizeValue(String sValue) throws ElasticsearchParseException {
|
public static ByteSizeValue parseBytesSizeValue(String sValue, String settingName) throws ElasticsearchParseException {
|
||||||
return parseBytesSizeValue(sValue, null);
|
return parseBytesSizeValue(sValue, null, settingName);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static ByteSizeValue parseBytesSizeValue(String sValue, ByteSizeValue defaultValue) throws ElasticsearchParseException {
|
public static ByteSizeValue parseBytesSizeValue(String sValue, ByteSizeValue defaultValue, String settingName) throws ElasticsearchParseException {
|
||||||
|
settingName = Objects.requireNonNull(settingName);
|
||||||
if (sValue == null) {
|
if (sValue == null) {
|
||||||
return defaultValue;
|
return defaultValue;
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,8 @@
|
||||||
|
|
||||||
package org.elasticsearch.common.unit;
|
package org.elasticsearch.common.unit;
|
||||||
|
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
import org.elasticsearch.ElasticsearchParseException;
|
import org.elasticsearch.ElasticsearchParseException;
|
||||||
import org.elasticsearch.monitor.jvm.JvmInfo;
|
import org.elasticsearch.monitor.jvm.JvmInfo;
|
||||||
|
|
||||||
|
@ -31,7 +33,8 @@ public enum MemorySizeValue {
|
||||||
/** Parse the provided string as a memory size. This method either accepts absolute values such as
|
/** Parse the provided string as a memory size. This method either accepts absolute values such as
|
||||||
* <tt>42</tt> (default assumed unit is byte) or <tt>2mb</tt>, or percentages of the heap size: if
|
* <tt>42</tt> (default assumed unit is byte) or <tt>2mb</tt>, or percentages of the heap size: if
|
||||||
* the heap is 1G, <tt>10%</tt> will be parsed as <tt>100mb</tt>. */
|
* the heap is 1G, <tt>10%</tt> will be parsed as <tt>100mb</tt>. */
|
||||||
public static ByteSizeValue parseBytesSizeValueOrHeapRatio(String sValue) {
|
public static ByteSizeValue parseBytesSizeValueOrHeapRatio(String sValue, String settingName) {
|
||||||
|
settingName = Objects.requireNonNull(settingName);
|
||||||
if (sValue != null && sValue.endsWith("%")) {
|
if (sValue != null && sValue.endsWith("%")) {
|
||||||
final String percentAsString = sValue.substring(0, sValue.length() - 1);
|
final String percentAsString = sValue.substring(0, sValue.length() - 1);
|
||||||
try {
|
try {
|
||||||
|
@ -44,7 +47,7 @@ public enum MemorySizeValue {
|
||||||
throw new ElasticsearchParseException("Failed to parse [" + percentAsString + "] as a double", e);
|
throw new ElasticsearchParseException("Failed to parse [" + percentAsString + "] as a double", e);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
return parseBytesSizeValue(sValue);
|
return parseBytesSizeValue(sValue, settingName);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -129,7 +129,7 @@ public final class EngineConfig {
|
||||||
public static final TimeValue DEFAULT_REFRESH_INTERVAL = new TimeValue(1, TimeUnit.SECONDS);
|
public static final TimeValue DEFAULT_REFRESH_INTERVAL = new TimeValue(1, TimeUnit.SECONDS);
|
||||||
public static final TimeValue DEFAULT_GC_DELETES = TimeValue.timeValueSeconds(60);
|
public static final TimeValue DEFAULT_GC_DELETES = TimeValue.timeValueSeconds(60);
|
||||||
public static final ByteSizeValue DEFAUTL_INDEX_BUFFER_SIZE = new ByteSizeValue(64, ByteSizeUnit.MB);
|
public static final ByteSizeValue DEFAUTL_INDEX_BUFFER_SIZE = new ByteSizeValue(64, ByteSizeUnit.MB);
|
||||||
public static final ByteSizeValue INACTIVE_SHARD_INDEXING_BUFFER = ByteSizeValue.parseBytesSizeValue("500kb");
|
public static final ByteSizeValue INACTIVE_SHARD_INDEXING_BUFFER = ByteSizeValue.parseBytesSizeValue("500kb", "INACTIVE_SHARD_INDEXING_BUFFER");
|
||||||
|
|
||||||
public static final String DEFAULT_VERSION_MAP_SIZE = "25%";
|
public static final String DEFAULT_VERSION_MAP_SIZE = "25%";
|
||||||
|
|
||||||
|
@ -180,7 +180,7 @@ public final class EngineConfig {
|
||||||
double percent = Double.parseDouble(versionMapSizeSetting.substring(0, versionMapSizeSetting.length() - 1));
|
double percent = Double.parseDouble(versionMapSizeSetting.substring(0, versionMapSizeSetting.length() - 1));
|
||||||
versionMapSize = new ByteSizeValue((long) (((double) indexingBufferSize.bytes() * (percent / 100))));
|
versionMapSize = new ByteSizeValue((long) (((double) indexingBufferSize.bytes() * (percent / 100))));
|
||||||
} else {
|
} else {
|
||||||
versionMapSize = ByteSizeValue.parseBytesSizeValue(versionMapSizeSetting);
|
versionMapSize = ByteSizeValue.parseBytesSizeValue(versionMapSizeSetting, "VersionMapSize");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -165,7 +165,7 @@ public class SourceFieldMapper extends AbstractFieldMapper implements RootMapper
|
||||||
builder.compressThreshold(((Number) fieldNode).longValue());
|
builder.compressThreshold(((Number) fieldNode).longValue());
|
||||||
builder.compress(true);
|
builder.compress(true);
|
||||||
} else {
|
} else {
|
||||||
builder.compressThreshold(ByteSizeValue.parseBytesSizeValue(fieldNode.toString()).bytes());
|
builder.compressThreshold(ByteSizeValue.parseBytesSizeValue(fieldNode.toString(), "compress_threshold").bytes());
|
||||||
builder.compress(true);
|
builder.compress(true);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -42,7 +42,7 @@ public final class TranslogConfig {
|
||||||
public static final String INDEX_TRANSLOG_FS_TYPE = "index.translog.fs.type";
|
public static final String INDEX_TRANSLOG_FS_TYPE = "index.translog.fs.type";
|
||||||
public static final String INDEX_TRANSLOG_BUFFER_SIZE = "index.translog.fs.buffer_size";
|
public static final String INDEX_TRANSLOG_BUFFER_SIZE = "index.translog.fs.buffer_size";
|
||||||
public static final String INDEX_TRANSLOG_SYNC_INTERVAL = "index.translog.sync_interval";
|
public static final String INDEX_TRANSLOG_SYNC_INTERVAL = "index.translog.sync_interval";
|
||||||
public static final ByteSizeValue INACTIVE_SHARD_TRANSLOG_BUFFER = ByteSizeValue.parseBytesSizeValue("1kb");
|
public static final ByteSizeValue INACTIVE_SHARD_TRANSLOG_BUFFER = ByteSizeValue.parseBytesSizeValue("1kb", "INACTIVE_SHARD_TRANSLOG_BUFFER");
|
||||||
|
|
||||||
private final TimeValue syncInterval;
|
private final TimeValue syncInterval;
|
||||||
private final BigArrays bigArrays;
|
private final BigArrays bigArrays;
|
||||||
|
@ -73,7 +73,7 @@ public final class TranslogConfig {
|
||||||
this.threadPool = threadPool;
|
this.threadPool = threadPool;
|
||||||
this.bigArrays = bigArrays;
|
this.bigArrays = bigArrays;
|
||||||
this.type = TranslogWriter.Type.fromString(indexSettings.get(INDEX_TRANSLOG_FS_TYPE, TranslogWriter.Type.BUFFERED.name()));
|
this.type = TranslogWriter.Type.fromString(indexSettings.get(INDEX_TRANSLOG_FS_TYPE, TranslogWriter.Type.BUFFERED.name()));
|
||||||
this.bufferSize = (int) indexSettings.getAsBytesSize(INDEX_TRANSLOG_BUFFER_SIZE, ByteSizeValue.parseBytesSizeValue("64k")).bytes(); // Not really interesting, updated by IndexingMemoryController...
|
this.bufferSize = (int) indexSettings.getAsBytesSize(INDEX_TRANSLOG_BUFFER_SIZE, ByteSizeValue.parseBytesSizeValue("64k", INDEX_TRANSLOG_BUFFER_SIZE)).bytes(); // Not really interesting, updated by IndexingMemoryController...
|
||||||
|
|
||||||
syncInterval = indexSettings.getAsTime(INDEX_TRANSLOG_SYNC_INTERVAL, TimeValue.timeValueSeconds(5));
|
syncInterval = indexSettings.getAsTime(INDEX_TRANSLOG_SYNC_INTERVAL, TimeValue.timeValueSeconds(5));
|
||||||
if (syncInterval.millis() > 0 && threadPool != null) {
|
if (syncInterval.millis() > 0 && threadPool != null) {
|
||||||
|
|
|
@ -65,7 +65,7 @@ public class IndicesFilterCache extends AbstractComponent implements QueryCache,
|
||||||
public IndicesFilterCache(Settings settings) {
|
public IndicesFilterCache(Settings settings) {
|
||||||
super(settings);
|
super(settings);
|
||||||
final String sizeString = settings.get(INDICES_CACHE_QUERY_SIZE, "10%");
|
final String sizeString = settings.get(INDICES_CACHE_QUERY_SIZE, "10%");
|
||||||
final ByteSizeValue size = MemorySizeValue.parseBytesSizeValueOrHeapRatio(sizeString);
|
final ByteSizeValue size = MemorySizeValue.parseBytesSizeValueOrHeapRatio(sizeString, INDICES_CACHE_QUERY_SIZE);
|
||||||
final int count = settings.getAsInt(INDICES_CACHE_QUERY_COUNT, 100000);
|
final int count = settings.getAsInt(INDICES_CACHE_QUERY_COUNT, 100000);
|
||||||
logger.debug("using [node] weighted filter cache with size [{}], actual_size [{}], max filter count [{}]",
|
logger.debug("using [node] weighted filter cache with size [{}], actual_size [{}], max filter count [{}]",
|
||||||
sizeString, size, count);
|
sizeString, size, count);
|
||||||
|
|
|
@ -128,7 +128,7 @@ public class IndicesQueryCache extends AbstractComponent implements RemovalListe
|
||||||
}
|
}
|
||||||
|
|
||||||
private void buildCache() {
|
private void buildCache() {
|
||||||
long sizeInBytes = MemorySizeValue.parseBytesSizeValueOrHeapRatio(size).bytes();
|
long sizeInBytes = MemorySizeValue.parseBytesSizeValueOrHeapRatio(size, INDICES_CACHE_QUERY_SIZE).bytes();
|
||||||
|
|
||||||
CacheBuilder<Key, Value> cacheBuilder = CacheBuilder.newBuilder()
|
CacheBuilder<Key, Value> cacheBuilder = CacheBuilder.newBuilder()
|
||||||
.maximumWeight(sizeInBytes).weigher(new QueryCacheWeigher()).removalListener(this);
|
.maximumWeight(sizeInBytes).weigher(new QueryCacheWeigher()).removalListener(this);
|
||||||
|
|
|
@ -73,7 +73,7 @@ public class RecoverySettings extends AbstractComponent implements Closeable {
|
||||||
public static final String INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT = "indices.recovery.internal_action_long_timeout";
|
public static final String INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT = "indices.recovery.internal_action_long_timeout";
|
||||||
|
|
||||||
|
|
||||||
public static final long SMALL_FILE_CUTOFF_BYTES = ByteSizeValue.parseBytesSizeValue("5mb").bytes();
|
public static final long SMALL_FILE_CUTOFF_BYTES = ByteSizeValue.parseBytesSizeValue("5mb", "SMALL_FILE_CUTOFF_BYTES").bytes();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Use {@link #INDICES_RECOVERY_MAX_BYTES_PER_SEC} instead
|
* Use {@link #INDICES_RECOVERY_MAX_BYTES_PER_SEC} instead
|
||||||
|
|
|
@ -144,7 +144,7 @@ public abstract class RestRequest extends ContextAndHeaderHolder implements ToXC
|
||||||
}
|
}
|
||||||
|
|
||||||
public ByteSizeValue paramAsSize(String key, ByteSizeValue defaultValue) {
|
public ByteSizeValue paramAsSize(String key, ByteSizeValue defaultValue) {
|
||||||
return parseBytesSizeValue(param(key), defaultValue);
|
return parseBytesSizeValue(param(key), defaultValue, key);
|
||||||
}
|
}
|
||||||
|
|
||||||
public String[] paramAsStringArray(String key, String[] defaultValue) {
|
public String[] paramAsStringArray(String key, String[] defaultValue) {
|
||||||
|
|
|
@ -39,8 +39,8 @@ public class FsAppendBenchmark {
|
||||||
Path path = PathUtils.get("work/test.log");
|
Path path = PathUtils.get("work/test.log");
|
||||||
IOUtils.deleteFilesIgnoringExceptions(path);
|
IOUtils.deleteFilesIgnoringExceptions(path);
|
||||||
|
|
||||||
int CHUNK = (int) ByteSizeValue.parseBytesSizeValue("1k").bytes();
|
int CHUNK = (int) ByteSizeValue.parseBytesSizeValue("1k", "CHUNK").bytes();
|
||||||
long DATA = ByteSizeValue.parseBytesSizeValue("10gb").bytes();
|
long DATA = ByteSizeValue.parseBytesSizeValue("10gb", "DATA").bytes();
|
||||||
|
|
||||||
byte[] data = new byte[CHUNK];
|
byte[] data = new byte[CHUNK];
|
||||||
new Random().nextBytes(data);
|
new Random().nextBytes(data);
|
||||||
|
|
|
@ -397,6 +397,6 @@ public class TermsAggregationSearchBenchmark {
|
||||||
totalQueryTime += searchResponse.getTookInMillis();
|
totalQueryTime += searchResponse.getTookInMillis();
|
||||||
}
|
}
|
||||||
System.out.println("--> Terms stats agg (" + name + "): " + (totalQueryTime / QUERY_COUNT) + "ms");
|
System.out.println("--> Terms stats agg (" + name + "): " + (totalQueryTime / QUERY_COUNT) + "ms");
|
||||||
return new StatsResult(name, totalQueryTime, ByteSizeValue.parseBytesSizeValue("0b"));
|
return new StatsResult(name, totalQueryTime, ByteSizeValue.parseBytesSizeValue("0b", "StatsResult"));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -131,7 +131,8 @@ public class SimpleClusterStateTests extends ElasticsearchIntegrationTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testLargeClusterStatePublishing() throws Exception {
|
public void testLargeClusterStatePublishing() throws Exception {
|
||||||
int estimatedBytesSize = scaledRandomIntBetween(ByteSizeValue.parseBytesSizeValue("10k").bytesAsInt(), ByteSizeValue.parseBytesSizeValue("256k").bytesAsInt());
|
int estimatedBytesSize = scaledRandomIntBetween(ByteSizeValue.parseBytesSizeValue("10k", "estimatedBytesSize").bytesAsInt(),
|
||||||
|
ByteSizeValue.parseBytesSizeValue("256k", "estimatedBytesSize").bytesAsInt());
|
||||||
XContentBuilder mapping = XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties");
|
XContentBuilder mapping = XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties");
|
||||||
int counter = 0;
|
int counter = 0;
|
||||||
int numberOfFields = 0;
|
int numberOfFields = 0;
|
||||||
|
|
|
@ -58,9 +58,9 @@ public class DiskThresholdDeciderUnitTests extends ElasticsearchTestCase {
|
||||||
};
|
};
|
||||||
DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY, nss, cis, null);
|
DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY, nss, cis, null);
|
||||||
|
|
||||||
assertThat(decider.getFreeBytesThresholdHigh(), equalTo(ByteSizeValue.parseBytesSizeValue("0b")));
|
assertThat(decider.getFreeBytesThresholdHigh(), equalTo(ByteSizeValue.parseBytesSizeValue("0b", "decider")));
|
||||||
assertThat(decider.getFreeDiskThresholdHigh(), equalTo(10.0d));
|
assertThat(decider.getFreeDiskThresholdHigh(), equalTo(10.0d));
|
||||||
assertThat(decider.getFreeBytesThresholdLow(), equalTo(ByteSizeValue.parseBytesSizeValue("0b")));
|
assertThat(decider.getFreeBytesThresholdLow(), equalTo(ByteSizeValue.parseBytesSizeValue("0b", "decider")));
|
||||||
assertThat(decider.getFreeDiskThresholdLow(), equalTo(15.0d));
|
assertThat(decider.getFreeDiskThresholdLow(), equalTo(15.0d));
|
||||||
assertThat(decider.getRerouteInterval().seconds(), equalTo(60L));
|
assertThat(decider.getRerouteInterval().seconds(), equalTo(60L));
|
||||||
assertTrue(decider.isEnabled());
|
assertTrue(decider.isEnabled());
|
||||||
|
@ -79,11 +79,11 @@ public class DiskThresholdDeciderUnitTests extends ElasticsearchTestCase {
|
||||||
applySettings.onRefreshSettings(newSettings);
|
applySettings.onRefreshSettings(newSettings);
|
||||||
|
|
||||||
assertThat("high threshold bytes should be unset",
|
assertThat("high threshold bytes should be unset",
|
||||||
decider.getFreeBytesThresholdHigh(), equalTo(ByteSizeValue.parseBytesSizeValue("0b")));
|
decider.getFreeBytesThresholdHigh(), equalTo(ByteSizeValue.parseBytesSizeValue("0b", "high threshold")));
|
||||||
assertThat("high threshold percentage should be changed",
|
assertThat("high threshold percentage should be changed",
|
||||||
decider.getFreeDiskThresholdHigh(), equalTo(30.0d));
|
decider.getFreeDiskThresholdHigh(), equalTo(30.0d));
|
||||||
assertThat("low threshold bytes should be set to 500mb",
|
assertThat("low threshold bytes should be set to 500mb",
|
||||||
decider.getFreeBytesThresholdLow(), equalTo(ByteSizeValue.parseBytesSizeValue("500mb")));
|
decider.getFreeBytesThresholdLow(), equalTo(ByteSizeValue.parseBytesSizeValue("500mb", "low threshold")));
|
||||||
assertThat("low threshold bytes should be unset",
|
assertThat("low threshold bytes should be unset",
|
||||||
decider.getFreeDiskThresholdLow(), equalTo(0.0d));
|
decider.getFreeDiskThresholdLow(), equalTo(0.0d));
|
||||||
assertThat("reroute interval should be changed to 30 seconds",
|
assertThat("reroute interval should be changed to 30 seconds",
|
||||||
|
|
|
@ -70,29 +70,29 @@ public class ByteSizeValueTests extends ElasticsearchTestCase {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testParsing() {
|
public void testParsing() {
|
||||||
assertThat(ByteSizeValue.parseBytesSizeValue("42pb").toString(), is("42pb"));
|
assertThat(ByteSizeValue.parseBytesSizeValue("42pb", "testParsing").toString(), is("42pb"));
|
||||||
assertThat(ByteSizeValue.parseBytesSizeValue("42P").toString(), is("42pb"));
|
assertThat(ByteSizeValue.parseBytesSizeValue("42P", "testParsing").toString(), is("42pb"));
|
||||||
assertThat(ByteSizeValue.parseBytesSizeValue("42PB").toString(), is("42pb"));
|
assertThat(ByteSizeValue.parseBytesSizeValue("42PB", "testParsing").toString(), is("42pb"));
|
||||||
assertThat(ByteSizeValue.parseBytesSizeValue("54tb").toString(), is("54tb"));
|
assertThat(ByteSizeValue.parseBytesSizeValue("54tb", "testParsing").toString(), is("54tb"));
|
||||||
assertThat(ByteSizeValue.parseBytesSizeValue("54T").toString(), is("54tb"));
|
assertThat(ByteSizeValue.parseBytesSizeValue("54T", "testParsing").toString(), is("54tb"));
|
||||||
assertThat(ByteSizeValue.parseBytesSizeValue("54TB").toString(), is("54tb"));
|
assertThat(ByteSizeValue.parseBytesSizeValue("54TB", "testParsing").toString(), is("54tb"));
|
||||||
assertThat(ByteSizeValue.parseBytesSizeValue("12gb").toString(), is("12gb"));
|
assertThat(ByteSizeValue.parseBytesSizeValue("12gb", "testParsing").toString(), is("12gb"));
|
||||||
assertThat(ByteSizeValue.parseBytesSizeValue("12G").toString(), is("12gb"));
|
assertThat(ByteSizeValue.parseBytesSizeValue("12G", "testParsing").toString(), is("12gb"));
|
||||||
assertThat(ByteSizeValue.parseBytesSizeValue("12GB").toString(), is("12gb"));
|
assertThat(ByteSizeValue.parseBytesSizeValue("12GB", "testParsing").toString(), is("12gb"));
|
||||||
assertThat(ByteSizeValue.parseBytesSizeValue("12M").toString(), is("12mb"));
|
assertThat(ByteSizeValue.parseBytesSizeValue("12M", "testParsing").toString(), is("12mb"));
|
||||||
assertThat(ByteSizeValue.parseBytesSizeValue("1b").toString(), is("1b"));
|
assertThat(ByteSizeValue.parseBytesSizeValue("1b", "testParsing").toString(), is("1b"));
|
||||||
assertThat(ByteSizeValue.parseBytesSizeValue("23kb").toString(), is("23kb"));
|
assertThat(ByteSizeValue.parseBytesSizeValue("23kb", "testParsing").toString(), is("23kb"));
|
||||||
assertThat(ByteSizeValue.parseBytesSizeValue("23k").toString(), is("23kb"));
|
assertThat(ByteSizeValue.parseBytesSizeValue("23k", "testParsing").toString(), is("23kb"));
|
||||||
assertThat(ByteSizeValue.parseBytesSizeValue("23").toString(), is("23b"));
|
assertThat(ByteSizeValue.parseBytesSizeValue("23", "testParsing").toString(), is("23b"));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(expected = ElasticsearchParseException.class)
|
@Test(expected = ElasticsearchParseException.class)
|
||||||
public void testFailOnEmptyParsing() {
|
public void testFailOnEmptyParsing() {
|
||||||
assertThat(ByteSizeValue.parseBytesSizeValue("").toString(), is("23kb"));
|
assertThat(ByteSizeValue.parseBytesSizeValue("", "emptyParsing").toString(), is("23kb"));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(expected = ElasticsearchParseException.class)
|
@Test(expected = ElasticsearchParseException.class)
|
||||||
public void testFailOnEmptyNumberParsing() {
|
public void testFailOnEmptyNumberParsing() {
|
||||||
assertThat(ByteSizeValue.parseBytesSizeValue("g").toString(), is("23b"));
|
assertThat(ByteSizeValue.parseBytesSizeValue("g", "emptyNumberParsing").toString(), is("23b"));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -56,7 +56,8 @@ public class DirectBufferNetworkTests extends ElasticsearchIntegrationTest {
|
||||||
public void verifySaneDirectBufferAllocations() throws Exception {
|
public void verifySaneDirectBufferAllocations() throws Exception {
|
||||||
createIndex("test");
|
createIndex("test");
|
||||||
|
|
||||||
int estimatedBytesSize = scaledRandomIntBetween(ByteSizeValue.parseBytesSizeValue("1.1mb").bytesAsInt(), ByteSizeValue.parseBytesSizeValue("1.5mb").bytesAsInt());
|
int estimatedBytesSize = scaledRandomIntBetween(ByteSizeValue.parseBytesSizeValue("1.1mb", "estimatedBytesSize").bytesAsInt(),
|
||||||
|
ByteSizeValue.parseBytesSizeValue("1.5mb", "estimatedBytesSize").bytesAsInt());
|
||||||
byte[] data = new byte[estimatedBytesSize];
|
byte[] data = new byte[estimatedBytesSize];
|
||||||
getRandom().nextBytes(data);
|
getRandom().nextBytes(data);
|
||||||
|
|
||||||
|
|
|
@ -123,7 +123,7 @@ public class RecoverySettingsTest extends ElasticsearchSingleNodeTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void innerTestSettings(String key, int newValue, TimeUnit timeUnit, Validator validator) {
|
private void innerTestSettings(String key, int newValue, TimeUnit timeUnit, Validator validator) {
|
||||||
client().admin().cluster().prepareUpdateSettings().setTransientSettings(ImmutableSettings.builder().put(key, newValue, timeUnit)).get();
|
client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder().put(key, newValue, timeUnit)).get();
|
||||||
validator.validate(getInstanceFromNode(RecoverySettings.class), newValue);
|
validator.validate(getInstanceFromNode(RecoverySettings.class), newValue);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue