Skip to content

Conversation

@merlimat
Copy link
Contributor

@merlimat merlimat commented Mar 4, 2023

Motivation

Note: this is stacked on top of #3830 & #3835

This change improves the way the AddRequests responses are send to client.

The current flow is:

  • The journal-force-thread issues the fsync on the journal file
  • We iterate over all the entries that were just synced and for each of them:
    1. Trigger channel.writeAndFlus()
    2. This will jump on the connection IO thread (Netty will use a write() to eventfd to post the task and wake the epoll)
    3. Write the object in the connection and trigger the serialization logic
    4. Grab a ByteBuf from the pool and write ~20 bytes with the response
    5. Write and flush the buffer on the channel
    6. With the flush consolidator we try to group multiple buffer into a single writev() syscall, though each call will have a long list of buffer, making the memcpy inefficient.
    7. Release all the buffers and return them to the pool

All these steps are quite expensive when the bookie is receiving a lot of small requests.

This PR changes the flow into:

  1. journal fsync
  2. go through each request and prepare the response into a per-connection ByteBuf which is not written on the channel as of yet
  3. after preparing all the responses, we flush them at once: Trigger an event on all the connections that will write the accumulated buffers.

The advantages are:

  1. 1 ByteBuf allocated per connection instead of 1 per request
    1. Less allocations and stress of buffer pool
    2. More efficient socket write() operations
  2. 1 task per connection posted on the Netty IO threads, instead of 1 per request.

@merlimat merlimat added this to the 4.16.0 milestone Mar 4, 2023
@merlimat merlimat self-assigned this Mar 4, 2023
@merlimat merlimat force-pushed the send-response branch 2 times, most recently from 4ea6b76 to fc7b13d Compare March 5, 2023 17:48
Copy link
Contributor

@hangc0276 hangc0276 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good job! Please rebase the master.

@codecov-commenter
Copy link

codecov-commenter commented Mar 6, 2023

Codecov Report

Merging #3837 (138ffb6) into master (73c5a0e) will increase coverage by 13.69%.
The diff coverage is 96.29%.

@@              Coverage Diff              @@
##             master    #3837       +/-   ##
=============================================
+ Coverage     54.67%   68.36%   +13.69%     
- Complexity     5346     6774     +1428     
=============================================
  Files           473      473               
  Lines         40910    40960       +50     
  Branches       5232     5240        +8     
=============================================
+ Hits          22366    28003     +5637     
+ Misses        16462    10706     -5756     
- Partials       2082     2251      +169     
Flag Coverage Δ
bookie 39.70% <83.33%> (-0.10%) ⬇️
client 44.14% <83.33%> (?)
remaining 29.70% <44.44%> (?)
replication 41.34% <92.59%> (+0.05%) ⬆️
tls 20.95% <88.88%> (?)

Flags with carried forward coverage won't be shown. Click here to find out more.

Impacted Files Coverage Δ
...main/java/org/apache/bookkeeper/bookie/Bookie.java 75.00% <ø> (ø)
...pache/bookkeeper/proto/BookieRequestProcessor.java 74.60% <85.71%> (+24.34%) ⬆️
.../apache/bookkeeper/proto/BookieRequestHandler.java 86.36% <95.83%> (+10.36%) ⬆️
.../java/org/apache/bookkeeper/bookie/BookieImpl.java 71.45% <100.00%> (+1.45%) ⬆️
...ain/java/org/apache/bookkeeper/bookie/Journal.java 79.67% <100.00%> (-0.33%) ⬇️
...g/apache/bookkeeper/proto/BookieProtoEncoding.java 80.93% <100.00%> (+6.23%) ⬆️
...java/org/apache/bookkeeper/proto/BookieServer.java 71.28% <100.00%> (+2.28%) ⬆️
...g/apache/bookkeeper/proto/WriteEntryProcessor.java 76.05% <100.00%> (+6.61%) ⬆️
...a/org/apache/bookkeeper/proto/ResponseBuilder.java 72.72% <0.00%> (-18.19%) ⬇️
.../java/org/apache/bookkeeper/bookie/SyncThread.java 69.69% <0.00%> (-3.04%) ⬇️
... and 169 more

📣 We’re building smart automated test selection to slash your CI/CD build times. Learn more

Comment on lines +1103 to +1106

