Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Support skip invalid journal record in replying journal stage #3956

Merged
merged 7 commits into from
Jun 19, 2023

Conversation

hangc0276
Copy link
Contributor

@hangc0276 hangc0276 commented May 15, 2023

Motivation

This PR is generated from #3437
Fix #2220

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.

@hangc0276
Copy link
Contributor Author

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

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Overall LGTM, but I would set the property to true by default

* @see #isSkipReplayJournalInvalidRecord .
*/
public boolean isSkipReplayJournalInvalidRecord() {
return this.getBoolean(SKIP_REPLAY_JOURNAL_INVALID_RECORD, false);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Comment on lines 831 to 839
try {
if (len == PADDING_MASK && journalVersion >= JournalChannel.V5) {
// skip padding bytes
lenBuff.clear();
fullRead(recLog, lenBuff);
if (lenBuff.remaining() != 0) {
break;
}
lenBuff.flip();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

            while (true) {
                // entry start offset
                long offset = recLog.fc.position();
                // start reading entry
                lenBuff.clear();
                fullRead(recLog, lenBuff);

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I caught all the fullRead throw exceptions.

Copy link
Member

@wenbingshen wenbingshen left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@wenbingshen
Copy link
Member

rerun failure checks

1 similar comment
@hangc0276
Copy link
Contributor Author

rerun failure checks

@frankjkelly
Copy link

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

@hangc0276 hangc0276 closed this Jun 16, 2023
@hangc0276 hangc0276 reopened this Jun 16, 2023
@zymap
Copy link
Member

zymap commented Jun 19, 2023

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

@hangc0276
Copy link
Contributor Author

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.

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

@eolivelli eolivelli merged commit 5e9fdc2 into apache:master Jun 19, 2023
16 checks passed
@frankjkelly
Copy link

Thanks everyone for this - much appreciated

hangc0276 added a commit to hangc0276/bookkeeper that referenced this pull request Jun 26, 2023
…#3956)

Co-authored-by: zhiyuanlei <zhiyuanlei@tencent.com>
(cherry picked from commit 5e9fdc2)
zymap pushed a commit that referenced this pull request Aug 29, 2023
Co-authored-by: zhiyuanlei <zhiyuanlei@tencent.com>
(cherry picked from commit 5e9fdc2)
zymap pushed a commit that referenced this pull request Dec 6, 2023
Co-authored-by: zhiyuanlei <zhiyuanlei@tencent.com>
(cherry picked from commit 5e9fdc2)
@vitalii-buchyn-exa
Copy link

hello community,
any documentation update for this feature?

Ghatage pushed a commit to sijie/bookkeeper that referenced this pull request Jul 12, 2024
…#3956)


Co-authored-by: zhiyuanlei <zhiyuanlei@tencent.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Exception while replaying journals, shutting down
6 participants