You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2013/02/13 19:05:55 UTC

svn commit: r1445782 [2/3] - in /hbase/branches/hbase-7290: hbase-common/src/main/java/org/apache/hadoop/hbase/ hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ hbase-protocol/src/main/protobuf/ hbase-server/src/main/java/org/ap...

Modified: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1445782&r1=1445781&r2=1445782&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Wed Feb 13 18:05:53 2013
@@ -781,27 +781,49 @@ public class HRegion implements HeapSize
     return this.memstoreSize.getAndAdd(memStoreSize);
   }
 
-  /*
-   * Write out an info file under the region directory.  Useful recovering
-   * mangled regions.
+  /**
+   * Write out an info file under the stored region directory. Useful recovering mangled regions.
    * @throws IOException
    */
   private void checkRegioninfoOnFilesystem() throws IOException {
-    Path regioninfoPath = new Path(this.regiondir, REGIONINFO_FILE);
-    // Compose the content of the file so we can compare to length in filesystem.  If not same,
-    // rewrite it (it may have been written in the old format using Writables instead of pb).  The
+    checkRegioninfoOnFilesystem(this.regiondir);
+  }
+
+  /**
+   * Write out an info file under the region directory. Useful recovering mangled regions.
+   * @param regiondir directory under which to write out the region info
+   * @throws IOException
+   */
+  private void checkRegioninfoOnFilesystem(Path regiondir) throws IOException {
+    writeRegioninfoOnFilesystem(regionInfo, regiondir, getFilesystem(), conf);
+  }
+
+  /**
+   * Write out an info file under the region directory. Useful recovering mangled regions. If the
+   * regioninfo already exists on disk and there is information in the file, then we fast exit.
+   * @param regionInfo information about the region
+   * @param regiondir directory under which to write out the region info
+   * @param fs {@link FileSystem} on which to write the region info
+   * @param conf {@link Configuration} from which to extract specific file locations
+   * @throws IOException on unexpected error.
+   */
+  public static void writeRegioninfoOnFilesystem(HRegionInfo regionInfo, Path regiondir,
+      FileSystem fs, Configuration conf) throws IOException {
+    Path regioninfoPath = new Path(regiondir, REGIONINFO_FILE);
+    // Compose the content of the file so we can compare to length in filesystem. If not same,
+    // rewrite it (it may have been written in the old format using Writables instead of pb). The
     // pb version is much shorter -- we write now w/o the toString version -- so checking length
-    // only should be sufficient.  I don't want to read the file every time to check if it pb
+    // only should be sufficient. I don't want to read the file every time to check if it pb
     // serialized.
-    byte [] content = getDotRegionInfoFileContent(this.getRegionInfo());
-    boolean exists = this.fs.exists(regioninfoPath);
-    FileStatus status = exists? this.fs.getFileStatus(regioninfoPath): null;
+    byte[] content = getDotRegionInfoFileContent(regionInfo);
+    boolean exists = fs.exists(regioninfoPath);
+    FileStatus status = exists ? fs.getFileStatus(regioninfoPath) : null;
     if (status != null && status.getLen() == content.length) {
       // Then assume the content good and move on.
       return;
     }
     // Create in tmpdir and then move into place in case we crash after
-    // create but before close.  If we don't successfully close the file,
+    // create but before close. If we don't successfully close the file,
     // subsequent region reopens will fail the below because create is
     // registered in NN.
 
@@ -809,7 +831,7 @@ public class HRegion implements HeapSize
     FsPermission perms = FSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
 
     // And then create the file
-    Path tmpPath = new Path(getTmpDir(), REGIONINFO_FILE);
+    Path tmpPath = new Path(getTmpDir(regiondir), REGIONINFO_FILE);
 
     // If datanode crashes or if the RS goes down just before the close is called while trying to
     // close the created regioninfo file in the .tmp directory then on next
@@ -1251,7 +1273,11 @@ public class HRegion implements HeapSize
    * will have its contents removed when the region is reopened.
    */
   Path getTmpDir() {
-    return new Path(getRegionDir(), REGION_TEMP_SUBDIR);
+    return getTmpDir(getRegionDir());
+  }
+
+  static Path getTmpDir(Path regionDir) {
+    return new Path(regionDir, REGION_TEMP_SUBDIR);
   }
 
   void triggerMajorCompaction() {

Modified: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/errorhandling/impl/ExceptionOrchestrator.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/errorhandling/impl/ExceptionOrchestrator.java?rev=1445782&r1=1445781&r2=1445782&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/errorhandling/impl/ExceptionOrchestrator.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/errorhandling/impl/ExceptionOrchestrator.java Wed Feb 13 18:05:53 2013
@@ -59,7 +59,7 @@ public class ExceptionOrchestrator<E ext
       .create();
 
   /** Error visitor for framework listeners */
-  final ForwardingErrorVisitor genericVisitor = new ForwardingErrorVisitor();
+  public final ForwardingErrorVisitor genericVisitor = new ForwardingErrorVisitor();
 
   public ExceptionOrchestrator() {
     this("generic-error-dispatcher");

Added: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/TakeSnapshotUtils.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/TakeSnapshotUtils.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/TakeSnapshotUtils.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/TakeSnapshotUtils.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,325 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.server.snapshot;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
+import org.apache.hadoop.hbase.server.errorhandling.ExceptionListener;
+import org.apache.hadoop.hbase.server.errorhandling.OperationAttemptTimer;
+import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.snapshot.exception.CorruptedSnapshotException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+
+/**
+ * Utilities for useful when taking a snapshot
+ */
+public class TakeSnapshotUtils {
+
+  private static final Log LOG = LogFactory.getLog(TakeSnapshotUtils.class);
+
+  private TakeSnapshotUtils() {
+    // private constructor for util class
+  }
+
+  /**
+   * Get the per-region snapshot description location.
+   * <p>
+   * Under the per-snapshot directory, specific files per-region are kept in a similar layout as per
+   * the current directory layout.
+   * @param desc description of the snapshot
+   * @param rootDir root directory for the hbase installation
+   * @param regionName encoded name of the region (see {@link HRegionInfo#encodeRegionName(byte[])})
+   * @return path to the per-region directory for the snapshot
+   */
+  public static Path getRegionSnapshotDirectory(SnapshotDescription desc, Path rootDir,
+      String regionName) {
+    Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir);
+    return HRegion.getRegionDir(snapshotDir, regionName);
+  }
+
+  /**
+   * Get the home directory for store-level snapshot files.
+   * <p>
+   * Specific files per store are kept in a similar layout as per the current directory layout.
+   * @param regionDir snapshot directory for the parent region, <b>not</b> the standard region
+   *          directory. See {@link #getRegionSnapshotDirectory(SnapshotDescription, Path, String)}
+   * @param family name of the store to snapshot
+   * @return path to the snapshot home directory for the store/family
+   */
+  public static Path getStoreSnapshotDirectory(Path regionDir, String family) {
+    return HStore.getStoreHomedir(regionDir, Bytes.toBytes(family));
+  }
+
+  /**
+   * Get the snapshot directory for each family to be added to the the snapshot
+   * @param snapshot description of the snapshot being take
+   * @param snapshotRegionDir directory in the snapshot where the region directory information
+   *          should be stored
+   * @param families families to be added (can be null)
+   * @return paths to the snapshot directory for each family, in the same order as the families
+   *         passed in
+   */
+  public static List<Path> getFamilySnapshotDirectories(SnapshotDescription snapshot,
+      Path snapshotRegionDir, FileStatus[] families) {
+    if (families == null || families.length == 0) return Collections.emptyList();
+
+    List<Path> familyDirs = new ArrayList<Path>(families.length);
+    for (FileStatus family : families) {
+      // build the reference directory name
+      familyDirs.add(getStoreSnapshotDirectory(snapshotRegionDir, family.getPath().getName()));
+    }
+    return familyDirs;
+  }
+
+  /**
+   * Create a snapshot timer for the master which notifies the monitor when an error occurs
+   * @param snapshot snapshot to monitor
+   * @param conf configuration to use when getting the max snapshot life
+   * @param monitor monitor to notify when the snapshot life expires
+   * @return the timer to use update to signal the start and end of the snapshot
+   */
+  @SuppressWarnings("rawtypes")
+  public static OperationAttemptTimer getMasterTimerAndBindToMonitor(SnapshotDescription snapshot,
+      Configuration conf, ExceptionListener monitor) {
+    long maxTime = SnapshotDescriptionUtils.getMaxMasterTimeout(conf, snapshot.getType(),
+      SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME);
+    return new OperationAttemptTimer(monitor, maxTime, snapshot);
+  }
+
+  /**
+   * Verify that all the expected logs got referenced
+   * @param fs filesystem where the logs live
+   * @param logsDir original logs directory
+   * @param serverNames names of the servers that involved in the snapshot
+   * @param snapshot description of the snapshot being taken
+   * @param snapshotLogDir directory for logs in the snapshot
+   * @throws IOException
+   */
+  public static void verifyAllLogsGotReferenced(FileSystem fs, Path logsDir,
+      Set<String> serverNames, SnapshotDescription snapshot, Path snapshotLogDir)
+      throws IOException {
+    assertTrue(snapshot, "Logs directory doesn't exist in snapshot", fs.exists(logsDir));
+    // for each of the server log dirs, make sure it matches the main directory
+    Multimap<String, String> snapshotLogs = getMapOfServersAndLogs(fs, snapshotLogDir, serverNames);
+    Multimap<String, String> realLogs = getMapOfServersAndLogs(fs, logsDir, serverNames);
+    if (realLogs != null) {
+      assertNotNull(snapshot, "No server logs added to snapshot", snapshotLogs);
+    } else if (realLogs == null) {
+      assertNull(snapshot, "Snapshotted server logs that don't exist", snapshotLogs);
+    }
+
+    // check the number of servers
+    Set<Entry<String, Collection<String>>> serverEntries = realLogs.asMap().entrySet();
+    Set<Entry<String, Collection<String>>> snapshotEntries = snapshotLogs.asMap().entrySet();
+    assertEquals(snapshot, "Not the same number of snapshot and original server logs directories",
+      serverEntries.size(), snapshotEntries.size());
+
+    // verify we snapshotted each of the log files
+    for (Entry<String, Collection<String>> serverLogs : serverEntries) {
+      // if the server is not the snapshot, skip checking its logs
+      if (!serverNames.contains(serverLogs.getKey())) continue;
+      Collection<String> snapshotServerLogs = snapshotLogs.get(serverLogs.getKey());
+      assertNotNull(snapshot, "Snapshots missing logs for server:" + serverLogs.getKey(),
+        snapshotServerLogs);
+
+      // check each of the log files
+      assertEquals(snapshot,
+        "Didn't reference all the log files for server:" + serverLogs.getKey(), serverLogs
+            .getValue().size(), snapshotServerLogs.size());
+      for (String log : serverLogs.getValue()) {
+        assertTrue(snapshot, "Snapshot logs didn't include " + log,
+          snapshotServerLogs.contains(log));
+      }
+    }
+  }
+
+  /**
+   * Verify one of a snapshot's region's recovered.edits, has been at the surface (file names,
+   * length), match the original directory.
+   * @param fs filesystem on which the snapshot had been taken
+   * @param rootDir full path to the root hbase directory
+   * @param regionInfo info for the region
+   * @param snapshot description of the snapshot that was taken
+   * @throws IOException if there is an unexpected error talking to the filesystem
+   */
+  public static void verifyRecoveredEdits(FileSystem fs, Path rootDir, HRegionInfo regionInfo,
+      SnapshotDescription snapshot) throws IOException {
+    Path regionDir = HRegion.getRegionDir(rootDir, regionInfo);
+    Path editsDir = HLogUtil.getRegionDirRecoveredEditsDir(regionDir);
+    Path snapshotRegionDir = TakeSnapshotUtils.getRegionSnapshotDirectory(snapshot, rootDir,
+      regionInfo.getEncodedName());
+    Path snapshotEditsDir = HLogUtil.getRegionDirRecoveredEditsDir(snapshotRegionDir);
+
+    FileStatus[] edits = FSUtils.listStatus(fs, editsDir);
+    FileStatus[] snapshotEdits = FSUtils.listStatus(fs, snapshotEditsDir);
+    if (edits == null) {
+      assertNull(snapshot, "Snapshot has edits but table doesn't", snapshotEdits);
+      return;
+    }
+
+    assertNotNull(snapshot, "Table has edits, but snapshot doesn't", snapshotEdits);
+
+    // check each of the files
+    assertEquals(snapshot, "Not same number of edits in snapshot as table", edits.length,
+      snapshotEdits.length);
+
+    // make sure we have a file with the same name as the original
+    // it would be really expensive to verify the content matches the original
+    for (FileStatus edit : edits) {
+      for (FileStatus sEdit : snapshotEdits) {
+        if (sEdit.getPath().equals(edit.getPath())) {
+          assertEquals(snapshot, "Snapshot file" + sEdit.getPath()
+              + " length not equal to the original: " + edit.getPath(), edit.getLen(),
+            sEdit.getLen());
+          break;
+        }
+      }
+      assertTrue(snapshot, "No edit in snapshot with name:" + edit.getPath(), false);
+    }
+  }
+
+  private static void assertNull(SnapshotDescription snapshot, String msg, Object isNull)
+      throws CorruptedSnapshotException {
+    if (isNull != null) {
+      throw new CorruptedSnapshotException(msg + ", Expected " + isNull + " to be null.", snapshot);
+    }
+  }
+
+  private static void assertNotNull(SnapshotDescription snapshot, String msg, Object notNull)
+      throws CorruptedSnapshotException {
+    if (notNull == null) {
+      throw new CorruptedSnapshotException(msg + ", Expected object to not be null, but was null.",
+          snapshot);
+    }
+  }
+
+  private static void assertTrue(SnapshotDescription snapshot, String msg, boolean isTrue)
+      throws CorruptedSnapshotException {
+    if (!isTrue) {
+      throw new CorruptedSnapshotException(msg + ", Expected true, but was false", snapshot);
+    }
+  }
+
+  /**
+   * Assert that the expect matches the gotten amount
+   * @param msg message to add the to exception
+   * @param expected
+   * @param gotten
+   * @throws CorruptedSnapshotException thrown if the two elements don't match
+   */
+  private static void assertEquals(SnapshotDescription snapshot, String msg, int expected,
+      int gotten) throws CorruptedSnapshotException {
+    if (expected != gotten) {
+      throw new CorruptedSnapshotException(msg + ". Expected:" + expected + ", got:" + gotten,
+          snapshot);
+    }
+  }
+
+  /**
+   * Assert that the expect matches the gotten amount
+   * @param msg message to add the to exception
+   * @param expected
+   * @param gotten
+   * @throws CorruptedSnapshotException thrown if the two elements don't match
+   */
+  private static void assertEquals(SnapshotDescription snapshot, String msg, long expected,
+      long gotten) throws CorruptedSnapshotException {
+    if (expected != gotten) {
+      throw new CorruptedSnapshotException(msg + ". Expected:" + expected + ", got:" + gotten,
+          snapshot);
+    }
+  }
+
+  /**
+   * @param logdir
+   * @param toInclude list of servers to include. If empty or null, returns all servers
+   * @return maps of servers to all their log files. If there is no log directory, returns
+   *         <tt>null</tt>
+   */
+  private static Multimap<String, String> getMapOfServersAndLogs(FileSystem fs, Path logdir,
+      Collection<String> toInclude) throws IOException {
+    // create a path filter based on the passed directories to include
+    PathFilter filter = toInclude == null || toInclude.size() == 0 ? null
+        : new MatchesDirectoryNames(toInclude);
+
+    // get all the expected directories
+    FileStatus[] serverLogDirs = FSUtils.listStatus(fs, logdir, filter);
+    if (serverLogDirs == null) return null;
+
+    // map those into a multimap of servername -> [log files]
+    Multimap<String, String> map = HashMultimap.create();
+    for (FileStatus server : serverLogDirs) {
+      FileStatus[] serverLogs = FSUtils.listStatus(fs, server.getPath(), null);
+      if (serverLogs == null) continue;
+      for (FileStatus log : serverLogs) {
+        map.put(server.getPath().getName(), log.getPath().getName());
+      }
+    }
+    return map;
+  }
+
+  /**
+   * Path filter that only accepts paths where that have a {@link Path#getName()} that is contained
+   * in the specified collection.
+   */
+  private static class MatchesDirectoryNames implements PathFilter {
+
+    Collection<String> paths;
+
+    public MatchesDirectoryNames(Collection<String> dirNames) {
+      this.paths = dirNames;
+    }
+
+    @Override
+    public boolean accept(Path path) {
+      return paths.contains(path.getName());
+    }
+  }
+
+  /**
+   * Get the log directory for a specific snapshot
+   * @param snapshotDir directory where the specific snapshot will be store
+   * @param serverName name of the parent regionserver for the log files
+   * @return path to the log home directory for the archive files.
+   */
+  public static Path getSnapshotHLogsDir(Path snapshotDir, String serverName) {
+    return new Path(snapshotDir, HLogUtil.getHLogDirectoryName(serverName));
+  }
+}
\ No newline at end of file

Added: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/error/SnapshotExceptionSnare.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/error/SnapshotExceptionSnare.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/error/SnapshotExceptionSnare.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/error/SnapshotExceptionSnare.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.server.snapshot.error;
+
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.server.errorhandling.impl.ExceptionSnare;
+import org.apache.hadoop.hbase.snapshot.exception.HBaseSnapshotException;
+import org.apache.hadoop.hbase.snapshot.exception.UnexpectedSnapshotException;
+
+/**
+ * {@link ExceptionSnare} for snapshot exceptions, ensuring that only the first exception is
+ * retained and always returned via {@link #failOnError()}.
+ * <p>
+ * Ensures that any generic exceptions received via
+ * {@link #receiveError(String, HBaseSnapshotException, Object...)} are in fact propagated as
+ * {@link HBaseSnapshotException}.
+ */
+public class SnapshotExceptionSnare extends ExceptionSnare<HBaseSnapshotException> implements
+    SnapshotFailureListener {
+
+  private SnapshotDescription snapshot;
+
+  /**
+   * Create a snare that expects errors for the passed snapshot. Any untyped exceptions passed to
+   * {@link #receiveError(String, HBaseSnapshotException, Object...)} are wrapped as an
+   * {@link UnexpectedSnapshotException} with the passed {@link SnapshotDescription}.
+   * @param snapshot
+   */
+  public SnapshotExceptionSnare(SnapshotDescription snapshot) {
+    this.snapshot = snapshot;
+  }
+
+  @Override
+  public void snapshotFailure(String reason, SnapshotDescription snapshot) {
+    this.receiveError(reason, null, snapshot);
+  }
+
+  @Override
+  public void snapshotFailure(String reason, SnapshotDescription snapshot, Exception t) {
+    this.receiveError(reason,
+      t instanceof HBaseSnapshotException ? (HBaseSnapshotException) t
+          : new UnexpectedSnapshotException(reason, t, snapshot), snapshot);
+  }
+
+  @Override
+  public void failOnError() throws HBaseSnapshotException {
+    try {
+      super.failOnError();
+    } catch (Exception e) {
+      if (e instanceof HBaseSnapshotException) {
+        throw (HBaseSnapshotException) e;
+      }
+      throw new UnexpectedSnapshotException(e.getMessage(), e, snapshot);
+    }
+  }
+}

Added: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/error/SnapshotFailureListener.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/error/SnapshotFailureListener.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/error/SnapshotFailureListener.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/error/SnapshotFailureListener.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.server.snapshot.error;
+
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+
+/**
+ * Generic running snapshot failure listener
+ */
+public interface SnapshotFailureListener {
+
+  /**
+   * Notification that a given snapshot failed because of an error on the local server
+   * @param snapshot snapshot that failed
+   * @param reason explanation of why the snapshot failed
+   */
+  public void snapshotFailure(String reason, SnapshotDescription snapshot);
+
+  /**
+   * Notification that a given snapshot failed because of an error on the local server
+   * @param reason reason the snapshot failed
+   * @param snapshot the snapshot that failed
+   * @param t the exception that caused the failure
+   */
+  public void snapshotFailure(String reason, SnapshotDescription snapshot, Exception t);
+}

Added: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/CopyRecoveredEditsTask.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/CopyRecoveredEditsTask.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/CopyRecoveredEditsTask.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/CopyRecoveredEditsTask.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.server.snapshot.task;
+
+import java.io.IOException;
+import java.util.NavigableSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
+import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
+
+/**
+ * Copy over each of the files in a region's recovered.edits directory to the region's snapshot
+ * directory.
+ * <p>
+ * This is a serial operation over each of the files in the recovered.edits directory and also
+ * streams all the bytes to the client and then back to the filesystem, so the files being copied
+ * should be <b>small</b> or it will (a) suck up a lot of bandwidth, and (b) take a long time.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class CopyRecoveredEditsTask extends SnapshotTask {
+
+  private static final Log LOG = LogFactory.getLog(CopyRecoveredEditsTask.class);
+  private final FileSystem fs;
+  private final Path regiondir;
+  private final Path outputDir;
+
+  /**
+   * @param snapshot Snapshot being taken
+   * @param monitor error monitor for the snapshot
+   * @param fs {@link FileSystem} where the snapshot is being taken
+   * @param regionDir directory for the region to examine for edits
+   * @param snapshotRegionDir directory for the region in the snapshot
+   */
+  public CopyRecoveredEditsTask(SnapshotDescription snapshot, SnapshotExceptionSnare monitor,
+      FileSystem fs, Path regionDir, Path snapshotRegionDir) {
+    super(snapshot, monitor, "Copy recovered.edits for region:" + regionDir.getName());
+    this.fs = fs;
+    this.regiondir = regionDir;
+    this.outputDir = HLogUtil.getRegionDirRecoveredEditsDir(snapshotRegionDir);
+  }
+
+  @Override
+  public void process() throws IOException {
+    NavigableSet<Path> files = HLogUtil.getSplitEditFilesSorted(this.fs, regiondir);
+    if (files == null || files.size() == 0) return;
+
+    // copy over each file.
+    // this is really inefficient (could be trivially parallelized), but is
+    // really simple to reason about.
+    for (Path source : files) {
+      // check to see if the file is zero length, in which case we can skip it
+      FileStatus stat = fs.getFileStatus(source);
+      if (stat.getLen() <= 0) continue;
+
+      // its not zero length, so copy over the file
+      Path out = new Path(outputDir, source.getName());
+      LOG.debug("Copying " + source + " to " + out);
+      FileUtil.copy(fs, source, fs, out, true, fs.getConf());
+
+      // check for errors to the running operation after each file
+      this.failOnError();
+    }
+  }
+}
\ No newline at end of file

Added: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/ReferenceRegionHFilesTask.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/ReferenceRegionHFilesTask.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/ReferenceRegionHFilesTask.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/ReferenceRegionHFilesTask.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.server.snapshot.task;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.server.snapshot.TakeSnapshotUtils;
+import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+/**
+ * Reference all the hfiles in a region for a snapshot.
+ * <p>
+ * Doesn't take into acccount if the hfiles are valid or not, just keeps track of what's in the
+ * region's directory.
+ */
+public class ReferenceRegionHFilesTask extends SnapshotTask {
+
+  public static final Log LOG = LogFactory.getLog(ReferenceRegionHFilesTask.class);
+  private final Path regiondir;
+  private final FileSystem fs;
+  private final PathFilter fileFilter;
+  private final Path snapshotDir;
+
+  /**
+   * Reference all the files in the given region directory
+   * @param snapshot snapshot for which to add references
+   * @param monitor to check/send error
+   * @param regionDir region directory to look for errors
+   * @param fs {@link FileSystem} where the snapshot/region live
+   * @param regionSnapshotDir directory in the snapshot to store region files
+   */
+  public ReferenceRegionHFilesTask(final SnapshotDescription snapshot,
+      SnapshotExceptionSnare monitor, Path regionDir, final FileSystem fs, Path regionSnapshotDir) {
+    super(snapshot, monitor, "Reference hfiles for region:" + regionDir.getName());
+    this.regiondir = regionDir;
+    this.fs = fs;
+
+    this.fileFilter = new PathFilter() {
+      @Override
+      public boolean accept(Path path) {
+        try {
+          return fs.isFile(path);
+        } catch (IOException e) {
+          LOG.error("Failed to reach fs to check file:" + path + ", marking as not file");
+          ReferenceRegionHFilesTask.this.snapshotFailure("Failed to reach fs to check file status",
+            e);
+          return false;
+        }
+      }
+    };
+    this.snapshotDir = regionSnapshotDir;
+  }
+
+  @Override
+  public void process() throws IOException {
+    FileStatus[] families = FSUtils.listStatus(fs, regiondir, new FSUtils.FamilyDirFilter(fs));
+
+    // if no families, then we are done again
+    if (families == null || families.length == 0) {
+      LOG.info("No families under region directory:" + regiondir
+          + ", not attempting to add references.");
+      return;
+    }
+
+    // snapshot directories to store the hfile reference
+    List<Path> snapshotFamilyDirs = TakeSnapshotUtils.getFamilySnapshotDirectories(snapshot,
+      snapshotDir, families);
+
+    LOG.debug("Add hfile references to snapshot directories:" + snapshotFamilyDirs);
+    for (int i = 0; i < families.length; i++) {
+      FileStatus family = families[i];
+      Path familyDir = family.getPath();
+      // get all the hfiles in the family
+      FileStatus[] hfiles = FSUtils.listStatus(fs, familyDir, fileFilter);
+
+      // if no hfiles, then we are done with this family
+      if (hfiles == null || hfiles.length == 0) {
+        LOG.debug("Not hfiles found for family: " + familyDir + ", skipping.");
+        continue;
+      }
+
+      // make the snapshot's family directory
+      Path snapshotFamilyDir = snapshotFamilyDirs.get(i);
+      fs.mkdirs(snapshotFamilyDir);
+
+      // create a reference for each hfile
+      for (FileStatus hfile : hfiles) {
+        Path referenceFile = new Path(snapshotFamilyDir, hfile.getPath().getName());
+        LOG.debug("Creating reference for:" + hfile.getPath() + " at " + referenceFile);
+        if (!fs.createNewFile(referenceFile)) {
+          throw new IOException("Failed to create reference file:" + referenceFile);
+        }
+      }
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Finished referencing hfiles, current region state:");
+      FSUtils.logFileSystemState(fs, regiondir, LOG);
+      LOG.debug("and the snapshot directory:");
+      FSUtils.logFileSystemState(fs, snapshotDir, LOG);
+    }
+  }
+}
\ No newline at end of file

Added: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/ReferenceServerWALsTask.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/ReferenceServerWALsTask.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/ReferenceServerWALsTask.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/ReferenceServerWALsTask.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,103 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.server.snapshot.task;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.server.snapshot.TakeSnapshotUtils;
+import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
+import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+/**
+ * Reference all the WAL files under a server's WAL directory
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ReferenceServerWALsTask extends SnapshotTask {
+  private static final Log LOG = LogFactory.getLog(ReferenceServerWALsTask.class);
+  // XXX does this need to be HasThread?
+  private final FileSystem fs;
+  private final Configuration conf;
+  private final String serverName;
+  private Path logDir;
+
+  /**
+   * @param snapshot snapshot being run
+   * @param failureListener listener to check for errors while running the operation and to
+   *          propagate errors found while running the task
+   * @param logDir log directory for the server. Name of the directory is taken as the name of the
+   *          server
+   * @param conf {@link Configuration} to extract fileystem information
+   * @param fs filesystem where the log files are stored and should be referenced
+   * @throws IOException
+   */
+  public ReferenceServerWALsTask(SnapshotDescription snapshot,
+      SnapshotExceptionSnare failureListener, final Path logDir, final Configuration conf,
+      final FileSystem fs) throws IOException {
+    super(snapshot, failureListener, "Reference WALs for server:" + logDir.getName());
+    this.fs = fs;
+    this.conf = conf;
+    this.serverName = logDir.getName();
+    this.logDir = logDir;
+  }
+
+  @Override
+  public void process() throws IOException {
+    // TODO switch to using a single file to reference all required WAL files
+    // Iterate through each of the log files and add a reference to it.
+    // assumes that all the files under the server's logs directory is a log
+    FileStatus[] serverLogs = FSUtils.listStatus(fs, logDir, null);
+    if (serverLogs == null) LOG.info("No logs for server directory:" + logDir
+        + ", done referencing files.");
+
+    if (LOG.isDebugEnabled()) LOG.debug("Adding references for WAL files:"
+        + Arrays.toString(serverLogs));
+
+    for (FileStatus file : serverLogs) {
+      this.failOnError();
+
+      // TODO - switch to using MonitoredTask
+      // add the reference to the file
+      // 0. Build a reference path based on the file name
+      // get the current snapshot directory
+      Path rootDir = FSUtils.getRootDir(conf);
+      Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(this.snapshot, rootDir);
+      Path snapshotLogDir = TakeSnapshotUtils.getSnapshotHLogsDir(snapshotDir, serverName);
+      // actually store the reference on disk (small file)
+      Path ref = new Path(snapshotLogDir, file.getPath().getName());
+      if (!fs.createNewFile(ref)) {
+        if (!fs.exists(ref)) {
+          throw new IOException("Couldn't create reference for:" + file.getPath());
+        }
+      }
+      LOG.debug("Completed WAL referencing for: " + file.getPath() + " to " + ref);
+    }
+    LOG.debug("Successfully completed WAL referencing for ALL files");
+  }
+}
\ No newline at end of file

