You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@bookkeeper.apache.org by Hang Chen <ch...@apache.org> on 2023/05/15 11:32:39 UTC

[DISCUSS] Support skip invalid journal records in replying journal stage

The journal file may be broken when we use the `kill -9` command to
stop one bookie pod. If we try to start up the bookie pod again, the
pod will start 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.

If we skip the broken journal files in the replying stage, it may lead
to some data loss, even though we enabled the fsync flag for the
journal. However, if we don't skip the broken journal files, we need
to decommission the bookie pod to make those ledgers located at the
bookie replicated to other bookie pods.

I think skipping those broken journal files is acceptable compared to
decommissioning the bookie pod. For those lost data, the auto-recovery
can replicate them from other replicas in other bookie pods.

The related PR is: https://github.com/apache/bookkeeper/pull/3956

Thanks,
Hang