sofastack / sofa-jraft

A production-grade java implementation of RAFT consensus algorithm.
https://www.sofastack.tech/projects/sofa-jraft/
Apache License 2.0
3.57k stars 1.14k forks source link

大量的byte[]创建引发频发GC,希望能做优化 #158

Closed shiftyman closed 5 years ago

shiftyman commented 5 years ago

压测期间发现jraft创建大量byte[],gc十分频繁,希望能优化一下,比如池化。

附内存分配采样,前6都是jraft生成的byte[]:

--- 5890105016 bytes (14.90%), 25629 samples [ 0] byte[] (out) [ 1] java.nio.HeapByteBuffer. [ 2] java.nio.ByteBuffer.allocate [ 3] com.alipay.sofa.jraft.util.Utils.expandByteBufferAtLeast [ 4] com.alipay.sofa.jraft.util.ByteBufferCollector.getBuffer [ 5] com.alipay.sofa.jraft.util.ByteBufferCollector.put [ 6] com.alipay.sofa.jraft.core.Replicator.prepareEntry [ 7] com.alipay.sofa.jraft.core.Replicator.sendEntries [ 8] com.alipay.sofa.jraft.core.Replicator.sendEntries [ 9] com.alipay.sofa.jraft.core.Replicator.onRpcReturned [10] com.alipay.sofa.jraft.core.Replicator$3.run [11] com.alipay.sofa.jraft.rpc.impl.AbstractBoltClientService$1.onResponse [12] com.alipay.remoting.rpc.RpcInvokeCallbackListener$CallbackTask.run [13] java.util.concurrent.ThreadPoolExecutor.runWorker [14] java.util.concurrent.ThreadPoolExecutor$Worker.run [15] java.lang.Thread.run

--- 2766546656 bytes (7.00%), 10930 samples [ 0] byte[] [ 1] java.nio.HeapByteBuffer. [ 2] java.nio.ByteBuffer.allocate [ 3] com.alipay.sofa.jraft.util.Utils.allocate [ 4] com.alipay.sofa.jraft.util.ByteBufferCollector. [ 5] com.alipay.sofa.jraft.util.ByteBufferCollector.allocate [ 6] com.alipay.sofa.jraft.core.Replicator.sendEntries [ 7] com.alipay.sofa.jraft.core.Replicator.sendEntries [ 8] com.alipay.sofa.jraft.core.Replicator.continueSending [ 9] com.alipay.sofa.jraft.core.Replicator.lambda$waitMoreEntries$5 [10] com.alipay.sofa.jraft.core.Replicator$$Lambda$12.1225022760.onNewLog [11] com.alipay.sofa.jraft.storage.impl.LogManagerImpl.runOnNewLog [12] com.alipay.sofa.jraft.storage.impl.LogManagerImpl.lambda$wakeupAllWaiter$3 [13] com.alipay.sofa.jraft.storage.impl.LogManagerImpl$$Lambda$84.1996144794.run [14] java.util.concurrent.Executors$RunnableAdapter.call [15] java.util.concurrent.FutureTask.run [16] java.util.concurrent.ThreadPoolExecutor.runWorker [17] java.util.concurrent.ThreadPoolExecutor$Worker.run [18] java.lang.Thread.run

--- 1381232480 bytes (3.49%), 1265 samples [ 0] byte[] [ 1] java.nio.HeapByteBuffer. [ 2] java.nio.ByteBuffer.allocate [ 3] com.alipay.sofa.jraft.util.Utils.expandByteBufferAtLeast [ 4] com.alipay.sofa.jraft.util.ByteBufferCollector.getBuffer [ 5] com.alipay.sofa.jraft.util.ByteBufferCollector.put [ 6] com.alipay.sofa.jraft.core.Replicator.prepareEntry [ 7] com.alipay.sofa.jraft.core.Replicator.sendEntries [ 8] com.alipay.sofa.jraft.core.Replicator.sendEntries [ 9] com.alipay.sofa.jraft.core.Replicator.onRpcReturned [10] com.alipay.sofa.jraft.core.Replicator$3.run [11] com.alipay.sofa.jraft.rpc.impl.AbstractBoltClientService$1.onResponse [12] com.alipay.remoting.rpc.RpcInvokeCallbackListener$CallbackTask.run [13] java.util.concurrent.ThreadPoolExecutor.runWorker [14] java.util.concurrent.ThreadPoolExecutor$Worker.run [15] java.lang.Thread.run