Added: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/SnapshotTask.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/SnapshotTask.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/SnapshotTask.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/SnapshotTask.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.server.snapshot.task;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.server.errorhandling.ExceptionCheckable;
+import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
+import org.apache.hadoop.hbase.snapshot.exception.HBaseSnapshotException;
+
+/**
+ * General snapshot operation taken on a regionserver
+ */
+public abstract class SnapshotTask implements ExceptionCheckable<HBaseSnapshotException>, Runnable {
+
+  private static final Log LOG = LogFactory.getLog(SnapshotTask.class);
+
+  private final SnapshotExceptionSnare errorMonitor;
+  private final String desc;
+
+  protected final SnapshotDescription snapshot;
+
+  /**
+   * @param snapshot Description of the snapshot we are going to operate on
+   * @param monitor listener interested in failures to the snapshot caused by this operation
+   * @param description description of the task being run, for logging
+   */
+  public SnapshotTask(SnapshotDescription snapshot, SnapshotExceptionSnare monitor,
+      String description) {
+    this.snapshot = snapshot;
+    this.errorMonitor = monitor;
+    this.desc = description;
+  }
+
+  protected final void snapshotFailure(String message, Exception e) {
+    this.errorMonitor.snapshotFailure(message, this.snapshot, e);
+  }
+
+  @Override
+  public void failOnError() throws HBaseSnapshotException {
+    this.errorMonitor.failOnError();
+  }
+
+  @Override
+  public boolean checkForError() {
+    return this.errorMonitor.checkForError();
+  }
+
+  @Override
+  public void run() {
+    try {
+      LOG.debug("Running: " + desc);
+      this.process();
+    } catch (Exception e) {
+      this.snapshotFailure("Failed to run " + this.desc, e);
+    }
+  }
+
+  /**
+   * Run the task for the snapshot.
+   * @throws Exception if the task fails. Will be propagated to any other tasks watching the same
+   *           {@link SnapshotErrorListener}.
+   */
+  protected abstract void process() throws Exception;
+}

