hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Todd Lipcon (JIRA)" <j...@apache.org>
Subject [jira] Commented: (HBASE-50) Snapshot of table
Date Sun, 13 Jun 2010 23:03:27 GMT

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

Todd Lipcon commented on HBASE-50:
----------------------------------

Finally had a chance to look over the doc. Great to see such a thorough writeup on the plan!
A couple thoughts:
- Snapshot creation: rather than causing all of the RS to roll the logs, they could simply
record the log sequence number of the snapshot, right? This will be a bit faster to do and
causes even less of a "hiccup" in concurrent operations (and I don't think it's any more complicated
to implement, is it?)
- Snapshot restore: I do think it's a good idea to allow snapshot restore to a new table name
while the original table is still online. And the restored snapshot should be able to share
HFiles with the original table
- Making the client orchestrate the snapshot process seems a little strange - could the client
simply initiate it and put the actual snapshot code in the master? I think we should keep
the client as thin as we can (in the future we may want to implement clients natively in other
languages)
- I'd be interested in a section about failure analysis - what happens when the snapshot coordinator
fails in the middle? You briefly touched on this, but would be good just to enumerate the
different points where a failure can happen and show that the operation is correctly aborted
and that you don't end up with an HFile "reference leak"

> Snapshot of table
> -----------------
>
>                 Key: HBASE-50
>                 URL: https://issues.apache.org/jira/browse/HBASE-50
>             Project: HBase
>          Issue Type: New Feature
>            Reporter: Billy Pearson
>            Assignee: Li Chongxin
>            Priority: Minor
>         Attachments: HBase Snapshot Design Report V2.pdf, HBase Snapshot Design Report
V3.pdf, snapshot-src.zip
>
>
> Havening an option to take a snapshot of a table would be vary useful in production.
> What I would like to see this option do is do a merge of all the data into one or more
files stored in the same folder on the dfs. This way we could save data in case of a software
bug in hadoop or user code. 
> The other advantage would be to be able to export a table to multi locations. Say I had
a read_only table that must be online. I could take a snapshot of it when needed and export
it to a separate data center and have it loaded there and then i would have it online at multi
data centers for load balancing and failover.
> I understand that hadoop takes the need out of havening backup to protect from failed
servers, but this does not protect use from software bugs that might delete or alter data
in ways we did not plan. We should have a way we can roll back a dataset.

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


Mime
View raw message