--- 1279626912 bytes (3.24%), 10652 samples [ 0] byte[] (out) [ 1] java.nio.HeapByteBuffer. [ 2] java.nio.ByteBuffer.allocate [ 3] com.alipay.sofa.jraft.util.Utils.expandByteBufferAtLeast [ 4] com.alipay.sofa.jraft.util.ByteBufferCollector.getBuffer [ 5] com.alipay.sofa.jraft.util.ByteBufferCollector.put [ 6] com.alipay.sofa.jraft.core.Replicator.prepareEntry [ 7] com.alipay.sofa.jraft.core.Replicator.sendEntries [ 8] com.alipay.sofa.jraft.core.Replicator.sendEntries [ 9] com.alipay.sofa.jraft.core.Replicator.onRpcReturned [10] com.alipay.sofa.jraft.core.Replicator$4.run [11] com.alipay.sofa.jraft.rpc.impl.AbstractBoltClientService$1.onResponse [12] com.alipay.remoting.rpc.RpcInvokeCallbackListener$CallbackTask.run [13] java.util.concurrent.ThreadPoolExecutor.runWorker [14] java.util.concurrent.ThreadPoolExecutor$Worker.run [15] java.lang.Thread.run

--- 860045616 bytes (2.18%), 21309 samples [ 0] byte[] [ 1] com.alipay.sofa.jraft.entity.LogEntry.encode [ 2] com.alipay.sofa.jraft.storage.impl.RocksDBLogStorage.addDataBatch [ 3] com.alipay.sofa.jraft.storage.impl.RocksDBLogStorage.lambda$appendEntries$1 [ 4] com.alipay.sofa.jraft.storage.impl.RocksDBLogStorage$$Lambda$11.900263968.execute [ 5] com.alipay.sofa.jraft.storage.impl.RocksDBLogStorage.executeBatch [ 6] com.alipay.sofa.jraft.storage.impl.RocksDBLogStorage.appendEntries [ 7] com.alipay.sofa.jraft.storage.impl.LogManagerImpl.appendToStorage [ 8] com.alipay.sofa.jraft.storage.impl.LogManagerImpl.access$300 [ 9] com.alipay.sofa.jraft.storage.impl.LogManagerImpl$AppendBatcher.flush [10] com.alipay.sofa.jraft.storage.impl.LogManagerImpl$StableClosureEventHandler.onEvent [11] com.alipay.sofa.jraft.storage.impl.LogManagerImpl$StableClosureEventHandler.onEvent [12] com.lmax.disruptor.BatchEventProcessor.run [13] java.lang.Thread.run

--- 796816392 bytes (2.02%), 13174 samples [ 0] byte[] [ 1] com.alipay.remoting.rpc.protocol.RpcCommandDecoder.decode [ 2] com.alipay.remoting.codec.ProtocolCodeBasedDecoder.decode [ 3] com.alipay.remoting.codec.AbstractBatchDecoder.callDecode [ 4] com.alipay.remoting.codec.AbstractBatchDecoder.channelRead [ 5] io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead [ 6] io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead [ 7] io.netty.channel.AbstractChannelHandlerContext.fireChannelRead [ 8] io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead [ 9] io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead [10] io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead [11] io.netty.channel.DefaultChannelPipeline.fireChannelRead [12] io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady [13] io.netty.channel.epoll.EpollEventLoop.processReady [14] io.netty.channel.epoll.EpollEventLoop.run [15] io.netty.util.concurrent.SingleThreadEventExecutor$5.run [16] java.lang.Thread.run

fengjiachun commented 5 years ago

可不可以再给一些详细数据? 比如 jraft 中的 metric log,jvm 配置等等,以及你的压测场景

shiftyman commented 5 years ago

1.metric log没开 2.jraft 默认配置 3.multi-raft,24region 5.三节点集群,都有24region,leader比: 11:9:4 5.jvm:-XX:+UseG1GC -Xmx32g -XX:MaxGCPauseMillis=30 6.测试场景:300并发,写,300字节。吞吐量大概到8-10w/s整集群,机器配置就不提了

反正感觉跟测试场景关系不大,怎么测只要压高了都是1s一次ygc,如果用cms+parnew,gc更凶,延迟更高。 观察到主要是jraft中产生了大量的byte[]。

麻烦看看?

killme2008 commented 5 years ago

这是一个可以优化的点,jraft 这个环节可以尽量做一些 buffer 复用,原来有考虑做过,但是觉的如果不从上到下都改造,可能效果不会太好。bolt rpc 目前的序列化还是基于 byte 数组来做的,要更好的优化,还是需要去改造 bolt 框架,尽量从上(网络)到下(rocksdb)都做到 zero copy 和 buffer 内存复用。

不过我觉的这里的瓶颈很可能不是这个问题,高负载系统 1 秒一次的 young gc 还是常见的。可以开下 metric 观察下。

fengjiachun commented 5 years ago

