Vanlightly commented on issue #2528:
URL: https://github.com/apache/bookkeeper/issues/2528#issuecomment-885479682
Hi,
This is an issue with entry log corruption, I was planning on posting this
very problem this week.
I was investigating an entry log corruption we had in production and ran
long running system tests to try and reproduce. I hit this issue on every
test within a few hours or a couple of days.
Normally you would use the DbLedgerStorage with Pulsar but I had
accidentally set BookKeeper to use SortedLedgerStorage for my tests. When I
switched to DbLedgerStorage I could no longer trigger issues related to
entry log corruption.
I have not nailed down the cause of why SortedLedgerStorage is corrupting
entry log files. I don't have the free capacity to devote to this issue,
given that we only use DbLedgerStorage. But this issue needs fixing
urgently for anyone using SortedLedgerStorage.
To reproduce simply run a test on a real deployed BookKeeper cluster with
SortedLedgerStorage and you will see errors in the garbage collector
related to:
- negative position in buffer
- bad ledger ids
- OutOfMemory exceptions because the entry length is bad (attempts to
allocate buffers as large as 2GB).
Jack
On Fri, Jul 23, 2021 at 5:07 AM choi se ***@***.***> wrote:
> One pulsar-2.8.0 Bookies node was a problem.
> pulsar-2.7.2 downgrade
>
> 18:00:37.715 [SortedLedgerStorage-0] INFO
org.apache.bookkeeper.bookie.EntryLogManagerForSingleEntryLog - Rolling entry
logger since it reached size limitation
> 18:00:37.715 [SortedLedgerStorage-0] INFO
org.apache.bookkeeper.bookie.EntryLogManagerBase - Creating a new entry log
file due to reaching log limit after flushing memtable : logIdBeforeFlush =
12784, logIdAfterFlush = 12784
> 18:00:37.716 [SortedLedgerStorage-0] INFO
org.apache.bookkeeper.bookie.EntryLogManagerBase - Flushing entry logger 12784
back to filesystem, pending for syncing entry loggers :
[BufferedChannel{logId=12784,
logFile=/data/bookkeeper/ledgers/current/31f0.log, ledgerIdAssigned=-1}].
> 18:00:37.716 [SyncThread-7-1] INFO
org.apache.bookkeeper.bookie.EntryLogManagerBase - Creating a new entry log
file due to preparing checkpoint : numBytesFlushed = 119500
> 18:00:37.716 [pool-5-thread-1] INFO
org.apache.bookkeeper.bookie.EntryLoggerAllocator - Created new entry log file
/data/bookkeeper/ledgers/current/31f2.log for logId 12786.
> 18:00:37.717 [SyncThread-7-1] INFO
org.apache.bookkeeper.bookie.EntryLogManagerBase - Flushing entry logger 12785
back to filesystem, pending for syncing entry loggers :
[BufferedChannel{logId=12784,
logFile=/data/bookkeeper/ledgers/current/31f0.log, ledgerIdAssigned=-1},
BufferedChannel{logId=12785, logFile=/data/bookkeeper/ledgers/current/31f1.log,
ledgerIdAssigned=-1}].
> 18:00:37.717 [pool-5-thread-1] INFO
org.apache.bookkeeper.bookie.EntryLoggerAllocator - Created new entry log file
/data/bookkeeper/ledgers/current/31f3.log for logId 12787.
> 18:00:37.757 [SyncThread-7-1] INFO
org.apache.bookkeeper.bookie.EntryLogManagerForSingleEntryLog - Synced entry
logger 12784 to disk.
> 18:00:37.766 [SyncThread-7-1] INFO
org.apache.bookkeeper.bookie.EntryLogManagerForSingleEntryLog - Synced entry
logger 12785 to disk.
> 18:01:02.256 [GarbageCollectorThread-8-1] INFO
org.apache.bookkeeper.bookie.GarbageCollectorThread - Extracting entry log meta
from entryLogId: 11410
> 18:01:02.256 [GarbageCollectorThread-8-1] INFO
org.apache.bookkeeper.bookie.EntryLogger - Failed to get ledgers map index
from: 11410.log : Cannot deserialize ledgers map from ledger 433791696895
> 18:01:02.271 [GarbageCollectorThread-8-1] ERROR
org.apache.bookkeeper.common.util.SafeRunnable - Unexpected throwable caught
> java.lang.IllegalArgumentException: Negative position
> at sun.nio.ch.FileChannelImpl.read(FileChannelImpl.java:785) ~[?:?]
> at
org.apache.bookkeeper.bookie.BufferedReadChannel.read(BufferedReadChannel.java:93)
~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
> at
org.apache.bookkeeper.bookie.BufferedReadChannel.read(BufferedReadChannel.java:65)
~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
> at
org.apache.bookkeeper.bookie.EntryLogger.readFromLogChannel(EntryLogger.java:418)
~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
> at
org.apache.bookkeeper.bookie.EntryLogger.scanEntryLog(EntryLogger.java:996)
~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
> at
org.apache.bookkeeper.bookie.EntryLogger.extractEntryLogMetadataByScanning(EntryLogger.java:1136)
~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
> at
org.apache.bookkeeper.bookie.EntryLogger.getEntryLogMetadata(EntryLogger.java:1045)
~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
> at
org.apache.bookkeeper.bookie.GarbageCollectorThread.extractMetaFromEntryLogs(GarbageCollectorThread.java:607)
~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
> at
org.apache.bookkeeper.bookie.GarbageCollectorThread.runWithFlags(GarbageCollectorThread.java:348)
~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
> at
org.apache.bookkeeper.bookie.GarbageCollectorThread.safeRun(GarbageCollectorThread.java:329)
~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
> at
org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36)
[org.apache.bookkeeper-bookkeeper-common-4.14.1.jar:4.14.1]
> at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) [?:?]
> at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:305)
[?:?]
> at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:305)
[?:?]
> at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
[?:?]
> at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
[?:?]
> at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
[io.netty-netty-common-4.1.63.Final.jar:4.1.63.Final]
> at java.lang.Thread.run(Thread.java:830) [?:?]
>
> —
> You are receiving this because you are subscribed to this thread.
> Reply to this email directly, view it on GitHub
> <https://github.com/apache/bookkeeper/issues/2528#issuecomment-885367602>,
> or unsubscribe
>
<https://github.com/notifications/unsubscribe-auth/ABG3JYHAZJJSBEPMSVRHJSLTZDMFVANCNFSM4VYJQB3Q>
> .
>
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]