Added: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/TableInfoCopyTask.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/TableInfoCopyTask.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/TableInfoCopyTask.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/server/snapshot/task/TableInfoCopyTask.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.server.snapshot.task;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.server.snapshot.error.SnapshotExceptionSnare;
+import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSTableDescriptors;
+
+/**
+ * Copy the table info into the snapshot directory
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class TableInfoCopyTask extends SnapshotTask {
+
+  public static final Log LOG = LogFactory.getLog(TableInfoCopyTask.class);
+  private final FileSystem fs;
+  private final Path rootDir;
+
+  /**
+   * Copy the table info for the given table into the snapshot
+   * @param failureListener listen for errors while running the snapshot
+   * @param snapshot snapshot for which we are copying the table info
+   * @param fs {@link FileSystem} where the tableinfo is stored (and where the copy will be written)
+   * @param rootDir root of the {@link FileSystem} where the tableinfo is stored
+   */
+  public TableInfoCopyTask(SnapshotExceptionSnare failureListener, SnapshotDescription snapshot,
+      FileSystem fs, Path rootDir) {
+    super(snapshot, failureListener, "Copy table info for table: " + snapshot.getTable());
+    this.rootDir = rootDir;
+    this.fs = fs;
+  }
+
+  @Override
+  public void process() throws IOException {
+    LOG.debug("Running table info copy.");
+    this.failOnError();
+    LOG.debug("Attempting to copy table info for snapshot:" + this.snapshot);
+    // get the HTable descriptor
+    HTableDescriptor orig = FSTableDescriptors.getTableDescriptor(fs, rootDir,
+      Bytes.toBytes(this.snapshot.getTable()));
+
+    this.failOnError();
+    // write a copy of descriptor to the snapshot directory
+    Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
+    FSTableDescriptors.createTableDescriptorForTableDirectory(fs, snapshotDir, orig, false);
+    LOG.debug("Finished copying tableinfo.");
+  }
+}
\ No newline at end of file

Modified: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java?rev=1445782&r1=1445781&r2=1445782&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java Wed Feb 13 18:05:53 2013
@@ -17,19 +17,128 @@
  */
 package org.apache.hadoop.hbase.snapshot;
 
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.snapshot.exception.CorruptedSnapshotException;
+import org.apache.hadoop.hbase.snapshot.exception.SnapshotCreationException;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
 
 /**
  * Utility class to help manage {@link SnapshotDescription SnapshotDesriptions}.
+ * <p>
+ * Snapshots are laid out on disk like this:
+ * 
+ * <pre>
+ * /hbase/.snapshots
+ *          /.tmp                <---- working directory
+ *          /[snapshot name]     <----- completed snapshot
+ * </pre>
+ * 
+ * A completed snapshot named 'completed' then looks like (multiple regions, servers, files, etc.
+ * signified by '...' on the same directory depth).
+ * 
+ * <pre>
+ * /hbase/.snapshots/completed
+ *                   .snapshotinfo          <--- Description of the snapshot
+ *                   .tableinfo             <--- Copy of the tableinfo
+ *                    /.logs
+ *                        /[server_name]
+ *                            /... [log files]
+ *                         ...
+ *                   /[region name]           <---- All the region's information
+ *                   .regioninfo              <---- Copy of the HRegionInfo
+ *                      /[column family name]
+ *                          /[hfile name]     <--- name of the hfile in the real region
+ *                          ...
+ *                      ...
+ *                    ...
+ * </pre>
+ * 
+ * Utility methods in this class are useful for getting the correct locations for different parts of
+ * the snapshot, as well as moving completed snapshots into place (see
+ * {@link #completeSnapshot(SnapshotDescription, Path, Path, FileSystem)}, and writing the
+ * {@link SnapshotDescription} to the working snapshot directory.
  */
 public class SnapshotDescriptionUtils {
 
+  /**
+   * Filter that only accepts completed snapshot directories
+   */
+  public static class CompletedSnaphotDirectoriesFilter extends FSUtils.DirFilter {
+
+    /**
+     * @param fs
+     */
+    public CompletedSnaphotDirectoriesFilter(FileSystem fs) {
+      super(fs);
+    }
+
+    @Override
+    public boolean accept(Path path) {
+      // only accept directories that aren't the tmp directory
+      if (super.accept(path)) {
+        return !path.getName().equals(SNAPSHOT_TMP_DIR_NAME);
+      }
+      return false;
+    }
+
+  }
+
+  private static final Log LOG = LogFactory.getLog(SnapshotDescriptionUtils.class);
+  /**
+   * Version of the fs layout for a snapshot. Future snapshots may have different file layouts,
+   * which we may need to read in differently.
+   */
+  public static final int SNAPSHOT_LAYOUT_VERSION = 0;
+
+  // snapshot directory constants
+  /**
+   * The file contains the snapshot basic information and it is under the directory of a snapshot.
+   */
+  public static final String SNAPSHOTINFO_FILE = ".snapshotinfo";
+
+  private static final String SNAPSHOT_TMP_DIR_NAME = ".tmp";
+  // snapshot operation values
+  /** Default value if no start time is specified */
+  public static final long NO_SNAPSHOT_START_TIME_SPECIFIED = 0;
+
+  public static final String MASTER_WAIT_TIME_GLOBAL_SNAPSHOT = "hbase.snapshot.global.master.timeout";
+  public static final String REGION_WAIT_TIME_GLOBAL_SNAPSHOT = "hbase.snapshot.global.region.timeout";
+  public static final String MASTER_WAIT_TIME_TIMESTAMP_SNAPSHOT = "hbase.snapshot.timestamp.master.timeout";
+  public static final String REGION_WAIT_TIME_TIMESTAMP_SNAPSHOT = "hbase.snapshot.timestamp.region.timeout";
+  public static final String MASTER_WAIT_TIME_DISABLED_SNAPSHOT = "hbase.snapshot.disabled.master.timeout";
+
+  /** Default timeout of 60 sec for a snapshot timeout on a region */
+  public static final long DEFAULT_REGION_SNAPSHOT_TIMEOUT = 60000;
+
+  /** By default, wait 60 seconds for a snapshot to complete */
+  public static final long DEFAULT_MAX_WAIT_TIME = 60000;
+
+  /**
+   * Conf key for amount of time the in the future a timestamp snapshot should be taken (ms).
+   * Defaults to {@value SnapshotDescriptionUtils#DEFAULT_TIMESTAMP_SNAPSHOT_SPLIT_IN_FUTURE}
+   */
+  public static final String TIMESTAMP_SNAPSHOT_SPLIT_POINT_ADDITION = "hbase.snapshot.timestamp.master.splittime";
+  /** Start 2 seconds in the future, if no start time given */
+  public static final long DEFAULT_TIMESTAMP_SNAPSHOT_SPLIT_IN_FUTURE = 2000;
+
   private SnapshotDescriptionUtils() {
     // private constructor for utility class
   }
-  
+
   /**
    * Check to make sure that the description of the snapshot requested is valid
    * @param snapshot description of the snapshot
@@ -44,4 +153,219 @@ public class SnapshotDescriptionUtils {
     // make sure the table name is valid
     HTableDescriptor.isLegalTableName(Bytes.toBytes(snapshot.getTable()));
   }
+
+  /**
+   * @param conf {@link Configuration} from which to check for the timeout
+   * @param type type of snapshot being taken
+   * @param defaultMaxWaitTime Default amount of time to wait, if none is in the configuration
+   * @return the max amount of time the master should wait for a snapshot to complete
+   */
+  public static long getMaxMasterTimeout(Configuration conf, SnapshotDescription.Type type,
+      long defaultMaxWaitTime) {
+    String confKey;
+    switch (type) {
+    case GLOBAL:
+      confKey = MASTER_WAIT_TIME_GLOBAL_SNAPSHOT;
+      break;
+    case TIMESTAMP:
+
+      confKey = MASTER_WAIT_TIME_TIMESTAMP_SNAPSHOT;
+    case DISABLED:
+    default:
+      confKey = MASTER_WAIT_TIME_DISABLED_SNAPSHOT;
+    }
+    return conf.getLong(confKey, defaultMaxWaitTime);
+  }
+
+  /**
+   * @param conf {@link Configuration} from which to check for the timeout
+   * @param type type of snapshot being taken
+   * @param defaultMaxWaitTime Default amount of time to wait, if none is in the configuration
+   * @return the max amount of time the region should wait for a snapshot to complete
+   */
+  public static long getMaxRegionTimeout(Configuration conf, SnapshotDescription.Type type,
+      long defaultMaxWaitTime) {
+    String confKey;
+    switch (type) {
+    case GLOBAL:
+      confKey = REGION_WAIT_TIME_GLOBAL_SNAPSHOT;
+      break;
+    case TIMESTAMP:
+    default:
+      confKey = REGION_WAIT_TIME_TIMESTAMP_SNAPSHOT;
+    }
+    return conf.getLong(confKey, defaultMaxWaitTime);
+  }
+
+  /**
+   * Get the snapshot root directory. All the snapshots are kept under this directory, i.e.
+   * ${hbase.rootdir}/.snapshot
+   * @param rootDir hbase root directory
+   * @return the base directory in which all snapshots are kept
+   */
+  public static Path getSnapshotRootDir(final Path rootDir) {
+    return new Path(rootDir, HConstants.SNAPSHOT_DIR_NAME);
+  }
+
+  /**
+   * Get the directory for a specified snapshot. This directory is a sub-directory of snapshot root
+   * directory and all the data files for a snapshot are kept under this directory.
+   * @param snapshot snapshot being taken
+   * @param rootDir hbase root directory
+   * @return the final directory for the completed snapshot
+   */
+  public static Path getCompletedSnapshotDir(final SnapshotDescription snapshot, final Path rootDir) {
+    return getCompletedSnapshotDir(snapshot.getName(), rootDir);
+  }
+
+  /**
+   * Get the directory for a completed snapshot. This directory is a sub-directory of snapshot root
+   * directory and all the data files for a snapshot are kept under this directory.
+   * @param snapshotName name of the snapshot being taken
+   * @param rootDir hbase root directory
+   * @return the final directory for the completed snapshot
+   */
+  public static Path getCompletedSnapshotDir(final String snapshotName, final Path rootDir) {
+    return getCompletedSnapshotDir(getSnapshotsDir(rootDir), snapshotName);
+  }
+
+  /**
+   * Get the directory to build a snapshot, before it is finalized
+   * @param snapshot snapshot that will be built
+   * @param rootDir root directory of the hbase installation
+   * @return {@link Path} where one can build a snapshot
+   */
+  public static Path getWorkingSnapshotDir(SnapshotDescription snapshot, final Path rootDir) {
+    return getCompletedSnapshotDir(new Path(getSnapshotsDir(rootDir), SNAPSHOT_TMP_DIR_NAME),
+      snapshot.getName());
+  }
+
+  /**
+   * Get the directory to store the snapshot instance
+   * @param snapshotsDir hbase-global directory for storing all snapshots
+   * @param snapshotName name of the snapshot to take
+   * @return
+   */
+  private static final Path getCompletedSnapshotDir(final Path snapshotsDir, String snapshotName) {
+    return new Path(snapshotsDir, snapshotName);
+  }
+
+  /**
+   * @param rootDir hbase root directory
+   * @return the directory for all completed snapshots;
+   */
+  public static final Path getSnapshotsDir(Path rootDir) {
+    return new Path(rootDir, HConstants.SNAPSHOT_DIR_NAME);
+  }
+
+  /**
+   * Convert the passed snapshot description into a 'full' snapshot description based on default
+   * parameters, if none have been supplied. This resolves any 'optional' parameters that aren't
+   * supplied to their default values.
+   * @param snapshot general snapshot descriptor
+   * @param conf Configuration to read configured snapshot defaults if snapshot is not complete
+   * @return a valid snapshot description
+   * @throws IllegalArgumentException if the {@link SnapshotDescription} is not a complete
+   *           {@link SnapshotDescription}.
+   */
+  public static SnapshotDescription validate(SnapshotDescription snapshot, Configuration conf)
+      throws IllegalArgumentException {
+    if (!snapshot.hasTable()) {
+      throw new IllegalArgumentException(
+        "Descriptor doesn't apply to a table, so we can't build it.");
+    }
+
+    // set the creation time, if one hasn't been set
+    long time = snapshot.getCreationTime();
+    if (time == SnapshotDescriptionUtils.NO_SNAPSHOT_START_TIME_SPECIFIED) {
+      time = EnvironmentEdgeManager.currentTimeMillis();
+      if (snapshot.getType().equals(SnapshotDescription.Type.TIMESTAMP)) {
+        long increment = conf.getLong(
+          SnapshotDescriptionUtils.TIMESTAMP_SNAPSHOT_SPLIT_POINT_ADDITION,
+          SnapshotDescriptionUtils.DEFAULT_TIMESTAMP_SNAPSHOT_SPLIT_IN_FUTURE);
+        LOG.debug("Setting timestamp snasphot in future by " + increment + " ms.");
+        time += increment;
+      }
+      LOG.debug("Creation time not specified, setting to:" + time + " (current time:"
+          + EnvironmentEdgeManager.currentTimeMillis() + ").");
+      SnapshotDescription.Builder builder = snapshot.toBuilder();
+      builder.setCreationTime(time);
+      snapshot = builder.build();
+    }
+    return snapshot;
+  }
+
+  /**
+   * Write the snapshot description into the working directory of a snapshot
+   * @param snapshot description of the snapshot being taken
+   * @param workingDir working directory of the snapshot
+   * @param fs {@link FileSystem} on which the snapshot should be taken
+   * @throws IOException if we can't reach the filesystem and the file cannot be cleaned up on
+   *           failure
+   */
+  public static void writeSnapshotInfo(SnapshotDescription snapshot, Path workingDir, FileSystem fs)
+      throws IOException {
+    FsPermission perms = FSUtils.getFilePermissions(fs, fs.getConf(),
+      HConstants.DATA_FILE_UMASK_KEY);
+    Path snapshotInfo = new Path(workingDir, SnapshotDescriptionUtils.SNAPSHOTINFO_FILE);
+    try {
+      FSDataOutputStream out = FSUtils.create(fs, snapshotInfo, perms, true);
+      try {
+        snapshot.writeTo(out);
+      } finally {
+        out.close();
+      }
+    } catch (IOException e) {
+      // if we get an exception, try to remove the snapshot info
+      if (!fs.delete(snapshotInfo, false)) {
+        String msg = "Couldn't delete snapshot info file: " + snapshotInfo;
+        LOG.error(msg);
+        throw new IOException(msg);
+      }
+    }
+  }
+
+  /**
+   * Read in the {@link SnapshotDescription} stored for the snapshot in the passed directory
+   * @param fs filesystem where the snapshot was taken
+   * @param snapshotDir directory where the snapshot was stored
+   * @return the stored snapshot description
+   * @throws CorruptedSnapshotException if the snapshot cannot be read
+   */
+  public static SnapshotDescription readSnapshotInfo(FileSystem fs, Path snapshotDir)
+      throws CorruptedSnapshotException {
+    Path snapshotInfo = new Path(snapshotDir, SNAPSHOTINFO_FILE);
+    try {
+      FSDataInputStream in = null;
+      try {
+        in = fs.open(snapshotInfo);
+        return SnapshotDescription.parseFrom(in);
+      } finally {
+        if (in != null) in.close();
+      }
+    } catch (IOException e) {
+      throw new CorruptedSnapshotException("Couldn't read snapshot info from:" + snapshotInfo, e);
+    }
+  }
+
+  /**
+   * Move the finished snapshot to its final, publicly visible directory - this marks the snapshot
+   * as 'complete'.
+   * @param snapshot description of the snapshot being tabken
+   * @param rootdir root directory of the hbase installation
+   * @param workingDir directory where the in progress snapshot was built
+   * @param fs {@link FileSystem} where the snapshot was built
+   * @throws SnapshotCreationException if the snapshot could not be moved
+   * @throws IOException the filesystem could not be reached
+   */
+  public static void completeSnapshot(SnapshotDescription snapshot, Path rootdir, Path workingDir,
+      FileSystem fs) throws SnapshotCreationException, IOException {
+    Path finishedDir = getCompletedSnapshotDir(snapshot, rootdir);
+    LOG.debug("Snapshot is done, just moving the snapshot from " + workingDir + " to "
+        + finishedDir);
+    if (!fs.rename(workingDir, finishedDir)) {
+      throw new SnapshotCreationException("Failed to move working directory(" + workingDir
+          + ") to completed directory(" + finishedDir + ").", snapshot);
+    }
+  }
 }
