[
https://issues.apache.org/jira/browse/HDFS-15869?focusedWorklogId=585171&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-585171
]
ASF GitHub Bot logged work on HDFS-15869:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 19/Apr/21 14:27
Start Date: 19/Apr/21 14:27
Worklog Time Spent: 10m
Work Description: amahussein commented on pull request #2737:
URL: https://github.com/apache/hadoop/pull/2737#issuecomment-822510445
> @amahussein Thanks for your quick response. I think this is not the same
concept/issue between
[HDFS-15957](https://issues.apache.org/jira/browse/HDFS-15957) and
[HDFS-15869](https://issues.apache.org/jira/browse/HDFS-15869). I have leave
comment at [HDFS-15869](https://issues.apache.org/jira/browse/HDFS-15869) and
suggest to change it to `improvement` rather than `bug fix`; (cc @functioner)
> IMO, this is a classic Producer-Consumer problem, and it is natural idea
to improve performance using parallel way. And Yiqun has reported the same
issue at [HDFS-15486](https://issues.apache.org/jira/browse/HDFS-15486). In my
own production env it was about 5% E2E saving time for write operation.
> My suggestion,
> A. Update description for improvement rather than bug fix.
> B. If any concerns about lambda expression, we could improve it rather
than reject it directly.
> Welcome any more discussion. Thanks everyone here.
Thanks @Hexiaoqiao Hexiaoqiao for the comment.
Ok, is the purpose of the change is to improve performance of the
`FSEditLogAsync.java` by executing `sendResponse()` in parallel?
In that case, please change the title of the Jira and the description to
remove references to "hanging" problems.
Then I will take another look. I am sorry for the inconvenience as I want to
make sure I understand the purpose of the change before revieweing.
> @amahussein Thanks for the comment.
> Can I send an email to you to explain more about the issue? @Hexiaoqiao
and I have some more discussion on it, and some discussion is inconvenient to
put in here. You can contact me via
[[email protected]](mailto:[email protected]) or
[[email protected]](mailto:[email protected]) and then I will reply.
Thanks @functioner ! I really appreciate that.
I think @Hexiaoqiao Hexiaoqiao reply already clarified some of the confusion
about the scope of the work.
Please feel free to reach me through email at anytime. I am on the
common-dev mailing list.
--
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.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 585171)
Time Spent: 4h 10m (was: 4h)
> Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can
> cause the namenode to hang
> ----------------------------------------------------------------------------------------------------
>
> Key: HDFS-15869
> URL: https://issues.apache.org/jira/browse/HDFS-15869
> Project: Hadoop HDFS
> Issue Type: Improvement
> Components: fs async, namenode
> Affects Versions: 3.2.2
> Reporter: Haoze Wu
> Assignee: Haoze Wu
> Priority: Major
> Labels: pull-request-available
> Time Spent: 4h 10m
> Remaining Estimate: 0h
>
> We were doing some testing of the latest Hadoop stable release 3.2.2 and
> found some network issue can cause the namenode to hang even with the async
> edit logging (FSEditLogAsync).
> The workflow of the FSEditLogAsync thread is basically:
> # get EditLog from a queue (line 229)
> # do the transaction (line 232)
> # sync the log if doSync (line 243)
> # do logSyncNotify (line 248)
> {code:java}
> //hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogAsync.java
> @Override
> public void run() {
> try {
> while (true) {
> boolean doSync;
> Edit edit = dequeueEdit(); //
> line 229
> if (edit != null) {
> // sync if requested by edit log.
> doSync = edit.logEdit(); //
> line 232
> syncWaitQ.add(edit);
> } else {
> // sync when editq runs dry, but have edits pending a sync.
> doSync = !syncWaitQ.isEmpty();
> }
> if (doSync) {
> // normally edit log exceptions cause the NN to terminate, but tests
> // relying on ExitUtil.terminate need to see the exception.
> RuntimeException syncEx = null;
> try {
> logSync(getLastWrittenTxId()); //
> line 243
> } catch (RuntimeException ex) {
> syncEx = ex;
> }
> while ((edit = syncWaitQ.poll()) != null) {
> edit.logSyncNotify(syncEx); //
> line 248
> }
> }
> }
> } catch (InterruptedException ie) {
> LOG.info(Thread.currentThread().getName() + " was interrupted,
> exiting");
> } catch (Throwable t) {
> terminate(t);
> }
> }
> {code}
> In terms of the step 4, FSEditLogAsync$RpcEdit.logSyncNotify is
> essentially doing some network write (line 365).
> {code:java}
> //hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogAsync.java
> private static class RpcEdit extends Edit {
> // ...
> @Override
> public void logSyncNotify(RuntimeException syncEx) {
> try {
> if (syncEx == null) {
> call.sendResponse(); // line
> 365
> } else {
> call.abortResponse(syncEx);
> }
> } catch (Exception e) {} // don't care if not sent.
> }
> // ...
> }{code}
> If the sendResponse operation in line 365 gets stuck, then the whole
> FSEditLogAsync thread is not able to proceed. In this case, the critical
> logSync (line 243) can’t be executed, for the incoming transactions. Then the
> namenode hangs. This is undesirable because FSEditLogAsync’s key feature is
> asynchronous edit logging that is supposed to tolerate slow I/O.
> To see why the sendResponse operation in line 365 may get stuck, here is
> the stack trace:
> {code:java}
> '(org.apache.hadoop.ipc.Server,channelWrite,3593)',
> '(org.apache.hadoop.ipc.Server,access$1700,139)',
> '(org.apache.hadoop.ipc.Server$Responder,processResponse,1657)',
> '(org.apache.hadoop.ipc.Server$Responder,doRespond,1727)',
> '(org.apache.hadoop.ipc.Server$Connection,sendResponse,2828)',
> '(org.apache.hadoop.ipc.Server$Connection,access$300,1799)',
> '(org.apache.hadoop.ipc.Server$RpcCall,doResponse,1111)',
> '(org.apache.hadoop.ipc.Server$Call,doResponse,903)',
> '(org.apache.hadoop.ipc.Server$Call,sendResponse,889)',
>
> '(org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync$RpcEdit,logSyncNotify,365)',
> '(org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync,run,248)',
> '(java.lang.Thread,run,748)'
> {code}
> The `channelWrite` function is defined as follows:
> {code:java}
> //hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
> private int channelWrite(WritableByteChannel channel,
> ByteBuffer buffer) throws IOException {
>
> int count = (buffer.remaining() <= NIO_BUFFER_LIMIT) ?
> channel.write(buffer) : channelIO(null, channel, buffer);
> // line 3594
> if (count > 0) {
> rpcMetrics.incrSentBytes(count);
> }
> return count;
> }{code}
> The `channel.write(buffer)` operation in line 3594 may be slow. Although
> for this specific stack trace, the channel is initialized in the non-blocking
> mode, there is still a chance of being slow depending on native write
> implementation in the OS (e.g., a kernel issue). Furthermore, the channelIO
> invocation in line 3594 may also get stuck, since it waits until the buffer
> is drained:
> {code:java}
> //hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
> private static int channelIO(...) throws IOException {
> //...
> while (buf.remaining() > 0) { //
> line 3637
> try {
> int ioSize = Math.min(buf.remaining(), NIO_BUFFER_LIMIT);
> buf.limit(buf.position() + ioSize);
>
> ret = (readCh == null) ? writeCh.write(buf) : readCh.read(buf); //
> line 3642
>
> if (ret < ioSize) {
> break;
> } } finally {
> buf.limit(originalLimit);
> }
> } int nBytes = initialRemaining - buf.remaining();
> return (nBytes > 0) ? nBytes : ret;
> }
> {code}
> For example, if the payload is split in two batches, the second batch
> will have to wait for the first batch to be sent out, which may encounter
> high packet loss rate and thus slow I/O.
> In summary, FSEditLogAsync is a critical service, but the potential delay
> occurring in `edit.logSyncNotify(syncEx)` can block the FSEditLogAsync
> thread. According to the comment
> ([https://github.com/apache/hadoop/blob/rel/release-3.2.2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogAsync.java#L369]),
> “don't care if not sent”, FSEditLogAsync thread in fact does not really need
> to be concerned with the sync notification.
> By the way, we found that this issue is related to HDFS-15486 in the old
> version Hadoop.
> *Reproduction*
> To show that the potential delay introduced by
> `FSEditLogAsync$RpcEdit.logSyncNotify` can cause the namenode to hang, we
> provide the scripts to reproduce the bug.
> The script basically blocks the FSEditLogAsync thread when it’s invoking
> `call.sendResponse()` in `FSEditLogAsync$RpcEdit#logSyncNotify`. Our
> reproduction scripts guarantee that the delay is injected once and only once.
> The reproduction script is provided in a gist
> ([https://gist.github.com/functioner/891108ee54e24f155ac395a65e46cbfd]).
> *Fix*
> Since the `logSyncNotify` is not a critical operation for
> `FSEditLogAsync`, we propose to put the
> `FSEditLogAsync$RpcEdit.logSyncNotify` invocation to a separate thread in
> `FSEditLogAsync`. In this way, even if the notifications get stuck, they will
> not affect the edit logging.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]