[ 
https://issues.apache.org/jira/browse/HDFS-10943?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15542701#comment-15542701
 ] 

Kihwal Lee commented on HDFS-10943:
-----------------------------------

Which version are you looking at?  {{rollEditLog()}} does not and cannot solely 
depend on {{FSEditLog}} synchronization.  That's why it acquires the FSN write 
lock. This blocks everyone from edit logging. While still in the write lock, 
the edits are synced. This is different from regular {{logSync()}}, which 
happens outside the write lock.  

If you are seeing such an exception, something is circumventing the FSN lock 
and edit logging.  There are some places where locking is more complicated like 
use of {{noInterruptsLock}} in {{DelegationTokenSecretManager}}, but they are 
also not supposed to edit log while rolling edits.  The exception clearly 
indicates a bug.  If you can reproduce it, make it dump the buffer before 
terminating so that we can find out who did edit logging.

> rollEditLog expects empty EditsDoubleBuffer.bufCurrent which is not guaranteed
> ------------------------------------------------------------------------------
>
>                 Key: HDFS-10943
>                 URL: https://issues.apache.org/jira/browse/HDFS-10943
>             Project: Hadoop HDFS
>          Issue Type: Bug
>            Reporter: Yongjun Zhang
>
> Per the following trace stack:
> {code}
> FATAL org.apache.hadoop.hdfs.server.namenode.FSEditLog: Error: finalize log 
> segment 10562075963, 10562174157 failed for required journal 
> (JournalAndStream(mgr=QJM to [0.0.0.1:8485, 0.0.0.2:8485, 0.0.0.3:8485, 
> 0.0.0.4:8485, 0.0.0.5:8485], stream=QuorumOutputStream starting at txid 
> 10562075963))
> java.io.IOException: FSEditStream has 49708 bytes still to be flushed and 
> cannot be closed.
>         at 
> org.apache.hadoop.hdfs.server.namenode.EditsDoubleBuffer.close(EditsDoubleBuffer.java:66)
>         at 
> org.apache.hadoop.hdfs.qjournal.client.QuorumOutputStream.close(QuorumOutputStream.java:65)
>         at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet$JournalAndStream.closeStream(JournalSet.java:115)
>         at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet$4.apply(JournalSet.java:235)
>         at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet.mapJournalsAndReportErrors(JournalSet.java:393)
>         at 
> org.apache.hadoop.hdfs.server.namenode.JournalSet.finalizeLogSegment(JournalSet.java:231)
>         at 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.endCurrentLogSegment(FSEditLog.java:1243)
>         at 
> org.apache.hadoop.hdfs.server.namenode.FSEditLog.rollEditLog(FSEditLog.java:1172)
>         at 
> org.apache.hadoop.hdfs.server.namenode.FSImage.rollEditLog(FSImage.java:1243)
>         at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.rollEditLog(FSNamesystem.java:6437)
>         at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.rollEditLog(NameNodeRpcServer.java:1002)
>         at 
> org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB.rollEditLog(NamenodeProtocolServerSideTranslatorPB.java:142)
>         at 
> org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos$NamenodeProtocolService$2.callBlockingMethod(NamenodeProtocolProtos.java:12025)
>         at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:617)
>         at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1060)
>         at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2086)
>         at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2082)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at javax.security.auth.Subject.doAs(Subject.java:422)
>         at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1671)
>         at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2080)
> 2016-09-23 21:40:59,618 WARN 
> org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager: Aborting 
> QuorumOutputStream starting at txid 10562075963
> {code}
> The exception is from  EditsDoubleBuffer
> {code}
>  public void close() throws IOException {
>     Preconditions.checkNotNull(bufCurrent);
>     Preconditions.checkNotNull(bufReady);
>     int bufSize = bufCurrent.size();
>     if (bufSize != 0) {
>       throw new IOException("FSEditStream has " + bufSize
>           + " bytes still to be flushed and cannot be closed.");
>     }
>     IOUtils.cleanup(null, bufCurrent, bufReady);
>     bufCurrent = bufReady = null;
>   }
> {code}
> We can see that FSNamesystem.rollEditLog expects  
> EditsDoubleBuffer.bufCurrent to be empty.
> Edits are recorded via FSEditLog$logSync, which does:
> {code}
>    * The data is double-buffered within each edit log implementation so that
>    * in-memory writing can occur in parallel with the on-disk writing.
>    *
>    * Each sync occurs in three steps:
>    *   1. synchronized, it swaps the double buffer and sets the isSyncRunning
>    *      flag.
>    *   2. unsynchronized, it flushes the data to storage
>    *   3. synchronized, it resets the flag and notifies anyone waiting on the
>    *      sync.
>    *
>    * The lack of synchronization on step 2 allows other threads to continue
>    * to write into the memory buffer while the sync is in progress.
>    * Because this step is unsynchronized, actions that need to avoid
>    * concurrency with sync() should be synchronized and also call
>    * waitForSyncToFinish() before assuming they are running alone.
>    */
> {code}
> We can see that step 2 is on-purposely not synchronized to let other threads 
> to write into the memory buffer, presumbaly EditsDoubleBuffer.bufCurrent. 
> This means that the EditsDoubleBuffer.bufCurrent  can be non-empty when 
> logSync is done.
> Now if rollEditLog happens, the above exception happens.
> Another interesting observation is, the size of the EditsDoubleBuffer can be 
> as large as "private int outputBufferCapacity = 512 * 1024;", which means a 
> lot of edits could get buffered before they are flushed to JNs. 
> How can rollEdit operation expect the EditsDoubleBuffer.bufCurrent to be 
> empty? Or ask in another way, ollEdit operation want to close the stream
> {code}
> org.apache.hadoop.hdfs.server.namenode.JournalSet$JournalAndStream.closeStream(JournalSet.java:115)
> {code}
> where the close operation expects EditsDoubleBuffer.bufCurrent to be empty. 
> It  seems rollEdit should make sure the EditsDoubleBuffer is flushed in a 
> synchronized way before it tries to close the stream. Why it doesn't do that?
> If my above theory is correct, wonder why this issue doesn't show up so often.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to