\ No newline at end of file

Added: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/CorruptedSnapshotException.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/CorruptedSnapshotException.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/CorruptedSnapshotException.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/CorruptedSnapshotException.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.snapshot.exception;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+
+
+/**
+ * Exception thrown when the found snapshot info from the filesystem is not valid
+ */
+@SuppressWarnings("serial")
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class CorruptedSnapshotException extends HBaseSnapshotException {
+
+  /**
+   * @param message message describing the exception
+   * @param e cause
+   */
+  public CorruptedSnapshotException(String message, Exception e) {
+    super(message, e);
+  }
+
+  /**
+   * Snapshot was corrupt for some reason
+   * @param message full description of the failure
+   * @param snapshot snapshot that was expected
+   */
+  public CorruptedSnapshotException(String message, SnapshotDescription snapshot) {
+    super(message, snapshot);
+  }
+}

Added: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/HBaseSnapshotException.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/HBaseSnapshotException.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/HBaseSnapshotException.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/HBaseSnapshotException.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.snapshot.exception;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+
+/**
+ * General exception base class for when a snapshot fails
+ */
+@SuppressWarnings("serial")
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class HBaseSnapshotException extends HBaseIOException {
+
+  private SnapshotDescription description;
+
+  /**
+   * Some exception happened for a snapshot and don't even know the snapshot that it was about
+   * @param msg Full description of the failure
+   */
+  public HBaseSnapshotException(String msg) {
+    super(msg);
+  }
+
+  /**
+   * Exception for the given snapshot that has no previous root cause
+   * @param msg reason why the snapshot failed
+   * @param desc description of the snapshot that is being failed
+   */
+  public HBaseSnapshotException(String msg, SnapshotDescription desc) {
+    super(msg);
+    this.description = desc;
+  }
+
+  /**
+   * Exception for the given snapshot due to another exception
+   * @param msg reason why the snapshot failed
+   * @param cause root cause of the failure
+   * @param desc description of the snapshot that is being failed
+   */
+  public HBaseSnapshotException(String msg, Throwable cause, SnapshotDescription desc) {
+    super(msg, cause);
+    this.description = desc;
+  }
+
+  /**
+   * Exception when the description of the snapshot cannot be determined, due to some root other
+   * root cause
+   * @param message description of what caused the failure
+   * @param e root cause
+   */
+  public HBaseSnapshotException(String message, Exception e) {
+    super(message, e);
+  }
+
+  public SnapshotDescription getSnapshotDescription() {
+    return this.description;
+  }
+}

Added: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/SnapshotCreationException.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/SnapshotCreationException.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/SnapshotCreationException.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/SnapshotCreationException.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.snapshot.exception;
+
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+
+/**
+ * Thrown when a snapshot could not be created due to a server-side error when taking the snapshot.
+ */
+@SuppressWarnings("serial")
+public class SnapshotCreationException extends HBaseSnapshotException {
+
+  /**
+   * Used internally by the RPC engine to pass the exception back to the client.
+   * @param msg error message to pass back
+   */
+  public SnapshotCreationException(String msg) {
+    super(msg);
+  }
+
+  /**
+   * Failure to create the specified snapshot
+   * @param msg reason why the snapshot couldn't be completed
+   * @param desc description of the snapshot attempted
+   */
+  public SnapshotCreationException(String msg, SnapshotDescription desc) {
+    super(msg, desc);
+  }
+
+  /**
+   * Failure to create the specified snapshot due to an external cause
+   * @param msg reason why the snapshot couldn't be completed
+   * @param cause root cause of the failure
+   * @param desc description of the snapshot attempted
+   */
+  public SnapshotCreationException(String msg, Throwable cause, SnapshotDescription desc) {
+    super(msg, cause, desc);
+  }
+}

Added: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/SnapshotDoesNotExistException.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/SnapshotDoesNotExistException.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/SnapshotDoesNotExistException.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/SnapshotDoesNotExistException.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.snapshot.exception;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+
+
+/**
+ * Thrown when the server is looking for a snapshot but can't find the snapshot on the filesystem
+ */
+@SuppressWarnings("serial")
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class SnapshotDoesNotExistException extends HBaseSnapshotException {
+
+  /**
+   * @param desc expected snapshot to find
+   */
+  public SnapshotDoesNotExistException(SnapshotDescription desc) {
+    super("Snapshot doesn't exist on the filesystem", desc);
+  }
+
+}

Added: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/SnapshotExistsException.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/SnapshotExistsException.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/SnapshotExistsException.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/SnapshotExistsException.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.snapshot.exception;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+
+/**
+ * Thrown when a snapshot exists but should not
+ */
+@SuppressWarnings("serial")
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class SnapshotExistsException extends HBaseSnapshotException {
+
+  /**
+   * Failure due to the snapshot already existing
+   * @param msg full description of the failure
+   * @param desc snapshot that was attempted
+   */
+  public SnapshotExistsException(String msg, SnapshotDescription desc) {
+    super(msg, desc);
+  }
+}
\ No newline at end of file

Added: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/TablePartiallyOpenException.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/TablePartiallyOpenException.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/TablePartiallyOpenException.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/TablePartiallyOpenException.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.snapshot.exception;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Thrown if a table should be online/offline but is partial open
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class TablePartiallyOpenException extends IOException {
+  private static final long serialVersionUID = 3571982660065058361L;
+
+  public TablePartiallyOpenException() {
+    super();
+  }
+
+  /**
+   * @param s message
+   */
+  public TablePartiallyOpenException(String s) {
+    super(s);
+  }
+
+  /**
+   * @param tableName Name of table that is partial open
+   */
+  public TablePartiallyOpenException(byte[] tableName) {
+    this(Bytes.toString(tableName));
+  }
+}