zero copy 和 pooled 肯定会有一些效果,比如我之前在 jupiter 这个 rpc 框架中扩展了 protostuff 、 kryo、hessian 等序列化组件使之支持 zero copy,有一些性能收益,见 benchmark 但感觉上只能算锦上添花

所以我觉得瓶颈应该不在这里,建议打开并提供下 metric 数据

shiftyman commented 5 years ago

抽样了1个region的leader的metrics,如下:

[2019-05-14 15:41:26:927] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=GAUGE, name=region_7.raft-rpc-client-thread-pool.completed, value=3022965 [2019-05-14 15:41:26:927] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=GAUGE, name=region_7.raft-rpc-client-thread-pool.pool-size, value=26 [2019-05-14 15:41:26:927] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=GAUGE, name=region_7.raft-rpc-client-thread-pool.queued, value=0 [2019-05-14 15:41:26:927] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=GAUGE, name=region_7.raft-utils-closure-thread-pool.active, value=3 [2019-05-14 15:41:26:927] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=GAUGE, name=region_7.raft-utils-closure-thread-pool.completed, value=37114447 [2019-05-14 15:41:26:927] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=GAUGE, name=region_7.raft-utils-closure-thread-pool.pool-size, value=64 [2019-05-14 15:41:26:927] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=GAUGE, name=region_7.raft-utils-closure-thread-pool.queued, value=0 [2019-05-14 15:41:26:927] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=GAUGE, name=region_7.replicator-group7/10.15.70.35:8958.append-entries-times, value=2 [2019-05-14 15:41:26:927] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=GAUGE, name=region_7.replicator-group7/10.15.70.35:8958.heartbeat-times, value=6485 [2019-05-14 15:41:26:927] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=GAUGE, name=region_7.replicator-group7/10.15.70.35:8958.install-snapshot-times, value=0 [2019-05-14 15:41:26:927] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=GAUGE, name=region_7.replicator-group7/10.15.70.35:8958.log-lags, value=893132 [2019-05-14 15:41:26:927] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=GAUGE, name=region_7.replicator-group7/10.15.70.35:8958.next-index, value=13271496 [2019-05-14 15:41:26:927] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=GAUGE, name=region_7.replicator-group7/10.15.72.2:8958.append-entries-times, value=122 [2019-05-14 15:41:26:927] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=GAUGE, name=region_7.replicator-group7/10.15.72.2:8958.heartbeat-times, value=6238 [2019-05-14 15:41:26:927] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=GAUGE, name=region_7.replicator-group7/10.15.72.2:8958.install-snapshot-times, value=0 [2019-05-14 15:41:26:927] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=GAUGE, name=region_7.replicator-group7/10.15.72.2:8958.log-lags, value=953951 [2019-05-14 15:41:26:927] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=GAUGE, name=region_7.replicator-group7/10.15.72.2:8958.next-index, value=13210677 [2019-05-14 15:41:26:928] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=HISTOGRAM, name=region_7.append-logs-bytes, count=800379, min=373, max=5996, mean=728.9320637378448, stddev=690.2230603239028, median=375.0, p75=750.0, p95=1875.0, p98=3000.0, p99=3748.0, p999=5996.0 [2019-05-14 15:41:26:928] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=HISTOGRAM, name=region_7.append-logs-count, count=800379, min=1, max=27, mean=2.145770463412822, stddev=2.0388778317873872, median=1.0, p75=3.0, p95=6.0, p98=8.0, p99=10.0, p999=24.0 [2019-05-14 15:41:26:928] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=HISTOGRAM, name=region_7.fsm-apply-tasks-count, count=618106, min=1, max=34, mean=2.5559129024082377, stddev=3.2845757446583494, median=1.0, p75=3.0, p95=9.0, p98=13.0, p99=16.0, p999=34.0 [2019-05-14 15:41:26:929] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=HISTOGRAM, name=region_7.replicate-entries-bytes, count=2998597, min=373, max=2625, mean=450.05417283469194, stddev=204.53854290132693, median=375.0, p75=375.0, p95=750.0, p98=1124.0, p99=1499.0, p999=2625.0 [2019-05-14 15:41:26:929] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=HISTOGRAM, name=region_7.replicate-entries-count, count=2998597, min=1, max=11, mean=1.2094732782598416, stddev=0.682644796024078, median=1.0, p75=1.0, p95=2.0, p98=3.0, p99=3.0, p999=11.0 [2019-05-14 15:41:26:930] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=HISTOGRAM, name=region_7.replicate-inflights-count, count=3003985, min=1, max=234, mean=18.39272773130996, stddev=25.148391464374953, median=12.0, p75=22.0, p95=58.0, p98=108.0, p99=156.0, p999=197.0 [2019-05-14 15:41:26:930] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=TIMER, name=region_7.append-logs, count=800379, min=0.0, max=77.0, mean=0.3702336178990099, stddev=2.55953310728969, median=0.0, p75=1.0, p95=1.0, p98=1.0, p99=2.0, p999=77.0, mean_rate=763.1235584423595, m1=1115.4732817521094, m5=972.2875942734611, m15=583.7790340510926, rate_unit=events/second, duration_unit=milliseconds [2019-05-14 15:41:26:931] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=TIMER, name=region_7.fsm-apply-tasks, count=618106, min=0.0, max=4.0, mean=0.16688535639932303, stddev=0.42721921696561094, median=0.0, p75=0.0, p95=1.0, p98=1.0, p99=1.0, p999=4.0, mean_rate=589.3391729715237, m1=882.9372241151191, m5=721.1984966349697, m15=440.14928614674335, rate_unit=events/second, duration_unit=milliseconds [2019-05-14 15:41:26:931] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=TIMER, name=region_7.fsm-commit, count=618105, min=0.0, max=3.0, mean=0.18387498658976564, stddev=0.4203490277230257, median=0.0, p75=0.0, p95=1.0, p98=1.0, p99=1.0, p999=3.0, mean_rate=589.3404028345439, m1=882.8944626940076, m5=721.1833281648045, m15=440.0844653918782, rate_unit=events/second, duration_unit=milliseconds [2019-05-14 15:41:26:931] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=TIMER, name=region_7.fsm-leader-stop, count=1, min=0.0, max=0.0, mean=0.0, stddev=0.0, median=0.0, p75=0.0, p95=0.0, p98=0.0, p99=0.0, p999=0.0, mean_rate=0.0028813944594439627, m1=6.918754672929542E-4, m5=0.06439165430753521, m15=0.1370765582061974, rate_unit=events/second, duration_unit=milliseconds [2019-05-14 15:41:26:931] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=TIMER, name=region_7.fsm-snapshot-load, count=1, min=3417.0, max=3417.0, mean=3417.0, stddev=0.0, median=3417.0, p75=3417.0, p95=3417.0, p98=3417.0, p99=3417.0, p999=3417.0, mean_rate=9.523179229806868E-4, m1=5.4584303014450984E-9, m5=0.006140978124982921, m15=0.06262761126843666, rate_unit=events/second, duration_unit=milliseconds [2019-05-14 15:41:26:931] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=TIMER, name=region_7.pre-vote, count=4, min=0.0, max=2.0, mean=1.0, stddev=0.9999999999999999, median=2.0, p75=2.0, p95=2.0, p98=2.0, p99=2.0, p999=2.0, mean_rate=0.0038137785080237783, m1=1.106504815620347E-4, m5=0.014616971727941457, m15=0.12747187135463725, rate_unit=events/second, duration_unit=milliseconds [2019-05-14 15:41:26:932] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=TIMER, name=region_7.replicate-entries, count=2998597, min=0.0, max=262.0, mean=7.182016803714525, stddev=18.66125102299114, median=3.0, p75=6.0, p95=23.0, p98=70.0, p99=96.0, p999=178.0, mean_rate=2859.023420742251, m1=3998.0307213285323, m5=3465.1290315931674, m15=2132.392027729686, rate_unit=events/second, duration_unit=milliseconds [2019-05-14 15:41:26:932] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=TIMER, name=region_7.request-vote, count=4, min=0.0, max=2.0, mean=0.5000131620416306, stddev=0.5000263232171103, median=1.0, p75=1.0, p95=1.0, p98=1.0, p99=1.0, p999=1.0, mean_rate=0.003813786616691856, m1=1.106504815620347E-4, m5=0.014616971727941457, m15=0.12747187135463725, rate_unit=events/second, duration_unit=milliseconds [2019-05-14 15:41:26:932] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=TIMER, name=region_7.save-raft-meta, count=2, min=0.0, max=4.0, mean=1.0529633304552593E-4, stddev=0.020522530176962855, median=0.0, p75=0.0, p95=0.0, p98=0.0, p99=0.0, p999=0.0, mean_rate=0.001906900236375883, m1=5.532524078101735E-5, m5=0.007308485863970728, m15=0.06373593567731863, rate_unit=events/second, duration_unit=milliseconds [2019-05-14 15:41:26:932] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=TIMER, name=region_7.truncate-log-prefix, count=1, min=0.0, max=0.0, mean=0.0, stddev=0.0, median=0.0, p75=0.0, p95=0.0, p98=0.0, p99=0.0, p999=0.0, mean_rate=9.523174519267224E-4, m1=5.4584303014450984E-9, m5=0.006140978124982921, m15=0.06262761126843666, rate_unit=events/second, duration_unit=milliseconds

