[ https://issues.apache.org/jira/browse/HDFS-16507?focusedWorklogId=747574&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-747574 ]
ASF GitHub Bot logged work on HDFS-16507: ----------------------------------------- Author: ASF GitHub Bot Created on: 25/Mar/22 02:00 Start Date: 25/Mar/22 02:00 Worklog Time Spent: 10m Work Description: tomscut commented on a change in pull request #4082: URL: https://github.com/apache/hadoop/pull/4082#discussion_r834896270 ########## File path: hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java ########## @@ -1509,13 +1509,18 @@ synchronized void abortCurrentLogSegment() { * effect. */ @Override - public synchronized void purgeLogsOlderThan(final long minTxIdToKeep) { + public synchronized void purgeLogsOlderThan(long minTxIdToKeep) { // Should not purge logs unless they are open for write. // This prevents the SBN from purging logs on shared storage, for example. if (!isOpenForWrite()) { return; } - + + // Reset purgeLogsFrom to avoid purging edit log which is in progress. + if (isSegmentOpen()) { + minTxIdToKeep = minTxIdToKeep > curSegmentTxId ? curSegmentTxId : minTxIdToKeep; Review comment: Hi @jojochuang @Hexiaoqiao @ayushtkn , please also take a look. Thank you very much. This problem begin from inprogress edits tail. And this issue [HDFS-14317](https://issues.apache.org/jira/browse/HDFS-14317) does a good job of avoiding this problem. However, if SNN's rolledit operation is disabled accidentally by configuration, and ANN's automatic roll period is very long, then edit log which is in progress may also be purged. Although we add assertions, assertion is generally disabled in a production. This bug also proves that we are not strictly ensure`(inTxIdToKeep <= curSegmentTxId)`. So it is dangerous for NameNode. We should reset `minTxIdToKeep` to ensure that the in progress edit log is not purged very strict. And wait for ANN to automatically roll to finalize the edit log. Then, after checkpoint, ANN automatically purged the finalized editlog(See the stack mentioned above). -- 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. To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 747574) Time Spent: 2h 40m (was: 2.5h) > [SBN read] Avoid purging edit log which is in progress > ------------------------------------------------------ > > Key: HDFS-16507 > URL: https://issues.apache.org/jira/browse/HDFS-16507 > Project: Hadoop HDFS > Issue Type: Bug > Affects Versions: 3.1.0 > Reporter: tomscut > Priority: Critical > Labels: pull-request-available > Time Spent: 2h 40m > Remaining Estimate: 0h > > We introduced [Standby Read] feature in branch-3.1.0, but found a FATAL > exception. It looks like it's purging edit logs which is in process. > According to the analysis, I suspect that the editlog which is in progress to > be purged(after SNN checkpoint) does not finalize(See HDFS-14317) before ANN > rolls edit its self. > The stack: > {code:java} > java.lang.Thread.getStackTrace(Thread.java:1552) > org.apache.hadoop.util.StringUtils.getStackTrace(StringUtils.java:1032) > > org.apache.hadoop.hdfs.server.namenode.FileJournalManager.purgeLogsOlderThan(FileJournalManager.java:185) > > org.apache.hadoop.hdfs.server.namenode.JournalSet$5.apply(JournalSet.java:623) > > org.apache.hadoop.hdfs.server.namenode.JournalSet.mapJournalsAndReportErrors(JournalSet.java:388) > > org.apache.hadoop.hdfs.server.namenode.JournalSet.purgeLogsOlderThan(JournalSet.java:620) > > org.apache.hadoop.hdfs.server.namenode.FSEditLog.purgeLogsOlderThan(FSEditLog.java:1512) > org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager.purgeOldStorage(NNStorageRetentionManager.java:177) > > org.apache.hadoop.hdfs.server.namenode.FSImage.purgeOldStorage(FSImage.java:1249) > > org.apache.hadoop.hdfs.server.namenode.ImageServlet$2.run(ImageServlet.java:617) > > org.apache.hadoop.hdfs.server.namenode.ImageServlet$2.run(ImageServlet.java:516) > java.security.AccessController.doPrivileged(Native Method) > javax.security.auth.Subject.doAs(Subject.java:422) > > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1730) > > org.apache.hadoop.hdfs.server.namenode.ImageServlet.doPut(ImageServlet.java:515) > javax.servlet.http.HttpServlet.service(HttpServlet.java:710) > javax.servlet.http.HttpServlet.service(HttpServlet.java:790) > org.eclipse.jetty.servlet.ServletHolder.handle(ServletHolder.java:848) > > org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1772) > > org.apache.hadoop.http.HttpServer2$QuotingInputFilter.doFilter(HttpServer2.java:1604) > > org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1759) > org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45) > > org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1759) > org.eclipse.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:582) > > org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:143) > > org.eclipse.jetty.security.SecurityHandler.handle(SecurityHandler.java:548) > > org.eclipse.jetty.server.session.SessionHandler.doHandle(SessionHandler.java:226) > > org.eclipse.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1180) > org.eclipse.jetty.servlet.ServletHandler.doScope(ServletHandler.java:512) > > org.eclipse.jetty.server.session.SessionHandler.doScope(SessionHandler.java:185) > > org.eclipse.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1112) > > org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:141) > > org.eclipse.jetty.server.handler.HandlerCollection.handle(HandlerCollection.java:119) > > org.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:134) > org.eclipse.jetty.server.Server.handle(Server.java:539) > org.eclipse.jetty.server.HttpChannel.handle(HttpChannel.java:333) > > org.eclipse.jetty.server.HttpConnection.onFillable(HttpConnection.java:251) > > org.eclipse.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:283) > org.eclipse.jetty.io.FillInterest.fillable(FillInterest.java:108) > > org.eclipse.jetty.io.SelectChannelEndPoint$2.run(SelectChannelEndPoint.java:93) > > org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.executeProduceConsume(ExecuteProduceConsume.java:303) > > org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.produceConsume(ExecuteProduceConsume.java:148) > > org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.run(ExecuteProduceConsume.java:136) > > org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671) > > org.eclipse.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589) > java.lang.Thread.run(Thread.java:745) {code} > > I post some key logs for your reference: > 1. ANN. Create editlog, > {color:#ff0000}edits_InProgress_0000000000024207987{color}. > {code:java} > 2022-03-15 17:24:52,558 INFO namenode.FSEditLog > (FSEditLog.java:startLogSegment(1394)) - Starting log segment at 24207987 > 2022-03-15 17:24:52,609 INFO namenode.FSEditLog > (FSEditLog.java:startLogSegment(1423)) - Ending log segment at 24207987 > 2022-03-15 17:24:52,610 INFO namenode.FSEditLog > (FSEditLog.java:startLogSegmentAndWriteHeaderTxn(1432)) - logEdit at 24207987 > 2022-03-15 17:24:52,624 INFO namenode.FSEditLog > (FSEditLog.java:startLogSegmentAndWriteHeaderTxn(1434)) - logSync at 24207987 > {code} > 2. SNN. Checkpoint. > The oldest image file is: fsimage_00000000000{color:#de350b}25892513{color}. > {color:#ff0000}25892513 + 1 - 1000000 = 24892514{color} > {color:#ff0000}dfs.namenode.num.extra.edits.retained=1000000{color} > {color:#172b4d}Code: {color}NNStorageRetentionManager#purgeOldStorage > {code:java} > void purgeOldStorage(NameNodeFile nnf) throws IOException { > FSImageTransactionalStorageInspector inspector = > new FSImageTransactionalStorageInspector(EnumSet.of(nnf)); > storage.inspectStorageDirs(inspector); > long minImageTxId = getImageTxIdToRetain(inspector); > purgeCheckpointsOlderThan(inspector, minImageTxId); > > {code} > {color:#910091}...{color} > {code:java} > long minimumRequiredTxId = minImageTxId + 1; > long purgeLogsFrom = Math.max(0, minimumRequiredTxId - > numExtraEditsToRetain); > > ArrayList<EditLogInputStream> editLogs = new > ArrayList<EditLogInputStream>(); > purgeableLogs.selectInputStreams(editLogs, purgeLogsFrom, false, false); > Collections.sort(editLogs, new Comparator<EditLogInputStream>() { > @Override > public int compare(EditLogInputStream a, EditLogInputStream b) { > return ComparisonChain.start() > .compare(a.getFirstTxId(), b.getFirstTxId()) > .compare(a.getLastTxId(), b.getLastTxId()) > .result(); > } > }); > // Remove from consideration any edit logs that are in fact required. > while (editLogs.size() > 0 && > editLogs.get(editLogs.size() - 1).getFirstTxId() >= > minimumRequiredTxId) { > editLogs.remove(editLogs.size() - 1); > } > > // Next, adjust the number of transactions to retain if doing so would mean > // keeping too many segments around. > while (editLogs.size() > maxExtraEditsSegmentsToRetain) { > purgeLogsFrom = editLogs.get(0).getLastTxId() + 1; > editLogs.remove(0); > } > ... > purgeableLogs.purgeLogsOlderThan(purgeLogsFrom); > }{code} > > {code:java} > 2022-03-15 17:28:02,640 INFO ha.StandbyCheckpointer > (StandbyCheckpointer.java:doWork(443)) - Triggering checkpoint because there > have been 1189661 txns since the last checkpoint, which exceeds the > configured threshold 20000 > 2022-03-15 17:28:02,648 INFO namenode.FSImage > (FSEditLogLoader.java:loadFSEdits(188)) - Edits file > ByteStringEditLog[27082175, 27082606], ByteStringEditLog[27082175, 27082606], > ByteStringEditLog[27082175, 27082606] of size 60008 edits # 432 loaded in 0 > seconds > 2022-03-15 17:28:02,649 INFO namenode.FSImage > (FSImage.java:saveNamespace(1121)) - Save namespace ... > 2022-03-15 17:28:02,650 INFO namenode.FSImageFormatProtobuf > (FSImageFormatProtobuf.java:save(718)) - Saving image file > /data/hadoop/hdfs/namenode/current/fsimage.ckpt_0000000000027082606 using no > compression > 2022-03-15 17:28:03,180 INFO namenode.FSImageFormatProtobuf > (FSImageFormatProtobuf.java:save(722)) - Image file > /data/hadoop/hdfs/namenode/current/fsimage.ckpt_0000000000027082606 of size > 17885002 bytes saved in 0 seconds . > 2022-03-15 17:28:03,183 INFO namenode.NNStorageRetentionManager > (NNStorageRetentionManager.java:getImageTxIdToRetain(211)) - Going to retain > 2 images with txid >= 25892513 > 2022-03-15 17:28:03,183 INFO namenode.NNStorageRetentionManager > (NNStorageRetentionManager.java:purgeImage(233)) - Purging old image > FSImageFile(file=/data/hadoop/hdfs/namenode/current/fsimage_0000000000024794305, > cpktTxId=0000000000024794305) > 2022-03-15 17:28:03,188 INFO namenode.NNStorageRetentionManager > (NNStorageRetentionManager.java:purgeOldStorage(169)) - purgeLogsFrom: > 24892514 > 2022-03-15 17:28:03,282 INFO namenode.TransferFsImage > (TransferFsImage.java:copyFileToStream(396)) - Sending fileName: > /data/hadoop/hdfs/namenode/current/fsimage_0000000000027082606, fileSize: > 17885002. Sent total: 17885002 bytes. Size of last segment intended to send: > -1 bytes. > 2022-03-15 17:28:03,536 INFO namenode.TransferFsImage > (TransferFsImage.java:uploadImageFromStorage(240)) - Uploaded image with txid > 27082606 to namenode at http://sg-test-ambari-nn1.bigdata.bigo.inner:50070 in > 0.343 seconds > 2022-03-15 17:28:03,640 INFO namenode.TransferFsImage > (TransferFsImage.java:copyFileToStream(396)) - Sending fileName: > /data/hadoop/hdfs/namenode/current/fsimage_0000000000027082606, fileSize: > 17885002. Sent total: 17885002 bytes. Size of last segment intended to send: > -1 bytes. > 2022-03-15 17:28:03,684 INFO namenode.TransferFsImage > (TransferFsImage.java:uploadImageFromStorage(240)) - Uploaded image with txid > 27082606 to namenode at http://sg-test-ambari-dn1.bigdata.bigo.inner:50070 in > 0.148 seconds > 2022-03-15 17:28:03,748 INFO namenode.TransferFsImage > (TransferFsImage.java:copyFileToStream(396)) - Sending fileName: > /data/hadoop/hdfs/namenode/current/fsimage_0000000000027082606, fileSize: > 17885002. Sent total: 17885002 bytes. Size of last segment intended to send: > -1 bytes. > 2022-03-15 17:28:03,798 INFO namenode.TransferFsImage > (TransferFsImage.java:uploadImageFromStorage(240)) - Uploaded image with txid > 27082606 to namenode at http://sg-test-ambari-dn2.bigdata.bigo.inner:50070 in > 0.113 seconds > 2022-03-15 17:28:03,798 INFO ha.StandbyCheckpointer > (StandbyCheckpointer.java:doWork(482)) - Checkpoint finished successfully. > {code} > 3. ANN. Purge edit logs. > The oldest image file is: fsimage_00000000000{color:#de350b}25892513{color}. > {color:#ff0000}25892513 + 1 - 1000000 = 24892514{color} > {color:#ff0000}dfs.namenode.num.extra.edits.retained=1000000{color} > {code:java} > 2022-03-15 17:28:03,515 INFO namenode.NNStorageRetentionManager > (NNStorageRetentionManager.java:getImageTxIdToRetain(211)) - Going to retain > 2 images with txid >= 25892513 {code} > {code:java} > 2022-03-15 17:28:03,523 INFO namenode.NNStorageRetentionManager > (NNStorageRetentionManager.java:purgeOldStorage(169)) - purgeLogsFrom: > 24892514 2022-03-15 17:28:03,523 INFO client.QuorumJournalManager > (QuorumJournalManager.java:purgeLogsOlderThan(458)) - Purging remote journals > older than txid 24892514 2022-03-15 17:28:03,524 INFO > namenode.FileJournalManager (FileJournalManager.java:purgeLogsOlderThan(184)) > - Purging logs older than 24892514 > {code} > *4. ANN. Purge {color:#ff0000}edits_inprogress_0000000000024207987{color}.* > {code:java} > 2022-03-15 17:28:03,525 INFO namenode.NNStorageRetentionManager > (NNStorageRetentionManager.java:purgeLog(227)) - Purging old edit log > EditLogFile(file=/data/hadoop/hdfs/namenode/current/edits_inprogress_0000000000024207987,first=0000000000024207987,last=-000000000000012345,inProgress=true,hasCorruptHeader=false) > {code} > 5. JN. Purge edit logs include {color:#ff0000}24207987{color}. > {color:#ff0000}No log file to finalize at transaction ID 24207987{color} > {code:java} > 2022-03-15 17:28:03,530 INFO common.Storage > (JNStorage.java:purgeMatching(196)) - Purging no-longer needed file 24207986 > 2022-03-15 17:28:03,564 INFO common.Storage > (JNStorage.java:purgeMatching(196)) - Purging no-longer needed file 24207987 > 2022-03-15 17:28:03,571 INFO common.Storage > (JNStorage.java:purgeMatching(196)) - Purging no-longer needed file 24207981 > 2022-03-15 17:28:03,572 INFO common.Storage > (JNStorage.java:purgeMatching(196)) - Purging no-longer needed file 24207984 > 2022-03-15 17:28:03,576 INFO common.Storage > (JNStorage.java:purgeMatching(196)) - Purging no-longer needed file 24207982 > org.apache.hadoop.hdfs.qjournal.protocol.JournalOutOfSyncException: No log > file to finalize at transaction ID 24207987 ; journal id: ambari-test > {code} > 6. ANN. Process exits because of the fatal exception from JNs. > {code:java} > 2022-03-15 17:28:52,709 INFO namenode.FSNamesystem > (FSNamesystem.java:run(4232)) - NameNode rolling its own edit log because > number of edits in open segment exceeds threshold of 100000 > 2022-03-15 17:28:52,709 INFO namenode.FSEditLog > (FSEditLog.java:rollEditLog(1346)) - Rolling edit logs > 2022-03-15 17:28:52,709 INFO namenode.FSEditLog > (FSEditLog.java:endCurrentLogSegment(1442)) - Ending log segment 24207987, > 27990691 > 2022-03-15 17:28:52,748 INFO namenode.FSEditLog > (FSEditLog.java:printStatistics(806)) - Number of transactions: 3782706 Total > time for transactions(ms): 25003 Number of transactions batched in Syncs: > 3774095 Number of syncs: 8611 SyncTimes(ms): 11347 151932 > 2022-03-15 17:28:52,867 FATAL namenode.FSEditLog > (JournalSet.java:mapJournalsAndReportErrors(393)) - Error: finalize log > segment 24207987, 27990692 failed for required journal > (JournalAndStream(mgr=QJM > to [10.152.106.55:8485, 10.152.115.74:8485, 10.152.124.150:8485, > 10.152.124.157:8485, 10.152.106.57:8485], stream=null)) > org.apache.hadoop.hdfs.qjournal.client.QuorumException: Got too many > exceptions to achieve quorum size 3/5. 5 exceptions thrown: > 10.152.124.157:8485: No log file to finalize at transaction ID 24207987 ; > journal id: ambari-test > at > org.apache.hadoop.hdfs.qjournal.server.Journal.finalizeLogSegment(Journal.java:656) > at > org.apache.hadoop.hdfs.qjournal.server.JournalNodeRpcServer.finalizeLogSegment(JournalNodeRpcServer.java:210) > at > org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolServerSideTranslatorPB.finalizeLogSegment(QJournalProtocolServerSideTranslatorPB.java:205) > at > org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos$QJournalProtocolService$2.callBlockingMethod(QJournalProtocolProtos.java:28890) > at > org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:550) > at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1094) > at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:1066) > at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:1000) > 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:1730) > at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2989) {code} -- This message was sent by Atlassian Jira (v8.20.1#820001) --------------------------------------------------------------------- To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org