Added: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/UnexpectedSnapshotException.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/UnexpectedSnapshotException.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/UnexpectedSnapshotException.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/UnexpectedSnapshotException.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.snapshot.exception;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+
+/**
+ * General exception when an unexpected error occurs while running a snapshot.
+ */
+@SuppressWarnings("serial")
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class UnexpectedSnapshotException extends HBaseSnapshotException {
+
+  /**
+   * General exception for some cause
+   * @param msg reason why the snapshot couldn't be completed
+   * @param cause root cause of the failure
+   * @param snapshot description of the snapshot attempted
+   */
+  public UnexpectedSnapshotException(String msg, Exception cause, SnapshotDescription snapshot) {
+    super(msg, cause, snapshot);
+  }
+
+}

Added: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/UnknownSnapshotException.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/UnknownSnapshotException.java?rev=1445782&view=auto
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/UnknownSnapshotException.java (added)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/exception/UnknownSnapshotException.java Wed Feb 13 18:05:53 2013
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.snapshot.exception;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Exception thrown when we get a request for a snapshot we don't recognize.
+ */
+@SuppressWarnings("serial")
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class UnknownSnapshotException extends HBaseSnapshotException {
+
+
+  /**
+   * @param msg full infomration about the failure
+   */
+  public UnknownSnapshotException(String msg) {
+    super(msg);
+  }
+}
\ No newline at end of file

Modified: hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java?rev=1445782&r1=1445781&r2=1445782&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java Wed Feb 13 18:05:53 2013
@@ -607,7 +607,25 @@ public class FSTableDescriptors implemen
   public static boolean createTableDescriptor(FileSystem fs, Path rootdir,
       HTableDescriptor htableDescriptor, boolean forceCreation)
   throws IOException {
-    FileStatus status = getTableInfoPath(fs, rootdir, htableDescriptor.getNameAsString());
+    Path tabledir = FSUtils.getTablePath(rootdir, htableDescriptor.getNameAsString());
+    return createTableDescriptorForTableDirectory(fs, tabledir, htableDescriptor, forceCreation);
+  }
+
+  /**
+   * Create a new HTableDescriptor in HDFS in the specified table directory. Happens when we create
+   * a new table or snapshot a table.
+   * @param fs filesystem where the descriptor should be written
+   * @param tabledir directory under which we should write the file
+   * @param htableDescriptor description of the table to write
+   * @param forceCreation if <tt>true</tt>,then even if previous table descriptor is present it will
+   *          be overwritten
+   * @return <tt>true</tt> if the we successfully created the file, <tt>false</tt> if the file
+   *         already exists and we weren't forcing the descriptor creation.
+   * @throws IOException if a filesystem error occurs
+   */
+  public static boolean createTableDescriptorForTableDirectory(FileSystem fs, Path tabledir,
+      HTableDescriptor htableDescriptor, boolean forceCreation) throws IOException {
+    FileStatus status = getTableInfoPath(fs, tabledir);
     if (status != null) {
       LOG.info("Current tableInfoPath = " + status.getPath());
       if (!forceCreation) {
@@ -617,8 +635,7 @@ public class FSTableDescriptors implemen
         }
       }
     }
-    Path p = writeTableDescriptor(fs, htableDescriptor,
-      FSUtils.getTablePath(rootdir, htableDescriptor.getNameAsString()), status);
+    Path p = writeTableDescriptor(fs, htableDescriptor, tabledir, status);
     return p != null;
   }
 }