[
https://issues.apache.org/jira/browse/HBASE-50?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12894706#action_12894706
]
HBase Review Board commented on HBASE-50:
-----------------------------------------
Message from: "Todd Lipcon" <[email protected]>
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
http://review.cloudera.org/r/467/#review618
-----------------------------------------------------------
looks pretty good! I didn't et a chance to look through the test cases in
detail, I'll try to look them over some more later this week.
src/main/java/org/apache/hadoop/hbase/HConstants.java
<http://review.cloudera.org/r/467/#comment2293>
since we also have a "log archive dir" somewhere, should specify this a
little more - this is archived HFiles that are still referenced by snapshots?
src/main/java/org/apache/hadoop/hbase/HSnapshotDescriptor.java
<http://review.cloudera.org/r/467/#comment2294>
license
src/main/java/org/apache/hadoop/hbase/HSnapshotDescriptor.java
<http://review.cloudera.org/r/467/#comment2295>
no need for @param javadoc if there is no actual description attached. same
thing below in a few places
src/main/java/org/apache/hadoop/hbase/HSnapshotDescriptor.java
<http://review.cloudera.org/r/467/#comment2296>
why not System.currentTimeMillis?
src/main/java/org/apache/hadoop/hbase/HSnapshotDescriptor.java
<http://review.cloudera.org/r/467/#comment2297>
empty @return
src/main/java/org/apache/hadoop/hbase/HSnapshotDescriptor.java
<http://review.cloudera.org/r/467/#comment2298>
since we're using the snapshot name as a directory name in HDFS, it has to
be a UTF8 string, so why not just keep it as a String above too?
src/main/java/org/apache/hadoop/hbase/TablePartialOpenException.java
<http://review.cloudera.org/r/467/#comment2299>
no need for this javadoc (it's obvious)
src/main/java/org/apache/hadoop/hbase/TablePartialOpenException.java
<http://review.cloudera.org/r/467/#comment2300>
same with this one
src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
<http://review.cloudera.org/r/467/#comment2301>
add "TODO" to this comment
src/main/java/org/apache/hadoop/hbase/io/Reference.java
<http://review.cloudera.org/r/467/#comment2302>
to keep compatibility with current storefiles, "entire" should be value 2,
and bottom should be 0
while we're at it, maybe rename these to be all caps - Range.TOP,
Range.BOTTOM, etc
src/main/java/org/apache/hadoop/hbase/master/BaseScanner.java
<http://review.cloudera.org/r/467/#comment2303>
no need to check size() - iterating the empty array should be fine
src/main/java/org/apache/hadoop/hbase/master/BaseScanner.java
<http://review.cloudera.org/r/467/#comment2304>
if we crash between step 1 and 2, we orphan the archived file. Instead, we
can do the delete first (ignoring failure if it doesn't exist) and then update
META.
src/main/java/org/apache/hadoop/hbase/master/HMaster.java
<http://review.cloudera.org/r/467/#comment2305>
you can just call mkdirs, I think, and it won't fail if it already exists
src/main/java/org/apache/hadoop/hbase/master/HMaster.java
<http://review.cloudera.org/r/467/#comment2306>
should this be an exception, rather than a result code? ie is it normal to
fail?
src/main/java/org/apache/hadoop/hbase/master/HMaster.java
<http://review.cloudera.org/r/467/#comment2309>
do we have a race here? what if the table gets enabled while the snapshot
is being processed? it seems we need some locking here around table status and
snapshot modification
src/main/java/org/apache/hadoop/hbase/master/HMaster.java
<http://review.cloudera.org/r/467/#comment2311>
shouldn't we rethrow in this error case? and in the above error case? ie
these should be clauses like:
boolean success=false;
try {
... make snapshot ...
success = true;
} finally {
if (!success) {
deleteSnapshot();
}
}
src/main/java/org/apache/hadoop/hbase/master/HMaster.java
<http://review.cloudera.org/r/467/#comment2313>
would it be problematic to create a partially written snapshotinfo file? or
would it get cleaned up at a higher layer?
(perhaps worth creating snapshotinfo.tmp, then atomically rename it to
snapshotinfo if it writes correctly)
src/main/java/org/apache/hadoop/hbase/master/SnapshotLogCleaner.java
<http://review.cloudera.org/r/467/#comment2314>
license
src/main/java/org/apache/hadoop/hbase/master/SnapshotLogCleaner.java
<http://review.cloudera.org/r/467/#comment2315>
worth noting that this class is not thread-safe? I don't know if these
classes need to be thread safe, but you're using an unsynchronized hashset.
Also, since refreshHLogsAndSearch clears hlogs before re-adding stuff, it needs
to be synchronized more than just using a synchronized collection.
src/main/java/org/apache/hadoop/hbase/master/SnapshotLogCleaner.java
<http://review.cloudera.org/r/467/#comment2317>
do we have a race here between the listStatus and creating a snapshot?
src/main/java/org/apache/hadoop/hbase/master/SnapshotLogCleaner.java
<http://review.cloudera.org/r/467/#comment2316>
document that it may be null, and what null means? in fact, do we ever call
this with null? it doesn't look like it.
src/main/java/org/apache/hadoop/hbase/master/SnapshotLogCleaner.java
<http://review.cloudera.org/r/467/#comment2318>
do we really want to swallow this IOE?
src/main/java/org/apache/hadoop/hbase/master/SnapshotMonitor.java
<http://review.cloudera.org/r/467/#comment2321>
this is basically a singleton per-master, right? not per-snapshot? ie this
is created at master startup and destroyed at master shutdown, and can
coordinate multiple snapshots over its lifetime? would be good to document how
this fits into the overall design, and perhaps refactor into one part that is
master-global and another part that is created once per-snapshot.
src/main/java/org/apache/hadoop/hbase/master/SnapshotMonitor.java
<http://review.cloudera.org/r/467/#comment2331>
needs to be volatile - waitForFinish accesses this outside of
synchronization
src/main/java/org/apache/hadoop/hbase/master/SnapshotMonitor.java
<http://review.cloudera.org/r/467/#comment2319>
what is this mutex? better to name it based on what objects it
synchronizes, and also use new Object() instead of new Integer(0)
src/main/java/org/apache/hadoop/hbase/master/SnapshotMonitor.java
<http://review.cloudera.org/r/467/#comment2320>
are you sure this is the way we name HMaster's ZKW instance? can we just
grab the existing zkWrapper instance out of the master object?
src/main/java/org/apache/hadoop/hbase/master/SnapshotMonitor.java
<http://review.cloudera.org/r/467/#comment2322>
what's the synchronization story here? who calls this method?
src/main/java/org/apache/hadoop/hbase/master/SnapshotMonitor.java
<http://review.cloudera.org/r/467/#comment2323>
useless doc
src/main/java/org/apache/hadoop/hbase/master/SnapshotMonitor.java
<http://review.cloudera.org/r/467/#comment2325>
better to just do something like:
if (!mkdirs(...)) {
throw IOE("could not create")
}
src/main/java/org/apache/hadoop/hbase/master/SnapshotMonitor.java
<http://review.cloudera.org/r/467/#comment2324>
include path or snapshot name in exception msg
src/main/java/org/apache/hadoop/hbase/master/SnapshotMonitor.java
<http://review.cloudera.org/r/467/#comment2328>
does ZKW automatically re-watch the nodes for you, here?
Also, how does this interact with region server failure? We just assume
that the snapshot will timeout and abort, instead of proactively detecting?
src/main/java/org/apache/hadoop/hbase/master/SnapshotMonitor.java
<http://review.cloudera.org/r/467/#comment2326>
add a LOG.debug perhaps for this case - it seems rare that we'd have the
correct count of servers but be missing one
src/main/java/org/apache/hadoop/hbase/master/SnapshotMonitor.java
<http://review.cloudera.org/r/467/#comment2327>
assert that we're in ALLREADY state here?
src/main/java/org/apache/hadoop/hbase/master/SnapshotMonitor.java
<http://review.cloudera.org/r/467/#comment2329>
log at least?
src/main/java/org/apache/hadoop/hbase/master/SnapshotMonitor.java
<http://review.cloudera.org/r/467/#comment2330>
this should probably be rethrown?
src/main/java/org/apache/hadoop/hbase/master/SnapshotMonitor.java
<http://review.cloudera.org/r/467/#comment2332>
consider rename to M_ALL_RS_READY and M_ALL_RS_FINISHED for clarity
also, what is M?
src/main/java/org/apache/hadoop/hbase/master/SnapshotMonitor.java
<http://review.cloudera.org/r/467/#comment2333>
rename to M_RS_READY and M_RS_FINISHED?
Should these RS-specific ones be in a separate enum? GlobalSnapshotStatus
vs RSSnapshotStatus?
src/main/java/org/apache/hadoop/hbase/master/TableSnapshot.java
<http://review.cloudera.org/r/467/#comment2334>
check return value of mkdirs instead
src/main/java/org/apache/hadoop/hbase/master/TableSnapshot.java
<http://review.cloudera.org/r/467/#comment2335>
info level instead, perhaps? seems like a common issue given we're very
flaky about region enable status currently.
src/main/java/org/apache/hadoop/hbase/master/TableSnapshot.java
<http://review.cloudera.org/r/467/#comment2336>
is there a process that scans for cases where the reference count has
gotten out of sync?
I'm worried about a case where a snapshot is half-done, and then it fails,
so the snapshot is considered aborted, but we never clean up the references
because META has been incremented.
src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
<http://review.cloudera.org/r/467/#comment2337>
update message to include snapshot case
src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
<http://review.cloudera.org/r/467/#comment2338>
rather than returning booleans, it might be better to throw back an
exception to indicate error - this way the snapshot coordinator can actually
show the reason for the failed snapshot, instead of forcing the user to grep
all of the RS logs.
src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
<http://review.cloudera.org/r/467/#comment2339>
what happens if the snapshot coordinator dies before completing the
snapshot? the region is left permanently in snapshot mode?
src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
<http://review.cloudera.org/r/467/#comment2340>
useless doc
src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
<http://review.cloudera.org/r/467/#comment2341>
check return value of mkdirs instead of separately calling exists
src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
<http://review.cloudera.org/r/467/#comment2342>
see above - just call fs.mkdirs
src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
<http://review.cloudera.org/r/467/#comment2343>
this code is duplicated from the master-driven snapshot - perhaps it can be
factored out somewhere
src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
<http://review.cloudera.org/r/467/#comment2344>
this code is also duplicated
src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
<http://review.cloudera.org/r/467/#comment2345>
in this case, though, we've already called startSnapshot on some other
regions - is this problematic?
src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
<http://review.cloudera.org/r/467/#comment2346>
missing " " before "on RS"
src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
<http://review.cloudera.org/r/467/#comment2348>
at this point don't we have to wait for the snapshot coordinator to
"commit" the snapshot?
src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
<http://review.cloudera.org/r/467/#comment2347>
!regionsToBackup.isEmpty()
src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
<http://review.cloudera.org/r/467/#comment2349>
"on RS" -> " on RS" (space)
src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
<http://review.cloudera.org/r/467/#comment2350>
perhaps write to a tmp file then move into place, so it's atomic
src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
<http://review.cloudera.org/r/467/#comment2351>
this seems prone to collision if it's multithreaded, since the exists check
and the use of the filename aren't atomic
src/main/java/org/apache/hadoop/hbase/regionserver/ZKSnapshotWatcher.java
<http://review.cloudera.org/r/467/#comment2352>
since createZNodeIfNotExists already does existance check, you don't need
the .exists above
src/main/java/org/apache/hadoop/hbase/regionserver/ZKSnapshotWatcher.java
<http://review.cloudera.org/r/467/#comment2353>
does ZKWatcher automatically re-watch for you?
src/main/java/org/apache/hadoop/hbase/regionserver/ZKSnapshotWatcher.java
<http://review.cloudera.org/r/467/#comment2354>
should actually wait for the snapshotThread to exit here - otherwise maybe
an aborting one is still doing some work, which might overlap with the next one
src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
<http://review.cloudera.org/r/467/#comment2355>
more examples of checking mkdirs
src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
<http://review.cloudera.org/r/467/#comment2356>
useless javadoc
src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java
<http://review.cloudera.org/r/467/#comment2357>
throw exceptions instead of returning false?
src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java
<http://review.cloudera.org/r/467/#comment2358>
throw exceptions instead of returning false
src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java
<http://review.cloudera.org/r/467/#comment2359>
throw exceptions to user
src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWrapper.java
<http://review.cloudera.org/r/467/#comment2360>
these checks are inherently racy
src/test/java/org/apache/hadoop/hbase/master/TestSnapshot.java
<http://review.cloudera.org/r/467/#comment2361>
license
src/test/java/org/apache/hadoop/hbase/master/TestSnapshotFailure.java
<http://review.cloudera.org/r/467/#comment2362>
license
src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionSnapshot.java
<http://review.cloudera.org/r/467/#comment2363>
license
- Todd
> 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, 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.
-
You can reply to this email to add a comment to the issue online.