[jira] [Updated] (HDFS-15869) Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang

2023-08-11 Thread caozhiqiang (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15869?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

caozhiqiang updated HDFS-15869:
---
Attachment: 2.png

> 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,)',
>  '(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 

[jira] [Updated] (HDFS-15869) Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang

2023-08-11 Thread caozhiqiang (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15869?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

caozhiqiang updated HDFS-15869:
---
Attachment: 1.png

> 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,)',
>  '(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 

[jira] [Updated] (HDFS-15869) Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang

2021-04-17 Thread Xiaoqiao He (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15869?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xiaoqiao He updated HDFS-15869:
---
Issue Type: Improvement  (was: Bug)
  Priority: Major  (was: Critical)

Add [~functioner] to the contributor list and assign this JIRA to him.
Thanks [~functioner] for your report and works.
Change this issue type to `improvement`. Just suggest to modify title to 
'Improve sync notify for FSEditLogAsync' or some other related title rather 
than about bug fix.

> 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: 3h
>  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,)',
>  '(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)',

[jira] [Updated] (HDFS-15869) Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang

2021-03-24 Thread Haoze Wu (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15869?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Haoze Wu updated HDFS-15869:

Description: 
    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,)',
 '(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) {

[jira] [Updated] (HDFS-15869) Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang

2021-03-02 Thread Haoze Wu (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15869?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Haoze Wu updated HDFS-15869:

Description: 
*Description*

    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,)',
 '(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) { 

[jira] [Updated] (HDFS-15869) Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang

2021-03-02 Thread Haoze Wu (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15869?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Haoze Wu updated HDFS-15869:

Description: 
*Description*

    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,)',
 '(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) { 

[jira] [Updated] (HDFS-15869) Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang

2021-03-02 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15869?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

ASF GitHub Bot updated HDFS-15869:
--
Labels: pull-request-available  (was: )

> 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: Bug
>  Components: fs async, namenode
>Affects Versions: 3.2.2
>Reporter: Haoze Wu
>Priority: Critical
>  Labels: pull-request-available
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> *Description*
>     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,)',
>  '(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 {
> 

[jira] [Updated] (HDFS-15869) Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang

2021-03-02 Thread Haoze Wu (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15869?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Haoze Wu updated HDFS-15869:

Description: 
*Description*

    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,)',
 '(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) 

[jira] [Updated] (HDFS-15869) Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang

2021-03-02 Thread Haoze Wu (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15869?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Haoze Wu updated HDFS-15869:

Description: 
*Description*

    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,)',
 '(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) 

[jira] [Updated] (HDFS-15869) Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang

2021-03-02 Thread Haoze Wu (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15869?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Haoze Wu updated HDFS-15869:

Description: 
*Description*

    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). We found that this issue is similar to 
HDFS-15486 in the old version Hadoop, but we have a more comprehensive study on 
it in the latest stable release 3.2.2.

    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,)',
 '(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}

[jira] [Updated] (HDFS-15869) Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang

2021-03-02 Thread Haoze Wu (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15869?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Haoze Wu updated HDFS-15869:

Description: 
*Description*

    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). We found that this issue is similar to 
HDFS-15486 in the old version Hadoop, but we have a more comprehensive study on 
it in the latest stable release 3.2.2.

    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,)',
 '(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}

[jira] [Updated] (HDFS-15869) Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang

2021-03-02 Thread Haoze Wu (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15869?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Haoze Wu updated HDFS-15869:

Description: 
*Description*

    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). We found that this issue is similar to 
HDFS-15486, but we have a more comprehensive study here.

    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,)',
 '(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

  

[jira] [Updated] (HDFS-15869) Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang

2021-03-02 Thread Haoze Wu (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15869?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Haoze Wu updated HDFS-15869:

Description: 
*Description*

    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). We found that this issue is similar to 
HDFS-15486, but we have a more comprehensive study here.

    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,)',
 '(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

  

[jira] [Updated] (HDFS-15869) Network issue while FSEditLogAsync is executing RpcEdit.logSyncNotify can cause the namenode to hang

2021-03-02 Thread Haoze Wu (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-15869?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Haoze Wu updated HDFS-15869:

Description: 
*Description*

    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). We found that this issue is similar to 
HDFS-15486, but we have a more comprehensive study here.

    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,)',
 '(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