if (forceWriteThread.requestProcessor != null) {
forceWriteThread.requestProcessor.flushPendingResponses();
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need to record journalStats.getJournalFlushStats() before trigger flushPendingResponses? That would lead the JournalFlushStats to include response time.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The flush would be very fast though. It's only posting an event on the channel group, not writing the actuals responses on the channels.

@hangc0276 hangc0276 merged commit d6748f9 into apache:master Mar 7, 2023
journalStats.getForceWriteGroupingCountStats()
.registerSuccessfulValue(numReqInLastForceWrite);

if (requestProcessor != null) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here we trigger all channels to flush pendingSendResponses, shall we only pick the requestHandler which is written to the logFile to flush pendingSendResponses, it may reduce the unnecessary iteration in the loop.

    public void flushPendingResponses() {
        for (Channel c : allChannels) {
            c.pipeline().fireUserEventTriggered(BookieRequestHandler.EVENT_FLUSH_ALL_PENDING_RESPONSES);
        }
    }

If there are 1000 channels, maybe only 2 channels need to flush pendingSendResponses

hangc0276 pushed a commit that referenced this pull request Mar 21, 2023
Descriptions of the changes in this PR:
This is an improvement for #3837

### Motivation
1. Now if the maxPendingResponsesSize is expanded large, it will not decrease. => We should make it flexible.
2. Now after prepareSendResponseV2 to the channel, then we trigger all channels to flush pendingSendResponses, maybe there is only a few channels that need to flush, but if we trigger all channels, it's a waste. => We only flush the channel which prepareSendResponseV2.
merlimat added a commit to merlimat/bookkeeper that referenced this pull request Mar 21, 2023
* Fix memory leak issue of reading small entries (apache#3844)

* Make read entry request recyclable (apache#3842)

* Make read entry request recyclable

* Move recycle to finally block

* Fix test and comments

* Fix test

* Avoid unnecessary force write. (apache#3847)

* Avoid unnecessary force write.

* code clean.

* fix style

* Correct the running job name for the test group (apache#3851)

---

### Motivation

The running tests job name doesn't match the tests. Correct
the job name.

* add timeout for two flaky timeout tests (apache#3855)

* add V2 protocal and warmupMessages support for benchMark (apache#3856)

* disable trimStackTrack for code-coverage profile (apache#3854)

* Fix bkperf log directory not found (apache#3858)

### Motivation
When using the bkperf command `bin/bkperf journal append -j data -n 100000000 --sync true` to test the BookKeeper journal performance, it failed with the following exception
```
[0.002s][error][logging] Error opening log file '/Users/hangc/Downloads/tmp/tc/batch/ta/bookkeeper-all-4.16.0-SNAPSHOT/logs/bkperf-gc.log': No such file or directory
[0.002s][error][logging] Initialization of output 'file=/Users/hangc/Downloads/tmp/tc/batch/ta/bookkeeper-all-4.16.0-SNAPSHOT/logs/bkperf-gc.log' using options 'filecount=5,filesize=64m' failed.
Invalid -Xlog option '-Xlog:gc=info:file=/Users/hangc/Downloads/tmp/tc/batch/ta/bookkeeper-all-4.16.0-SNAPSHOT/logs/bkperf-gc.log::filecount=5,filesize=64m', see error log for details.
Error: Could not create the Java Virtual Machine.
Error: A fatal exception has occurred. Program will exit.
```

The root cause is that the `logs` directory was not created.

### Modifications
Create the `logs` directory before bkperf started.

* [improve] Fix indexDirs upgrade failed (apache#3762)

* fix indexDirs upgrade failed

* Bump checkstyle-plugin from 3.1.2 to 3.2.1 (apache#3850)

* [Flaky] Fix flaky test in testRaceGuavaEvictAndReleaseBeforeRetain (apache#3857)

* Fix flaky test in testRaceGuavaEvictAndReleaseBeforeRetain

* format code

* Fix NPE in BenchThroughputLatency (apache#3859)

* Update website to 4.15.4 (apache#3862)

---

### Motivation

Update website to 4.15.4

* change rocksDB config level_compaction_dynamic_level_bytes to CFOptions (apache#3860)

### Motivation
After PR apache#3056 , Bookkeeper set `level_compaction_dynamic_level_bytes=true` as `TableOptions` in `entry_location_rocksdb.conf.default` , which will cause `level_compaction_dynamic_level_bytes` lose efficacy and will cause rocksDB .sst file compact sort chaos when update bookie release.
As RocksDB  conf, `level_compaction_dynamic_level_bytes` need set as `CFOptions` https://github.com/facebook/rocksdb/blob/master/examples/rocksdb_option_file_example.ini

<img width="703" alt="image" src="https://user-images.githubusercontent.com/84127069/224640399-d5481fe5-7b75-4229-ac06-3d280aa9ae6d.png">


<img width="240" alt="image" src="https://user-images.githubusercontent.com/84127069/224640621-737d0a42-4e01-4f38-bd5a-862a93bc4b32.png">

### Changes

1. Change `level_compaction_dynamic_level_bytes=true` from `TableOptions` to `CFOptions`  in `entry_location_rocksdb.conf.default` ;

* Correct the running job flag for the test group. (apache#3865)

* Release note for 4.15.4 (apache#3831)

---

### Motivation

Release note for 4.15.4

* Add trigger entry location index rocksDB compact interface. (apache#3802)

### Motivation
After the bookie instance running long time, the bookie entry location index rocksDB `.sst` file size maybe expand to 20-30GB as one ledger data dir's location index in some case, which will cause the rocksDB scan operator cost more time and cause the bookie client request timeout.

Add trigger entry location index rocksDB compact REST API which can trigger  entry location rocksDB compaction and get the compaction status. 

The full range entry location index rocksDB compact will cause the entry location index dir express higher IOUtils. So we'd better trigger the entry location rocksDB compact by the api in low data flow period.

**Some case before rocksDB compact:**
<img width="232" alt="image" src="https://user-images.githubusercontent.com/84127069/220893469-e6fbc1a3-c767-4ffe-8ae9-f05ad1833c50.png">


<img width="288" alt="image" src="https://user-images.githubusercontent.com/84127069/220891359-dc37e139-37b0-461b-8001-dcc48517366c.png">

**After rocksDB compact:**
<img width="255" alt="image" src="https://user-images.githubusercontent.com/84127069/220891419-24267fa7-348c-4fbd-8b3e-70a99840bce5.png">

### Changes
1. Add  REST API  to trigger entry location index rocksDB compact.

* Pick the higher leak detection level between netty and bookkeeper. (apache#3794)

### Motivation
1. Pick the higher leak detection level between netty and bookkeeper.
2. Enhance the bookkeeper leak detection value match rule, now it's case insensitive.

There are detailed information about it: https://lists.apache.org/thread/d3zw8bxhlg0wxfhocyjglq0nbxrww3sg

* Disable code coverage and codecov report (apache#3863)

### Motivation

There're two reasons that we want to disable the code coverage.

1. The current report result is not accurate.
2. We can't get the PR's unit test's code coverage because of the apache Codecov permission.

* Add small files check in garbage collection (apache#3631)

### Motivation
When we use `TransactionalEntryLogCompactor` to compact the entry log files, it will generate a lot of small entry log files, and for those files, the file usage is usually greater than 90%, which can not be compacted unless the file usage decreased.

![image](https://user-images.githubusercontent.com/5436568/201135615-4d6072f5-e353-483d-9afb-48fad8134044.png)


### Changes
We introduce the entry log file size check during compaction, and the checker is controlled by `gcEntryLogSizeRatio`. 
If the total entry log file size is less than `gcEntryLogSizeRatio * logSizeLimit`, the entry log file will be compacted even though the file usage is greater than 90%. This feature is disabled by default and the `gcEntryLogSizeRatio` default value is `0.0`

* [improvement] Delay all audit task when have a already delayed bookie check task (apache#3818)

### Motivation

Fixes apache#3817 

For details, see: apache#3817 

### Changes

When there is an `auditTask` during the `lostBookieRecoveryDelay` delay, other detection tasks should be skipped.

* Change order of doGcLedgers and extractMetaFromEntryLogs (apache#3869)

* [Bugfix] make metadataDriver initialization more robust (apache#3873)

Co-authored-by: zengqiang.xu <zengqiang.xu@shopee.com>

* Enable CI for the streamstorage python client (apache#3875)

* Fix compaction threshold default value precision problem. (apache#3871)

* Fix compaction threshold precision problem.

* Fix compaction threshold precision problem.

* Single buffer for small add requests (apache#3783)

* Single buffer for small add requests

* Fixed checkstyle

* Fixed treating of ComposityByteBuf

* Fixed merge issues

* Fixed merge issues

* WIP

* Fixed test and removed dead code

* Removed unused import

* Fixed BookieJournalTest

* removed unused import

* fix the checkstyle

* fix failed test

* fix failed test

---------

Co-authored-by: chenhang <chenhang@apache.org>

* Add log for entry log file delete. (apache#3872)

* Add log for entry log file delete.

* add log info.

* Address the comment.

* Address the comment.

* revert the code.

* Improve group and flush add-responses after journal sync (apache#3848)

Descriptions of the changes in this PR:
This is an improvement for apache#3837

### Motivation
1. Now if the maxPendingResponsesSize is expanded large, it will not decrease. => We should make it flexible.
2. Now after prepareSendResponseV2 to the channel, then we trigger all channels to flush pendingSendResponses, maybe there is only a few channels that need to flush, but if we trigger all channels, it's a waste. => We only flush the channel which prepareSendResponseV2.

---------

Co-authored-by: Penghui Li <penghui@apache.org>
Co-authored-by: Yong Zhang <zhangyong1025.zy@gmail.com>
Co-authored-by: Hang Chen <chenhang@apache.org>
Co-authored-by: wenbingshen <oliver.shen999@gmail.com>
Co-authored-by: ZhangJian He <shoothzj@gmail.com>
Co-authored-by: lixinyang <84127069+Nicklee007@users.noreply.github.com>
Co-authored-by: YANGLiiN <ielin@qq.com>
Co-authored-by: Lishen Yao <yaalsn@gmail.com>
Co-authored-by: Andrey Yegorov <8622884+dlg99@users.noreply.github.com>
Co-authored-by: ZanderXu <zanderxu@apache.org>
Co-authored-by: zengqiang.xu <zengqiang.xu@shopee.com>
Co-authored-by: Matteo Merli <mmerli@apache.org>
hangc0276 added a commit that referenced this pull request Mar 26, 2023
### Motivation
#3837 introduced group flush add responses triggered by journal sync. However, if we skip writing journals, the add responses won't be flushed to the netty channel and the client will receive write entries timeout.

### Changes
- Flush the add responses when skipping writing journals
- Add tests to cover V2 protocol and skip writing journal cases.
hangc0276 added a commit that referenced this pull request Mar 26, 2023
### Motivation
#3837 introduced group flush add responses triggered by journal sync. However, if we skip writing journals, the add responses won't be flushed to the netty channel and the client will receive write entries timeout.

### Changes
- Flush the add responses when skipping writing journals
- Add tests to cover V2 protocol and skip writing journal cases.

(cherry picked from commit 5deadcc)
Ghatage pushed a commit to sijie/bookkeeper that referenced this pull request Jul 12, 2024
### Motivation

Note: this is stacked on top of apache#3830 & apache#3835

This change improves the way the AddRequests responses are send to client. 

The current flow is: 
 * The journal-force-thread issues the fsync on the journal file
 * We iterate over all the entries that were just synced and for each of them:
     1. Trigger channel.writeAndFlus()
     2. This will jump on the connection IO thread (Netty will use a `write()` to `eventfd` to post the task and wake the epoll)
     3. Write the object in the connection and trigger the serialization logic
     4. Grab a `ByteBuf` from the pool and write ~20 bytes with the response
     5. Write and flush the buffer on the channel
     6. With the flush consolidator we try to group multiple buffer into a single `writev()` syscall, though each call will have a long list of buffer, making the memcpy inefficient.
     7. Release all the buffers and return them to the pool

All these steps are quite expensive when the bookie is receiving a lot of small requests. 

This PR changes the flow into: 

1. journal fsync
2. go through each request and prepare the response into a per-connection `ByteBuf` which is not written on the channel as of yet
3. after preparing all the responses, we flush them at once: Trigger an event on all the connections that will write the accumulated buffers.

The advantages are: 
 1. 1 ByteBuf allocated per connection instead of 1 per request
    1. Less allocations and stress of buffer pool
    2. More efficient socket write() operations
 3. 1 task per connection posted on the Netty IO threads, instead of 1 per request.
Ghatage pushed a commit to sijie/bookkeeper that referenced this pull request Jul 12, 2024
Descriptions of the changes in this PR:
This is an improvement for apache#3837

### Motivation
1. Now if the maxPendingResponsesSize is expanded large, it will not decrease. => We should make it flexible.
2. Now after prepareSendResponseV2 to the channel, then we trigger all channels to flush pendingSendResponses, maybe there is only a few channels that need to flush, but if we trigger all channels, it's a waste. => We only flush the channel which prepareSendResponseV2.
Ghatage pushed a commit to sijie/bookkeeper that referenced this pull request Jul 12, 2024
### Motivation
apache#3837 introduced group flush add responses triggered by journal sync. However, if we skip writing journals, the add responses won't be flushed to the netty channel and the client will receive write entries timeout.

### Changes
- Flush the add responses when skipping writing journals
- Add tests to cover V2 protocol and skip writing journal cases.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants