You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@bookkeeper.apache.org by Enrico Olivelli <eo...@gmail.com> on 2017/09/19 12:42:06 UTC

Cannot compact bookie due to NegativeArraySizeException on BK 4.5

Hi
in one of my staging sites I got this error and the bookie cannot reclaim
disk space

I am running a pure 4.5 BookKeeper bookie
below is the error

Surely it is a broken EntryLog file.
Do you think the bookie could be recoverable ?
I did a backup, it does not contain sensitive data, I could share it

-- Enrico

17-09-19-14-37-22    Unexpected throwable caught
17-09-19-14-37-22    java.lang.NegativeArraySizeException
java.lang.NegativeArraySizeException
    at
org.apache.bookkeeper.bookie.EntryLogger.scanEntryLog(EntryLogger.java:1006)
    at
org.apache.bookkeeper.bookie.EntryLogger.extractEntryLogMetadataByScanning(EntryLogger.java:1118)
    at
org.apache.bookkeeper.bookie.EntryLogger.getEntryLogMetadata(EntryLogger.java:1031)
    at
org.apache.bookkeeper.bookie.GarbageCollectorThread.extractMetaFromEntryLogs(GarbageCollectorThread.java:569)
    at
org.apache.bookkeeper.bookie.GarbageCollectorThread.safeRun(GarbageCollectorThread.java:342)
    at org.apache.bookkeeper.util.SafeRunnable.run(SafeRunnable.java:33)
    at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
    at java.util.concurrent.FutureTask.run(FutureTask.java:266)
    at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
    at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
    at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
    at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
    at java.lang.Thread.run(Thread.java:745)

Re: Cannot compact bookie due to NegativeArraySizeException on BK 4.5

Posted by Enrico Olivelli <eo...@gmail.com>.
Thank you Jia,
The error is clear.
The bookie is in fact able to work but it cannot reclaim space
I am not sure if a good patch would to simply drop the broken file...

Enrico

On mer 20 set 2017, 10:45 Jia Zhai <zh...@gmail.com> wrote:

> Seems the failing place was scan the old EntryLog, not doing write, maybe
> it not broken the EntryLog file.
>
> The error indicates that we read a negative entrySize from entryLog file.
>
> /**
>  * Scan entry log.
>  *
>  * @param entryLogId Entry Log Id
>  * @param scanner Entry Log Scanner
>  * @throws IOException
>  */
> protected void scanEntryLog(long entryLogId, EntryLogScanner scanner) throws IOException {
>     ByteBuffer sizeBuff = ByteBuffer.allocate(4);
>     ByteBuffer lidBuff = ByteBuffer.allocate(8);
>     BufferedReadChannel bc;
>     // Get the BufferedChannel for the current entry log file
>     try {
>         bc = getChannelForLogId(entryLogId);
>     } catch (IOException e) {
>         LOG.warn("Failed to get channel to scan entry log: " + entryLogId + ".log");
>         throw e;
>     }
>     // Start the read position in the current entry log file to be after
>     // the header where all of the ledger entries are.
>     long pos = LOGFILE_HEADER_SIZE;
>
>     // Read through the entry log file and extract the ledger ID's.
>     while (true) {
>         // Check if we've finished reading the entry log file.
>         if (pos >= bc.size()) {
>             break;
>         }
>         if (readFromLogChannel(entryLogId, bc, sizeBuff, pos) != sizeBuff.capacity()) {
>             LOG.warn("Short read for entry size from entrylog {}", entryLogId);
>             return;
>         }
>         long offset = pos;
>         pos += 4;
>         sizeBuff.flip();
>         int entrySize = sizeBuff.getInt();   < === 2, here we get entrySize from buffer.
>
>         sizeBuff.clear();
>         // try to read ledger id first
>         if (readFromLogChannel(entryLogId, bc, lidBuff, pos) != lidBuff.capacity()) {
>             LOG.warn("Short read for ledger id from entrylog {}", entryLogId);
>             return;
>         }
>         lidBuff.flip();
>         long lid = lidBuff.getLong();
>         lidBuff.clear();
>         if (lid == INVALID_LID || !scanner.accept(lid)) {
>             // skip this entry
>             pos += entrySize;
>             continue;
>         }
>         // read the entry
>         byte data[] = new byte[entrySize];   < === 1, here is line 1006, seems it get a negative entrySize.
>         ByteBuffer buff = ByteBuffer.wrap(data);
>         int rc = readFromLogChannel(entryLogId, bc, buff, pos);
>         if (rc != data.length) {
>             LOG.warn("Short read for ledger entry from entryLog {}@{} ({} != {})", new Object[] { entryLogId, pos,
>                     rc, data.length });
>             return;
>         }
>         buff.flip();
>         // process the entry
>         scanner.process(lid, offset, buff);
>         // Advance position to the next entry
>         pos += entrySize;
>     }
> }
>
>
>
>
>
> On Tue, Sep 19, 2017 at 8:42 PM, Enrico Olivelli <eo...@gmail.com>
> wrote:
>
>> Hi
>> in one of my staging sites I got this error and the bookie cannot reclaim
>> disk space
>>
>> I am running a pure 4.5 BookKeeper bookie
>> below is the error
>>
>> Surely it is a broken EntryLog file.
>> Do you think the bookie could be recoverable ?
>> I did a backup, it does not contain sensitive data, I could share it
>>
>> -- Enrico
>>
>> 17-09-19-14-37-22    Unexpected throwable caught
>> 17-09-19-14-37-22    java.lang.NegativeArraySizeException
>> java.lang.NegativeArraySizeException
>>     at
>> org.apache.bookkeeper.bookie.EntryLogger.scanEntryLog(EntryLogger.java:1006)
>>     at
>> org.apache.bookkeeper.bookie.EntryLogger.extractEntryLogMetadataByScanning(EntryLogger.java:1118)
>>     at
>> org.apache.bookkeeper.bookie.EntryLogger.getEntryLogMetadata(EntryLogger.java:1031)
>>     at
>> org.apache.bookkeeper.bookie.GarbageCollectorThread.extractMetaFromEntryLogs(GarbageCollectorThread.java:569)
>>     at
>> org.apache.bookkeeper.bookie.GarbageCollectorThread.safeRun(GarbageCollectorThread.java:342)
>>     at org.apache.bookkeeper.util.SafeRunnable.run(SafeRunnable.java:33)
>>     at
>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>     at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>     at
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>>     at
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>>     at
>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>>     at
>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>>     at java.lang.Thread.run(Thread.java:745)
>>
>
> --


