Skip to content
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

Journal thread fetch entry from queue cost too much time on acquireInterruptibly #2820

Open
hangc0276 opened this issue Oct 12, 2021 · 3 comments
Labels

Comments

@hangc0276
Copy link
Contributor

BUG REPORT

Describe the bug
The bookie configuration list as follow.

  • 24 cores
  • 48 bookie-io threads
  • 1 journal directory on NVME SSD
  • 3 ledger directories on NVME SSD
  • journalQueueSize is 10000

The write throughput keeps on 500MB/s.
When read request increase and the read throughput reaches about 600MB/s, the journal queue will be full for a long time. I check the journal sync latency is low. I print the stack of the bookie, and found the BookieJournal-3181 thread keeps on acquireInterruptibly in ArrayBlockingQueue.poll operation.

"BookieJournal-3181" #25 prio=5 os_prio=0 tid=0x00007f89b8001000 nid=0x14e waiting on condition [0x00007f8daebee000]
   java.lang.Thread.State: WAITING (parking)
	at sun.misc.Unsafe.park(Native Method)
	- parking to wait for  <0x00007f948dfb55c8> (a java.util.concurrent.locks.ReentrantLock$NonfairSync)
	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
	at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
	at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireInterruptibly(AbstractQueuedSynchronizer.java:897)
	at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireInterruptibly(AbstractQueuedSynchronizer.java:1222)
	at java.util.concurrent.locks.ReentrantLock.lockInterruptibly(ReentrantLock.java:335)
	at java.util.concurrent.ArrayBlockingQueue.poll(ArrayBlockingQueue.java:413)
	at org.apache.bookkeeper.bookie.Journal.run(Journal.java:980)

   Locked ownable synchronizers:
	- None

And the bookie-io threads are keeps on ArrayBlockingQueue.put operation

"bookie-io-1-2" #105 prio=5 os_prio=0 tid=0x00007f8fd0007800 nid=0x155 waiting on condition [0x00007f8dac3cd000]
   java.lang.Thread.State: WAITING (parking)
	at sun.misc.Unsafe.park(Native Method)
	- parking to wait for  <0x00007f948dfb55a0> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
	at java.util.concurrent.ArrayBlockingQueue.put(ArrayBlockingQueue.java:353)
	at org.apache.bookkeeper.bookie.Journal.logAddEntry(Journal.java:868)
	at org.apache.bookkeeper.bookie.Journal.logAddEntry(Journal.java:857)
	at org.apache.bookkeeper.bookie.Bookie.addEntryInternal(Bookie.java:1297)
	at org.apache.bookkeeper.bookie.Bookie.addEntry(Bookie.java:1402)
	- locked <0x00007f96c39c1a18> (a org.apache.bookkeeper.bookie.LedgerDescriptorImpl)
	at org.apache.bookkeeper.proto.WriteEntryProcessor.processPacket(WriteEntryProcessor.java:80)
	at org.apache.bookkeeper.proto.PacketProcessorBase.safeRun(PacketProcessorBase.java:85)
	at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36)
	at org.apache.bookkeeper.proto.BookieRequestProcessor.processAddRequest(BookieRequestProcessor.java:625)
	at org.apache.bookkeeper.proto.BookieRequestProcessor.processRequest(BookieRequestProcessor.java:365)
	at org.apache.bookkeeper.proto.BookieRequestHandler.channelRead(BookieRequestHandler.java:80)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
	at io.netty.channel.ChannelInboundHandlerAdapter.channelRead(ChannelInboundHandlerAdapter.java:93)
	at org.apache.bookkeeper.proto.AuthHandler$ServerSideHandler.channelRead(AuthHandler.java:95)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
	at org.apache.bookkeeper.proto.BookieProtoEncoding$RequestDecoder.channelRead(BookieProtoEncoding.java:450)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
	at io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:324)
	at io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:311)
	at io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:432)
	at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:276)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
	at io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
	at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
	at io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:795)
	at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:480)
	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:378)
	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	at java.lang.Thread.run(Thread.java:748)

Does anyone has any ideas for this situation?

The jstack list as follow
jstack.log

@hangc0276
Copy link
Contributor Author

@merlimat @eolivelli @dlg99 Would you please help take a look?

@eolivelli
Copy link
Contributor

it looks like the Journal is not able to serve write requests at the requested pace, so we are blocked on appending to the Queue

@dlg99
Copy link
Contributor

dlg99 commented Oct 12, 2021

@hangc0276 I'd start with monitoring of the situation. Is the disk a bottleneck?
I understand that NVME SSDs are very fast, but you have:

  • journal writes with fsync
  • memtable flush / entrylog writes with fsync
  • possibly reads (concurrent reads+writes can affect write throughput)
  • compaction
  • index writes (flat files or RocksDB) - small but with fsync

iostat, sar, etc should help with this.
Watch for io - writes/reads, io queues, latencies, paging etc.
Most likely something like memtable flush causes slower journal writes and results in a longer journal queue, queue put blocks.
there are metrics around that, do you see queue size maxing up?

As a simple test you can try configuring 2-3GB ramdrive (if node memory permits), move the journal there + disable journalSyncData. Is the journal/queue still a bottleneck?

Check if the disk configuration is optimized (deadline or noop scheduler for disk, ec, google for ideas to optimize IO for nvme ssd)

After that you can try tuning some parameters, like increase journalBufferedWritesThreshold, increase journalPreAllocSizeMB, journalWriteBufferSizeKB, journalQueueSize, etc.

Experiment with read/write buffers for entry log, maybe try flushEntrylogBytes of e.g.64-128MB (more frequent but faster flushes instead of)

tune compaction intervals to make it run less frequently and with smaller data volumes to rewrite, if the disk space permits.

I don't remember that well, but I think disabling journalRemoveFromPageCache might help if the node has a lot of memory.

If you can confirm that the queue (and not the disk) is definitely a bottleneck, we have JCTools as a dependency already and can try swapping the queue implementation. See #1682

Hope this helps.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

No branches or pull requests

3 participants