Remove duplicate methods in ByteSizeValue (#20560)

This commit removes `ByteSizeValue`'s methods that are duplicated (ex: `mbFrac()` and `getMbFrac()`) in order to only keep the `getN` form.
    
It also renames `mb()` -> `getMb()`, `kb()` -> `getKB()` in order to be more coherent with the `ByteSizeUnit` method names.
This commit is contained in:
Tanguy Leroux 2016-09-20 14:07:23 +02:00 committed by GitHub
parent 0fee59127e
commit 7645abaad9
53 changed files with 246 additions and 301 deletions

View File

@ -68,14 +68,14 @@ public class StoreRateLimiting {
}
public void setMaxRate(ByteSizeValue rate) {
if (rate.bytes() <= 0) {
if (rate.getBytes() <= 0) {
actualRateLimiter = null;
} else if (actualRateLimiter == null) {
actualRateLimiter = rateLimiter;
actualRateLimiter.setMBPerSec(rate.mbFrac());
actualRateLimiter.setMBPerSec(rate.getMbFrac());
} else {
assert rateLimiter == actualRateLimiter;
rateLimiter.setMBPerSec(rate.mbFrac());
rateLimiter.setMBPerSec(rate.getMbFrac());
}
}

View File

@ -220,7 +220,7 @@ public class NodeInfo extends BaseNodeResponse {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeLong(totalIndexingBuffer.bytes());
out.writeLong(totalIndexingBuffer.getBytes());
}
if (settings == null) {
out.writeBoolean(false);

View File

@ -250,11 +250,11 @@ public class ClusterStatsNodes implements ToXContent {
long freeMemory = 0;
for (NodeStats nodeStats : nodeStatsList) {
if (nodeStats.getOs() != null) {
long total = nodeStats.getOs().getMem().getTotal().bytes();
long total = nodeStats.getOs().getMem().getTotal().getBytes();
if (total > 0) {
totalMemory += total;
}
long free = nodeStats.getOs().getMem().getFree().bytes();
long free = nodeStats.getOs().getMem().getFree().getBytes();
if (free > 0) {
freeMemory += free;
}
@ -423,8 +423,8 @@ public class ClusterStatsNodes implements ToXContent {
}
maxUptime = Math.max(maxUptime, js.getUptime().millis());
if (js.getMem() != null) {
heapUsed += js.getMem().getHeapUsed().bytes();
heapMax += js.getMem().getHeapMax().bytes();
heapUsed += js.getMem().getHeapUsed().getBytes();
heapMax += js.getMem().getHeapMax().getBytes();
}
}
this.threads = threads;

View File

@ -190,7 +190,7 @@ public class BulkProcessor implements Closeable {
BulkProcessor(Client client, BackoffPolicy backoffPolicy, Listener listener, @Nullable String name, int concurrentRequests, int bulkActions, ByteSizeValue bulkSize, @Nullable TimeValue flushInterval) {
this.bulkActions = bulkActions;
this.bulkSize = bulkSize.bytes();
this.bulkSize = bulkSize.getBytes();
this.bulkRequest = new BulkRequest();
this.bulkRequestHandler = (concurrentRequests == 0) ? BulkRequestHandler.syncHandler(client, backoffPolicy, listener) : BulkRequestHandler.asyncHandler(client, backoffPolicy, listener, concurrentRequests);

View File

@ -415,9 +415,9 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
if (leastAvailablePath == null) {
assert mostAvailablePath == null;
mostAvailablePath = leastAvailablePath = info;
} else if (leastAvailablePath.getAvailable().bytes() > info.getAvailable().bytes()){
} else if (leastAvailablePath.getAvailable().getBytes() > info.getAvailable().getBytes()){
leastAvailablePath = info;
} else if (mostAvailablePath.getAvailable().bytes() < info.getAvailable().bytes()) {
} else if (mostAvailablePath.getAvailable().getBytes() < info.getAvailable().getBytes()) {
mostAvailablePath = info;
}
}
@ -428,21 +428,21 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
nodeId, mostAvailablePath.getTotal(), leastAvailablePath.getAvailable(),
leastAvailablePath.getTotal(), leastAvailablePath.getAvailable());
}
if (leastAvailablePath.getTotal().bytes() < 0) {
if (leastAvailablePath.getTotal().getBytes() < 0) {
if (logger.isTraceEnabled()) {
logger.trace("node: [{}] least available path has less than 0 total bytes of disk [{}], skipping",
nodeId, leastAvailablePath.getTotal().bytes());
nodeId, leastAvailablePath.getTotal().getBytes());
}
} else {
newLeastAvaiableUsages.put(nodeId, new DiskUsage(nodeId, nodeName, leastAvailablePath.getPath(), leastAvailablePath.getTotal().bytes(), leastAvailablePath.getAvailable().bytes()));
newLeastAvaiableUsages.put(nodeId, new DiskUsage(nodeId, nodeName, leastAvailablePath.getPath(), leastAvailablePath.getTotal().getBytes(), leastAvailablePath.getAvailable().getBytes()));
}
if (mostAvailablePath.getTotal().bytes() < 0) {
if (mostAvailablePath.getTotal().getBytes() < 0) {
if (logger.isTraceEnabled()) {
logger.trace("node: [{}] most available path has less than 0 total bytes of disk [{}], skipping",
nodeId, mostAvailablePath.getTotal().bytes());
nodeId, mostAvailablePath.getTotal().getBytes());
}
} else {
newMostAvaiableUsages.put(nodeId, new DiskUsage(nodeId, nodeName, mostAvailablePath.getPath(), mostAvailablePath.getTotal().bytes(), mostAvailablePath.getAvailable().bytes()));
newMostAvaiableUsages.put(nodeId, new DiskUsage(nodeId, nodeName, mostAvailablePath.getPath(), mostAvailablePath.getTotal().getBytes(), mostAvailablePath.getAvailable().getBytes()));
}
}

View File

@ -62,10 +62,10 @@ public class DiskThresholdMonitor extends AbstractComponent implements ClusterIn
*/
private void warnAboutDiskIfNeeded(DiskUsage usage) {
// Check absolute disk values
if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdHigh().bytes()) {
if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdHigh().getBytes()) {
logger.warn("high disk watermark [{}] exceeded on {}, shards will be relocated away from this node",
diskThresholdSettings.getFreeBytesThresholdHigh(), usage);
} else if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdLow().bytes()) {
} else if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdLow().getBytes()) {
logger.info("low disk watermark [{}] exceeded on {}, replicas will not be assigned to this node",
diskThresholdSettings.getFreeBytesThresholdLow(), usage);
}
@ -100,7 +100,7 @@ public class DiskThresholdMonitor extends AbstractComponent implements ClusterIn
String node = entry.key;
DiskUsage usage = entry.value;
warnAboutDiskIfNeeded(usage);
if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdHigh().bytes() ||
if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdHigh().getBytes() ||
usage.getFreeDiskAsPercentage() < diskThresholdSettings.getFreeDiskThresholdHigh()) {
if ((System.nanoTime() - lastRunNS) > diskThresholdSettings.getRerouteInterval().nanos()) {
lastRunNS = System.nanoTime();
@ -112,7 +112,7 @@ public class DiskThresholdMonitor extends AbstractComponent implements ClusterIn
node, diskThresholdSettings.getRerouteInterval());
}
nodeHasPassedWatermark.add(node);
} else if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdLow().bytes() ||
} else if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdLow().getBytes() ||
usage.getFreeDiskAsPercentage() < diskThresholdSettings.getFreeDiskThresholdLow()) {
nodeHasPassedWatermark.add(node);
} else {

View File

@ -128,7 +128,7 @@ public class DiskThresholdDecider extends AllocationDecider {
shardRouting.active() == false && shardRouting.recoverySource().getType() == RecoverySource.Type.EMPTY_STORE;
// checks for exact byte comparisons
if (freeBytes < diskThresholdSettings.getFreeBytesThresholdLow().bytes()) {
if (freeBytes < diskThresholdSettings.getFreeBytesThresholdLow().getBytes()) {
if (skipLowTresholdChecks == false) {
if (logger.isDebugEnabled()) {
logger.debug("less than the required {} free bytes threshold ({} bytes free) on node {}, preventing allocation",
@ -137,7 +137,7 @@ public class DiskThresholdDecider extends AllocationDecider {
return allocation.decision(Decision.NO, NAME,
"the node is above the low watermark and has less than required [%s] free, free: [%s]",
diskThresholdSettings.getFreeBytesThresholdLow(), new ByteSizeValue(freeBytes));
} else if (freeBytes > diskThresholdSettings.getFreeBytesThresholdHigh().bytes()) {
} else if (freeBytes > diskThresholdSettings.getFreeBytesThresholdHigh().getBytes()) {
// Allow the shard to be allocated because it is primary that
// has never been allocated if it's under the high watermark
if (logger.isDebugEnabled()) {
@ -205,7 +205,7 @@ public class DiskThresholdDecider extends AllocationDecider {
final long shardSize = getExpectedShardSize(shardRouting, allocation, 0);
double freeSpaceAfterShard = freeDiskPercentageAfterShardAssigned(usage, shardSize);
long freeBytesAfterShard = freeBytes - shardSize;
if (freeBytesAfterShard < diskThresholdSettings.getFreeBytesThresholdHigh().bytes()) {
if (freeBytesAfterShard < diskThresholdSettings.getFreeBytesThresholdHigh().getBytes()) {
logger.warn("after allocating, node [{}] would have less than the required " +
"{} free bytes threshold ({} bytes free), preventing allocation",
node.nodeId(), diskThresholdSettings.getFreeBytesThresholdHigh(), freeBytesAfterShard);
@ -258,7 +258,7 @@ public class DiskThresholdDecider extends AllocationDecider {
return allocation.decision(Decision.YES, NAME,
"this shard is not allocated on the most utilized disk and can remain");
}
if (freeBytes < diskThresholdSettings.getFreeBytesThresholdHigh().bytes()) {
if (freeBytes < diskThresholdSettings.getFreeBytesThresholdHigh().getBytes()) {
if (logger.isDebugEnabled()) {
logger.debug("less than the required {} free bytes threshold ({} bytes free) on node {}, shard cannot remain",
diskThresholdSettings.getFreeBytesThresholdHigh(), freeBytes, node.nodeId());

View File

@ -46,7 +46,7 @@ public class FsBlobStore extends AbstractComponent implements BlobStore {
super(settings);
this.path = path;
Files.createDirectories(path);
this.bufferSizeInBytes = (int) settings.getAsBytesSize("repositories.fs.buffer_size", new ByteSizeValue(100, ByteSizeUnit.KB)).bytes();
this.bufferSizeInBytes = (int) settings.getAsBytesSize("repositories.fs.buffer_size", new ByteSizeValue(100, ByteSizeUnit.KB)).getBytes();
}
@Override

View File

@ -55,7 +55,7 @@ public class URLBlobStore extends AbstractComponent implements BlobStore {
public URLBlobStore(Settings settings, URL path) {
super(settings);
this.path = path;
this.bufferSizeInBytes = (int) settings.getAsBytesSize("repositories.uri.buffer_size", new ByteSizeValue(100, ByteSizeUnit.KB)).bytes();
this.bufferSizeInBytes = (int) settings.getAsBytesSize("repositories.uri.buffer_size", new ByteSizeValue(100, ByteSizeUnit.KB)).getBytes();
}
/**

View File

@ -57,7 +57,7 @@ public class MemoryCircuitBreaker implements CircuitBreaker {
* @param oldBreaker the previous circuit breaker to inherit the used value from (starting offset)
*/
public MemoryCircuitBreaker(ByteSizeValue limit, double overheadConstant, MemoryCircuitBreaker oldBreaker, Logger logger) {
this.memoryBytesLimit = limit.bytes();
this.memoryBytesLimit = limit.getBytes();
this.overheadConstant = overheadConstant;
if (oldBreaker == null) {
this.used = new AtomicLong(0);

View File

@ -584,10 +584,10 @@ public class Setting<T> extends ToXContentToBytes {
public static ByteSizeValue parseByteSize(String s, ByteSizeValue minValue, ByteSizeValue maxValue, String key) {
ByteSizeValue value = ByteSizeValue.parseBytesSizeValue(s, key);
if (value.bytes() < minValue.bytes()) {
if (value.getBytes() < minValue.getBytes()) {
throw new IllegalArgumentException("Failed to parse value [" + s + "] for setting [" + key + "] must be >= " + minValue);
}
if (value.bytes() > maxValue.bytes()) {
if (value.getBytes() > maxValue.getBytes()) {
throw new IllegalArgumentException("Failed to parse value [" + s + "] for setting [" + key + "] must be <= " + maxValue);
}
return value;

View File

@ -41,7 +41,7 @@ public class ByteSizeValue implements Writeable {
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVLong(bytes());
out.writeVLong(getBytes());
}
public ByteSizeValue(long bytes) {
@ -54,120 +54,76 @@ public class ByteSizeValue implements Writeable {
}
public int bytesAsInt() {
long bytes = bytes();
long bytes = getBytes();
if (bytes > Integer.MAX_VALUE) {
throw new IllegalArgumentException("size [" + toString() + "] is bigger than max int");
}
return (int) bytes;
}
public long bytes() {
public long getBytes() {
return sizeUnit.toBytes(size);
}
public long getBytes() {
return bytes();
}
public long kb() {
public long getKb() {
return sizeUnit.toKB(size);
}
public long getKb() {
return kb();
}
public long mb() {
public long getMb() {
return sizeUnit.toMB(size);
}
public long getMb() {
return mb();
}
public long gb() {
public long getGb() {
return sizeUnit.toGB(size);
}
public long getGb() {
return gb();
}
public long tb() {
public long getTb() {
return sizeUnit.toTB(size);
}
public long getTb() {
return tb();
}
public long pb() {
public long getPb() {
return sizeUnit.toPB(size);
}
public long getPb() {
return pb();
}
public double kbFrac() {
return ((double) bytes()) / ByteSizeUnit.C1;
}
public double getKbFrac() {
return kbFrac();
}
public double mbFrac() {
return ((double) bytes()) / ByteSizeUnit.C2;
return ((double) getBytes()) / ByteSizeUnit.C1;
}
public double getMbFrac() {
return mbFrac();
}
public double gbFrac() {
return ((double) bytes()) / ByteSizeUnit.C3;
return ((double) getBytes()) / ByteSizeUnit.C2;
}
public double getGbFrac() {
return gbFrac();
}
public double tbFrac() {
return ((double) bytes()) / ByteSizeUnit.C4;
return ((double) getBytes()) / ByteSizeUnit.C3;
}
public double getTbFrac() {
return tbFrac();
}
public double pbFrac() {
return ((double) bytes()) / ByteSizeUnit.C5;
return ((double) getBytes()) / ByteSizeUnit.C4;
}
public double getPbFrac() {
return pbFrac();
return ((double) getBytes()) / ByteSizeUnit.C5;
}
@Override
public String toString() {
long bytes = bytes();
long bytes = getBytes();
double value = bytes;
String suffix = "b";
if (bytes >= ByteSizeUnit.C5) {
value = pbFrac();
value = getPbFrac();
suffix = "pb";
} else if (bytes >= ByteSizeUnit.C4) {
value = tbFrac();
value = getTbFrac();
suffix = "tb";
} else if (bytes >= ByteSizeUnit.C3) {
value = gbFrac();
value = getGbFrac();
suffix = "gb";
} else if (bytes >= ByteSizeUnit.C2) {
value = mbFrac();
value = getMbFrac();
suffix = "mb";
} else if (bytes >= ByteSizeUnit.C1) {
value = kbFrac();
value = getKbFrac();
suffix = "kb";
}
return Strings.format1Decimals(value, suffix);
@ -237,7 +193,7 @@ public class ByteSizeValue implements Writeable {
ByteSizeValue sizeValue = (ByteSizeValue) o;
return bytes() == sizeValue.bytes();
return getBytes() == sizeValue.getBytes();
}
@Override

View File

@ -42,7 +42,7 @@ public enum MemorySizeValue {
if (percent < 0 || percent > 100) {
throw new ElasticsearchParseException("percentage should be in [0-100], got [{}]", percentAsString);
}
return new ByteSizeValue((long) ((percent / 100) * JvmInfo.jvmInfo().getMem().getHeapMax().bytes()), ByteSizeUnit.BYTES);
return new ByteSizeValue((long) ((percent / 100) * JvmInfo.jvmInfo().getMem().getHeapMax().getBytes()), ByteSizeUnit.BYTES);
} catch (NumberFormatException e) {
throw new ElasticsearchParseException("failed to parse [{}] as a double", e, percentAsString);
}

View File

@ -20,7 +20,6 @@
package org.elasticsearch.common.util;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.recycler.AbstractRecyclerC;
@ -29,7 +28,6 @@ import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import java.util.Arrays;
@ -70,7 +68,7 @@ public class PageCacheRecycler extends AbstractComponent implements Releasable {
protected PageCacheRecycler(Settings settings) {
super(settings);
final Type type = TYPE_SETTING .get(settings);
final long limit = LIMIT_HEAP_SETTING .get(settings).bytes();
final long limit = LIMIT_HEAP_SETTING .get(settings).getBytes();
final int availableProcessors = EsExecutors.boundedNumberOfProcessors(settings);
// We have a global amount of memory that we need to divide across data types.

View File

@ -932,7 +932,7 @@ public final class XContentBuilder implements BytesStream, Releasable, Flushable
if (humanReadable) {
field(readableFieldName, byteSizeValue.toString());
}
field(rawFieldName, byteSizeValue.bytes());
field(rawFieldName, byteSizeValue.getBytes());
return this;
}

View File

@ -171,10 +171,10 @@ public final class MergePolicyConfig {
maxMergeAtOnce = adjustMaxMergeAtOnceIfNeeded(maxMergeAtOnce, segmentsPerTier);
mergePolicy.setNoCFSRatio(indexSettings.getValue(INDEX_COMPOUND_FORMAT_SETTING));
mergePolicy.setForceMergeDeletesPctAllowed(forceMergeDeletesPctAllowed);
mergePolicy.setFloorSegmentMB(floorSegment.mbFrac());
mergePolicy.setFloorSegmentMB(floorSegment.getMbFrac());
mergePolicy.setMaxMergeAtOnce(maxMergeAtOnce);
mergePolicy.setMaxMergeAtOnceExplicit(maxMergeAtOnceExplicit);
mergePolicy.setMaxMergedSegmentMB(maxMergedSegment.mbFrac());
mergePolicy.setMaxMergedSegmentMB(maxMergedSegment.getMbFrac());
mergePolicy.setSegmentsPerTier(segmentsPerTier);
mergePolicy.setReclaimDeletesWeight(reclaimDeletesWeight);
if (logger.isTraceEnabled()) {
@ -192,7 +192,7 @@ public final class MergePolicyConfig {
}
void setMaxMergedSegment(ByteSizeValue maxMergedSegment) {
mergePolicy.setMaxMergedSegmentMB(maxMergedSegment.mbFrac());
mergePolicy.setMaxMergedSegmentMB(maxMergedSegment.getMbFrac());
}
void setMaxMergesAtOnceExplicit(Integer maxMergeAtOnceExplicit) {
@ -204,7 +204,7 @@ public final class MergePolicyConfig {
}
void setFloorSegmentSetting(ByteSizeValue floorSegementSetting) {
mergePolicy.setFloorSegmentMB(floorSegementSetting.mbFrac());
mergePolicy.setFloorSegmentMB(floorSegementSetting.getMbFrac());
}
void setExpungeDeletesAllowed(Double value) {

View File

@ -1095,7 +1095,7 @@ public class InternalEngine extends Engine {
mergePolicy = new ElasticsearchMergePolicy(mergePolicy);
iwc.setMergePolicy(mergePolicy);
iwc.setSimilarity(engineConfig.getSimilarity());
iwc.setRAMBufferSizeMB(engineConfig.getIndexingBufferSize().mbFrac());
iwc.setRAMBufferSizeMB(engineConfig.getIndexingBufferSize().getMbFrac());
iwc.setCodec(engineConfig.getCodec());
iwc.setUseCompoundFile(true); // always use compound on flush - reduces # of file-handles on refresh
return new IndexWriter(store.directory(), iwc);

View File

@ -1222,7 +1222,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
if (engine != null) {
try {
Translog translog = engine.getTranslog();
return translog.sizeInBytes() > indexSettings.getFlushThresholdSize().bytes();
return translog.sizeInBytes() > indexSettings.getFlushThresholdSize().getBytes();
} catch (AlreadyClosedException | EngineClosedException ex) {
// that's fine we are already close - no need to flush
}

View File

@ -70,7 +70,7 @@ public class BlobStoreIndexShardSnapshot implements ToXContent, FromXContentBuil
long partBytes = Long.MAX_VALUE;
if (partSize != null) {
partBytes = partSize.bytes();
partBytes = partSize.getBytes();
}
long totalLength = metaData.length();
@ -261,7 +261,7 @@ public class BlobStoreIndexShardSnapshot implements ToXContent, FromXContentBuil
builder.field(CHECKSUM, file.metadata.checksum());
}
if (file.partSize != null) {
builder.field(PART_SIZE, file.partSize.bytes());
builder.field(PART_SIZE, file.partSize.getBytes());
}
if (file.metadata.writtenBy() != null) {

View File

@ -106,10 +106,10 @@ public class IndexingMemoryController extends AbstractComponent implements Index
// We only apply the min/max when % value was used for the index buffer:
ByteSizeValue minIndexingBuffer = MIN_INDEX_BUFFER_SIZE_SETTING.get(this.settings);
ByteSizeValue maxIndexingBuffer = MAX_INDEX_BUFFER_SIZE_SETTING.get(this.settings);
if (indexingBuffer.bytes() < minIndexingBuffer.bytes()) {
if (indexingBuffer.getBytes() < minIndexingBuffer.getBytes()) {
indexingBuffer = minIndexingBuffer;
}
if (maxIndexingBuffer.bytes() != -1 && indexingBuffer.bytes() > maxIndexingBuffer.bytes()) {
if (maxIndexingBuffer.getBytes() != -1 && indexingBuffer.getBytes() > maxIndexingBuffer.getBytes()) {
indexingBuffer = maxIndexingBuffer;
}
}
@ -245,13 +245,13 @@ public class IndexingMemoryController extends AbstractComponent implements Index
public void bytesWritten(int bytes) {
long totalBytes = bytesWrittenSinceCheck.addAndGet(bytes);
assert totalBytes >= 0;
while (totalBytes > indexingBuffer.bytes()/30) {
while (totalBytes > indexingBuffer.getBytes()/30) {
if (runLock.tryLock()) {
try {
// Must pull this again because it may have changed since we first checked:
totalBytes = bytesWrittenSinceCheck.get();
if (totalBytes > indexingBuffer.bytes()/30) {
if (totalBytes > indexingBuffer.getBytes()/30) {
bytesWrittenSinceCheck.addAndGet(-totalBytes);
// NOTE: this is only an approximate check, because bytes written is to the translog, vs indexing memory buffer which is
// typically smaller but can be larger in extreme cases (many unique terms). This logic is here only as a safety against
@ -320,9 +320,9 @@ public class IndexingMemoryController extends AbstractComponent implements Index
// If we are using more than 50% of our budget across both indexing buffer and bytes we are still moving to disk, then we now
// throttle the top shards to send back-pressure to ongoing indexing:
boolean doThrottle = (totalBytesWriting + totalBytesUsed) > 1.5 * indexingBuffer.bytes();
boolean doThrottle = (totalBytesWriting + totalBytesUsed) > 1.5 * indexingBuffer.getBytes();
if (totalBytesUsed > indexingBuffer.bytes()) {
if (totalBytesUsed > indexingBuffer.getBytes()) {
// OK we are now over-budget; fill the priority queue and ask largest shard(s) to refresh:
PriorityQueue<ShardAndBytesUsed> queue = new PriorityQueue<>();
@ -357,7 +357,7 @@ public class IndexingMemoryController extends AbstractComponent implements Index
logger.debug("now write some indexing buffers: total indexing heap bytes used [{}] vs {} [{}], currently writing bytes [{}], [{}] shards with non-zero indexing buffer",
new ByteSizeValue(totalBytesUsed), INDEX_BUFFER_SIZE_SETTING.getKey(), indexingBuffer, new ByteSizeValue(totalBytesWriting), queue.size());
while (totalBytesUsed > indexingBuffer.bytes() && queue.isEmpty() == false) {
while (totalBytesUsed > indexingBuffer.getBytes() && queue.isEmpty() == false) {
ShardAndBytesUsed largest = queue.poll();
logger.debug("write indexing buffer to disk for shard [{}] to free up its [{}] indexing buffer", largest.shard.shardId(), new ByteSizeValue(largest.bytesUsed));
writeIndexingBufferAsync(largest.shard);

View File

@ -74,9 +74,9 @@ public class IndicesQueryCache extends AbstractComponent implements QueryCache,
logger.debug("using [node] query cache with size [{}] max filter count [{}]",
size, count);
if (INDICES_QUERIES_CACHE_ALL_SEGMENTS_SETTING.get(settings)) {
cache = new ElasticsearchLRUQueryCache(count, size.bytes(), context -> true);
cache = new ElasticsearchLRUQueryCache(count, size.getBytes(), context -> true);
} else {
cache = new ElasticsearchLRUQueryCache(count, size.bytes());
cache = new ElasticsearchLRUQueryCache(count, size.getBytes());
}
sharedRamBytesUsed = 0;
}

View File

@ -86,7 +86,7 @@ public final class IndicesRequestCache extends AbstractComponent implements Remo
super(settings);
this.size = INDICES_CACHE_QUERY_SIZE.get(settings);
this.expire = INDICES_CACHE_QUERY_EXPIRE.exists(settings) ? INDICES_CACHE_QUERY_EXPIRE.get(settings) : null;
long sizeInBytes = size.bytes();
long sizeInBytes = size.getBytes();
CacheBuilder<Key, Value> cacheBuilder = CacheBuilder.<Key, Value>builder()
.setMaximumWeight(sizeInBytes).weigher((k, v) -> k.ramBytesUsed() + v.ramBytesUsed()).removalListener(this);
if (expire != null) {

View File

@ -81,25 +81,25 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
public HierarchyCircuitBreakerService(Settings settings, ClusterSettings clusterSettings) {
super(settings);
this.fielddataSettings = new BreakerSettings(CircuitBreaker.FIELDDATA,
FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING.get(settings).bytes(),
FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING.get(settings).getBytes(),
FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING.get(settings),
FIELDDATA_CIRCUIT_BREAKER_TYPE_SETTING.get(settings)
);
this.inFlightRequestsSettings = new BreakerSettings(CircuitBreaker.IN_FLIGHT_REQUESTS,
IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_LIMIT_SETTING.get(settings).bytes(),
IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_LIMIT_SETTING.get(settings).getBytes(),
IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_OVERHEAD_SETTING.get(settings),
IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_TYPE_SETTING.get(settings)
);
this.requestSettings = new BreakerSettings(CircuitBreaker.REQUEST,
REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING.get(settings).bytes(),
REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING.get(settings).getBytes(),
REQUEST_CIRCUIT_BREAKER_OVERHEAD_SETTING.get(settings),
REQUEST_CIRCUIT_BREAKER_TYPE_SETTING.get(settings)
);
this.parentSettings = new BreakerSettings(CircuitBreaker.PARENT,
TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING.get(settings).bytes(), 1.0,
TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING.get(settings).getBytes(), 1.0,
CircuitBreaker.Type.PARENT);
if (logger.isTraceEnabled()) {
@ -117,7 +117,7 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
}
private void setRequestBreakerLimit(ByteSizeValue newRequestMax, Double newRequestOverhead) {
BreakerSettings newRequestSettings = new BreakerSettings(CircuitBreaker.REQUEST, newRequestMax.bytes(), newRequestOverhead,
BreakerSettings newRequestSettings = new BreakerSettings(CircuitBreaker.REQUEST, newRequestMax.getBytes(), newRequestOverhead,
HierarchyCircuitBreakerService.this.requestSettings.getType());
registerBreaker(newRequestSettings);
HierarchyCircuitBreakerService.this.requestSettings = newRequestSettings;
@ -125,7 +125,7 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
}
private void setInFlightRequestsBreakerLimit(ByteSizeValue newInFlightRequestsMax, Double newInFlightRequestsOverhead) {
BreakerSettings newInFlightRequestsSettings = new BreakerSettings(CircuitBreaker.IN_FLIGHT_REQUESTS, newInFlightRequestsMax.bytes(),
BreakerSettings newInFlightRequestsSettings = new BreakerSettings(CircuitBreaker.IN_FLIGHT_REQUESTS, newInFlightRequestsMax.getBytes(),
newInFlightRequestsOverhead, HierarchyCircuitBreakerService.this.inFlightRequestsSettings.getType());
registerBreaker(newInFlightRequestsSettings);
HierarchyCircuitBreakerService.this.inFlightRequestsSettings = newInFlightRequestsSettings;
@ -133,7 +133,7 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
}
private void setFieldDataBreakerLimit(ByteSizeValue newFielddataMax, Double newFielddataOverhead) {
long newFielddataLimitBytes = newFielddataMax == null ? HierarchyCircuitBreakerService.this.fielddataSettings.getLimit() : newFielddataMax.bytes();
long newFielddataLimitBytes = newFielddataMax == null ? HierarchyCircuitBreakerService.this.fielddataSettings.getLimit() : newFielddataMax.getBytes();
newFielddataOverhead = newFielddataOverhead == null ? HierarchyCircuitBreakerService.this.fielddataSettings.getOverhead() : newFielddataOverhead;
BreakerSettings newFielddataSettings = new BreakerSettings(CircuitBreaker.FIELDDATA, newFielddataLimitBytes, newFielddataOverhead,
HierarchyCircuitBreakerService.this.fielddataSettings.getType());
@ -143,13 +143,13 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
}
private boolean validateTotalCircuitBreakerLimit(ByteSizeValue byteSizeValue) {
BreakerSettings newParentSettings = new BreakerSettings(CircuitBreaker.PARENT, byteSizeValue.bytes(), 1.0, CircuitBreaker.Type.PARENT);
BreakerSettings newParentSettings = new BreakerSettings(CircuitBreaker.PARENT, byteSizeValue.getBytes(), 1.0, CircuitBreaker.Type.PARENT);
validateSettings(new BreakerSettings[]{newParentSettings});
return true;
}
private void setTotalCircuitBreakerLimit(ByteSizeValue byteSizeValue) {
BreakerSettings newParentSettings = new BreakerSettings(CircuitBreaker.PARENT, byteSizeValue.bytes(), 1.0, CircuitBreaker.Type.PARENT);
BreakerSettings newParentSettings = new BreakerSettings(CircuitBreaker.PARENT, byteSizeValue.getBytes(), 1.0, CircuitBreaker.Type.PARENT);
this.parentSettings = newParentSettings;
}

View File

@ -58,7 +58,7 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
public IndicesFieldDataCache(Settings settings, IndexFieldDataCache.Listener indicesFieldDataCacheListener) {
super(settings);
this.indicesFieldDataCacheListener = indicesFieldDataCacheListener;
final long sizeInBytes = INDICES_FIELDDATA_CACHE_SIZE_KEY.get(settings).bytes();
final long sizeInBytes = INDICES_FIELDDATA_CACHE_SIZE_KEY.get(settings).getBytes();
CacheBuilder<Key, Accountable> cacheBuilder = CacheBuilder.<Key, Accountable>builder()
.removalListener(this);
if (sizeInBytes > 0) {

View File

@ -99,10 +99,10 @@ public class RecoverySettings extends AbstractComponent {
this.activityTimeout = INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING.get(settings);
this.maxBytesPerSec = INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.get(settings);
if (maxBytesPerSec.bytes() <= 0) {
if (maxBytesPerSec.getBytes() <= 0) {
rateLimiter = null;
} else {
rateLimiter = new SimpleRateLimiter(maxBytesPerSec.mbFrac());
rateLimiter = new SimpleRateLimiter(maxBytesPerSec.getMbFrac());
}
@ -172,12 +172,12 @@ public class RecoverySettings extends AbstractComponent {
private void setMaxBytesPerSec(ByteSizeValue maxBytesPerSec) {
this.maxBytesPerSec = maxBytesPerSec;
if (maxBytesPerSec.bytes() <= 0) {
if (maxBytesPerSec.getBytes() <= 0) {
rateLimiter = null;
} else if (rateLimiter != null) {
rateLimiter.setMBPerSec(maxBytesPerSec.mbFrac());
rateLimiter.setMBPerSec(maxBytesPerSec.getMbFrac());
} else {
rateLimiter = new SimpleRateLimiter(maxBytesPerSec.mbFrac());
rateLimiter = new SimpleRateLimiter(maxBytesPerSec.getMbFrac());
}
}
}

View File

@ -241,7 +241,7 @@ public class OsStats implements Writeable, ToXContent {
}
public short getUsedPercent() {
return calculatePercentage(getUsed().bytes(), total);
return calculatePercentage(getUsed().getBytes(), total);
}
public ByteSizeValue getFree() {

View File

@ -621,10 +621,10 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
private RateLimiter getRateLimiter(Settings repositorySettings, String setting, ByteSizeValue defaultRate) {
ByteSizeValue maxSnapshotBytesPerSec = repositorySettings.getAsBytesSize(setting,
settings.getAsBytesSize(setting, defaultRate));
if (maxSnapshotBytesPerSec.bytes() <= 0) {
if (maxSnapshotBytesPerSec.getBytes() <= 0) {
return null;
} else {
return new RateLimiter.SimpleRateLimiter(maxSnapshotBytesPerSec.mbFrac());
return new RateLimiter.SimpleRateLimiter(maxSnapshotBytesPerSec.getMbFrac());
}
}

View File

@ -126,10 +126,10 @@ public class RestAllocationAction extends AbstractCatAction {
//if we don't know how much we use (non data nodes), it means 0
long used = 0;
short diskPercent = -1;
if (total.bytes() > 0) {
used = total.bytes() - avail.bytes();
if (used >= 0 && avail.bytes() >= 0) {
diskPercent = (short) (used * 100 / (used + avail.bytes()));
if (total.getBytes() > 0) {
used = total.getBytes() - avail.getBytes();
if (used >= 0 && avail.getBytes() >= 0) {
diskPercent = (short) (used * 100 / (used + avail.getBytes()));
}
}
@ -137,8 +137,8 @@ public class RestAllocationAction extends AbstractCatAction {
table.addCell(shardCount);
table.addCell(nodeStats.getIndices().getStore().getSize());
table.addCell(used < 0 ? null : new ByteSizeValue(used));
table.addCell(avail.bytes() < 0 ? null : avail);
table.addCell(total.bytes() < 0 ? null : total);
table.addCell(avail.getBytes() < 0 ? null : avail);
table.addCell(total.getBytes() < 0 ? null : total);
table.addCell(diskPercent < 0 ? null : diskPercent);
table.addCell(node.getHostName());
table.addCell(node.getHostAddress());

View File

@ -302,17 +302,17 @@ public class RestTable {
ByteSizeValue v = (ByteSizeValue) value;
String resolution = request.param("bytes");
if ("b".equals(resolution)) {
return Long.toString(v.bytes());
return Long.toString(v.getBytes());
} else if ("k".equals(resolution) || "kb".equals(resolution)) {
return Long.toString(v.kb());
return Long.toString(v.getKb());
} else if ("m".equals(resolution) || "mb".equals(resolution)) {
return Long.toString(v.mb());
return Long.toString(v.getMb());
} else if ("g".equals(resolution) || "gb".equals(resolution)) {
return Long.toString(v.gb());
return Long.toString(v.getGb());
} else if ("t".equals(resolution) || "tb".equals(resolution)) {
return Long.toString(v.tb());
return Long.toString(v.getTb());
} else if ("p".equals(resolution) || "pb".equals(resolution)) {
return Long.toString(v.pb());
return Long.toString(v.getPb());
} else {
return v.toString();
}

View File

@ -145,7 +145,7 @@ public abstract class TcpTransport<Channel> extends AbstractLifecycleComponent i
Setting.byteSizeSetting("transport.tcp.receive_buffer_size", NetworkService.TcpSettings.TCP_RECEIVE_BUFFER_SIZE,
Setting.Property.NodeScope);
private static final long NINETY_PER_HEAP_SIZE = (long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.9);
private static final long NINETY_PER_HEAP_SIZE = (long) (JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.9);
private static final int PING_DATA_SIZE = -1;
protected final int connectionsPerNodeRecovery;

View File

@ -170,7 +170,7 @@ public class ClusterStatsIT extends ESIntegTestCase {
assertThat(msg, response.getTimestamp(), Matchers.greaterThan(946681200000L)); // 1 Jan 2000
assertThat(msg, response.indicesStats.getStore().getSizeInBytes(), Matchers.greaterThan(0L));
assertThat(msg, response.nodesStats.getFs().getTotal().bytes(), Matchers.greaterThan(0L));
assertThat(msg, response.nodesStats.getFs().getTotal().getBytes(), Matchers.greaterThan(0L));
assertThat(msg, response.nodesStats.getJvm().getVersions().size(), Matchers.greaterThan(0));
assertThat(msg, response.nodesStats.getVersions().size(), Matchers.greaterThan(0));
@ -189,13 +189,13 @@ public class ClusterStatsIT extends ESIntegTestCase {
long free = 0;
long used = 0;
for (NodeStats nodeStats : nodesStatsResponse.getNodes()) {
total += nodeStats.getOs().getMem().getTotal().bytes();
free += nodeStats.getOs().getMem().getFree().bytes();
used += nodeStats.getOs().getMem().getUsed().bytes();
total += nodeStats.getOs().getMem().getTotal().getBytes();
free += nodeStats.getOs().getMem().getFree().getBytes();
used += nodeStats.getOs().getMem().getUsed().getBytes();
}
assertEquals(msg, free, response.nodesStats.getOs().getMem().getFree().bytes());
assertEquals(msg, total, response.nodesStats.getOs().getMem().getTotal().bytes());
assertEquals(msg, used, response.nodesStats.getOs().getMem().getUsed().bytes());
assertEquals(msg, free, response.nodesStats.getOs().getMem().getFree().getBytes());
assertEquals(msg, total, response.nodesStats.getOs().getMem().getTotal().getBytes());
assertEquals(msg, used, response.nodesStats.getOs().getMem().getUsed().getBytes());
assertEquals(msg, OsStats.calculatePercentage(used, total), response.nodesStats.getOs().getMem().getUsedPercent());
assertEquals(msg, OsStats.calculatePercentage(free, total), response.nodesStats.getOs().getMem().getFreePercent());
}

View File

@ -26,7 +26,6 @@ import org.elasticsearch.action.admin.indices.stats.ShardStats;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.RecoverySource;
import org.elasticsearch.cluster.routing.RecoverySource.PeerRecoverySource;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingHelper;
@ -269,8 +268,8 @@ public class DiskUsageTests extends ESTestCase {
assertNotNull(usage);
assertNotNull(path);
assertEquals(usage.toString(), usage.getPath(), path.getPath());
assertEquals(usage.toString(), usage.getTotalBytes(), path.getTotal().bytes());
assertEquals(usage.toString(), usage.getFreeBytes(), path.getAvailable().bytes());
assertEquals(usage.toString(), usage.getTotalBytes(), path.getTotal().getBytes());
assertEquals(usage.toString(), usage.getFreeBytes(), path.getAvailable().getBytes());
}
}

View File

@ -38,33 +38,33 @@ public class MemorySizeSettingsTests extends ESTestCase {
public void testPageCacheLimitHeapSetting() {
assertMemorySizeSetting(PageCacheRecycler.LIMIT_HEAP_SETTING, "cache.recycler.page.limit.heap",
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.1)));
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.1)));
}
public void testIndexBufferSizeSetting() {
assertMemorySizeSetting(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "indices.memory.index_buffer_size",
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.1)));
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.1)));
}
public void testQueryCacheSizeSetting() {
assertMemorySizeSetting(IndicesQueryCache.INDICES_CACHE_QUERY_SIZE_SETTING, "indices.queries.cache.size",
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.1)));
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.1)));
}
public void testIndicesRequestCacheSetting() {
assertMemorySizeSetting(IndicesRequestCache.INDICES_CACHE_QUERY_SIZE, "indices.requests.cache.size",
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.01)));
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.01)));
}
public void testCircuitBreakerSettings() {
assertMemorySizeSetting(HierarchyCircuitBreakerService.TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING, "indices.breaker.total.limit",
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.7)));
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.7)));
assertMemorySizeSetting(HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING, "indices.breaker.fielddata.limit",
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.6)));
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.6)));
assertMemorySizeSetting(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, "indices.breaker.request.limit",
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.6)));
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.6)));
assertMemorySizeSetting(HierarchyCircuitBreakerService.IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_LIMIT_SETTING,
"network.breaker.inflight_requests.limit", new ByteSizeValue((JvmInfo.jvmInfo().getMem().getHeapMax().bytes())));
"network.breaker.inflight_requests.limit", new ByteSizeValue((JvmInfo.jvmInfo().getMem().getHeapMax().getBytes())));
}
public void testIndicesFieldDataCacheSetting() {
@ -80,7 +80,7 @@ public class MemorySizeSettingsTests extends ESTestCase {
equalTo(defaultValue));
Settings settingWithPercentage = Settings.builder().put(settingKey, "25%").build();
assertThat(setting.get(settingWithPercentage),
equalTo(new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.25))));
equalTo(new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.25))));
Settings settingWithBytesValue = Settings.builder().put(settingKey, "1024b").build();
assertThat(setting.get(settingWithBytesValue), equalTo(new ByteSizeValue(1024)));
}

View File

@ -48,11 +48,11 @@ public class SettingTests extends ESTestCase {
Setting.byteSizeSetting("a.byte.size", new ByteSizeValue(1024), Property.Dynamic, Property.NodeScope);
assertFalse(byteSizeValueSetting.isGroupSetting());
ByteSizeValue byteSizeValue = byteSizeValueSetting.get(Settings.EMPTY);
assertEquals(byteSizeValue.bytes(), 1024);
assertEquals(byteSizeValue.getBytes(), 1024);
byteSizeValueSetting = Setting.byteSizeSetting("a.byte.size", s -> "2048b", Property.Dynamic, Property.NodeScope);
byteSizeValue = byteSizeValueSetting.get(Settings.EMPTY);
assertEquals(byteSizeValue.bytes(), 2048);
assertEquals(byteSizeValue.getBytes(), 2048);
AtomicReference<ByteSizeValue> value = new AtomicReference<>(null);
@ -75,20 +75,20 @@ public class SettingTests extends ESTestCase {
assertFalse(memorySizeValueSetting.isGroupSetting());
ByteSizeValue memorySizeValue = memorySizeValueSetting.get(Settings.EMPTY);
assertEquals(memorySizeValue.bytes(), 1024);
assertEquals(memorySizeValue.getBytes(), 1024);
memorySizeValueSetting = Setting.memorySizeSetting("a.byte.size", s -> "2048b", Property.Dynamic, Property.NodeScope);
memorySizeValue = memorySizeValueSetting.get(Settings.EMPTY);
assertEquals(memorySizeValue.bytes(), 2048);
assertEquals(memorySizeValue.getBytes(), 2048);
memorySizeValueSetting = Setting.memorySizeSetting("a.byte.size", "50%", Property.Dynamic, Property.NodeScope);
assertFalse(memorySizeValueSetting.isGroupSetting());
memorySizeValue = memorySizeValueSetting.get(Settings.EMPTY);
assertEquals(memorySizeValue.bytes(), JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.5, 1.0);
assertEquals(memorySizeValue.getBytes(), JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.5, 1.0);
memorySizeValueSetting = Setting.memorySizeSetting("a.byte.size", s -> "25%", Property.Dynamic, Property.NodeScope);
memorySizeValue = memorySizeValueSetting.get(Settings.EMPTY);
assertEquals(memorySizeValue.bytes(), JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.25, 1.0);
assertEquals(memorySizeValue.getBytes(), JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.25, 1.0);
AtomicReference<ByteSizeValue> value = new AtomicReference<>(null);
ClusterSettings.SettingUpdater<ByteSizeValue> settingUpdater = memorySizeValueSetting.newUpdater(value::set, logger);
@ -104,7 +104,7 @@ public class SettingTests extends ESTestCase {
assertEquals(new ByteSizeValue(12), value.get());
assertTrue(settingUpdater.apply(Settings.builder().put("a.byte.size", "20%").build(), Settings.EMPTY));
assertEquals(new ByteSizeValue((int) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.2)), value.get());
assertEquals(new ByteSizeValue((int) (JvmInfo.jvmInfo().getMem().getHeapMax().getBytes() * 0.2)), value.get());
}
public void testSimpleUpdate() {

View File

@ -36,24 +36,24 @@ import static org.hamcrest.Matchers.is;
*/
public class ByteSizeValueTests extends ESTestCase {
public void testActualPeta() {
MatcherAssert.assertThat(new ByteSizeValue(4, ByteSizeUnit.PB).bytes(), equalTo(4503599627370496L));
MatcherAssert.assertThat(new ByteSizeValue(4, ByteSizeUnit.PB).getBytes(), equalTo(4503599627370496L));
}
public void testActualTera() {
MatcherAssert.assertThat(new ByteSizeValue(4, ByteSizeUnit.TB).bytes(), equalTo(4398046511104L));
MatcherAssert.assertThat(new ByteSizeValue(4, ByteSizeUnit.TB).getBytes(), equalTo(4398046511104L));
}
public void testActual() {
MatcherAssert.assertThat(new ByteSizeValue(4, ByteSizeUnit.GB).bytes(), equalTo(4294967296L));
MatcherAssert.assertThat(new ByteSizeValue(4, ByteSizeUnit.GB).getBytes(), equalTo(4294967296L));
}
public void testSimple() {
assertThat(ByteSizeUnit.BYTES.toBytes(10), is(new ByteSizeValue(10, ByteSizeUnit.BYTES).bytes()));
assertThat(ByteSizeUnit.KB.toKB(10), is(new ByteSizeValue(10, ByteSizeUnit.KB).kb()));
assertThat(ByteSizeUnit.MB.toMB(10), is(new ByteSizeValue(10, ByteSizeUnit.MB).mb()));
assertThat(ByteSizeUnit.GB.toGB(10), is(new ByteSizeValue(10, ByteSizeUnit.GB).gb()));
assertThat(ByteSizeUnit.TB.toTB(10), is(new ByteSizeValue(10, ByteSizeUnit.TB).tb()));
assertThat(ByteSizeUnit.PB.toPB(10), is(new ByteSizeValue(10, ByteSizeUnit.PB).pb()));
assertThat(ByteSizeUnit.BYTES.toBytes(10), is(new ByteSizeValue(10, ByteSizeUnit.BYTES).getBytes()));
assertThat(ByteSizeUnit.KB.toKB(10), is(new ByteSizeValue(10, ByteSizeUnit.KB).getKb()));
assertThat(ByteSizeUnit.MB.toMB(10), is(new ByteSizeValue(10, ByteSizeUnit.MB).getMb()));
assertThat(ByteSizeUnit.GB.toGB(10), is(new ByteSizeValue(10, ByteSizeUnit.GB).getGb()));
assertThat(ByteSizeUnit.TB.toTB(10), is(new ByteSizeValue(10, ByteSizeUnit.TB).getTb()));
assertThat(ByteSizeUnit.PB.toPB(10), is(new ByteSizeValue(10, ByteSizeUnit.PB).getPb()));
}
public void testEquality() {

View File

@ -20,13 +20,9 @@ package org.elasticsearch.index;
import org.apache.lucene.index.NoMergePolicy;
import org.apache.lucene.index.TieredMergePolicy;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.MergePolicyConfig;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.test.ESTestCase;
@ -85,9 +81,9 @@ public class MergePolicySettingsTests extends ESTestCase {
indexSettings.updateIndexMetaData(newIndexMeta("index", Settings.builder().put(MergePolicyConfig.INDEX_MERGE_POLICY_EXPUNGE_DELETES_ALLOWED_SETTING.getKey(), MergePolicyConfig.DEFAULT_EXPUNGE_DELETES_ALLOWED + 1.0d).build()));
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getForceMergeDeletesPctAllowed(), MergePolicyConfig.DEFAULT_EXPUNGE_DELETES_ALLOWED + 1.0d, 0.0d);
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getFloorSegmentMB(), MergePolicyConfig.DEFAULT_FLOOR_SEGMENT.mbFrac(), 0);
indexSettings.updateIndexMetaData(newIndexMeta("index", Settings.builder().put(MergePolicyConfig.INDEX_MERGE_POLICY_FLOOR_SEGMENT_SETTING.getKey(), new ByteSizeValue(MergePolicyConfig.DEFAULT_FLOOR_SEGMENT.mb() + 1, ByteSizeUnit.MB)).build()));
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getFloorSegmentMB(), new ByteSizeValue(MergePolicyConfig.DEFAULT_FLOOR_SEGMENT.mb() + 1, ByteSizeUnit.MB).mbFrac(), 0.001);
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getFloorSegmentMB(), MergePolicyConfig.DEFAULT_FLOOR_SEGMENT.getMbFrac(), 0);
indexSettings.updateIndexMetaData(newIndexMeta("index", Settings.builder().put(MergePolicyConfig.INDEX_MERGE_POLICY_FLOOR_SEGMENT_SETTING.getKey(), new ByteSizeValue(MergePolicyConfig.DEFAULT_FLOOR_SEGMENT.getMb() + 1, ByteSizeUnit.MB)).build()));
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getFloorSegmentMB(), new ByteSizeValue(MergePolicyConfig.DEFAULT_FLOOR_SEGMENT.getMb() + 1, ByteSizeUnit.MB).getMbFrac(), 0.001);
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getMaxMergeAtOnce(), MergePolicyConfig.DEFAULT_MAX_MERGE_AT_ONCE);
indexSettings.updateIndexMetaData(newIndexMeta("index", Settings.builder().put(MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_SETTING.getKey(), MergePolicyConfig.DEFAULT_MAX_MERGE_AT_ONCE - 1).build()));
@ -97,9 +93,9 @@ public class MergePolicySettingsTests extends ESTestCase {
indexSettings.updateIndexMetaData(newIndexMeta("index", Settings.builder().put(MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_EXPLICIT_SETTING.getKey(), MergePolicyConfig.DEFAULT_MAX_MERGE_AT_ONCE_EXPLICIT - 1).build()));
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getMaxMergeAtOnceExplicit(), MergePolicyConfig.DEFAULT_MAX_MERGE_AT_ONCE_EXPLICIT-1);
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getMaxMergedSegmentMB(), MergePolicyConfig.DEFAULT_MAX_MERGED_SEGMENT.mbFrac(), 0.0001);
indexSettings.updateIndexMetaData(newIndexMeta("index", Settings.builder().put(MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGED_SEGMENT_SETTING.getKey(), new ByteSizeValue(MergePolicyConfig.DEFAULT_MAX_MERGED_SEGMENT.bytes() + 1)).build()));
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getMaxMergedSegmentMB(), new ByteSizeValue(MergePolicyConfig.DEFAULT_MAX_MERGED_SEGMENT.bytes() + 1).mbFrac(), 0.0001);
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getMaxMergedSegmentMB(), MergePolicyConfig.DEFAULT_MAX_MERGED_SEGMENT.getMbFrac(), 0.0001);
indexSettings.updateIndexMetaData(newIndexMeta("index", Settings.builder().put(MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGED_SEGMENT_SETTING.getKey(), new ByteSizeValue(MergePolicyConfig.DEFAULT_MAX_MERGED_SEGMENT.getBytes() + 1)).build()));
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getMaxMergedSegmentMB(), new ByteSizeValue(MergePolicyConfig.DEFAULT_MAX_MERGED_SEGMENT.getBytes() + 1).getMbFrac(), 0.0001);
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getReclaimDeletesWeight(), MergePolicyConfig.DEFAULT_RECLAIM_DELETES_WEIGHT, 0);
indexSettings.updateIndexMetaData(newIndexMeta("index", Settings.builder().put(MergePolicyConfig.INDEX_MERGE_POLICY_RECLAIM_DELETES_WEIGHT_SETTING.getKey(), MergePolicyConfig.DEFAULT_RECLAIM_DELETES_WEIGHT + 1).build()));
@ -111,10 +107,10 @@ public class MergePolicySettingsTests extends ESTestCase {
indexSettings.updateIndexMetaData(newIndexMeta("index", EMPTY_SETTINGS)); // see if defaults are restored
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getForceMergeDeletesPctAllowed(), MergePolicyConfig.DEFAULT_EXPUNGE_DELETES_ALLOWED, 0.0d);
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getFloorSegmentMB(), new ByteSizeValue(MergePolicyConfig.DEFAULT_FLOOR_SEGMENT.mb(), ByteSizeUnit.MB).mbFrac(), 0.00);
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getFloorSegmentMB(), new ByteSizeValue(MergePolicyConfig.DEFAULT_FLOOR_SEGMENT.getMb(), ByteSizeUnit.MB).getMbFrac(), 0.00);
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getMaxMergeAtOnce(), MergePolicyConfig.DEFAULT_MAX_MERGE_AT_ONCE);
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getMaxMergeAtOnceExplicit(), MergePolicyConfig.DEFAULT_MAX_MERGE_AT_ONCE_EXPLICIT);
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getMaxMergedSegmentMB(), new ByteSizeValue(MergePolicyConfig.DEFAULT_MAX_MERGED_SEGMENT.bytes() + 1).mbFrac(), 0.0001);
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getMaxMergedSegmentMB(), new ByteSizeValue(MergePolicyConfig.DEFAULT_MAX_MERGED_SEGMENT.getBytes() + 1).getMbFrac(), 0.0001);
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getReclaimDeletesWeight(), MergePolicyConfig.DEFAULT_RECLAIM_DELETES_WEIGHT, 0);
assertEquals(((TieredMergePolicy) indexSettings.getMergePolicy()).getSegmentsPerTier(), MergePolicyConfig.DEFAULT_SEGMENTS_PER_TIER, 0);
}

View File

@ -46,7 +46,6 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -766,7 +765,7 @@ public class StoreTests extends ESTestCase {
initialStoreSize += store.directory().fileLength(extraFiles);
}
StoreStats stats = store.stats();
assertEquals(stats.getSize().bytes(), initialStoreSize);
assertEquals(stats.getSize().getBytes(), initialStoreSize);
Directory dir = store.directory();
final long length;

View File

@ -31,7 +31,6 @@ import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
import org.elasticsearch.common.settings.Settings;
@ -216,7 +215,7 @@ public class IndicesRequestCacheTests extends ESTestCase {
IOUtils.close(reader, secondReader, writer, dir, cache);
}
IndicesRequestCache cache = new IndicesRequestCache(Settings.builder()
.put(IndicesRequestCache.INDICES_CACHE_QUERY_SIZE.getKey(), size.bytes()+1 +"b")
.put(IndicesRequestCache.INDICES_CACHE_QUERY_SIZE.getKey(), size.getBytes()+1 +"b")
.build());
AtomicBoolean indexShard = new AtomicBoolean(true);
ShardRequestCache requestCacheStats = new ShardRequestCache();

View File

@ -48,7 +48,6 @@ import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.junit.After;
import org.junit.Before;
@ -468,11 +467,11 @@ public class CircuitBreakerServiceIT extends ESIntegTestCase {
for (BulkItemResponse bulkItemResponse : response) {
Throwable cause = ExceptionsHelper.unwrapCause(bulkItemResponse.getFailure().getCause());
assertThat(cause, instanceOf(CircuitBreakingException.class));
assertEquals(((CircuitBreakingException) cause).getByteLimit(), inFlightRequestsLimit.bytes());
assertEquals(((CircuitBreakingException) cause).getByteLimit(), inFlightRequestsLimit.getBytes());
}
}
} catch (CircuitBreakingException ex) {
assertEquals(ex.getByteLimit(), inFlightRequestsLimit.bytes());
assertEquals(ex.getByteLimit(), inFlightRequestsLimit.getBytes());
}
}
}

View File

@ -37,7 +37,7 @@ import static org.hamcrest.Matchers.notNullValue;
*/
public class CircuitBreakerUnitTests extends ESTestCase {
public static long pctBytes(String percentString) {
return Settings.EMPTY.getAsMemory("", percentString).bytes();
return Settings.EMPTY.getAsMemory("", percentString).getBytes();
}
public void testBreakerSettingsValidationWithValidSettings() {

View File

@ -136,7 +136,7 @@ public class IndexRecoveryIT extends ESIntegTestCase {
}
private void slowDownRecovery(ByteSizeValue shardSize) {
long chunkSize = Math.max(1, shardSize.bytes() / 10);
long chunkSize = Math.max(1, shardSize.getBytes() / 10);
for(RecoverySettings settings : internalCluster().getInstances(RecoverySettings.class)) {
setChunkSize(settings, new ByteSizeValue(chunkSize, ByteSizeUnit.BYTES));
}

View File

@ -115,7 +115,7 @@ public class UpdateSettingsIT extends ESIntegTestCase {
IndexService indexService = service.indexService(resolveIndex("test"));
if (indexService != null) {
assertEquals(indexService.getIndexSettings().getRefreshInterval().millis(), -1);
assertEquals(indexService.getIndexSettings().getFlushThresholdSize().bytes(), 1024);
assertEquals(indexService.getIndexSettings().getFlushThresholdSize().getBytes(), 1024);
}
}
client().admin().indices().prepareUpdateSettings("test")
@ -129,7 +129,7 @@ public class UpdateSettingsIT extends ESIntegTestCase {
IndexService indexService = service.indexService(resolveIndex("test"));
if (indexService != null) {
assertEquals(indexService.getIndexSettings().getRefreshInterval().millis(), 1000);
assertEquals(indexService.getIndexSettings().getFlushThresholdSize().bytes(), 1024);
assertEquals(indexService.getIndexSettings().getFlushThresholdSize().getBytes(), 1024);
}
}
}

View File

@ -64,7 +64,7 @@ public class JvmGcMonitorServiceTests extends ESTestCase {
when(gc.getCollectionCount()).thenReturn(totalCollectionCount);
when(gc.getCollectionTime()).thenReturn(totalCollectionTime);
final ByteSizeValue maxHeapUsed = new ByteSizeValue(Math.max(lastHeapUsed.bytes(), currentHeapUsed.bytes()) + 1 << 10);
final ByteSizeValue maxHeapUsed = new ByteSizeValue(Math.max(lastHeapUsed.getBytes(), currentHeapUsed.getBytes()) + 1 << 10);
JvmGcMonitorService.JvmMonitor.SlowGcEvent slowGcEvent = new JvmGcMonitorService.JvmMonitor.SlowGcEvent(
gc,

View File

@ -45,7 +45,7 @@ public class JvmStatsTests extends ESTestCase {
assertNotNull(mem);
for (ByteSizeValue heap : Arrays.asList(mem.getHeapCommitted(), mem.getHeapMax(), mem.getHeapUsed(), mem.getNonHeapCommitted())) {
assertNotNull(heap);
assertThat(heap.bytes(), greaterThanOrEqualTo(0L));
assertThat(heap.getBytes(), greaterThanOrEqualTo(0L));
}
assertNotNull(mem.getHeapUsedPercent());
assertThat(mem.getHeapUsedPercent(), anyOf(equalTo((short) -1), greaterThanOrEqualTo((short) 0)));
@ -78,9 +78,9 @@ public class JvmStatsTests extends ESTestCase {
assertTrue(Strings.hasText(bufferPool.getName()));
assertThat(bufferPool.getCount(), greaterThanOrEqualTo(0L));
assertNotNull(bufferPool.getTotalCapacity());
assertThat(bufferPool.getTotalCapacity().bytes(), greaterThanOrEqualTo(0L));
assertThat(bufferPool.getTotalCapacity().getBytes(), greaterThanOrEqualTo(0L));
assertNotNull(bufferPool.getUsed());
assertThat(bufferPool.getUsed().bytes(), anyOf(equalTo(-1L), greaterThanOrEqualTo(0L)));
assertThat(bufferPool.getUsed().getBytes(), anyOf(equalTo(-1L), greaterThanOrEqualTo(0L)));
}
}

View File

@ -88,25 +88,25 @@ public class OsProbeTests extends ESTestCase {
}
assertNotNull(stats.getMem());
assertThat(stats.getMem().getTotal().bytes(), greaterThan(0L));
assertThat(stats.getMem().getFree().bytes(), greaterThan(0L));
assertThat(stats.getMem().getTotal().getBytes(), greaterThan(0L));
assertThat(stats.getMem().getFree().getBytes(), greaterThan(0L));
assertThat(stats.getMem().getFreePercent(), allOf(greaterThanOrEqualTo((short) 0), lessThanOrEqualTo((short) 100)));
assertThat(stats.getMem().getUsed().bytes(), greaterThan(0L));
assertThat(stats.getMem().getUsed().getBytes(), greaterThan(0L));
assertThat(stats.getMem().getUsedPercent(), allOf(greaterThanOrEqualTo((short) 0), lessThanOrEqualTo((short) 100)));
assertNotNull(stats.getSwap());
assertNotNull(stats.getSwap().getTotal());
long total = stats.getSwap().getTotal().bytes();
long total = stats.getSwap().getTotal().getBytes();
if (total > 0) {
assertThat(stats.getSwap().getTotal().bytes(), greaterThan(0L));
assertThat(stats.getSwap().getFree().bytes(), greaterThan(0L));
assertThat(stats.getSwap().getUsed().bytes(), greaterThanOrEqualTo(0L));
assertThat(stats.getSwap().getTotal().getBytes(), greaterThan(0L));
assertThat(stats.getSwap().getFree().getBytes(), greaterThan(0L));
assertThat(stats.getSwap().getUsed().getBytes(), greaterThanOrEqualTo(0L));
} else {
// On platforms with no swap
assertThat(stats.getSwap().getTotal().bytes(), equalTo(0L));
assertThat(stats.getSwap().getFree().bytes(), equalTo(0L));
assertThat(stats.getSwap().getUsed().bytes(), equalTo(0L));
assertThat(stats.getSwap().getTotal().getBytes(), equalTo(0L));
assertThat(stats.getSwap().getFree().getBytes(), equalTo(0L));
assertThat(stats.getSwap().getUsed().getBytes(), equalTo(0L));
}
}
}

View File

@ -70,6 +70,6 @@ public class ProcessProbeTests extends ESTestCase {
ProcessStats.Mem mem = stats.getMem();
assertNotNull(mem);
// Commited total virtual memory can return -1 if not supported, let's see which platforms fail
assertThat(mem.getTotalVirtual().bytes(), greaterThan(0L));
assertThat(mem.getTotalVirtual().getBytes(), greaterThan(0L));
}
}

View File

@ -97,22 +97,22 @@ public class SimpleNodesInfoIT extends ESIntegTestCase {
assertThat(response.getNodes().size(), is(2));
assertThat(response.getNodesMap().get(server1NodeId), notNullValue());
assertNotNull(response.getNodesMap().get(server1NodeId).getTotalIndexingBuffer());
assertThat(response.getNodesMap().get(server1NodeId).getTotalIndexingBuffer().bytes(), greaterThan(0L));
assertThat(response.getNodesMap().get(server1NodeId).getTotalIndexingBuffer().getBytes(), greaterThan(0L));
assertThat(response.getNodesMap().get(server2NodeId), notNullValue());
assertNotNull(response.getNodesMap().get(server2NodeId).getTotalIndexingBuffer());
assertThat(response.getNodesMap().get(server2NodeId).getTotalIndexingBuffer().bytes(), greaterThan(0L));
assertThat(response.getNodesMap().get(server2NodeId).getTotalIndexingBuffer().getBytes(), greaterThan(0L));
// again, using only the indices flag
response = client().admin().cluster().prepareNodesInfo().clear().setIndices(true).execute().actionGet();
assertThat(response.getNodes().size(), is(2));
assertThat(response.getNodesMap().get(server1NodeId), notNullValue());
assertNotNull(response.getNodesMap().get(server1NodeId).getTotalIndexingBuffer());
assertThat(response.getNodesMap().get(server1NodeId).getTotalIndexingBuffer().bytes(), greaterThan(0L));
assertThat(response.getNodesMap().get(server1NodeId).getTotalIndexingBuffer().getBytes(), greaterThan(0L));
assertThat(response.getNodesMap().get(server2NodeId), notNullValue());
assertNotNull(response.getNodesMap().get(server2NodeId).getTotalIndexingBuffer());
assertThat(response.getNodesMap().get(server2NodeId).getTotalIndexingBuffer().bytes(), greaterThan(0L));
assertThat(response.getNodesMap().get(server2NodeId).getTotalIndexingBuffer().getBytes(), greaterThan(0L));
}
public void testAllocatedProcessors() throws Exception {

View File

@ -148,9 +148,9 @@ public class Netty3HttpServerTransport extends AbstractLifecycleComponent implem
Setting.byteSizeSetting("transport.netty.receive_predictor_size",
settings -> {
long defaultReceiverPredictor = 512 * 1024;
if (JvmInfo.jvmInfo().getMem().getDirectMemoryMax().bytes() > 0) {
if (JvmInfo.jvmInfo().getMem().getDirectMemoryMax().getBytes() > 0) {
// we can guess a better default...
long l = (long) ((0.3 * JvmInfo.jvmInfo().getMem().getDirectMemoryMax().bytes()) / SETTING_HTTP_WORKER_COUNT.get
long l = (long) ((0.3 * JvmInfo.jvmInfo().getMem().getDirectMemoryMax().getBytes()) / SETTING_HTTP_WORKER_COUNT.get
(settings));
defaultReceiverPredictor = Math.min(defaultReceiverPredictor, Math.max(l, 64 * 1024));
}
@ -246,11 +246,11 @@ public class Netty3HttpServerTransport extends AbstractLifecycleComponent implem
// See AdaptiveReceiveBufferSizePredictor#DEFAULT_XXX for default values in netty..., we can use higher ones for us, even fixed one
ByteSizeValue receivePredictorMin = SETTING_HTTP_NETTY_RECEIVE_PREDICTOR_MIN.get(settings);
ByteSizeValue receivePredictorMax = SETTING_HTTP_NETTY_RECEIVE_PREDICTOR_MAX.get(settings);
if (receivePredictorMax.bytes() == receivePredictorMin.bytes()) {
receiveBufferSizePredictorFactory = new FixedReceiveBufferSizePredictorFactory((int) receivePredictorMax.bytes());
if (receivePredictorMax.getBytes() == receivePredictorMin.getBytes()) {
receiveBufferSizePredictorFactory = new FixedReceiveBufferSizePredictorFactory((int) receivePredictorMax.getBytes());
} else {
receiveBufferSizePredictorFactory = new AdaptiveReceiveBufferSizePredictorFactory(
(int) receivePredictorMin.bytes(), (int) receivePredictorMin.bytes(), (int) receivePredictorMax.bytes());
(int) receivePredictorMin.getBytes(), (int) receivePredictorMin.getBytes(), (int) receivePredictorMax.getBytes());
}
this.compression = SETTING_HTTP_COMPRESSION.get(settings);
@ -260,7 +260,7 @@ public class Netty3HttpServerTransport extends AbstractLifecycleComponent implem
this.corsConfig = buildCorsConfig(settings);
// validate max content length
if (maxContentLength.bytes() > Integer.MAX_VALUE) {
if (maxContentLength.getBytes() > Integer.MAX_VALUE) {
logger.warn("maxContentLength[{}] set to high value, resetting it to [100mb]", maxContentLength);
maxContentLength = new ByteSizeValue(100, ByteSizeUnit.MB);
}
@ -298,12 +298,12 @@ public class Netty3HttpServerTransport extends AbstractLifecycleComponent implem
serverBootstrap.setOption("child.tcpNoDelay", tcpNoDelay);
serverBootstrap.setOption("child.keepAlive", tcpKeepAlive);
if (tcpSendBufferSize.bytes() > 0) {
if (tcpSendBufferSize.getBytes() > 0) {
serverBootstrap.setOption("child.sendBufferSize", tcpSendBufferSize.bytes());
serverBootstrap.setOption("child.sendBufferSize", tcpSendBufferSize.getBytes());
}
if (tcpReceiveBufferSize.bytes() > 0) {
serverBootstrap.setOption("child.receiveBufferSize", tcpReceiveBufferSize.bytes());
if (tcpReceiveBufferSize.getBytes() > 0) {
serverBootstrap.setOption("child.receiveBufferSize", tcpReceiveBufferSize.getBytes());
}
serverBootstrap.setOption("receiveBufferSizePredictorFactory", receiveBufferSizePredictorFactory);
serverBootstrap.setOption("child.receiveBufferSizePredictorFactory", receiveBufferSizePredictorFactory);
@ -466,7 +466,7 @@ public class Netty3HttpServerTransport extends AbstractLifecycleComponent implem
if (boundTransportAddress == null) {
return null;
}
return new HttpInfo(boundTransportAddress, maxContentLength.bytes());
return new HttpInfo(boundTransportAddress, maxContentLength.getBytes());
}
@Override
@ -531,15 +531,15 @@ public class Netty3HttpServerTransport extends AbstractLifecycleComponent implem
ChannelPipeline pipeline = Channels.pipeline();
pipeline.addLast("openChannels", transport.serverOpenChannels);
HttpRequestDecoder requestDecoder = new HttpRequestDecoder(
(int) transport.maxInitialLineLength.bytes(),
(int) transport.maxHeaderSize.bytes(),
(int) transport.maxChunkSize.bytes()
(int) transport.maxInitialLineLength.getBytes(),
(int) transport.maxHeaderSize.getBytes(),
(int) transport.maxChunkSize.getBytes()
);
if (transport.maxCumulationBufferCapacity.bytes() >= 0) {
if (transport.maxCumulationBufferCapacity.bytes() > Integer.MAX_VALUE) {
if (transport.maxCumulationBufferCapacity.getBytes() >= 0) {
if (transport.maxCumulationBufferCapacity.getBytes() > Integer.MAX_VALUE) {
requestDecoder.setMaxCumulationBufferCapacity(Integer.MAX_VALUE);
} else {
requestDecoder.setMaxCumulationBufferCapacity((int) transport.maxCumulationBufferCapacity.bytes());
requestDecoder.setMaxCumulationBufferCapacity((int) transport.maxCumulationBufferCapacity.getBytes());
}
}
if (transport.maxCompositeBufferComponents != -1) {
@ -547,7 +547,7 @@ public class Netty3HttpServerTransport extends AbstractLifecycleComponent implem
}
pipeline.addLast("decoder", requestDecoder);
pipeline.addLast("decoder_compress", new HttpContentDecompressor());
HttpChunkAggregator httpChunkAggregator = new HttpChunkAggregator((int) transport.maxContentLength.bytes());
HttpChunkAggregator httpChunkAggregator = new HttpChunkAggregator((int) transport.maxContentLength.getBytes());
if (transport.maxCompositeBufferComponents != -1) {
httpChunkAggregator.setMaxCumulationBufferComponents(transport.maxCompositeBufferComponents);
}

View File

@ -110,9 +110,9 @@ public class Netty3Transport extends TcpTransport<Channel> {
"transport.netty.receive_predictor_size",
settings -> {
long defaultReceiverPredictor = 512 * 1024;
if (JvmInfo.jvmInfo().getMem().getDirectMemoryMax().bytes() > 0) {
if (JvmInfo.jvmInfo().getMem().getDirectMemoryMax().getBytes() > 0) {
// we can guess a better default...
long l = (long) ((0.3 * JvmInfo.jvmInfo().getMem().getDirectMemoryMax().bytes()) / WORKER_COUNT.get(settings));
long l = (long) ((0.3 * JvmInfo.jvmInfo().getMem().getDirectMemoryMax().getBytes()) / WORKER_COUNT.get(settings));
defaultReceiverPredictor = Math.min(defaultReceiverPredictor, Math.max(l, 64 * 1024));
}
return new ByteSizeValue(defaultReceiverPredictor).toString();
@ -147,11 +147,11 @@ public class Netty3Transport extends TcpTransport<Channel> {
// See AdaptiveReceiveBufferSizePredictor#DEFAULT_XXX for default values in netty..., we can use higher ones for us, even fixed one
this.receivePredictorMin = NETTY_RECEIVE_PREDICTOR_MIN.get(settings);
this.receivePredictorMax = NETTY_RECEIVE_PREDICTOR_MAX.get(settings);
if (receivePredictorMax.bytes() == receivePredictorMin.bytes()) {
receiveBufferSizePredictorFactory = new FixedReceiveBufferSizePredictorFactory((int) receivePredictorMax.bytes());
if (receivePredictorMax.getBytes() == receivePredictorMin.getBytes()) {
receiveBufferSizePredictorFactory = new FixedReceiveBufferSizePredictorFactory((int) receivePredictorMax.getBytes());
} else {
receiveBufferSizePredictorFactory = new AdaptiveReceiveBufferSizePredictorFactory((int) receivePredictorMin.bytes(),
(int) receivePredictorMin.bytes(), (int) receivePredictorMax.bytes());
receiveBufferSizePredictorFactory = new AdaptiveReceiveBufferSizePredictorFactory((int) receivePredictorMin.getBytes(),
(int) receivePredictorMin.getBytes(), (int) receivePredictorMax.getBytes());
}
}
@ -211,13 +211,13 @@ public class Netty3Transport extends TcpTransport<Channel> {
clientBootstrap.setOption("keepAlive", tcpKeepAlive);
ByteSizeValue tcpSendBufferSize = TCP_SEND_BUFFER_SIZE.get(settings);
if (tcpSendBufferSize.bytes() > 0) {
clientBootstrap.setOption("sendBufferSize", tcpSendBufferSize.bytes());
if (tcpSendBufferSize.getBytes() > 0) {
clientBootstrap.setOption("sendBufferSize", tcpSendBufferSize.getBytes());
}
ByteSizeValue tcpReceiveBufferSize = TCP_RECEIVE_BUFFER_SIZE.get(settings);
if (tcpReceiveBufferSize.bytes() > 0) {
clientBootstrap.setOption("receiveBufferSize", tcpReceiveBufferSize.bytes());
if (tcpReceiveBufferSize.getBytes() > 0) {
clientBootstrap.setOption("receiveBufferSize", tcpReceiveBufferSize.getBytes());
}
clientBootstrap.setOption("receiveBufferSizePredictorFactory", receiveBufferSizePredictorFactory);
@ -252,13 +252,13 @@ public class Netty3Transport extends TcpTransport<Channel> {
ByteSizeValue fallbackTcpSendBufferSize = settings.getAsBytesSize("transport.netty.tcp_send_buffer_size",
TCP_SEND_BUFFER_SIZE.get(settings));
if (fallbackTcpSendBufferSize.bytes() >= 0) {
if (fallbackTcpSendBufferSize.getBytes() >= 0) {
fallbackSettingsBuilder.put("tcp_send_buffer_size", fallbackTcpSendBufferSize);
}
ByteSizeValue fallbackTcpBufferSize = settings.getAsBytesSize("transport.netty.tcp_receive_buffer_size",
TCP_RECEIVE_BUFFER_SIZE.get(settings));
if (fallbackTcpBufferSize.bytes() >= 0) {
if (fallbackTcpBufferSize.getBytes() >= 0) {
fallbackSettingsBuilder.put("tcp_receive_buffer_size", fallbackTcpBufferSize);
}
@ -305,11 +305,11 @@ public class Netty3Transport extends TcpTransport<Channel> {
if (!"default".equals(tcpKeepAlive)) {
serverBootstrap.setOption("child.keepAlive", Booleans.parseBoolean(tcpKeepAlive, null));
}
if (tcpSendBufferSize != null && tcpSendBufferSize.bytes() > 0) {
serverBootstrap.setOption("child.sendBufferSize", tcpSendBufferSize.bytes());
if (tcpSendBufferSize != null && tcpSendBufferSize.getBytes() > 0) {
serverBootstrap.setOption("child.sendBufferSize", tcpSendBufferSize.getBytes());
}
if (tcpReceiveBufferSize != null && tcpReceiveBufferSize.bytes() > 0) {
serverBootstrap.setOption("child.receiveBufferSize", tcpReceiveBufferSize.bytes());
if (tcpReceiveBufferSize != null && tcpReceiveBufferSize.getBytes() > 0) {
serverBootstrap.setOption("child.receiveBufferSize", tcpReceiveBufferSize.getBytes());
}
serverBootstrap.setOption("receiveBufferSizePredictorFactory", receiveBufferSizePredictorFactory);
serverBootstrap.setOption("child.receiveBufferSizePredictorFactory", receiveBufferSizePredictorFactory);
@ -417,11 +417,11 @@ public class Netty3Transport extends TcpTransport<Channel> {
public ChannelPipeline getPipeline() throws Exception {
ChannelPipeline channelPipeline = Channels.pipeline();
Netty3SizeHeaderFrameDecoder sizeHeader = new Netty3SizeHeaderFrameDecoder();
if (nettyTransport.maxCumulationBufferCapacity.bytes() >= 0) {
if (nettyTransport.maxCumulationBufferCapacity.bytes() > Integer.MAX_VALUE) {
if (nettyTransport.maxCumulationBufferCapacity.getBytes() >= 0) {
if (nettyTransport.maxCumulationBufferCapacity.getBytes() > Integer.MAX_VALUE) {
sizeHeader.setMaxCumulationBufferCapacity(Integer.MAX_VALUE);
} else {
sizeHeader.setMaxCumulationBufferCapacity((int) nettyTransport.maxCumulationBufferCapacity.bytes());
sizeHeader.setMaxCumulationBufferCapacity((int) nettyTransport.maxCumulationBufferCapacity.getBytes());
}
}
if (nettyTransport.maxCompositeBufferComponents != -1) {
@ -455,11 +455,11 @@ public class Netty3Transport extends TcpTransport<Channel> {
ChannelPipeline channelPipeline = Channels.pipeline();
channelPipeline.addLast("openChannels", nettyTransport.serverOpenChannels);
Netty3SizeHeaderFrameDecoder sizeHeader = new Netty3SizeHeaderFrameDecoder();
if (nettyTransport.maxCumulationBufferCapacity.bytes() > 0) {
if (nettyTransport.maxCumulationBufferCapacity.bytes() > Integer.MAX_VALUE) {
if (nettyTransport.maxCumulationBufferCapacity.getBytes() > 0) {
if (nettyTransport.maxCumulationBufferCapacity.getBytes() > Integer.MAX_VALUE) {
sizeHeader.setMaxCumulationBufferCapacity(Integer.MAX_VALUE);
} else {
sizeHeader.setMaxCumulationBufferCapacity((int) nettyTransport.maxCumulationBufferCapacity.bytes());
sizeHeader.setMaxCumulationBufferCapacity((int) nettyTransport.maxCumulationBufferCapacity.getBytes());
}
}
if (nettyTransport.maxCompositeBufferComponents != -1) {

View File

@ -150,9 +150,9 @@ public class Netty4HttpServerTransport extends AbstractLifecycleComponent implem
Setting.byteSizeSetting("transport.netty.receive_predictor_size",
settings -> {
long defaultReceiverPredictor = 512 * 1024;
if (JvmInfo.jvmInfo().getMem().getDirectMemoryMax().bytes() > 0) {
if (JvmInfo.jvmInfo().getMem().getDirectMemoryMax().getBytes() > 0) {
// we can guess a better default...
long l = (long) ((0.3 * JvmInfo.jvmInfo().getMem().getDirectMemoryMax().bytes()) / SETTING_HTTP_WORKER_COUNT.get
long l = (long) ((0.3 * JvmInfo.jvmInfo().getMem().getDirectMemoryMax().getBytes()) / SETTING_HTTP_WORKER_COUNT.get
(settings));
defaultReceiverPredictor = Math.min(defaultReceiverPredictor, Math.max(l, 64 * 1024));
}
@ -247,13 +247,13 @@ public class Netty4HttpServerTransport extends AbstractLifecycleComponent implem
// See AdaptiveReceiveBufferSizePredictor#DEFAULT_XXX for default values in netty..., we can use higher ones for us, even fixed one
ByteSizeValue receivePredictorMin = SETTING_HTTP_NETTY_RECEIVE_PREDICTOR_MIN.get(settings);
ByteSizeValue receivePredictorMax = SETTING_HTTP_NETTY_RECEIVE_PREDICTOR_MAX.get(settings);
if (receivePredictorMax.bytes() == receivePredictorMin.bytes()) {
recvByteBufAllocator = new FixedRecvByteBufAllocator(Math.toIntExact(receivePredictorMax.bytes()));
if (receivePredictorMax.getBytes() == receivePredictorMin.getBytes()) {
recvByteBufAllocator = new FixedRecvByteBufAllocator(Math.toIntExact(receivePredictorMax.getBytes()));
} else {
recvByteBufAllocator = new AdaptiveRecvByteBufAllocator(
Math.toIntExact(receivePredictorMin.bytes()),
Math.toIntExact(receivePredictorMin.bytes()),
Math.toIntExact(receivePredictorMax.bytes()));
Math.toIntExact(receivePredictorMin.getBytes()),
Math.toIntExact(receivePredictorMin.getBytes()),
Math.toIntExact(receivePredictorMax.getBytes()));
}
this.compression = SETTING_HTTP_COMPRESSION.get(settings);
@ -263,7 +263,7 @@ public class Netty4HttpServerTransport extends AbstractLifecycleComponent implem
this.corsConfig = buildCorsConfig(settings);
// validate max content length
if (maxContentLength.bytes() > Integer.MAX_VALUE) {
if (maxContentLength.getBytes() > Integer.MAX_VALUE) {
logger.warn("maxContentLength[{}] set to high value, resetting it to [100mb]", maxContentLength);
maxContentLength = new ByteSizeValue(100, ByteSizeUnit.MB);
}
@ -303,13 +303,13 @@ public class Netty4HttpServerTransport extends AbstractLifecycleComponent implem
serverBootstrap.childOption(ChannelOption.SO_KEEPALIVE, SETTING_HTTP_TCP_KEEP_ALIVE.get(settings));
final ByteSizeValue tcpSendBufferSize = SETTING_HTTP_TCP_SEND_BUFFER_SIZE.get(settings);
if (tcpSendBufferSize.bytes() > 0) {
serverBootstrap.childOption(ChannelOption.SO_SNDBUF, Math.toIntExact(tcpSendBufferSize.bytes()));
if (tcpSendBufferSize.getBytes() > 0) {
serverBootstrap.childOption(ChannelOption.SO_SNDBUF, Math.toIntExact(tcpSendBufferSize.getBytes()));
}
final ByteSizeValue tcpReceiveBufferSize = SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE.get(settings);
if (tcpReceiveBufferSize.bytes() > 0) {
serverBootstrap.childOption(ChannelOption.SO_RCVBUF, Math.toIntExact(tcpReceiveBufferSize.bytes()));
if (tcpReceiveBufferSize.getBytes() > 0) {
serverBootstrap.childOption(ChannelOption.SO_RCVBUF, Math.toIntExact(tcpReceiveBufferSize.getBytes()));
}
serverBootstrap.option(ChannelOption.RCVBUF_ALLOCATOR, recvByteBufAllocator);
@ -483,7 +483,7 @@ public class Netty4HttpServerTransport extends AbstractLifecycleComponent implem
if (boundTransportAddress == null) {
return null;
}
return new HttpInfo(boundTransportAddress, maxContentLength.bytes());
return new HttpInfo(boundTransportAddress, maxContentLength.getBytes());
}
@Override
@ -548,14 +548,14 @@ public class Netty4HttpServerTransport extends AbstractLifecycleComponent implem
protected void initChannel(Channel ch) throws Exception {
ch.pipeline().addLast("openChannels", transport.serverOpenChannels);
final HttpRequestDecoder decoder = new HttpRequestDecoder(
Math.toIntExact(transport.maxInitialLineLength.bytes()),
Math.toIntExact(transport.maxHeaderSize.bytes()),
Math.toIntExact(transport.maxChunkSize.bytes()));
Math.toIntExact(transport.maxInitialLineLength.getBytes()),
Math.toIntExact(transport.maxHeaderSize.getBytes()),
Math.toIntExact(transport.maxChunkSize.getBytes()));
decoder.setCumulator(ByteToMessageDecoder.COMPOSITE_CUMULATOR);
ch.pipeline().addLast("decoder", decoder);
ch.pipeline().addLast("decoder_compress", new HttpContentDecompressor());
ch.pipeline().addLast("encoder", new HttpResponseEncoder());
final HttpObjectAggregator aggregator = new HttpObjectAggregator(Math.toIntExact(transport.maxContentLength.bytes()));
final HttpObjectAggregator aggregator = new HttpObjectAggregator(Math.toIntExact(transport.maxContentLength.getBytes()));
if (transport.maxCompositeBufferComponents != -1) {
aggregator.setMaxCumulationBufferComponents(transport.maxCompositeBufferComponents);
}

View File

@ -113,9 +113,9 @@ public class Netty4Transport extends TcpTransport<Channel> {
"transport.netty.receive_predictor_size",
settings -> {
long defaultReceiverPredictor = 512 * 1024;
if (JvmInfo.jvmInfo().getMem().getDirectMemoryMax().bytes() > 0) {
if (JvmInfo.jvmInfo().getMem().getDirectMemoryMax().getBytes() > 0) {
// we can guess a better default...
long l = (long) ((0.3 * JvmInfo.jvmInfo().getMem().getDirectMemoryMax().bytes()) / WORKER_COUNT.get(settings));
long l = (long) ((0.3 * JvmInfo.jvmInfo().getMem().getDirectMemoryMax().getBytes()) / WORKER_COUNT.get(settings));
defaultReceiverPredictor = Math.min(defaultReceiverPredictor, Math.max(l, 64 * 1024));
}
return new ByteSizeValue(defaultReceiverPredictor).toString();
@ -152,11 +152,11 @@ public class Netty4Transport extends TcpTransport<Channel> {
// See AdaptiveReceiveBufferSizePredictor#DEFAULT_XXX for default values in netty..., we can use higher ones for us, even fixed one
this.receivePredictorMin = NETTY_RECEIVE_PREDICTOR_MIN.get(settings);
this.receivePredictorMax = NETTY_RECEIVE_PREDICTOR_MAX.get(settings);
if (receivePredictorMax.bytes() == receivePredictorMin.bytes()) {
recvByteBufAllocator = new FixedRecvByteBufAllocator((int) receivePredictorMax.bytes());
if (receivePredictorMax.getBytes() == receivePredictorMin.getBytes()) {
recvByteBufAllocator = new FixedRecvByteBufAllocator((int) receivePredictorMax.getBytes());
} else {
recvByteBufAllocator = new AdaptiveRecvByteBufAllocator((int) receivePredictorMin.bytes(),
(int) receivePredictorMin.bytes(), (int) receivePredictorMax.bytes());
recvByteBufAllocator = new AdaptiveRecvByteBufAllocator((int) receivePredictorMin.getBytes(),
(int) receivePredictorMin.getBytes(), (int) receivePredictorMax.getBytes());
}
}
@ -208,13 +208,13 @@ public class Netty4Transport extends TcpTransport<Channel> {
bootstrap.option(ChannelOption.SO_KEEPALIVE, TCP_KEEP_ALIVE.get(settings));
final ByteSizeValue tcpSendBufferSize = TCP_SEND_BUFFER_SIZE.get(settings);
if (tcpSendBufferSize.bytes() > 0) {
bootstrap.option(ChannelOption.SO_SNDBUF, Math.toIntExact(tcpSendBufferSize.bytes()));
if (tcpSendBufferSize.getBytes() > 0) {
bootstrap.option(ChannelOption.SO_SNDBUF, Math.toIntExact(tcpSendBufferSize.getBytes()));
}
final ByteSizeValue tcpReceiveBufferSize = TCP_RECEIVE_BUFFER_SIZE.get(settings);
if (tcpReceiveBufferSize.bytes() > 0) {
bootstrap.option(ChannelOption.SO_RCVBUF, Math.toIntExact(tcpReceiveBufferSize.bytes()));
if (tcpReceiveBufferSize.getBytes() > 0) {
bootstrap.option(ChannelOption.SO_RCVBUF, Math.toIntExact(tcpReceiveBufferSize.getBytes()));
}
bootstrap.option(ChannelOption.RCVBUF_ALLOCATOR, recvByteBufAllocator);
@ -251,13 +251,13 @@ public class Netty4Transport extends TcpTransport<Channel> {
ByteSizeValue fallbackTcpSendBufferSize = settings.getAsBytesSize("transport.netty.tcp_send_buffer_size",
TCP_SEND_BUFFER_SIZE.get(settings));
if (fallbackTcpSendBufferSize.bytes() >= 0) {
if (fallbackTcpSendBufferSize.getBytes() >= 0) {
fallbackSettingsBuilder.put("tcp_send_buffer_size", fallbackTcpSendBufferSize);
}
ByteSizeValue fallbackTcpBufferSize = settings.getAsBytesSize("transport.netty.tcp_receive_buffer_size",
TCP_RECEIVE_BUFFER_SIZE.get(settings));
if (fallbackTcpBufferSize.bytes() >= 0) {
if (fallbackTcpBufferSize.getBytes() >= 0) {
fallbackSettingsBuilder.put("tcp_receive_buffer_size", fallbackTcpBufferSize);
}
@ -291,12 +291,12 @@ public class Netty4Transport extends TcpTransport<Channel> {
serverBootstrap.childOption(ChannelOption.SO_KEEPALIVE, TCP_KEEP_ALIVE.get(settings));
final ByteSizeValue tcpSendBufferSize = TCP_SEND_BUFFER_SIZE.getDefault(settings);
if (tcpSendBufferSize != null && tcpSendBufferSize.bytes() > 0) {
serverBootstrap.childOption(ChannelOption.SO_SNDBUF, Math.toIntExact(tcpSendBufferSize.bytes()));
if (tcpSendBufferSize != null && tcpSendBufferSize.getBytes() > 0) {
serverBootstrap.childOption(ChannelOption.SO_SNDBUF, Math.toIntExact(tcpSendBufferSize.getBytes()));
}
final ByteSizeValue tcpReceiveBufferSize = TCP_RECEIVE_BUFFER_SIZE.getDefault(settings);
if (tcpReceiveBufferSize != null && tcpReceiveBufferSize.bytes() > 0) {
if (tcpReceiveBufferSize != null && tcpReceiveBufferSize.getBytes() > 0) {
serverBootstrap.childOption(ChannelOption.SO_RCVBUF, Math.toIntExact(tcpReceiveBufferSize.bytesAsInt()));
}

View File

@ -19,7 +19,6 @@
package org.elasticsearch.repositories.s3;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.Protocol;
import org.elasticsearch.cloud.aws.AwsS3Service;

View File

@ -98,7 +98,7 @@ public class MockTcpTransport extends TcpTransport<MockTcpTransport.MockChannel>
socket.bind(address);
socket.setReuseAddress(TCP_REUSE_ADDRESS.get(settings()));
ByteSizeValue tcpReceiveBufferSize = TCP_RECEIVE_BUFFER_SIZE.get(settings);
if (tcpReceiveBufferSize.bytes() > 0) {
if (tcpReceiveBufferSize.getBytes() > 0) {
socket.setReceiveBufferSize(tcpReceiveBufferSize.bytesAsInt());
}
MockChannel serverMockChannel = new MockChannel(socket, name);
@ -210,11 +210,11 @@ public class MockTcpTransport extends TcpTransport<MockTcpTransport.MockChannel>
private void configureSocket(Socket socket) throws SocketException {
socket.setTcpNoDelay(TCP_NO_DELAY.get(settings));
ByteSizeValue tcpSendBufferSize = TCP_SEND_BUFFER_SIZE.get(settings);
if (tcpSendBufferSize.bytes() > 0) {
if (tcpSendBufferSize.getBytes() > 0) {
socket.setSendBufferSize(tcpSendBufferSize.bytesAsInt());
}
ByteSizeValue tcpReceiveBufferSize = TCP_RECEIVE_BUFFER_SIZE.get(settings);
if (tcpReceiveBufferSize.bytes() > 0) {
if (tcpReceiveBufferSize.getBytes() > 0) {
socket.setReceiveBufferSize(tcpReceiveBufferSize.bytesAsInt());
}
socket.setReuseAddress(TCP_REUSE_ADDRESS.get(settings()));