-- Enrico Olivelli

Re: Cannot compact bookie due to NegativeArraySizeException on BK 4.5

Posted by Jia Zhai <zh...@gmail.com>.
Seems the failing place was scan the old EntryLog, not doing write, maybe
it not broken the EntryLog file.

The error indicates that we read a negative entrySize from entryLog file.

/**
 * Scan entry log.
 *
 * @param entryLogId Entry Log Id
 * @param scanner Entry Log Scanner
 * @throws IOException
 */
protected void scanEntryLog(long entryLogId, EntryLogScanner scanner)
throws IOException {
    ByteBuffer sizeBuff = ByteBuffer.allocate(4);
    ByteBuffer lidBuff = ByteBuffer.allocate(8);
    BufferedReadChannel bc;
    // Get the BufferedChannel for the current entry log file
    try {
        bc = getChannelForLogId(entryLogId);
    } catch (IOException e) {
        LOG.warn("Failed to get channel to scan entry log: " +
entryLogId + ".log");
        throw e;
    }
    // Start the read position in the current entry log file to be after
    // the header where all of the ledger entries are.
    long pos = LOGFILE_HEADER_SIZE;

    // Read through the entry log file and extract the ledger ID's.
    while (true) {
        // Check if we've finished reading the entry log file.
        if (pos >= bc.size()) {
            break;
        }
        if (readFromLogChannel(entryLogId, bc, sizeBuff, pos) !=
sizeBuff.capacity()) {
            LOG.warn("Short read for entry size from entrylog {}", entryLogId);
            return;
        }
        long offset = pos;
        pos += 4;
        sizeBuff.flip();
        int entrySize = sizeBuff.getInt();   < === 2, here we get
entrySize from buffer.

        sizeBuff.clear();
        // try to read ledger id first
        if (readFromLogChannel(entryLogId, bc, lidBuff, pos) !=
lidBuff.capacity()) {
            LOG.warn("Short read for ledger id from entrylog {}", entryLogId);
            return;
        }
        lidBuff.flip();
        long lid = lidBuff.getLong();
        lidBuff.clear();
        if (lid == INVALID_LID || !scanner.accept(lid)) {
            // skip this entry
            pos += entrySize;
            continue;
        }
        // read the entry
        byte data[] = new byte[entrySize];   < === 1, here is line
1006, seems it get a negative entrySize.
        ByteBuffer buff = ByteBuffer.wrap(data);
        int rc = readFromLogChannel(entryLogId, bc, buff, pos);
        if (rc != data.length) {
            LOG.warn("Short read for ledger entry from entryLog {}@{}
({} != {})", new Object[] { entryLogId, pos,
                    rc, data.length });
            return;
        }
        buff.flip();
        // process the entry
        scanner.process(lid, offset, buff);
        // Advance position to the next entry
        pos += entrySize;
    }
}





On Tue, Sep 19, 2017 at 8:42 PM, Enrico Olivelli <eo...@gmail.com>
wrote:

> Hi
> in one of my staging sites I got this error and the bookie cannot reclaim
> disk space
>
> I am running a pure 4.5 BookKeeper bookie
> below is the error
>
> Surely it is a broken EntryLog file.
> Do you think the bookie could be recoverable ?
> I did a backup, it does not contain sensitive data, I could share it
>
> -- Enrico
>
> 17-09-19-14-37-22    Unexpected throwable caught
> 17-09-19-14-37-22    java.lang.NegativeArraySizeException
> java.lang.NegativeArraySizeException
>     at org.apache.bookkeeper.bookie.EntryLogger.scanEntryLog(
> EntryLogger.java:1006)
>     at org.apache.bookkeeper.bookie.EntryLogger.
> extractEntryLogMetadataByScanning(EntryLogger.java:1118)
>     at org.apache.bookkeeper.bookie.EntryLogger.getEntryLogMetadata(
> EntryLogger.java:1031)
>     at org.apache.bookkeeper.bookie.GarbageCollectorThread.
> extractMetaFromEntryLogs(GarbageCollectorThread.java:569)
>     at org.apache.bookkeeper.bookie.GarbageCollectorThread.safeRun(
> GarbageCollectorThread.java:342)
>     at org.apache.bookkeeper.util.SafeRunnable.run(SafeRunnable.java:33)
>     at java.util.concurrent.Executors$RunnableAdapter.
> call(Executors.java:511)
>     at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>     at java.util.concurrent.ScheduledThreadPoolExecutor$
> ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>     at java.util.concurrent.ScheduledThreadPoolExecutor$
> ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>     at java.util.concurrent.ThreadPoolExecutor.runWorker(
> ThreadPoolExecutor.java:1142)
>     at java.util.concurrent.ThreadPoolExecutor$Worker.run(
> ThreadPoolExecutor.java:617)
>     at java.lang.Thread.run(Thread.java:745)
>