killme2008 commented 5 years ago

从这个指标看

type=HISTOGRAM, name=region_7.append-logs-count, count=800379, min=1, max=27, mean=2.145770463412822, stddev=2.0388778317873872, median=1.0, p75=3.0, p95=6.0, p98=8.0, p99=10.0, p999=24.0

中位数 median 是 1,单次 append 的日志条数不高,攒批的效果没有达到,这可能是吞吐上不去的主要原因。

你的 300 并发是单个客户端? 如果机器足够的话,可以采用多客户端压测看下,我们的压测数据也是在多个客户端并且客户端启用了 batch 写入的情况下才能达到。因为你的 range 比较多,请求分散到单个 range 后并发应该不高了。

fengjiachun commented 5 years ago

对于楼主的内存采样中频繁的 com.alipay.sofa.jraft.util.Utils.expandByteBufferAtLeast,也就是扩容 我新增了一个 AdaptiveBufAllocator 尝试规避 #161

shiftyman commented 5 years ago

1.我的应用对延迟敏感,如果并发增加,延迟就会增加不少,判断可能是单raft串行排队导致的。(吞吐量貌似也没增加太明显,观察cpu、disk io都没有达瓶颈,网络延迟在0.05ms)

2.gc频繁对延迟还是有不小影响的,之前用cms现在换g1收集器延迟有所减少且稳定了一些,吞吐量也有所提高的,所以能优化一下还是很好的。

