[jira] [Updated] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-30 Thread Jean-Daniel Cryans (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10958?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jean-Daniel Cryans updated HBASE-10958:
---

  Resolution: Fixed
Hadoop Flags: Reviewed
  Status: Resolved  (was: Patch Available)

Now committed to 0.94 (took some time because I wanted to double check the 
tests I modified were all green). Thanks everyone.

 [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.98.2, 0.96.3, 0.94.20

 Attachments: HBASE-10958-0.94.patch, 
 HBASE-10958-less-intrusive-hack-0.96.patch, 
 HBASE-10958-quick-hack-0.96.patch, HBASE-10958-v2.patch, 
 HBASE-10958-v3.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)


[jira] [Updated] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-25 Thread Jean-Daniel Cryans (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10958?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jean-Daniel Cryans updated HBASE-10958:
---

Release Note: Bulk loading with sequence IDs, an option in late 0.94 
releases and the default since 0.96.0, will now trigger a flush per region that 
loads an HFile (if there's data that needs to flushed).

Committed to 0.96 and up. Like for HBASE-11008, I'm waiting to commit to 0.94 
or I can open a backport jira.

 [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.98.2, 0.96.3, 0.94.20

 Attachments: HBASE-10958-0.94.patch, 
 HBASE-10958-less-intrusive-hack-0.96.patch, 
 HBASE-10958-quick-hack-0.96.patch, HBASE-10958-v2.patch, 
 HBASE-10958-v3.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)


[jira] [Updated] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-21 Thread Lars Hofhansl (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10958?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Lars Hofhansl updated HBASE-10958:
--

Fix Version/s: (was: 0.94.19)
   0.94.20

Moving to 0.94.20.

 [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.98.2, 0.96.3, 0.94.20

 Attachments: HBASE-10958-0.94.patch, 
 HBASE-10958-less-intrusive-hack-0.96.patch, 
 HBASE-10958-quick-hack-0.96.patch, HBASE-10958-v2.patch, 
 HBASE-10958-v3.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)


[jira] [Updated] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-17 Thread Jean-Daniel Cryans (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10958?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jean-Daniel Cryans updated HBASE-10958:
---

Attachment: HBASE-10958-0.94.patch
HBASE-10958-v3.patch

Attaching 2 patches.

One is a backport for 0.94. While doing the backport I saw that a 
TestSnapshotFromMaster was failing and Matteo was able to see that it was an 
error in my patch, flushing always returned that it needed compaction. I need 
to rerun all the tests now but it was the only one that failed (haven't tried 
with security either). I also added a test in TestHRegion for that.

The second patch is for trunk, in which I ported the same test to TestHRegion. 
Interestingly, it didn't work. I found that in 0.94 we compact if num_files  
compactionThreshold, but in trunk it's =, so it seems that we compact more 
often now. This patch also has the fixes from [~stack]'s comments.

 [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-0.94.patch, 
 HBASE-10958-less-intrusive-hack-0.96.patch, 
 HBASE-10958-quick-hack-0.96.patch, HBASE-10958-v2.patch, 
 HBASE-10958-v3.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)


[jira] [Updated] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-15 Thread Jean-Daniel Cryans (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10958?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jean-Daniel Cryans updated HBASE-10958:
---

Attachment: HBASE-10958-v2.patch

New patch with a unit test inside {{TestWALReplay}}. I've done a bit of 
refactoring and I'm tempted to go a step further and somehow extract the common 
bits from testRegionMadeOfBulkLoadedFilesOnly and testCompactedBulkLoadedFiles 
but it seems a bit messy.

 [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)


[jira] [Updated] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-11 Thread Jean-Daniel Cryans (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10958?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jean-Daniel Cryans updated HBASE-10958:
---

Attachment: HBASE-10958.patch

Patch that addresses most of Stack's comments and based on trunk. Still no unit 
test, spent too much time today trying to understand why {{TestHRegion}} fails 
so much (see HBASE-10312).

 [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
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.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)


[jira] [Updated] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-11 Thread Jean-Daniel Cryans (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10958?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jean-Daniel Cryans updated HBASE-10958:
---

Assignee: Jean-Daniel Cryans
  Status: Patch Available  (was: Open)

 [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.94.18, 0.98.1, 0.96.2
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.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)


[jira] [Updated] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-10 Thread Jean-Daniel Cryans (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10958?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jean-Daniel Cryans updated HBASE-10958:
---

Attachment: HBASE-10958-quick-hack-0.96.patch

Here's a quick hack I put together to show one solution. In this case I inline 
a flush with the {{bulkLoadHFiles}} call and I modified the 
{{internalFlushcache}} code to be able to get more state back and also return a 
sequential ID that ends up being in between two memstores.

I tested that it works following the steps listed in this jira's description.

I don't really like having to wait for the flush to happen since it could take 
a lot of time to finish making bulk loading way slower. On the other hand, it's 
safer than just requesting a flush asynchronously and then grabbing a new 
sequence ID from HLog. Maybe it would still be fine since you need to also have 
a compaction to trigger the bug...

 [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
Priority: Blocker
 Fix For: 0.99.0, 0.94.19, 0.98.2, 0.96.3

 Attachments: HBASE-10958-quick-hack-0.96.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)


[jira] [Updated] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-10 Thread Jean-Daniel Cryans (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10958?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jean-Daniel Cryans updated HBASE-10958:
---

Attachment: HBASE-10958-less-intrusive-hack-0.96.patch

Here's a less intrusive hack that shows what it looks like to request an 
asynchronous flush from {{bulkLoadHFiles}}. I also tested that it works like 
the previous patch.

The benefits are that the bulk loader doesn't have to wait and it's a lot less 
code, but the blindspot is still there and it widens as the region server gets 
more load; for example, if a lot of flushing is happening, which is likely with 
this patch, then the flush requests are queued and a compacting can kick in and 
at that moment if the region server dies then data is lost.

 [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
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


 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)