[ https://issues.apache.org/jira/browse/HDFS-15869?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17811439#comment-17811439 ]
ASF GitHub Bot commented on HDFS-15869: --------------------------------------- functioner commented on PR #2737: URL: https://github.com/apache/hadoop/pull/2737#issuecomment-1912891005 > @functioner Can you move forward with this MR? I can help with review. If you don't have time, I can resign this ticket to other contributor. @ZanderXu According to the discussion so far, Daryn has some doubt on the issue and patch. Do you have any idea to move forward the case? > 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 > Attachments: 1.png, 2.png > > Time Spent: 6.5h > 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.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org