3.从这个metrics看,日志复制为什么延迟这么高? [2019-05-14 15:41:26:932] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=TIMER, name=region_7.replicate-entries, count=2998597, min=0.0, max=262.0, mean=7.182016803714525, stddev=18.66125102299114, median=3.0, p75=6.0, p95=23.0, p98=70.0, p99=96.0, p999=178.0, mean_rate=2859.023420742251, m1=3998.0307213285323, m5=3465.1290315931674, m15=2132.392027729686, rate_unit=events/second, duration_unit=milliseconds

4.最后,之前用jprofile看,有一些block占用了相当长时间:

线程:JRaft-RPC-Processor-n block堆栈: com.alipay.sofa.jraft.util.NonReentrantLock.tryLock(long,java.util.concurrent.TimeUtil) java.util.concurrent.locks.Lock.tryLock(long,java.util.concurrent.TimeUnit) com.alipay.sofa.jraft.util.ThreadId.lock() com.alipay.sofa.jraf.core.Replicator.onRpcReturned(xxxxxxx) com.alipay.sofa.jraft.core.Replicator$4.run(com.alipay.sofa.jraft.Status) com.alipay.sofa.jraft.rpc.impl.AbstractBoltClientService$1.onResponse(java.lang.Object) ...

这种block有不少,个别block时间超过1s,很神奇。

killme2008 commented 5 years ago

对于 latency 敏感的话,确实 gc 影响很大,如果能减少 young gc 频率,效果还是会立竿见影的。 jraft 目前的设计还是重吞吐。

下面这个指标可以说明大部分请求的 bytes 范围是多少

 type=HISTOGRAM, name=region_7.replicate-entries-bytes, count=2998597, min=373, max=2625, mean=450.05417283469194, stddev=204.53854290132693, median=375.0, p75=375.0, p95=750.0, p98=1124.0, p99=1499.0, p999=2625.0

默认 ByteBufferCollector 分配的是 1k,不够的情况会去做扩展,家纯的实现可以缓解这个重新分配的问题。现有版本暂时可以先尝试调整下 -Djraft.byte_buf.size=3072 启动参数的环境变量来避免扩展分配,减少一点创建。

核心的问题还是 bolt 框架的序列化问题,暂时还没有好的解决办法。也许可以考虑复用下这里的 byte 数组,自己实现一个基于弱引用或者软引用的缓存池。基于 thread local 是不行的,因为 rpc 整体是异步的,在不同的线程上下文。

fengjiachun commented 5 years ago

@shiftyman

  1. 考虑 gc 对 latency 影响的话,我近几天会再提一个 pooled 的 PR
  2. 关于 bolt 框架未来也会考虑提一个 PR 实现序列化/反序列化的 zero copy, 这个时间上暂时还不能确定
shiftyman commented 5 years ago

赞,很高效!

shiftyman commented 5 years ago

请问这种错误一般什么情况引起?压测过程中偶尔会出现。

[2019-05-15 11:28:33:130] [JRaft-RPC-Processor-14] [ERROR] - com.alipay.sofa.jraft.rpc.impl.AbstractBoltClientService$1.onResponse(AbstractBoltClientService.java:176) - Fail to run RpcResponseClosure, the request is group_id: "group20" server_id: "10.15.70.35:8958" peer_id: "10.15.72.2:8958" term: 1 prev_log_term: 1 prev_log_index: 1706046 committed_index: 1757056

java.util.ConcurrentModificationException: null at java.util.ArrayDeque$DeqIterator.next(ArrayDeque.java:643) ~[?:1.8.0_45] at com.alipay.sofa.jraft.core.Replicator.onError(Replicator.java:836) ~[jraft-core-1.2.5.jar:?] at com.alipay.sofa.jraft.util.ThreadId.unlock(ThreadId.java:96) ~[jraft-core-1.2.5.jar:?] at com.alipay.sofa.jraft.core.Replicator.waitMoreEntries(Replicator.java:1300) ~[jraft-core-1.2.5.jar:?] at com.alipay.sofa.jraft.core.Replicator.sendEntries(Replicator.java:1362) ~[jraft-core-1.2.5.jar:?] at com.alipay.sofa.jraft.core.Replicator.sendEntries(Replicator.java:1314) ~[jraft-core-1.2.5.jar:?] at com.alipay.sofa.jraft.core.Replicator.onRpcReturned(Replicator.java:1112) ~[jraft-core-1.2.5.jar:?] at com.alipay.sofa.jraft.core.Replicator$3.run(Replicator.java:633) ~[jraft-core-1.2.5.jar:?] at com.alipay.sofa.jraft.rpc.impl.AbstractBoltClientService$1.onResponse(AbstractBoltClientService.java:174) ~[jraft-core-1.2.5.jar:?] at com.alipay.remoting.rpc.RpcInvokeCallbackListener$CallbackTask.run(RpcInvokeCallbackListener.java:169) ~[bolt-1.5.3.jar:?] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [?:1.8.0_45] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [?:1.8.0_45] at java.lang.Thread.run(Thread.java:745) [?:1.8.0_45]

killme2008 commented 5 years ago

@shiftyman 看情况是链接关闭过程中的一个并发 bug,取消 infly 的请求 future 的时候需要遍历 inflights 链表的时候,理论上不会造成什么影响,可以单独提个 issue 吗?我们修复下。

shiftyman commented 5 years ago

[2019-05-15 17:48:15:527] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=HISTOGRAM, name=region_7.replicate-entries-bytes, count=215422, min=70, max=523764, mean=1006.0615057571989, stddev=21828.50797919573, median=73.0, p75=73.0, p95=219.0, p98=291.0, p99=364.0, p999=523764.0 [2019-05-15 17:48:15:721] [Raft-Metrics-reporter-thread-1] [INFO] - com.codahale.metrics.Slf4jReporter$InfoLoggerProxy.log(Slf4jReporter.java:374) - type=HISTOGRAM, name=region_11.replicate-entries-bytes, count=201407, min=70, max=523775, mean=3348.002775726735, stddev=41069.623281819215, median=73.0, p75=73.0, p95=219.0, p98=364.0, p99=657.0, p999=523775.0

这种p999这么大是什么情况?可能是什么原因导致的?这边已经把jraft.byte_buf.size调大了,但是采样内存分配,还是大量的expandByteBufferAtLeast出现

现在每次写入就10byte,依旧大量expandByteBufferAtLeast,经查发现有这种特大的批量传输。

killme2008 commented 5 years ago

这个应该是复制较慢情况下攒批的效果了,p999 应该这种请求量很少的,目前没有 pool 的方式,还是会存在这种离散请求。你可以限制单个批次大小的,通过 RaftOptions.setMaxBodySize(bytes) 来避免这种大请求。

shiftyman commented 5 years ago

我怀疑哪里有bug。

我描述一个很奇怪的现象,刚启动时,写size=300压测,tps能达到10w。这个时候,内存分配采样是这样的,基本不会触发扩容: --- 2477707184 bytes (15.64%), 7013 samples [ 0] byte[] [ 1] java.nio.HeapByteBuffer. [ 2] java.nio.ByteBuffer.allocate [ 3] com.alipay.sofa.jraft.util.Utils.allocate [ 4] com.alipay.sofa.jraft.util.ByteBufferCollector. [ 5] com.alipay.sofa.jraft.util.ByteBufferCollector.allocate [ 6] com.alipay.sofa.jraft.core.Replicator.sendEntries [ 7] com.alipay.sofa.jraft.core.Replicator.sendEntries [ 8] com.alipay.sofa.jraft.core.Replicator.continueSending [ 9] com.alipay.sofa.jraft.core.Replicator.lambda$waitMoreEntries$5 [10] com.alipay.sofa.jraft.core.Replicator$$Lambda$24.1379661060.onNewLog [11] com.alipay.sofa.jraft.storage.impl.LogManagerImpl.runOnNewLog [12] com.alipay.sofa.jraft.storage.impl.LogManagerImpl.lambda$wakeupAllWaiter$3 [13] com.alipay.sofa.jraft.storage.impl.LogManagerImpl$$Lambda$99.1558250526.run [14] java.util.concurrent.Executors$RunnableAdapter.call [15] java.util.concurrent.FutureTask.run [16] java.util.concurrent.ThreadPoolExecutor.runWorker [17] java.util.concurrent.ThreadPoolExecutor$Worker.run [18] java.lang.Thread.run

