[
https://issues.apache.org/jira/browse/BOOKKEEPER-643?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13875082#comment-13875082
]
Ivan Kelly commented on BOOKKEEPER-643:
---------------------------------------
[~hustlmsp] [~i0exception]
Patch is good. A few comments below. Just to clarify my understanding of what
the change is, you've moved reads into a different set of channels so there
needs to be no synchronization in the read and write paths? And also
preallocation of write files (I thought I saw this somewhere before in a patch
that was pushed in, but I guess not).
Hierarchy BufferedChannel -> BufferedReadChannel -> BufferedChannelBase is a
bit screwy. Firstly, BufferedChannelBase is only inherited by
BufferedReadChannel. Secondly, BufferedChannel reimplements all the the read
functionallity from BufferedReadChannel. Why not use BufferedReadChannel's
logic if the position to read is lower than the write buffer?
serverCfg is called conf everywhere else, why the change?
logid2channel -> logId2ReadChannel
likewise, camelcase logid2filechannel
putInChannels -> putInReadChannels
INVALID_LID defined, but -1 used further down in two places.
LOG.info("Flush and sync current entry logger {}.", logChannel.getLogId());
will spam the logs, better to have as a statistic
One concern I have is that FileChannels in logid2filechannel never get closed.
If there are a lot of logs this could eat your fds. Perhaps this isn't a
problem though, but it's worth thinking about harder.
> Improve concurrency of entry logger
> -----------------------------------
>
> Key: BOOKKEEPER-643
> URL: https://issues.apache.org/jira/browse/BOOKKEEPER-643
> Project: Bookkeeper
> Issue Type: Sub-task
> Components: bookkeeper-server
> Affects Versions: 4.2.0
> Reporter: Aniruddha
> Assignee: Aniruddha
> Fix For: 4.3.0
>
> Attachments: BOOKKEEPER-643.diff, BOOKKEEPER-643.diff
>
>
> the jira is created as part of BOOKKEEPER-429 to improve concurrency of
> current bookie implementation by leverage concurrent structures.
--
This message was sent by Atlassian JIRA
(v6.1.5#6160)