You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by "hangc0276 (via GitHub)" <gi...@apache.org> on 2023/05/15 11:20:39 UTC

[GitHub] [bookkeeper] hangc0276 opened a new pull request, #3956: Support skip invalid journal record in replying journal stage

hangc0276 opened a new pull request, #3956:
URL: https://github.com/apache/bookkeeper/pull/3956

   ### Motivation
   This PR is generated from https://github.com/apache/bookkeeper/pull/3437
   
   ### Changes
   When we use `kill -9` command to stop one bookie pod, the journal file may be broken. If we try to start up the bookie pod again, the pod will startup to fail with the following exception.
   ```
   10:15:55.026 [main] ERROR org.apache.bookkeeper.bookie.Bookie - Exception while replaying journals, shutting down
   java.io.IOException: Invalid record found with negative length -448299468
           at org.apache.bookkeeper.bookie.Journal.scanJournal(Journal.java:821) ~[org.apache.bookkeeper-bookkeeper-server-4.12.0.jar:4.12.0]
           at org.apache.bookkeeper.bookie.Bookie.replay(Bookie.java:945) ~[org.apache.bookkeeper-bookkeeper-server-4.12.0.jar:4.12.0]
           at org.apache.bookkeeper.bookie.Bookie.readJournal(Bookie.java:911) ~[org.apache.bookkeeper-bookkeeper-server-4.12.0.jar:4.12.0]
           at org.apache.bookkeeper.bookie.Bookie.start(Bookie.java:965) ~[org.apache.bookkeeper-bookkeeper-server-4.12.0.jar:4.12.0]
           at org.apache.bookkeeper.proto.BookieServer.start(BookieServer.java:156) ~[org.apache.bookkeeper-bookkeeper-server-4.12.0.jar:4.12.0]
           at org.apache.bookkeeper.server.service.BookieService.doStart(BookieService.java:68) ~[org.apache.bookkeeper-bookkeeper-server-4.12.0.jar:4.12.0]
           at org.apache.bookkeeper.common.component.AbstractLifecycleComponent.start(AbstractLifecycleComponent.java:83) ~[org.apache.bookkeeper-bookkeeper-common-4.12.0.jar:4.12.0]
           at org.apache.bookkeeper.common.component.LifecycleComponentStack.lambda$start$4(LifecycleComponentStack.java:144) ~[org.apache.bookkeeper-bookkeeper-common-4.12.0.jar:4.12.0]
           at com.google.common.collect.ImmutableList.forEach(ImmutableList.java:406) [com.google.guava-guava-30.1-jre.jar:?]
           at org.apache.bookkeeper.common.component.LifecycleComponentStack.start(LifecycleComponentStack.java:144) [org.apache.bookkeeper-bookkeeper-common-4.12.0.jar:4.12.0]
           at org.apache.bookkeeper.common.component.ComponentStarter.startComponent(ComponentStarter.java:85) [org.apache.bookkeeper-bookkeeper-common-4.12.0.jar:4.12.0]
           at org.apache.bookkeeper.server.Main.doMain(Main.java:234) [org.apache.bookkeeper-bookkeeper-server-4.12.0.jar:4.12.0]
           at org.apache.bookkeeper.server.Main.main(Main.java:208) [org.apache.bookkeeper-bookkeeper-server-4.12.0.jar:4.12.0]
   10:15:55.134 [main] INFO  org.apache.zookeeper.ZooKeeper - Session: 0x200064a14681be2 closed
   ```
   
   We should have a way to make the bookie pod startup instead of decommissioning it.
   
   
   ### Changes
   1. Add a configuration to allow skipping the invalid entries when journal replying to journal files.
   


-- 
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: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] hangc0276 commented on a diff in pull request #3956: Support skip invalid journal record in replying journal stage

Posted by "hangc0276 (via GitHub)" <gi...@apache.org>.
hangc0276 commented on code in PR #3956:
URL: https://github.com/apache/bookkeeper/pull/3956#discussion_r1193852701


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java:
##########
@@ -4010,6 +4012,25 @@ public boolean isDataIntegrityStampMissingCookiesEnabled() {
         return this.getBoolean(DATA_INTEGRITY_COOKIE_STAMPING_ENABLED, false);
     }
 
+
+    /**
+     * When this config is set to true,if we replay journal failed, we will skip.
+     * @param skipReplayJournalInvalidRecord
+     * @return
+     */
+    public ServerConfiguration setSkipReplayJournalInvalidRecord(boolean skipReplayJournalInvalidRecord) {
+        this.setProperty(SKIP_REPLAY_JOURNAL_INVALID_RECORD,
+                Boolean.toString(skipReplayJournalInvalidRecord));
+        return this;
+    }
+
+    /**
+     * @see #isSkipReplayJournalInvalidRecord .
+     */
+    public boolean isSkipReplayJournalInvalidRecord() {
+        return this.getBoolean(SKIP_REPLAY_JOURNAL_INVALID_RECORD, false);

Review Comment:
   The previous behavior is to throw the exception out and block the bookie start-up. We should keep the same behavior.



-- 
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: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] zymap commented on pull request #3956: Support skip invalid journal record in replying journal stage

