hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Lars Hofhansl (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed
Date Wed, 16 Apr 2014 18:00:29 GMT

    [ https://issues.apache.org/jira/browse/HBASE-10958?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13971738#comment-13971738
] 

Lars Hofhansl commented on HBASE-10958:
---------------------------------------

Maybe "CREATE" no longer expresses what it now implies...?

I can see that folks would not want to grant users CREATE (or ADMIN) so that they cannot create/drop/enable/disable
tables, but still do allow them to load data via bulk load. That would now no longer possible.

Also it would seem more sensible to me that if a user bulk loads some data and then for *technical*
reasons the region server decides to flush there should be no additional right needed; just
a user does not need permission to flush only because a Put happens to cause a flush.
Have we dismissed this option?

> [dataloss] Bulk loading with seqids can prevent some log entries from being replayed
> ------------------------------------------------------------------------------------
>
>                 Key: HBASE-10958
>                 URL: https://issues.apache.org/jira/browse/HBASE-10958
>             Project: HBase
>          Issue Type: Bug
>    Affects Versions: 0.96.2, 0.98.1, 0.94.18
>            Reporter: Jean-Daniel Cryans
>            Assignee: Jean-Daniel Cryans
>            Priority: Blocker
>             Fix For: 0.99.0, 0.94.19, 0.98.2, 0.96.3
>
>         Attachments: HBASE-10958-less-intrusive-hack-0.96.patch, HBASE-10958-quick-hack-0.96.patch,
HBASE-10958-v2.patch, HBASE-10958.patch
>
>
> We found an issue with bulk loads causing data loss when assigning sequence ids (HBASE-6630)
that is triggered when replaying recovered edits. We're nicknaming this issue *Blindspot*.
> The problem is that the sequence id given to a bulk loaded file is higher than those
of the edits in the region's memstore. When replaying recovered edits, the rule to skip some
of them is that they have to be _lower than the highest sequence id_. In other words, the
edits that have a sequence id lower than the highest one in the store files *should* have
also been flushed. This is not the case with bulk loaded files since we now have an HFile
with a sequence id higher than unflushed edits.
> The log recovery code takes this into account by simply skipping the bulk loaded files,
but this "bulk loaded status" is *lost* on compaction. The edits in the logs that have a sequence
id lower than the bulk loaded file that got compacted are put in a blind spot and are skipped
during replay.
> Here's the easiest way to recreate this issue:
>  - Create an empty table
>  - Put one row in it (let's say it gets seqid 1)
>  - Bulk load one file (it gets seqid 2). I used ImporTsv and set hbase.mapreduce.bulkload.assign.sequenceNumbers.
>  - Bulk load a second file the same way (it gets seqid 3).
>  - Major compact the table (the new file has seqid 3 and isn't considered bulk loaded).
>  - Kill the region server that holds the table's region.
>  - Scan the table once the region is made available again. The first row, at seqid 1,
will be missing since the HFile with seqid 3 makes us believe that everything that came before
it was flushed.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Mime
View raw message