Set network receive predictor size to 32kb (#23284)

Previously we calculated Netty' receive predictor size for HTTP and transport
traffic based on available memory and worker nodes. This resulted in a receive
predictor size between 64kb and 512kb. In our benchmarks this leads to increased
GC pressure.

With this commit we set Netty's receive predictor size to 32kb. This value is in
a sweet spot between heap memory waste (-> GC pressure) and effect on request
metrics (achieved throughput and latency numbers).

Closes #23185
This commit is contained in:
Daniel Mitterdorfer 2017-02-21 14:45:33 +01:00 committed by GitHub
parent f933f80902
commit 0744a00001
3 changed files with 8 additions and 26 deletions

View File

@ -143,17 +143,7 @@ public class Netty4HttpServerTransport extends AbstractLifecycleComponent implem
Setting.byteSizeSetting("http.tcp.receive_buffer_size", NetworkService.TcpSettings.TCP_RECEIVE_BUFFER_SIZE,
Property.NodeScope, Property.Shared);
public static final Setting<ByteSizeValue> SETTING_HTTP_NETTY_RECEIVE_PREDICTOR_SIZE =
Setting.byteSizeSetting("transport.netty.receive_predictor_size",
settings -> {
long defaultReceiverPredictor = 512 * 1024;
if (JvmInfo.jvmInfo().getMem().getDirectMemoryMax().getBytes() > 0) {
// we can guess a better default...
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));
}
return new ByteSizeValue(defaultReceiverPredictor).toString();
}, Property.NodeScope);
Setting.byteSizeSetting("http.netty.receive_predictor_size", new ByteSizeValue(32, ByteSizeUnit.KB), Property.NodeScope);
public static final Setting<ByteSizeValue> SETTING_HTTP_NETTY_RECEIVE_PREDICTOR_MIN =
byteSizeSetting("http.netty.receive_predictor_min", SETTING_HTTP_NETTY_RECEIVE_PREDICTOR_SIZE, Property.NodeScope);
public static final Setting<ByteSizeValue> SETTING_HTTP_NETTY_RECEIVE_PREDICTOR_MAX =

View File

@ -61,6 +61,9 @@ public class Netty4Plugin extends Plugin implements NetworkPlugin {
Netty4HttpServerTransport.SETTING_HTTP_TCP_REUSE_ADDRESS,
Netty4HttpServerTransport.SETTING_HTTP_TCP_SEND_BUFFER_SIZE,
Netty4HttpServerTransport.SETTING_HTTP_TCP_RECEIVE_BUFFER_SIZE,
Netty4HttpServerTransport.SETTING_HTTP_NETTY_RECEIVE_PREDICTOR_SIZE,
Netty4HttpServerTransport.SETTING_HTTP_NETTY_RECEIVE_PREDICTOR_MIN,
Netty4HttpServerTransport.SETTING_HTTP_NETTY_RECEIVE_PREDICTOR_MAX,
Netty4Transport.WORKER_COUNT,
Netty4Transport.NETTY_MAX_CUMULATION_BUFFER_CAPACITY,
Netty4Transport.NETTY_MAX_COMPOSITE_BUFFER_COMPONENTS,

View File

@ -50,6 +50,7 @@ import org.elasticsearch.common.network.NetworkService.TcpSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArrays;
@ -106,24 +107,12 @@ public class Netty4Transport extends TcpTransport<Channel> {
public static final Setting<Integer> NETTY_MAX_COMPOSITE_BUFFER_COMPONENTS =
Setting.intSetting("transport.netty.max_composite_buffer_components", -1, -1, Property.NodeScope, Property.Shared);
// See AdaptiveReceiveBufferSizePredictor#DEFAULT_XXX for default values in netty..., we can use higher ones for us, even fixed one
public static final Setting<ByteSizeValue> NETTY_RECEIVE_PREDICTOR_SIZE = Setting.byteSizeSetting(
"transport.netty.receive_predictor_size",
settings -> {
long defaultReceiverPredictor = 512 * 1024;
if (JvmInfo.jvmInfo().getMem().getDirectMemoryMax().getBytes() > 0) {
// we can guess a better default...
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();
},
Property.NodeScope,
Property.Shared);
"transport.netty.receive_predictor_size", new ByteSizeValue(32, ByteSizeUnit.KB), Property.NodeScope);
public static final Setting<ByteSizeValue> NETTY_RECEIVE_PREDICTOR_MIN =
byteSizeSetting("transport.netty.receive_predictor_min", NETTY_RECEIVE_PREDICTOR_SIZE, Property.NodeScope, Property.Shared);
byteSizeSetting("transport.netty.receive_predictor_min", NETTY_RECEIVE_PREDICTOR_SIZE, Property.NodeScope);
public static final Setting<ByteSizeValue> NETTY_RECEIVE_PREDICTOR_MAX =
byteSizeSetting("transport.netty.receive_predictor_max", NETTY_RECEIVE_PREDICTOR_SIZE, Property.NodeScope, Property.Shared);
byteSizeSetting("transport.netty.receive_predictor_max", NETTY_RECEIVE_PREDICTOR_SIZE, Property.NodeScope);
public static final Setting<Integer> NETTY_BOSS_COUNT =
intSetting("transport.netty.boss_count", 1, 1, Property.NodeScope, Property.Shared);