Posted by "zymap (via GitHub)" <gi...@apache.org>.
zymap commented on PR #3956:
URL: https://github.com/apache/bookkeeper/pull/3956#issuecomment-1596354121

   @eolivelli Would you like to take another look at this PR? 


-- 
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: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] wenbingshen commented on pull request #3956: Support skip invalid journal record in replying journal stage

Posted by "wenbingshen (via GitHub)" <gi...@apache.org>.
wenbingshen commented on PR #3956:
URL: https://github.com/apache/bookkeeper/pull/3956#issuecomment-1567712831

   rerun failure checks


-- 
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: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] hangc0276 commented on a diff in pull request #3956: Support skip invalid journal record in replying journal stage

Posted by "hangc0276 (via GitHub)" <gi...@apache.org>.
hangc0276 commented on code in PR #3956:
URL: https://github.com/apache/bookkeeper/pull/3956#discussion_r1200244274


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java:
##########
@@ -4010,6 +4012,25 @@ public boolean isDataIntegrityStampMissingCookiesEnabled() {
         return this.getBoolean(DATA_INTEGRITY_COOKIE_STAMPING_ENABLED, false);
     }
 
+
+    /**
+     * When this config is set to true,if we replay journal failed, we will skip.
+     * @param skipReplayJournalInvalidRecord
+     * @return
+     */
+    public ServerConfiguration setSkipReplayJournalInvalidRecord(boolean skipReplayJournalInvalidRecord) {
+        this.setProperty(SKIP_REPLAY_JOURNAL_INVALID_RECORD,
+                Boolean.toString(skipReplayJournalInvalidRecord));
+        return this;
+    }
+
+    /**
+     * @see #isSkipReplayJournalInvalidRecord .
+     */
+    public boolean isSkipReplayJournalInvalidRecord() {
+        return this.getBoolean(SKIP_REPLAY_JOURNAL_INVALID_RECORD, false);

Review Comment:
   Even though the previous default behavior is `false`, I changed the default value to `true`, because I think it's valuable to skip the the invalid record instead of make the bookie node can't start up.



-- 
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: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] hangc0276 closed pull request #3956: Support skip invalid journal record in replying journal stage

Posted by "hangc0276 (via GitHub)" <gi...@apache.org>.
hangc0276 closed pull request #3956: Support skip invalid journal record in replying journal stage
URL: https://github.com/apache/bookkeeper/pull/3956


-- 
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: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] frankjkelly commented on pull request #3956: Support skip invalid journal record in replying journal stage

Posted by "frankjkelly (via GitHub)" <gi...@apache.org>.
frankjkelly commented on PR #3956:
URL: https://github.com/apache/bookkeeper/pull/3956#issuecomment-1590035221

   Hi folks - thanks for the efforts here - what are the remaining issues on this? Thanks! 


-- 
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: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] eolivelli merged pull request #3956: Support skip invalid journal record in replying journal stage

Posted by "eolivelli (via GitHub)" <gi...@apache.org>.
eolivelli merged PR #3956:
URL: https://github.com/apache/bookkeeper/pull/3956


-- 
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: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] hangc0276 commented on pull request #3956: Support skip invalid journal record in replying journal stage

Posted by "hangc0276 (via GitHub)" <gi...@apache.org>.
hangc0276 commented on PR #3956:
URL: https://github.com/apache/bookkeeper/pull/3956#issuecomment-1547674507

   @frankjkelly I have created a new PR to fix the bookie that can't startup due to the journal reply failing.


-- 
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: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] wenbingshen commented on a diff in pull request #3956: Support skip invalid journal record in replying journal stage

