BewareMyPower commented on PR #23983:
URL: https://github.com/apache/pulsar/pull/23983#issuecomment-3324545694
Let me close this PR first.
> An extra context switch for each entry is costly
It's true, but we still need a benchmark. I ran a simple test by adding the
metrics to `ManagedLedgerMBeanImpl`
```java
@Getter
private final StatsBuckets threadSwitchingStats = new
StatsBuckets(ENTRY_LATENCY_BUCKETS_USEC);
@Getter
private final StatsBuckets executeStats = new
StatsBuckets(ENTRY_LATENCY_BUCKETS_USEC);
@Getter
private final StatsBuckets asyncAddEntryStats = new
StatsBuckets(ENTRY_LATENCY_BUCKETS_USEC);
```
and then record the latency in `asyncAddEntry`:
```java
final var beforeExecute = System.nanoTime();
executor.execute(() -> {
final var afterExecute = System.nanoTime();
OpAddEntry addOperation = OpAddEntry.createNoRetainBuffer(this,
buffer, numberOfMessages, callback, ctx,
currentLedgerTimeoutTriggered);
internalAsyncAddEntry(addOperation);
final var completeAsyncAddEntry = System.nanoTime();
mbean.getThreadSwitchingStats().addValue(afterExecute -
beforeExecute);
mbean.getAsyncAddEntryStats().addValue(completeAsyncAddEntry -
afterExecute);
});
final var afterExecute = System.nanoTime();
mbean.getExecuteStats().addValue(afterExecute - beforeExecute);
```
Then I added a test:
```java
@Test
public void testThreadSwitchingLatency() throws Exception {
ManagedLedgerImpl ml = (ManagedLedgerImpl) factory.open("ml");
final var data = new byte[1000];
for (int i = 0; i < 1000; i++) {
ml.addEntry(data);
}
ml.mbean.getThreadSwitchingStats().refresh();
ml.mbean.getAsyncAddEntryStats().refresh();
ml.mbean.getExecuteStats().refresh();
log.info("1: {}", ml.mbean.getThreadSwitchingStats().getAvg());
log.info("2: {}", ml.mbean.getAsyncAddEntryStats().getAvg());
log.info("3: {}", ml.mbean.getExecuteStats().getAvg());
}
```
Outputs:
```
1: 26924.005
2: 35933.837
3: 8866.484
```
1. Task execution delay: 27 us
2. Asynchronous operation itself: 36 us
3. The `execute` method call itself: 9 us
All of these should not be a significant impact because the whole add
latency is milliseconds level.
The main concern is that if `internalAsyncAddEntry` could cost much time,
blocking the Netty I/O thread might have an impact for other requests. Hence,
switching to a dedicated thread pool (managed ledger's executor) might make
sense.
We should not use `synchronized` to replace `executor.execute` here, because
currently `ManagedLedgerImpl` uses `synchronized` as a coarse-grained lock in
many other places as well:
- `asyncOpenCursor`
- cursor initialization callback in `asyncOpenCursor`
- `asyncDeleteCursor`
- `newNonDurableCursor`
- `getEstimatedBacklogSize`
- `asyncTerminate`
- ...
It's hard to analyze how much time is blocked by acquiring the lock.
Using a dedicated lock for `asyncAddEntry` should make sense, which is much
more efficient than thread switching. However, we cannot simply replace
`synchronized` with a different lock on `internalAsyncAddEntry` because it
could access fields that are protected by `synchronized` in other threads.
In summary, the current design should not be changed, including:
- Adding `synchronized` on `internalAsyncAddEntry`
- Switching to managed ledger's executor to execute `internalAsyncAddEntry`
We should not care much about the thread switching because its latency is
microseconds level.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]