-
Notifications
You must be signed in to change notification settings - Fork 1.1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
大量的byte[]创建引发频发GC,希望能做优化 #158
Comments
可不可以再给一些详细数据? 比如 jraft 中的 metric log,jvm 配置等等,以及你的压测场景 |
1.metric log没开 反正感觉跟测试场景关系不大,怎么测只要压高了都是1s一次ygc,如果用cms+parnew,gc更凶,延迟更高。 麻烦看看? |
这是一个可以优化的点,jraft 这个环节可以尽量做一些 buffer 复用,原来有考虑做过,但是觉的如果不从上到下都改造,可能效果不会太好。bolt rpc 目前的序列化还是基于 byte 数组来做的,要更好的优化,还是需要去改造 bolt 框架,尽量从上(网络)到下(rocksdb)都做到 zero copy 和 buffer 内存复用。 不过我觉的这里的瓶颈很可能不是这个问题,高负载系统 1 秒一次的 young gc 还是常见的。可以开下 metric 观察下。 |
抽样了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 |
从这个指标看
中位数 median 是 1,单次 append 的日志条数不高,攒批的效果没有达到,这可能是吞吐上不去的主要原因。 你的 300 并发是单个客户端? 如果机器足够的话,可以采用多客户端压测看下,我们的压测数据也是在多个客户端并且客户端启用了 batch 写入的情况下才能达到。因为你的 range 比较多,请求分散到单个 range 后并发应该不高了。 |
对于楼主的内存采样中频繁的 com.alipay.sofa.jraft.util.Utils.expandByteBufferAtLeast,也就是扩容 |
1.我的应用对延迟敏感,如果并发增加,延迟就会增加不少,判断可能是单raft串行排队导致的。(吞吐量貌似也没增加太明显,观察cpu、disk io都没有达瓶颈,网络延迟在0.05ms) 2.gc频繁对延迟还是有不小影响的,之前用cms现在换g1收集器延迟有所减少且稳定了一些,吞吐量也有所提高的,所以能优化一下还是很好的。 3.从这个metrics看,日志复制为什么延迟这么高? 4.最后,之前用jprofile看,有一些block占用了相当长时间: 线程:JRaft-RPC-Processor-n 这种block有不少,个别block时间超过1s,很神奇。 |
对于 latency 敏感的话,确实 gc 影响很大,如果能减少 young gc 频率,效果还是会立竿见影的。 jraft 目前的设计还是重吞吐。 下面这个指标可以说明大部分请求的 bytes 范围是多少
默认 核心的问题还是 bolt 框架的序列化问题,暂时还没有好的解决办法。也许可以考虑复用下这里的 byte 数组,自己实现一个基于弱引用或者软引用的缓存池。基于 thread local 是不行的,因为 rpc 整体是异步的,在不同的线程上下文。 |
|
赞,很高效! |
请问这种错误一般什么情况引起?压测过程中偶尔会出现。 [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" java.util.ConcurrentModificationException: null |
@shiftyman 看情况是链接关闭过程中的一个并发 bug,取消 infly 的请求 future 的时候需要遍历 |
[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 这种p999这么大是什么情况?可能是什么原因导致的?这边已经把jraft.byte_buf.size调大了,但是采样内存分配,还是大量的expandByteBufferAtLeast出现 现在每次写入就10byte,依旧大量expandByteBufferAtLeast,经查发现有这种特大的批量传输。 |
这个应该是复制较慢情况下攒批的效果了,p999 应该这种请求量很少的,目前没有 pool 的方式,还是会存在这种离散请求。你可以限制单个批次大小的,通过 |
我怀疑哪里有bug。 我描述一个很奇怪的现象,刚启动时,写size=300压测,tps能达到10w。这个时候,内存分配采样是这样的,基本不会触发扩容: --- 380017808 bytes (2.40%), 5747 samples 但是,经过一段时间后,或者用size=1500去压一段时间,内存采样就大量的扩容,gc增多,tps下降,这个可以理解: --- 59670324192 bytes (23.38%), 21642 samples 这貌似很正常,但是,我换回来size=300甚至size=1去压,内存分配采样还是大量的扩容,理论上应该恢复到第一次采样的情况才对的,而且tps只有一半不到,甚至会到只有几千,不管怎么压都上不去了,除非重启。 我隐约觉得可能日志复制处有问题,在上述第三步换回小size后tps上不去时,观察磁盘、cpu等资源都很空闲,gc会严重些,感觉应该还是jraft的问题,可以帮助排查下吗? |
这个现象比较奇怪,建议按照下列思路排查下
|
1.没有堆积 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] "JRaft-RPC-Processor-4" #630 prio=5 os_prio=0 tid=0x00007fdee8003800 nid=0xf631 waiting on condition [0x00007fdea4712000] "JRaft-RPC-Processor-7" #633 prio=5 os_prio=0 tid=0x00007fdedc005000 nid=0xf634 waiting on condition [0x00007fdea440f000] "JRaft-RPC-Processor-0" #663 prio=5 os_prio=0 tid=0x00007fdf7c001800 nid=0xf652 waiting on condition [0x00007fde9d5ec000] "JRaft-RPC-Processor-4" #669 prio=5 os_prio=0 tid=0x00007fdfb0006800 nid=0xf658 waiting on condition [0x00007fde9bfe5000] "JRaft-RPC-Processor-6" #671 prio=5 os_prio=0 tid=0x00007fdfc4006800 nid=0xf65a waiting on condition [0x00007fde9bde3000] |
从你贴的堆栈看,是 replcator 的锁争抢较为激烈。 可以看下 replicator 的相关的堆栈,看看任务在做什么。 看到你有回复说是 jdk g1 gc 某个版本的问题,后来好像又删除了评论。不知道现在具体是什么情况? |
现在来看是java8 g1的问题,昨天换了几个工具,采样发现是gc的问题,虽然gc频率和耗时跟正常时差不多,但是却在申请内存时产生了大量block,且短时间内无法自动恢复(jstack还看不出来被block了,汗!)。 后来换openjdk11后貌似稳定了,还在观察。 |
赞,pooled 和 zero copy 我们正在 benchmark 选取最优方案 |
@shiftyman 有两个优化建议看看是不是能对你们业务场景的性能提供一点助力
|
关于 replicator 锁竞争激烈的问题,我感觉可以尝试从线程模型上优化一下,目前是共享 bolt rpc 的 client executor 线程池,可以考虑把每个 replicator 都分别绑定到一个固定的 thread 来避免这种锁竞争 |
* (feat) add AdaptiveBufAllocator * (feat) pooled for ByteBufferCollector #158 * (feat) pooled for ByteBufferCollector #158 * (feat) pooled for ByteBufferCollector #158 * (feat) pooled for ByteBufferCollector #158 * (fix) rename method name * (fix) minor fix * (fix) add metric for recyclers (#164) * (fix) add metric for recyclers * (fix) add metric for ByteBufferCollector.capacity * (fix) code format * (fix) by review comment * feat/zero copy with replicator (#167) * (fix) zero copy with replicator * (fix) support zero copy and add benchmark * (fix) rename field * (fix) rm zero copy and unnecessary metric * (fix) by review comment * (feat) add unit test AdaptiveBufAllocatorTest * (feat) add unit test RecyclersTest * (feat) add unit test RecyclableByteBufferListTest * (feat) add unit test ByteBufferCollectorTest
* (feat) add AdaptiveBufAllocator * (feat) pooled for ByteBufferCollector #158 * (feat) pooled for ByteBufferCollector #158 * (feat) pooled for ByteBufferCollector #158 * (feat) pooled for ByteBufferCollector #158 * (fix) rename method name * (fix) minor fix * (fix) add metric for recyclers (#164) * (fix) add metric for recyclers * (fix) add metric for ByteBufferCollector.capacity * (fix) code format * (fix) by review comment * feat/zero copy with replicator (#167) * (fix) zero copy with replicator * (fix) support zero copy and add benchmark * (fix) rename field * (fix) rm zero copy and unnecessary metric * (fix) by review comment * (feat) add unit test AdaptiveBufAllocatorTest * (feat) add unit test RecyclersTest * (feat) add unit test RecyclableByteBufferListTest * (feat) add unit test ByteBufferCollectorTest
* (feat) add AdaptiveBufAllocator * (feat) pooled for ByteBufferCollector #158 * (feat) pooled for ByteBufferCollector #158 * (feat) pooled for ByteBufferCollector #158 * (feat) pooled for ByteBufferCollector #158 * (fix) rename method name * (fix) minor fix * (fix) add metric for recyclers (#164) * (fix) add metric for recyclers * (fix) add metric for ByteBufferCollector.capacity * (fix) code format * (fix) by review comment * feat/zero copy with replicator (#167) * (fix) zero copy with replicator * (fix) support zero copy and add benchmark * (fix) rename field * (fix) rm zero copy and unnecessary metric * (fix) by review comment * (feat) add unit test AdaptiveBufAllocatorTest * (feat) add unit test RecyclersTest * (feat) add unit test RecyclableByteBufferListTest * (feat) add unit test ByteBufferCollectorTest
* (feat) add FixedThreadsExecutorGroup #168 * (feat) rename method * (feat) add MpscSingleThreadExecutor and benchmark #168 * (fix) forget to warmup producers * (fix) fix some bugs and add unit test * (fix) add more unit test * (fix) add more unit test * (fix) add more unit test * (fix) add some comments * (fix) unit test * (fix) add some comments * (fix) refactoring Utils class * (fix) refactoring Utils class * (fix) jraft.closure.threadpool.size.max update default value * (fix) fix unit test * (fix) fix unit test * (feat) refactor ThreadId and replicator (#169) * (feat) refactor ThreadId and replicator * (feat) Adds javadoc * (feat) add pooled buf allocator (#161) * (feat) add AdaptiveBufAllocator * (feat) pooled for ByteBufferCollector #158 * (feat) pooled for ByteBufferCollector #158 * (feat) pooled for ByteBufferCollector #158 * (feat) pooled for ByteBufferCollector #158 * (fix) rename method name * (fix) minor fix * (fix) add metric for recyclers (#164) * (fix) add metric for recyclers * (fix) add metric for ByteBufferCollector.capacity * (fix) code format * (fix) by review comment * feat/zero copy with replicator (#167) * (fix) zero copy with replicator * (fix) support zero copy and add benchmark * (fix) rename field * (fix) rm zero copy and unnecessary metric * (fix) by review comment * (feat) add unit test AdaptiveBufAllocatorTest * (feat) add unit test RecyclersTest * (feat) add unit test RecyclableByteBufferListTest * (feat) add unit test ByteBufferCollectorTest * Add unit tests for com.alipay.sofa.jraft.util.BytesUtil (#166) These tests were written using Diffblue Cover. * (fix) Utils.java format * (feat) add FixedThreadsExecutorGroup #168 * (feat) rename method * (feat) add MpscSingleThreadExecutor and benchmark #168 * (fix) forget to warmup producers * (fix) fix some bugs and add unit test * (fix) add more unit test * (fix) add more unit test * (fix) add more unit test * (fix) add some comments * (fix) unit test * (fix) add some comments * (fix) refactoring Utils class * (fix) refactoring Utils class * (fix) jraft.closure.threadpool.size.max update default value * (fix) fix unit test * (fix) fix unit test * (feat) add pooled buf allocator (#161) * (feat) add AdaptiveBufAllocator * (feat) pooled for ByteBufferCollector #158 * (feat) pooled for ByteBufferCollector #158 * (feat) pooled for ByteBufferCollector #158 * (feat) pooled for ByteBufferCollector #158 * (fix) rename method name * (fix) minor fix * (fix) add metric for recyclers (#164) * (fix) add metric for recyclers * (fix) add metric for ByteBufferCollector.capacity * (fix) code format * (fix) by review comment * feat/zero copy with replicator (#167) * (fix) zero copy with replicator * (fix) support zero copy and add benchmark * (fix) rename field * (fix) rm zero copy and unnecessary metric * (fix) by review comment * (feat) add unit test AdaptiveBufAllocatorTest * (feat) add unit test RecyclersTest * (feat) add unit test RecyclableByteBufferListTest * (feat) add unit test ByteBufferCollectorTest * (fix) Utils.java format * (fix) fix bad key with executor map * (fix) bad import * (fix) fix unit test * (feat) add mor benchmark * (fix) code format * (fix) code format * (fix) benchmark with jmh * (fix) benchmark with jmh * (fix) set common daemon * (fix) fix unit test * (fix) should be no radical changes, especially if they are not fully tested. * (feat) add jctools * (feat) configure the number of processors #180 (#181) * (fix) format
压测期间发现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
The text was updated successfully, but these errors were encountered: