You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2015/07/16 03:32:11 UTC

[2/3] hbase git commit: HBASE-8642 [Snapshot] List and delete snapshot by table

HBASE-8642 [Snapshot] List and delete snapshot by table

Signed-off-by: Andrew Purtell <ap...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/789d2a94
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/789d2a94
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/789d2a94

Branch: refs/heads/branch-1
Commit: 789d2a94b7e8c2d97c1b52f4f1f0d47922b711a2
Parents: 2446da0
Author: Ashish Singhi <as...@huawei.com>
Authored: Wed Jul 15 15:56:08 2015 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Jul 15 15:56:08 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |  41 +++++
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  73 +++++++++
 .../hbase/client/TestSnapshotFromClient.java    | 152 +++++++++++++++++++
 hbase-shell/src/main/ruby/hbase/admin.rb        |  12 ++
 hbase-shell/src/main/ruby/shell.rb              |   2 +
 .../shell/commands/delete_table_snapshots.rb    |  69 +++++++++
 .../ruby/shell/commands/list_table_snapshots.rb |  56 +++++++
 7 files changed, 405 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/789d2a94/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index 9445930..d7d9588 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -1255,6 +1255,28 @@ public interface Admin extends Abortable, Closeable {
   List<HBaseProtos.SnapshotDescription> listSnapshots(Pattern pattern) throws IOException;
 
   /**
+   * List all the completed snapshots matching the given table name regular expression and snapshot
+   * name regular expression.
+   * @param tableNameRegex The table name regular expression to match against
+   * @param snapshotNameRegex The snapshot name regular expression to match against
+   * @return - returns a List of completed SnapshotDescription
+   * @throws IOException if a remote or network exception occurs
+   */
+  List<HBaseProtos.SnapshotDescription> listTableSnapshots(String tableNameRegex,
+      String snapshotNameRegex) throws IOException;
+
+  /**
+   * List all the completed snapshots matching the given table name regular expression and snapshot
+   * name regular expression.
+   * @param tableNamePattern The compiled table name regular expression to match against
+   * @param snapshotNamePattern The compiled snapshot name regular expression to match against
+   * @return - returns a List of completed SnapshotDescription
+   * @throws IOException if a remote or network exception occurs
+   */
+  List<HBaseProtos.SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
+      Pattern snapshotNamePattern) throws IOException;
+
+  /**
    * Delete an existing snapshot.
    *
    * @param snapshotName name of the snapshot
@@ -1287,6 +1309,25 @@ public interface Admin extends Abortable, Closeable {
   void deleteSnapshots(final Pattern pattern) throws IOException;
   
   /**
+   * Delete all existing snapshots matching the given table name regular expression and snapshot
+   * name regular expression.
+   * @param tableNameRegex The table name regular expression to match against
+   * @param snapshotNameRegex The snapshot name regular expression to match against
+   * @throws IOException if a remote or network exception occurs
+   */
+  void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex) throws IOException;
+
+  /**
+   * Delete all existing snapshots matching the given table name regular expression and snapshot
+   * name regular expression.
+   * @param tableNamePattern The compiled table name regular expression to match against
+   * @param snapshotNamePattern The compiled snapshot name regular expression to match against
+   * @throws IOException if a remote or network exception occurs
+   */
+  void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
+      throws IOException;
+
+  /**
    * Apply the new quota settings.
    * @param quota the quota settings
    * @throws IOException if a remote or network exception occurs

http://git-wip-us.apache.org/repos/asf/hbase/blob/789d2a94/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index f446d96..2564493 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -3826,6 +3826,45 @@ public class HBaseAdmin implements Admin {
   }
 
   /**
+   * List all the completed snapshots matching the given table name regular expression and snapshot
+   * name regular expression.
+   * @param tableNameRegex The table name regular expression to match against
+   * @param snapshotNameRegex The snapshot name regular expression to match against
+   * @return returns a List of completed SnapshotDescription
+   * @throws IOException if a remote or network exception occurs
+   */
+  @Override
+  public List<SnapshotDescription> listTableSnapshots(String tableNameRegex,
+      String snapshotNameRegex) throws IOException {
+    return listTableSnapshots(Pattern.compile(tableNameRegex), Pattern.compile(snapshotNameRegex));
+  }
+
+  /**
+   * List all the completed snapshots matching the given table name regular expression and snapshot
+   * name regular expression.
+   * @param tableNamePattern The compiled table name regular expression to match against
+   * @param snapshotNamePattern The compiled snapshot name regular expression to match against
+   * @return returns a List of completed SnapshotDescription
+   * @throws IOException if a remote or network exception occurs
+   */
+  @Override
+  public List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
+      Pattern snapshotNamePattern) throws IOException {
+    TableName[] tableNames = listTableNames(tableNamePattern);
+
+    List<SnapshotDescription> tableSnapshots = new LinkedList<SnapshotDescription>();
+    List<SnapshotDescription> snapshots = listSnapshots(snapshotNamePattern);
+
+    List<TableName> listOfTableNames = Arrays.asList(tableNames);
+    for (SnapshotDescription snapshot : snapshots) {
+      if (listOfTableNames.contains(TableName.valueOf(snapshot.getTable()))) {
+        tableSnapshots.add(snapshot);
+      }
+    }
+    return tableSnapshots;
+  }
+
+  /**
    * Delete an existing snapshot.
    * @param snapshotName name of the snapshot
    * @throws IOException if a remote or network exception occurs
@@ -3898,6 +3937,40 @@ public class HBaseAdmin implements Admin {
   }
 
   /**
+   * Delete all existing snapshots matching the given table name regular expression and snapshot
+   * name regular expression.
+   * @param tableNameRegex The table name regular expression to match against
+   * @param snapshotNameRegex The snapshot name regular expression to match against
+   * @throws IOException if a remote or network exception occurs
+   */
+  @Override
+  public void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex)
+      throws IOException {
+    deleteTableSnapshots(Pattern.compile(tableNameRegex), Pattern.compile(snapshotNameRegex));
+  }
+
+  /**
+   * Delete all existing snapshots matching the given table name regular expression and snapshot
+   * name regular expression.
+   * @param tableNamePattern The compiled table name regular expression to match against
+   * @param snapshotNamePattern The compiled snapshot name regular expression to match against
+   * @throws IOException if a remote or network exception occurs
+   */
+  @Override
+  public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
+      throws IOException {
+    List<SnapshotDescription> snapshots = listTableSnapshots(tableNamePattern, snapshotNamePattern);
+    for (SnapshotDescription snapshot : snapshots) {
+      try {
+        internalDeleteSnapshot(snapshot);
+        LOG.debug("Successfully deleted snapshot: " + snapshot.getName());
+      } catch (IOException e) {
+        LOG.error("Failed to delete snapshot: " + snapshot.getName(), e);
+      }
+    }
+  }
+
+  /**
    * Apply the new quota settings.
    * @param quota the quota settings
    * @throws IOException if a remote or network exception occurs

http://git-wip-us.apache.org/repos/asf/hbase/blob/789d2a94/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
index c87305a..9295d88 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
@@ -18,8 +18,11 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
@@ -37,6 +40,7 @@ import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
+import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifestV1;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -301,4 +305,152 @@ public class TestSnapshotFromClient {
     snapshots = admin.listSnapshots();
     SnapshotTestingUtils.assertNoSnapshots(admin);
   }
+
+  @Test(timeout = 300000)
+  public void testListTableSnapshots() throws Exception {
+    Admin admin = null;
+    TableName tableName2 = TableName.valueOf("testListTableSnapshots");
+    try {
+      admin = UTIL.getHBaseAdmin();
+
+      HTableDescriptor htd = new HTableDescriptor(tableName2);
+      UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration());
+
+      String table1Snapshot1 = "Table1Snapshot1";
+      admin.snapshot(table1Snapshot1, TABLE_NAME);
+      LOG.debug("Snapshot1 completed.");
+
+      String table1Snapshot2 = "Table1Snapshot2";
+      admin.snapshot(table1Snapshot2, TABLE_NAME);
+      LOG.debug("Snapshot2 completed.");
+
+      String table2Snapshot1 = "Table2Snapshot1";
+      admin.snapshot(Bytes.toBytes(table2Snapshot1), tableName2);
+      LOG.debug(table2Snapshot1 + " completed.");
+
+      List<SnapshotDescription> listTableSnapshots = admin.listTableSnapshots("test.*", ".*");
+      List<String> listTableSnapshotNames = new ArrayList<String>();
+      assertEquals(3, listTableSnapshots.size());
+      for (SnapshotDescription s : listTableSnapshots) {
+        listTableSnapshotNames.add(s.getName());
+      }
+      assertTrue(listTableSnapshotNames.contains(table1Snapshot1));
+      assertTrue(listTableSnapshotNames.contains(table1Snapshot2));
+      assertTrue(listTableSnapshotNames.contains(table2Snapshot1));
+    } finally {
+      if (admin != null) {
+        try {
+          admin.deleteSnapshots("Table.*");
+        } catch (SnapshotDoesNotExistException ignore) {
+        }
+        if (admin.tableExists(tableName2)) {
+          UTIL.deleteTable(tableName2);
+        }
+        admin.close();
+      }
+    }
+  }
+
+  @Test(timeout = 300000)
+  public void testListTableSnapshotsWithRegex() throws Exception {
+    Admin admin = null;
+    try {
+      admin = UTIL.getHBaseAdmin();
+
+      String table1Snapshot1 = "Table1Snapshot1";
+      admin.snapshot(table1Snapshot1, TABLE_NAME);
+      LOG.debug("Snapshot1 completed.");
+
+      String table1Snapshot2 = "Table1Snapshot2";
+      admin.snapshot(table1Snapshot2, TABLE_NAME);
+      LOG.debug("Snapshot2 completed.");
+
+      String table2Snapshot1 = "Table2Snapshot1";
+      admin.snapshot(Bytes.toBytes(table2Snapshot1), TABLE_NAME);
+      LOG.debug(table2Snapshot1 + " completed.");
+
+      List<SnapshotDescription> listTableSnapshots = admin.listTableSnapshots("test.*", "Table1.*");
+      List<String> listTableSnapshotNames = new ArrayList<String>();
+      assertEquals(2, listTableSnapshots.size());
+      for (SnapshotDescription s : listTableSnapshots) {
+        listTableSnapshotNames.add(s.getName());
+      }
+      assertTrue(listTableSnapshotNames.contains(table1Snapshot1));
+      assertTrue(listTableSnapshotNames.contains(table1Snapshot2));
+      assertFalse(listTableSnapshotNames.contains(table2Snapshot1));
+    } finally {
+      if (admin != null) {
+        try {
+          admin.deleteSnapshots("Table.*");
+        } catch (SnapshotDoesNotExistException ignore) {
+        }
+        admin.close();
+      }
+    }
+  }
+
+  @Test(timeout = 300000)
+  public void testDeleteTableSnapshots() throws Exception {
+    Admin admin = null;
+    TableName tableName2 = TableName.valueOf("testListTableSnapshots");
+    try {
+      admin = UTIL.getHBaseAdmin();
+
+      HTableDescriptor htd = new HTableDescriptor(tableName2);
+      UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration());
+
+      String table1Snapshot1 = "Table1Snapshot1";
+      admin.snapshot(table1Snapshot1, TABLE_NAME);
+      LOG.debug("Snapshot1 completed.");
+
+      String table1Snapshot2 = "Table1Snapshot2";
+      admin.snapshot(table1Snapshot2, TABLE_NAME);
+      LOG.debug("Snapshot2 completed.");
+
+      String table2Snapshot1 = "Table2Snapshot1";
+      admin.snapshot(Bytes.toBytes(table2Snapshot1), tableName2);
+      LOG.debug(table2Snapshot1 + " completed.");
+
+      admin.deleteTableSnapshots("test.*", ".*");
+      assertEquals(0, admin.listTableSnapshots("test.*", ".*").size());
+    } finally {
+      if (admin != null) {
+        if (admin.tableExists(tableName2)) {
+          UTIL.deleteTable(tableName2);
+        }
+        admin.close();
+      }
+    }
+  }
+
+  @Test(timeout = 300000)
+  public void testDeleteTableSnapshotsWithRegex() throws Exception {
+    Admin admin = null;
+    try {
+      admin = UTIL.getHBaseAdmin();
+
+      String table1Snapshot1 = "Table1Snapshot1";
+      admin.snapshot(table1Snapshot1, TABLE_NAME);
+      LOG.debug("Snapshot1 completed.");
+
+      String table1Snapshot2 = "Table1Snapshot2";
+      admin.snapshot(table1Snapshot2, TABLE_NAME);
+      LOG.debug("Snapshot2 completed.");
+
+      String table2Snapshot1 = "Table2Snapshot1";
+      admin.snapshot(Bytes.toBytes(table2Snapshot1), TABLE_NAME);
+      LOG.debug(table2Snapshot1 + " completed.");
+
+      admin.deleteTableSnapshots("test.*", "Table1.*");
+      assertEquals(1, admin.listTableSnapshots("test.*", ".*").size());
+    } finally {
+      if (admin != null) {
+        try {
+          admin.deleteTableSnapshots("test.*", ".*");
+        } catch (SnapshotDoesNotExistException ignore) {
+        }
+        admin.close();
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/789d2a94/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index e10e2be..84cf619 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -868,11 +868,23 @@ module Hbase
     end
 
     #----------------------------------------------------------------------------------------------
+    # Deletes the table snapshots matching the given regex
+    def delete_table_snapshots(tableNameRegex, snapshotNameRegex = ".*")
+      @admin.deleteTableSnapshots(tableNameRegex, snapshotNameRegex).to_a
+    end
+
+    #----------------------------------------------------------------------------------------------
     # Returns a list of snapshots
     def list_snapshot(regex = ".*")
       @admin.listSnapshots(regex).to_a
     end
 
+    #----------------------------------------------------------------------------------------------
+    # Returns a list of table snapshots
+    def list_table_snapshots(tableNameRegex, snapshotNameRegex = ".*")
+      @admin.listTableSnapshots(tableNameRegex, snapshotNameRegex).to_a
+    end
+
     # Apply config specific to a table/column to its descriptor
     def set_descriptor_config(descriptor, config)
       raise(ArgumentError, "#{CONFIGURATION} must be a Hash type") unless config.kind_of?(Hash)

http://git-wip-us.apache.org/repos/asf/hbase/blob/789d2a94/hbase-shell/src/main/ruby/shell.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb
index c47aa4b..b94cb05 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -365,7 +365,9 @@ Shell.load_command_group(
     restore_snapshot
     delete_snapshot
     delete_all_snapshot
+    delete_table_snapshots
     list_snapshots
+    list_table_snapshots
   ]
 )
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/789d2a94/hbase-shell/src/main/ruby/shell/commands/delete_table_snapshots.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/delete_table_snapshots.rb b/hbase-shell/src/main/ruby/shell/commands/delete_table_snapshots.rb
new file mode 100644
index 0000000..686bb8e
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/delete_table_snapshots.rb
@@ -0,0 +1,69 @@
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class DeleteTableSnapshots < Command
+      def help
+        return <<-EOF
+Delete all of the snapshots matching the given table name regular expression
+and snapshot name regular expression.
+By default snapshot name regular expression will delete all the snapshots of the
+matching table name regular expression.
+
+Examples:
+  hbase> delete_table_snapshots 'tableName'
+  hbase> delete_table_snapshots 'tableName.*'
+  hbase> delete_table_snapshots 'tableName' 'snapshotName'
+  hbase> delete_table_snapshots 'tableName' 'snapshotName.*'
+  hbase> delete_table_snapshots 'tableName.*' 'snapshotName.*'
+  hbase> delete_table_snapshots 'ns:tableName.*' 'snapshotName.*'
+
+EOF
+      end
+
+      def command(tableNameregex, snapshotNameRegex = ".*")
+        formatter.header([ "SNAPSHOT", "TABLE + CREATION TIME"])
+        list = admin.list_table_snapshots(tableNameregex, snapshotNameRegex)
+        count = list.size
+        list.each do |snapshot|
+          creation_time = Time.at(snapshot.getCreationTime() / 1000).to_s
+          formatter.row([ snapshot.getName, snapshot.getTable + " (" + creation_time + ")" ])
+        end
+        puts "\nDelete the above #{count} snapshots (y/n)?" unless count == 0
+        answer = 'n'
+        answer = gets.chomp unless count == 0
+        puts "No snapshots matched the table name regular expression #{tableNameregex.to_s} and the snapshot name regular expression #{snapshotNameRegex.to_s}" if count == 0
+        return unless answer =~ /y.*/i
+
+        format_simple_command do
+          list.each do |deleteSnapshot|
+            begin
+              admin.delete_snapshot(deleteSnapshot.getName)
+              puts "Successfully deleted snapshot: #{deleteSnapshot.getName}"
+              puts "\n"
+            rescue RuntimeError
+              puts "Failed to delete snapshot: #{deleteSnapshot.getName}, due to below exception,\n" + $!
+              puts "\n"
+            end
+          end
+        end
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/789d2a94/hbase-shell/src/main/ruby/shell/commands/list_table_snapshots.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_table_snapshots.rb b/hbase-shell/src/main/ruby/shell/commands/list_table_snapshots.rb
new file mode 100644
index 0000000..3f8a528
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/list_table_snapshots.rb
@@ -0,0 +1,56 @@
+#
+# 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.
+#
+
+require 'time'
+
+module Shell
+  module Commands
+    class ListTableSnapshots < Command
+      def help
+        return <<-EOF
+List all completed snapshots matching the table name regular expression and the
+snapshot name regular expression (by printing the names and relative information).
+Optional snapshot name regular expression parameter could be used to filter the output
+by snapshot name.
+
+Examples:
+  hbase> list_table_snapshots 'tableName'
+  hbase> list_table_snapshots 'tableName.*'
+  hbase> list_table_snapshots 'tableName' 'snapshotName'
+  hbase> list_table_snapshots 'tableName' 'snapshotName.*'
+  hbase> list_table_snapshots 'tableName.*' 'snapshotName.*'
+  hbase> list_table_snapshots 'ns:tableName.*' 'snapshotName.*'
+EOF
+      end
+
+      def command(tableNameRegex, snapshotNameRegex = ".*")
+        now = Time.now
+        formatter.header([ "SNAPSHOT", "TABLE + CREATION TIME"])
+
+        list = admin.list_table_snapshots(tableNameRegex, snapshotNameRegex)
+        list.each do |snapshot|
+          creation_time = Time.at(snapshot.getCreationTime() / 1000).to_s
+          formatter.row([ snapshot.getName, snapshot.getTable + " (" + creation_time + ")" ])
+        end
+
+        formatter.footer(now, list.size)
+        return list.map { |s| s.getName() }
+      end
+    end
+  end
+end