--- 380017808 bytes (2.40%), 5747 samples [ 0] byte[] [ 1] com.alipay.remoting.rpc.protocol.RpcCommandDecoder.decode [ 2] com.alipay.remoting.codec.ProtocolCodeBasedDecoder.decode [ 3] com.alipay.remoting.codec.AbstractBatchDecoder.callDecode [ 4] com.alipay.remoting.codec.AbstractBatchDecoder.channelRead [ 5] io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead [ 6] io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead [ 7] io.netty.channel.AbstractChannelHandlerContext.fireChannelRead [ 8] io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead [ 9] io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead [10] io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead [11] io.netty.channel.DefaultChannelPipeline.fireChannelRead [12] io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady [13] io.netty.channel.epoll.EpollEventLoop.processReady [14] io.netty.channel.epoll.EpollEventLoop.run [15] io.netty.util.concurrent.SingleThreadEventExecutor$5.run [16] java.lang.Thread.run

但是,经过一段时间后,或者用size=1500去压一段时间,内存采样就大量的扩容,gc增多,tps下降,这个可以理解: --- 80285548320 bytes (31.46%), 239820 samples [ 0] byte[] (out) [ 1] java.nio.HeapByteBuffer. [ 2] java.nio.ByteBuffer.allocate [ 3] com.alipay.sofa.jraft.util.Utils.expandByteBufferAtLeast [ 4] com.alipay.sofa.jraft.util.ByteBufferCollector.getBuffer [ 5] com.alipay.sofa.jraft.util.ByteBufferCollector.put [ 6] com.alipay.sofa.jraft.core.Replicator.prepareEntry [ 7] com.alipay.sofa.jraft.core.Replicator.sendEntries [ 8] com.alipay.sofa.jraft.core.Replicator.sendEntries [ 9] com.alipay.sofa.jraft.core.Replicator.onRpcReturned [10] com.alipay.sofa.jraft.core.Replicator$3.run [11] com.alipay.sofa.jraft.rpc.impl.AbstractBoltClientService$1.onResponse [12] com.alipay.remoting.rpc.RpcInvokeCallbackListener$CallbackTask.run [13] java.util.concurrent.ThreadPoolExecutor.runWorker [14] java.util.concurrent.ThreadPoolExecutor$Worker.run [15] java.lang.Thread.run

--- 59670324192 bytes (23.38%), 21642 samples [ 0] byte[] [ 1] java.nio.HeapByteBuffer. [ 2] java.nio.ByteBuffer.allocate [ 3] com.alipay.sofa.jraft.util.Utils.expandByteBufferAtLeast [ 4] com.alipay.sofa.jraft.util.ByteBufferCollector.getBuffer [ 5] com.alipay.sofa.jraft.util.ByteBufferCollector.put [ 6] com.alipay.sofa.jraft.core.Replicator.prepareEntry [ 7] com.alipay.sofa.jraft.core.Replicator.sendEntries [ 8] com.alipay.sofa.jraft.core.Replicator.sendEntries [ 9] com.alipay.sofa.jraft.core.Replicator.onRpcReturned [10] com.alipay.sofa.jraft.core.Replicator$3.run [11] com.alipay.sofa.jraft.rpc.impl.AbstractBoltClientService$1.onResponse [12] com.alipay.remoting.rpc.RpcInvokeCallbackListener$CallbackTask.run [13] java.util.concurrent.ThreadPoolExecutor.runWorker [14] java.util.concurrent.ThreadPoolExecutor$Worker.run [15] java.lang.Thread.run

这貌似很正常,但是,我换回来size=300甚至size=1去压,内存分配采样还是大量的扩容,理论上应该恢复到第一次采样的情况才对的,而且tps只有一半不到,甚至会到只有几千,不管怎么压都上不去了,除非重启。

我隐约觉得可能日志复制处有问题,在上述第三步换回小size后tps上不去时,观察磁盘、cpu等资源都很空闲,gc会严重些,感觉应该还是jraft的问题,可以帮助排查下吗?