Posted by "wenbingshen (via GitHub)" <gi...@apache.org>.
wenbingshen commented on code in PR #3956:
URL: https://github.com/apache/bookkeeper/pull/3956#discussion_r1198624139


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java:
##########
@@ -827,22 +828,32 @@ public long scanJournal(long journalId, long journalPos, JournalScanner scanner)
                 }
                 boolean isPaddingRecord = false;
                 if (len < 0) {
-                    if (len == PADDING_MASK && journalVersion >= JournalChannel.V5) {
-                        // skip padding bytes
-                        lenBuff.clear();
-                        fullRead(recLog, lenBuff);
-                        if (lenBuff.remaining() != 0) {
-                            break;
+                    try {
+                        if (len == PADDING_MASK && journalVersion >= JournalChannel.V5) {
+                            // skip padding bytes
+                            lenBuff.clear();
+                            fullRead(recLog, lenBuff);
+                            if (lenBuff.remaining() != 0) {
+                                break;
+                            }
+                            lenBuff.flip();

Review Comment:
   line 820 may throw IOException, this may cause bookie startup failed ==>  fullRead(recLog, lenBuff);
   In this PR, line 835 also may throw IOException, because this future the bookie can continue starting.
   Same behavior here, two results, right?
   
   https://github.com/apache/bookkeeper/pull/3956/files#diff-0a79d54e3dfd4bab1d85510ef957f8f4b7ab6f6acd09aadd634b4d7bbee6e1f4R815-R821
   ```java
               while (true) {
                   // entry start offset
                   long offset = recLog.fc.position();
                   // start reading entry
                   lenBuff.clear();
                   fullRead(recLog, lenBuff);
   ```



-- 
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: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Support skip invalid journal record in replying journal stage [bookkeeper]

Posted by "vitalii-buchyn-exa (via GitHub)" <gi...@apache.org>.
vitalii-buchyn-exa commented on PR #3956:
URL: https://github.com/apache/bookkeeper/pull/3956#issuecomment-1867818800

   hello community,
   any documentation update for this feature?


-- 
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: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] hangc0276 commented on pull request #3956: Support skip invalid journal record in replying journal stage

Posted by "hangc0276 (via GitHub)" <gi...@apache.org>.
hangc0276 commented on PR #3956:
URL: https://github.com/apache/bookkeeper/pull/3956#issuecomment-1596629600

   > Hi folks - thanks for the efforts here - what are the remaining issues on this? Thanks!
   
   @frankjkelly I addressed all the comments, please help take a look, thanks.


-- 
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: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] frankjkelly commented on pull request #3956: Support skip invalid journal record in replying journal stage

Posted by "frankjkelly (via GitHub)" <gi...@apache.org>.
frankjkelly commented on PR #3956:
URL: https://github.com/apache/bookkeeper/pull/3956#issuecomment-1600871092

   Thanks everyone for this - much appreciated


-- 
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: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] hangc0276 commented on a diff in pull request #3956: Support skip invalid journal record in replying journal stage

Posted by "hangc0276 (via GitHub)" <gi...@apache.org>.
hangc0276 commented on code in PR #3956:
URL: https://github.com/apache/bookkeeper/pull/3956#discussion_r1200242610


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java:
##########
@@ -827,22 +828,32 @@ public long scanJournal(long journalId, long journalPos, JournalScanner scanner)
                 }
                 boolean isPaddingRecord = false;
                 if (len < 0) {
-                    if (len == PADDING_MASK && journalVersion >= JournalChannel.V5) {
-                        // skip padding bytes
-                        lenBuff.clear();
-                        fullRead(recLog, lenBuff);
-                        if (lenBuff.remaining() != 0) {
-                            break;
+                    try {
+                        if (len == PADDING_MASK && journalVersion >= JournalChannel.V5) {
+                            // skip padding bytes
+                            lenBuff.clear();
+                            fullRead(recLog, lenBuff);
+                            if (lenBuff.remaining() != 0) {
+                                break;
+                            }
+                            lenBuff.flip();

Review Comment:
   I caught all the `fullRead` throw exceptions.



-- 
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: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] hangc0276 commented on pull request #3956: Support skip invalid journal record in replying journal stage

Posted by "hangc0276 (via GitHub)" <gi...@apache.org>.
hangc0276 commented on PR #3956:
URL: https://github.com/apache/bookkeeper/pull/3956#issuecomment-1567956019

   rerun failure checks


-- 
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: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] eolivelli commented on a diff in pull request #3956: Support skip invalid journal record in replying journal stage

Posted by "eolivelli (via GitHub)" <gi...@apache.org>.
eolivelli commented on code in PR #3956:
URL: https://github.com/apache/bookkeeper/pull/3956#discussion_r1193813416


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java:
##########
@@ -4010,6 +4012,25 @@ public boolean isDataIntegrityStampMissingCookiesEnabled() {
         return this.getBoolean(DATA_INTEGRITY_COOKIE_STAMPING_ENABLED, false);
     }
 
+
+    /**
+     * When this config is set to true,if we replay journal failed, we will skip.
+     * @param skipReplayJournalInvalidRecord
+     * @return
+     */
+    public ServerConfiguration setSkipReplayJournalInvalidRecord(boolean skipReplayJournalInvalidRecord) {
+        this.setProperty(SKIP_REPLAY_JOURNAL_INVALID_RECORD,
+                Boolean.toString(skipReplayJournalInvalidRecord));
+        return this;
+    }
+
+    /**
+     * @see #isSkipReplayJournalInvalidRecord .
+     */
+    public boolean isSkipReplayJournalInvalidRecord() {
+        return this.getBoolean(SKIP_REPLAY_JOURNAL_INVALID_RECORD, false);

Review Comment:
   the previous behaviour was to skip errors, we should keep the same value



-- 
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: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org