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

Jesse Yates commented on HBASE-50:
----------------------------------

We had a meetup within a meetup at the HBase User Group meetup tonight to talk 
about the difficulties and next steps with snapshotting. The main takeaways 
were:
* exact time is inconsistent across a cluster (even with NTP) as we need 
millisecond exactness for point-in-time snapshots
* two-phase commit where we block writes in the first phase (completes in a set 
timeout) seems the most reasonable approach for ensuring a fully consistent view
* maybe fully consistent across the whole table isn't necessary, maybe per RS 
consistency within a window is acceptable
** possibly achieved by scheduling a time for a snapshot sometime in the future 
in zk and letting each RS snapshot makes it 'close enough'
* zk triggered snapshots make it even harder to ensure timeout boundaries due 
to RS no hard guarantees on notifications and even then zk timeouts causing 
presence issues

Even with all of this, I'm planning the first pass to be zk based (until we 
decide that unavailability suffers too much) and with a simple two-phase, 
write-locking per involved region approach. We can probably iterate on that to 
bring down the latency.

The main issue here is I don't see a way to ensure that in a snapshot, all RS 
take a snapshot "now" but still allow reading/writing on either side (pretty 
sure this is a CAP limitation). 
                
> Snapshot of table
> -----------------
>
>                 Key: HBASE-50
>                 URL: https://issues.apache.org/jira/browse/HBASE-50
>             Project: HBase
>          Issue Type: New Feature
>    Affects Versions: 0.96.0
>            Reporter: Billy Pearson
>            Assignee: Li Chongxin
>            Priority: Minor
>              Labels: gsoc
>             Fix For: 0.96.0
>
>         Attachments: HBase Snapshot Design Report V2.pdf, HBase Snapshot 
> Design Report V3.pdf, HBase Snapshot Implementation Plan.pdf, Snapshot Class 
> Diagram.png
>
>
> 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.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to