[ https://issues.apache.org/jira/browse/HDFS-16659?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17599239#comment-17599239 ]
ASF GitHub Bot commented on HDFS-16659: --------------------------------------- ZanderXu commented on code in PR #4560: URL: https://github.com/apache/hadoop/pull/4560#discussion_r961257965 ########## hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java: ########## @@ -1101,6 +1113,59 @@ public void testSelectViaRpcTwoJNsError() throws Exception { } } + /** + * Test selecting EditLogInputStream after some journalNode jitter. + * And the corner case as below: + * 1. Journal 0 has some abnormal cases when journaling Edits with start txId 11. + * 2. NameNode just ignore the abnormal journal 0 and continue to write Edits to Journal 1 and 2. + * 3. Journal 0 backed to health. + * 4. Observer NameNode try to select EditLogInputStream vis PRC with start txId 21. + * 5. Journal 1 has some abnormal cases caused slow response. + * + * And the expected selecting result is: Response should contain 20 Edits from txId 21 to txId 40. + * Because there is no Edits from id 21 to 40 in the cache of JournalNode0. Review Comment: Copy, Sir. I have updated it. > JournalNode should throw NewerTxnIdException if SinceTxId is bigger than > HighestWrittenTxId > ------------------------------------------------------------------------------------------- > > Key: HDFS-16659 > URL: https://issues.apache.org/jira/browse/HDFS-16659 > Project: Hadoop HDFS > Issue Type: Bug > Reporter: ZanderXu > Assignee: ZanderXu > Priority: Critical > Labels: pull-request-available > Time Spent: 1h > Remaining Estimate: 0h > > JournalNode should throw `CacheMissException` if `sinceTxId` is bigger than > `highestWrittenTxId` during handling `getJournaledEdits` rpc from NNs. > Current logic may cause in-progress EditlogTailer cannot replay any Edits > from JournalNodes in some corner cases, resulting in ObserverNameNode cannot > handle requests from clients. > Suppose there are 3 journalNodes, JN0 ~ JN1. > * JN0 has some abnormal cases when Active Namenode is syncing 10 Edits with > first txid 11 > * NameNode just ignore the abnormal JN0 and continue to sync Edits to Journal > 1 and 2 > * JN0 backed to health > * NameNode continue sync 10 Edits with first txid 21. > * At this point, there are no Edits 11 ~ 30 in the cache of JN0 > * Observer NameNode try to select EditLogInputStream through > `getJournaledEdits` with since txId 21 > * Journal 2 has some abnormal cases and caused a slow response > The expected result is: Response should contain 20 Edits from txId 21 to txId > 30 from JN1 and JN2. Because Active NameNode successfully write these Edits > to JN1 and JN2 and failed write these edits to JN0. > But in the current implementation, the response is [Response(0) from JN0, > Response(10) from JN1], because there are some abnormal cases in JN2, such > as GC, bad network, cause a slow response. So the `maxAllowedTxns` will be > 0, NameNode will not replay any Edits. > As above, the root case is that JournalNode should throw Miss Cache Exception > when `sinceTxid` is more than `highestWrittenTxId`. > And the bug code as blew: > {code:java} > if (sinceTxId > getHighestWrittenTxId()) { > // Requested edits that don't exist yet; short-circuit the cache here > metrics.rpcEmptyResponses.incr(); > return > GetJournaledEditsResponseProto.newBuilder().setTxnCount(0).build(); > } > {code} -- 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