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

Todd Lipcon commented on HBASE-2231:
------------------------------------

I've been thinking about this a bit, and I want to think about a different 
design.

The flaw with the design as I outlined above is that we have to read through 
the entire reconstruction HLog until we find the special entries telling us to 
"finish off" the compaction for the dead regionserver. This is fine in today's 
world, but I think in some instances it would be better to finish the 
compaction and get the directory in a reasonable state *before* replaying the 
logs. This would allow, for example, the recovering regionserver to start 
serving the region in "stale readonly" mode before log replay is done.

I'm thinking about a different design in which we add an 
/hbase/ops-in-progress/regionserver/ directory in zookeeper. When we are about 
to start moving files around after a compaction, we write our intention in 
there, and when we're done, we remove it. Thus the HMaster can take of 
finishing or aborting the compaction immediately when it starts recovery, and 
the recovering regionserver doesn't have to worry about the transition in 
progress.

Thoughts?

> Compaction events should be written to HLog
> -------------------------------------------
>
>                 Key: HBASE-2231
>                 URL: https://issues.apache.org/jira/browse/HBASE-2231
>             Project: Hadoop HBase
>          Issue Type: Improvement
>          Components: regionserver
>            Reporter: Todd Lipcon
>            Assignee: Todd Lipcon
>             Fix For: 0.21.0
>
>         Attachments: hbase-2231-testcase.txt, hbase-2231.txt
>
>
> The sequence for a compaction should look like this:
> # Compact region to "new" files
> # Write a "Compacted Region" entry to the HLog
> # Delete "old" files
> This deals with a case where the RS has paused between step 1 and 2 and the 
> regions have since been reassigned.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to