killme2008 commented 5 years ago

这个现象比较奇怪,建议按照下列思路排查下

  1. 在换回到 size=300 或者 1 的时候,前面的积压流量已经没有了? 是否有积压流量引起的。
  2. 如果 cpu , 磁盘正常,可以看下 jstack 打印下堆栈,看看 jraft 的线程都在干什么。
  3. jstat 是否有 full gc 频繁的现象?
  4. 既然你们有用 jrofiler,也可以看下 cpu 和 thread monitor 锁的采样。
shiftyman commented 5 years ago

1.没有堆积 3.没有full gc,ygc没有太大变化,不影响 4.这生产机没法连了

2.线程堆栈,有大量block,624个JRaft-RPC-Processor线程,block了120个左右,(下面列出了一部分),不压测时没有block,也不知道是不是这个导致了吞吐量降低、、、其他没发现什么异常。。。

"JRaft-RPC-Processor-5" #631 prio=5 os_prio=0 tid=0x00007fdee8005800 nid=0xf632 waiting on condition [0x00007fdea4611000] java.lang.Thread.State: TIMED_WAITING (parking) at sun.misc.Unsafe.park(Native Method)

"JRaft-RPC-Processor-4" #630 prio=5 os_prio=0 tid=0x00007fdee8003800 nid=0xf631 waiting on condition [0x00007fdea4712000] java.lang.Thread.State: TIMED_WAITING (parking) at sun.misc.Unsafe.park(Native Method)

"JRaft-RPC-Processor-7" #633 prio=5 os_prio=0 tid=0x00007fdedc005000 nid=0xf634 waiting on condition [0x00007fdea440f000] java.lang.Thread.State: TIMED_WAITING (parking) at sun.misc.Unsafe.park(Native Method)

"JRaft-RPC-Processor-0" #663 prio=5 os_prio=0 tid=0x00007fdf7c001800 nid=0xf652 waiting on condition [0x00007fde9d5ec000] java.lang.Thread.State: TIMED_WAITING (parking) at sun.misc.Unsafe.park(Native Method)

"JRaft-RPC-Processor-4" #669 prio=5 os_prio=0 tid=0x00007fdfb0006800 nid=0xf658 waiting on condition [0x00007fde9bfe5000] java.lang.Thread.State: TIMED_WAITING (parking) at sun.misc.Unsafe.park(Native Method)

"JRaft-RPC-Processor-6" #671 prio=5 os_prio=0 tid=0x00007fdfc4006800 nid=0xf65a waiting on condition [0x00007fde9bde3000] java.lang.Thread.State: TIMED_WAITING (parking) at sun.misc.Unsafe.park(Native Method)

killme2008 commented 5 years ago

从你贴的堆栈看,是 replcator 的锁争抢较为激烈。 可以看下 replicator 的相关的堆栈,看看任务在做什么。

看到你有回复说是 jdk g1 gc 某个版本的问题,后来好像又删除了评论。不知道现在具体是什么情况?

shiftyman commented 5 years ago

现在来看是java8 g1的问题,昨天换了几个工具,采样发现是gc的问题,虽然gc频率和耗时跟正常时差不多,但是却在申请内存时产生了大量block,且短时间内无法自动恢复(jstack还看不出来被block了,汗!)。 线程堆栈和这个相似:https://bugs.java.com/bugdatabase/view_bug.do?bug_id=7147724

后来换openjdk11后貌似稳定了,还在观察。 ps:还尝鲜了下zgc,很爽,应用rt稳定多了,所以还是很期待你们的pooled优化的^^

fengjiachun commented 5 years ago

赞,pooled 和 zero copy 我们正在 benchmark 选取最优方案

fengjiachun commented 5 years ago

@shiftyman 有两个优化建议看看是不是能对你们业务场景的性能提供一点助力

  1. rheakv 中有一种 client batching 的方式可以将 tps 提高接近一个数量级,建议可以参考下,看看是不是能对你们的场景有所帮助,理论上是不影响 latency 的,具体见 DefaultRheaKVStore 的 get/put 方法
  2. 在 StateMachine.onApply 中在保证顺序的情况下合并同类 state,这样你的状态机就可以 group/batch 方式落盘了
fengjiachun commented 5 years ago

从你贴的堆栈看,是 replcator 的锁争抢较为激烈。 可以看下 replicator 的相关的堆栈,看看任务在做什么。

关于 replicator 锁竞争激烈的问题,我感觉可以尝试从线程模型上优化一下,目前是共享 bolt rpc 的 client executor 线程池,可以考虑把每个 replicator 都分别绑定到一个固定的 thread 来避免这种锁竞争