[jira] [Commented] (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:comment-tabpanel&focusedCommentId=13965642#comment-13965642
 ] 

Jean-Daniel Cryans commented on HBASE-10958:


One workaround we found is to completely disable compactions, then when you 
need to run them you have to force flush the regions that have bulk loaded file 
first and ensure that bulk loads aren't coming in at the same time.

Workloads that are strictly doing incremental bulk loads aren't affected, you 
need a mix of bulk loaded files and normal Puts.

A hacky solution could be to force flush when bulk loading with seqids and grab 
the next sequence id that comes after the memstore flush to go to the bulk 
loaded file. This means that bulk loading needs to initiate a flush, get the 
sequence id under the region write lock, then do the bulk load. We don't need 
to wait for the flush to happen... unless the possibility for the bulk loaded 
file to be compacted before the flush is done is high enough.

> [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
>
>
> 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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-10 Thread Lars Hofhansl (JIRA)

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

Lars Hofhansl commented on HBASE-10958:
---

Seems fine to force a flush before bulk loading.

> [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
>
>
> 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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-10 Thread Alexandre Normand (JIRA)

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

Alexandre Normand commented on HBASE-10958:
---

I'll throw my 2 canadien cents and say that I prefer the 
"removing-the-second-lane" solution (the one where we do a synchronous flush 
prior to bulk load). My rationale is that the other approach still leaves us 
with a blindspot, even if greatly reduced and we would effectively still have 
the bug (potentially). 

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


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

2014-04-10 Thread stack (JIRA)

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

stack commented on HBASE-10958:
---

I'm w/ [~alexandre.normand].

FlushState should be FlushResult?  Does it have to be public?

Should below be HBaseIOE (you know who is watching) or some specialization on 
HBaseIOE, FlushFailedIOE(FlushState)?

throw new IOException(

What is happening here:

+  seqId = fs.flushSequenceId;

This is the 'next' seqid after the flush or the seqid that was written into the 
hfile that was flushed?

Do we need to up the seqid if the flush one is being used for a bulk load so 
the next edit in memstore has a different number?  Or that is done already 
elsewhere?

Good stuff JD.  A unit test would be too hard to conjure?  Maybe describe then 
instead how you replicate.

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


[jira] [Commented] (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:comment-tabpanel&focusedCommentId=13966722#comment-13966722
 ] 

Jean-Daniel Cryans commented on HBASE-10958:


bq. FlushState should be FlushResult? Does it have to be public?

Some external classes to that package call {{flushcache()}} directly. Also, 
that method is public so whatever it returns needs to be as visible.

bq. Should below be HBaseIOE (you know who is watching) or some specialization 
on HBaseIOE, FlushFailedIOE(FlushState)?

It's inline with the rest of that method, which the bulk loading client seems 
to process correctly. Not that it shouldn't be considered, but maybe in a 
different jira?

{quote}
This is the 'next' seqid after the flush or the seqid that was written into the 
hfile that was flushed?
Do we need to up the seqid if the flush one is being used for a bulk load so 
the next edit in memstore has a different number? Or that is done already 
elsewhere?
{quote}

Yeah that part is hard to follow, needs at least some documentation (hey it is 
a hack!). So it's set here:

+return new FlushState(flushSeqId, compactionRequested);

That {{flushSeqId}} comes from here:

{code}
Long startSeqId = 
wal.startCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
if (startSeqId == null) {
  status.setStatus("Flush will not be started for [" + 
this.getRegionInfo().getEncodedName()
  + "] - WAL is going away");
  return false;
}
flushSeqId = startSeqId.longValue();
{code}

So it's a sequence id that's the same one as the one used to signal the flush. 
I thought about creating a new one just after, but I'm not sure if it's 
necessary since that {{startSeqId}} will be after all the MemStore edits.

bq.  A unit test would be too hard to conjure? Maybe describe then instead how 
you replicate.

Doesn't seem too hard to write... at least the recreating what I'm doing 
manually shouldn't take too long to write, but testing the corner cases seems 
much harder.

I'm still wondering if there's a more elegant solution.

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


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

2014-04-11 Thread Hadoop QA (JIRA)

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

Hadoop QA commented on HBASE-10958:
---

{color:red}-1 overall{color}.  Here are the results of testing the latest 
attachment 
  http://issues.apache.org/jira/secure/attachment/12639893/HBASE-10958.patch
  against trunk revision .
  ATTACHMENT ID: 12639893

{color:green}+1 @author{color}.  The patch does not contain any @author 
tags.

{color:green}+1 tests included{color}.  The patch appears to include 6 new 
or modified tests.

{color:green}+1 javadoc{color}.  The javadoc tool did not generate any 
warning messages.

{color:green}+1 javac{color}.  The applied patch does not increase the 
total number of javac compiler warnings.

{color:red}-1 findbugs{color}.  The patch appears to introduce 2 new 
Findbugs (version 1.3.9) warnings.

{color:green}+1 release audit{color}.  The applied patch does not increase 
the total number of release audit warnings.

{color:green}+1 lineLengths{color}.  The patch does not introduce lines 
longer than 100

  {color:green}+1 site{color}.  The mvn site goal succeeds with this patch.

 {color:red}-1 core tests{color}.  The patch failed these unit tests:
   
org.apache.hadoop.hbase.security.access.TestAccessController

Test results: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9267//testReport/
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9267//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-protocol.html
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9267//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-thrift.html
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9267//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-client.html
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9267//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop2-compat.html
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9267//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-examples.html
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9267//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-prefix-tree.html
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9267//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-common.html
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9267//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-server.html
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9267//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop-compat.html
Console output: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9267//console

This message is automatically generated.

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

[jira] [Commented] (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:comment-tabpanel&focusedCommentId=13970015#comment-13970015
 ] 

Jean-Daniel Cryans commented on HBASE-10958:


Oh and I found that testRegionMadeOfBulkLoadedFilesOnly wasn't really testing 
WAL replay with a bulk loaded file, the KV was being added at LATEST_TIMESTAMP 
so it wasn't visible being so far in the future. I made it so we check that we 
got all the rows back.

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-15 Thread stack (JIRA)

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

stack commented on HBASE-10958:
---

Patch looks great [~jdcryans].  Minor nit is that you do

+if (fs.flushSucceeded()) {
+  seqId = fs.flushSequenceId;
+} else if (fs.result == 
FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) {

flushSucceeded method (should it be isFlushSucceeded) and then you go get the 
result by accessing the data member directly.  Minor inconsistency.

+1 on commit if hadoopqa ok.  Can address above if you want on commit.

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-15 Thread Hadoop QA (JIRA)

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

Hadoop QA commented on HBASE-10958:
---

{color:red}-1 overall{color}.  Here are the results of testing the latest 
attachment 
  http://issues.apache.org/jira/secure/attachment/12640324/HBASE-10958-v2.patch
  against trunk revision .
  ATTACHMENT ID: 12640324

{color:green}+1 @author{color}.  The patch does not contain any @author 
tags.

{color:green}+1 tests included{color}.  The patch appears to include 9 new 
or modified tests.

{color:green}+1 javadoc{color}.  The javadoc tool did not generate any 
warning messages.

{color:green}+1 javac{color}.  The applied patch does not increase the 
total number of javac compiler warnings.

{color:green}+1 findbugs{color}.  The patch does not introduce any new 
Findbugs (version 1.3.9) warnings.

{color:green}+1 release audit{color}.  The applied patch does not increase 
the total number of release audit warnings.

{color:green}+1 lineLengths{color}.  The patch does not introduce lines 
longer than 100

  {color:green}+1 site{color}.  The mvn site goal succeeds with this patch.

 {color:red}-1 core tests{color}.  The patch failed these unit tests:
   
org.apache.hadoop.hbase.security.access.TestAccessController

Test results: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9296//testReport/
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9296//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-protocol.html
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9296//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-thrift.html
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9296//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-client.html
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9296//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop2-compat.html
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9296//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-examples.html
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9296//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-prefix-tree.html
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9296//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-common.html
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9296//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-server.html
Findbugs warnings: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9296//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop-compat.html
Console output: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9296//console

This message is automatically generated.

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

[jira] [Commented] (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:comment-tabpanel&focusedCommentId=13970118#comment-13970118
 ] 

Jean-Daniel Cryans commented on HBASE-10958:


That's a real test failure, the user that bulk loads doesn't have the 
permission to flush. Looking.

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-15 Thread Lars Hofhansl (JIRA)

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

Lars Hofhansl commented on HBASE-10958:
---

Oh; that's an unforeseen problem. Does it make sense for a user to be able to 
bulkload but not to flush? (I suppose it does as bulk loading is not in 
principle different from inserting data via Put/Delete).


> [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] [Commented] (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:comment-tabpanel&focusedCommentId=13970158#comment-13970158
 ] 

Jean-Daniel Cryans commented on HBASE-10958:


Yeah, and bulk loading itself is kind of like a flush since you end up with an 
HFile.

> [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] [Commented] (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:comment-tabpanel&focusedCommentId=13970166#comment-13970166
 ] 

Jean-Daniel Cryans commented on HBASE-10958:


Right now bulk loading is WRITE and flush is ADMIN. Problematic!

> [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] [Commented] (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:comment-tabpanel&focusedCommentId=13970173#comment-13970173
 ] 

Jean-Daniel Cryans commented on HBASE-10958:


Solutions on top of my head:

- Do doAs inside the method. The rationale being that it's the region server 
that's trying to do that here, not the user. I'm not sure if this is doable, 
and [~mbertozzi] is laughing at me.

- Matteo thinks that we should just make bulk load an ADMIN method. I agree but 
I'm not fond of breaking secure setups that use bulk loads. [~apurtell]?

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-15 Thread Lars Hofhansl (JIRA)

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

Lars Hofhansl commented on HBASE-10958:
---

Wait. Are you saying the region server itself cannot issue a flush as part of 
the bulkLoadHFiles RPC?
The region server is flushing all the time on its own behalf (when the memstore 
is full, etc).


> [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] [Commented] (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:comment-tabpanel&focusedCommentId=13970228#comment-13970228
 ] 

Jean-Daniel Cryans commented on HBASE-10958:


bq. Wait. Are you saying the region server itself cannot issue a flush as part 
of the bulkLoadHFiles RPC?

Exact. That's why the User.runAs (and then run as the region server itself) 
solution seems to make sense to me.

I read some more code, and it seems bulk load actually requires CREATE even 
though the call itself requires WRITE. From TestAccessController:

{code}
// User performing bulk loads must have privilege to read table metadata
// (ADMIN or CREATE)
verifyAllowed(bulkLoadAction, SUPERUSER, USER_ADMIN, USER_OWNER, 
USER_CREATE);
verifyDenied(bulkLoadAction, USER_RW, USER_NONE, USER_RO);
{code}

So another options is to set flush and compact as CREATE actions (in line with 
create table, alter, disable, enable, delete).

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-15 Thread Lars Hofhansl (JIRA)

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

Lars Hofhansl commented on HBASE-10958:
---

I guess I did not realize that the authorization extends to any action issued 
from an RPC, rather than just authorizing the RPC call itself.


> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-15 Thread Andrew Purtell (JIRA)

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

Andrew Purtell commented on HBASE-10958:


We expect READ and WRITE perms granted in a fine grained way to constraint who 
can do individual ops that only collectively add up to cluster impacting events 
like compactions, splits, and flushes. For actions that can have a global 
cluster impact, we'd like ADMIN to be granted sparingly to admins or delegates. 
IIRC enable and disable are ADMIN actions also, since disabling or enabling a 
1 region table has consequences. CREATE is kind of a middle ground for 
schema reads and updates, but in terms of schema update that's splitting hairs 
I suppose since a schema update of said large table would also have 
consequences of the same scale.

Bulk load is a special snowflake because it's a series of puts (so, WRITE) yet 
obviously more than that as mentioned, we need to flush, and moving files in 
place will probably kick off compaction. Making bulk load an ADMIN action, or 
CREATE, makes sense to me also.

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-15 Thread Andrew Purtell (JIRA)

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

Andrew Purtell commented on HBASE-10958:


And the bit about needing CREATE or ADMIN to read schema metadata, this is to 
protect potentially sensitive information in the metadata that an ordinary user 
granted only READ or READ+WRITE access to the table has no need to see, that 
was HBASE-8692

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-15 Thread Jerry He (JIRA)

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

Jerry He commented on HBASE-10958:
--

Very nice explanation and insights on these permissions!
Looking previously at this page: 
https://hbase.apache.org/book/hbase.accesscontrol.configuration.html
we only have pretty vague info over there.
Also, the 'write' permission for 'flush' and 'compact' in 'Table 8.1'. Are they 
even correct?

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

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

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

Jean-Daniel Cryans commented on HBASE-10958:


bq. we'd like ADMIN to be granted sparingly to admins or delegates

+1

bq.  IIRC enable and disable are ADMIN actions also, since disabling or 
enabling a 1 region table has consequences.

This is what I see in the code in trunk:

{code}
requirePermission("preBulkLoadHFile", getTableDesc().getTableName(), 
el.getFirst(), null, Permission.Action.WRITE);
requirePermission("enableTable", tableName, null, null, Action.ADMIN, 
Action.CREATE);
requirePermission("disableTable", tableName, null, null, Action.ADMIN, 
Action.CREATE);
requirePermission("compact", getTableName(e.getEnvironment()), null, null, 
Action.ADMIN);
requirePermission("flush", getTableName(e.getEnvironment()), null, null, 
Action.ADMIN);
{code}

IMO flush should have lower or same perms as disableTable.

So here's a list of changes I believe are needed:

 - preBulkLoadHFile goes from WRITE to CREATE (seems more in line with what's 
really needed to bulk load given the code I posted yesterday)
 - compact/flush go from ADMIN to ADMIN or CREATE

This should not have an impact on the current users. If we can agree on the 
changes, I'll open a new jira that's going to be blocking this one.

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-16 Thread Lars Hofhansl (JIRA)

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


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

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

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

Jean-Daniel Cryans commented on HBASE-10958:


bq. 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.

Bulk load already needs CREATE as shown above in TestAccessController.

bq. Have we dismissed this option?

I don't think so, but no one has been pushing for it. It creates a precedent, 
nowhere else in the code do we use User.runAs to override the current user that 
came in via a RPC (I'm not even sure if it works, but that's because I don't 
know that code very well).

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-16 Thread Lars Hofhansl (JIRA)

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

Lars Hofhansl commented on HBASE-10958:
---

Missed the test comment. In that case let's do what you suggest.
(Since this in an existing issue I might sill want release 0.94.19 before we 
fix it depending on whether this needs more discussion)

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-16 Thread Jerry He (JIRA)

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

Jerry He commented on HBASE-10958:
--

Just did a quick test. The requirement on 'CREATE' for bulk load seems to come 
from here.  Is this even intended?
{code}
Exception in thread "main" 
org.apache.hadoop.hbase.security.AccessDeniedException: 
org.apache.hadoop.hbase.security.AccessDeniedException: Insufficient 
permissions (user=us...@ibm.com, scope=TestTable, family=, action=CREATE)
at 
org.apache.hadoop.hbase.security.access.AccessController.requirePermission(AccessController.java:356)
at 
org.apache.hadoop.hbase.security.access.AccessController.preGetTableDescriptors(AccessController.java:1513)
at 
org.apache.hadoop.hbase.master.MasterCoprocessorHost.preGetTableDescriptors(MasterCoprocessorHost.java:1260)
at 
org.apache.hadoop.hbase.master.HMaster.getTableDescriptors(HMaster.java:2569)
at 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos$MasterService$2.callBlockingMethod(MasterProtos.java:40438)
at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2150)
 ...
at 
org.apache.hadoop.hbase.protobuf.ProtobufUtil.getRemoteException(ProtobufUtil.java:235)
at 
org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.getHTableDescriptor(HConnectionManager.java:2632)
at 
org.apache.hadoop.hbase.client.HTable.getTableDescriptor(HTable.java:548)
at 
org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles.doBulkLoad(LoadIncrementalHFiles.java:233)
at 
org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles.run(LoadIncrementalHFiles.java:820)
{code}

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

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

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

Jean-Daniel Cryans commented on HBASE-10958:


bq. The requirement on 'CREATE' for bulk load seems to come from here. Is this 
even intended?

Thanks for doing this. There's also another place where this is called, 
splitStoreFile(), in order to get an HCD. I don't think it's necessary to call 
it twice, but it seems necessary to call it at least once else you can't:

- verify that the families exist
- get the schema for each family so that we create the HFiles with the correct 
configurations when splitting

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

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

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

Jean-Daniel Cryans commented on HBASE-10958:


Forgot to reply to [~stack]'s comment:

bq. flushSucceeded method (should it be isFlushSucceeded) and then you go get 
the result by accessing the data member directly. Minor inconsistency.

flushSucceeded() isn't just looking up a field though, it's checking two 
things. 

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-16 Thread Andrew Purtell (JIRA)

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

Andrew Purtell commented on HBASE-10958:


bq. IMO flush should have lower or same perms as disableTable.

That seems fine. 

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

At the least we have an imperfect idea of when a user should be able to create 
tables and administer them, just "not administer them too much"

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-16 Thread Andrew Purtell (JIRA)

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

Andrew Purtell commented on HBASE-10958:


bq.  The requirement on 'CREATE' for bulk load seems to come from [ 
getTableDescriptors ]. Is this even intended?

Yes.

CREATE is overloaded to mean "RESTRICTED ADMIN", with ADMIN-ish privilege 
required because table schema is considered potentially sensitive. 

On another issue Francis Liu and I discussed the notion of creating a new 
permission 'SCHEMA' which would grant permission to read schema metadata. Now 
as then it seems maybe not quite needed (yet). CREATE and ADMIN would have such 
SCHEMA permission implicitly, so how useful would SCHEMA be, and there would 
still need a grant beyond WRITE for bulk loading.

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-17 Thread Hadoop QA (JIRA)

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

Hadoop QA commented on HBASE-10958:
---

{color:red}-1 overall{color}.  Here are the results of testing the latest 
attachment 
  
http://issues.apache.org/jira/secure/attachment/12640751/HBASE-10958-0.94.patch
  against trunk revision .
  ATTACHMENT ID: 12640751

{color:green}+1 @author{color}.  The patch does not contain any @author 
tags.

{color:green}+1 tests included{color}.  The patch appears to include 15 new 
or modified tests.

{color:red}-1 patch{color}.  The patch command could not apply the patch.

Console output: 
https://builds.apache.org/job/PreCommit-HBASE-Build/9328//console

This message is automatically generated.

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-17 Thread stack (JIRA)

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

stack commented on HBASE-10958:
---

FYI rather than do this:

+String method = "testFlushResult";

You can do this:

method = name.getMethodName();

... because in TestHRegion it does this:

  @Rule public TestName name = new TestName();

See here 
http://stackoverflow.com/questions/473401/get-name-of-currently-executing-test-in-junit-4

On the sometime an accessor, sometime not... not going to argue.  nit.

Patch LGTM (where G==Great)

> [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] [Commented] (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:comment-tabpanel&focusedCommentId=13973789#comment-13973789
 ] 

Jean-Daniel Cryans commented on HBASE-10958:


bq. method = name.getMethodName();

Will fix (looks like I copied that from one of the few methods in that class 
that doesn't do it).

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-17 Thread stack (JIRA)

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

stack commented on HBASE-10958:
---

bq. Will fix (looks like I copied that from one of the few methods in that 
class that doesn't do it).

it is a nit.  fix on commit?

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-18 Thread Lars Hofhansl (JIRA)

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

Lars Hofhansl commented on HBASE-10958:
---

Nice. +1

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-25 Thread Hudson (JIRA)

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

Hudson commented on HBASE-10958:


FAILURE: Integrated in HBase-TRUNK #5118 (See 
[https://builds.apache.org/job/HBase-TRUNK/5118/])
HBASE-10958 [dataloss] Bulk loading with seqids can prevent some log entries 
from being replayed (jdcryans: rev 1590144)
* 
/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
* 
/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
* 
/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
* 
/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
* 
/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
* 
/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
* 
/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
* 
/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
* 
/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java


> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-25 Thread Hudson (JIRA)

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

Hudson commented on HBASE-10958:


SUCCESS: Integrated in HBase-0.98 #296 (See 
[https://builds.apache.org/job/HBase-0.98/296/])
HBASE-10958 [dataloss] Bulk loading with seqids can prevent some log entries 
from being replayed (jdcryans: rev 1590145)
* 
/hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
* 
/hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
* 
/hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
* 
/hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
* 
/hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
* 
/hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
* 
/hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
* 
/hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
* 
/hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java


> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-25 Thread Hudson (JIRA)

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

Hudson commented on HBASE-10958:


SUCCESS: Integrated in hbase-0.96-hadoop2 #274 (See 
[https://builds.apache.org/job/hbase-0.96-hadoop2/274/])
HBASE-10958 [dataloss] Bulk loading with seqids can prevent some log entries 
from being replayed (jdcryans: rev 1590146)
* 
/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
* 
/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
* 
/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
* 
/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
* 
/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
* 
/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
* 
/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
* 
/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
* 
/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java


> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-25 Thread Hudson (JIRA)

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

Hudson commented on HBASE-10958:


FAILURE: Integrated in hbase-0.96 #395 (See 
[https://builds.apache.org/job/hbase-0.96/395/])
HBASE-10958 [dataloss] Bulk loading with seqids can prevent some log entries 
from being replayed (jdcryans: rev 1590146)
* 
/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
* 
/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
* 
/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
* 
/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
* 
/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
* 
/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
* 
/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
* 
/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
* 
/hbase/branches/0.96/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java


> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-25 Thread Hudson (JIRA)

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

Hudson commented on HBASE-10958:


SUCCESS: Integrated in HBase-0.98-on-Hadoop-1.1 #281 (See 
[https://builds.apache.org/job/HBase-0.98-on-Hadoop-1.1/281/])
HBASE-10958 [dataloss] Bulk loading with seqids can prevent some log entries 
from being replayed (jdcryans: rev 1590145)
* 
/hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
* 
/hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
* 
/hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
* 
/hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
* 
/hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
* 
/hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
* 
/hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
* 
/hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
* 
/hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java


> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-30 Thread Lars Hofhansl (JIRA)

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

Lars Hofhansl commented on HBASE-10958:
---

Are you waiting for me to commit [~jdcryans]? Just making sure we're not 
mutually waiting.

> [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] [Commented] (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:comment-tabpanel&focusedCommentId=13986053#comment-13986053
 ] 

Jean-Daniel Cryans commented on HBASE-10958:


Sorry, went to do something else, lemme get that in (with HBASE-11008 first).

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-04-30 Thread Lars Hofhansl (JIRA)

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

Lars Hofhansl commented on HBASE-10958:
---

You 'd man.

> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-05-02 Thread Hudson (JIRA)

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

Hudson commented on HBASE-10958:


FAILURE: Integrated in HBase-0.94-JDK7 #132 (See 
[https://builds.apache.org/job/HBase-0.94-JDK7/132/])
HBASE-10958 [dataloss] Bulk loading with seqids can prevent some log entries 
from being replayed (jdcryans: rev 1591495)
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java


> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-05-02 Thread Hudson (JIRA)

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

Hudson commented on HBASE-10958:


FAILURE: Integrated in HBase-0.94-on-Hadoop-2 #82 (See 
[https://builds.apache.org/job/HBase-0.94-on-Hadoop-2/82/])
HBASE-10958 [dataloss] Bulk loading with seqids can prevent some log entries 
from being replayed (jdcryans: rev 1591495)
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java


> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-05-02 Thread Hudson (JIRA)

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

Hudson commented on HBASE-10958:


SUCCESS: Integrated in HBase-0.94 #1365 (See 
[https://builds.apache.org/job/HBase-0.94/1365/])
HBASE-10958 [dataloss] Bulk loading with seqids can prevent some log entries 
from being replayed (jdcryans: rev 1591495)
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java


> [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] [Commented] (HBASE-10958) [dataloss] Bulk loading with seqids can prevent some log entries from being replayed

2014-05-04 Thread Hudson (JIRA)

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

Hudson commented on HBASE-10958:


FAILURE: Integrated in HBase-0.94-security #480 (See 
[https://builds.apache.org/job/HBase-0.94-security/480/])
HBASE-10958 [dataloss] Bulk loading with seqids can prevent some log entries 
from being replayed (jdcryans: rev 1591495)
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
* 
/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
* 
/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java


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