You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2018/04/12 02:34:05 UTC

hbase git commit: HBASE-20243 [Shell] Add shell command to create a new table by cloning the existent table

Repository: hbase
Updated Branches:
  refs/heads/master e4b51bb27 -> 5a69465ea


HBASE-20243 [Shell] Add shell command to create a new table by cloning the existent table

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/master
Commit: 5a69465ea0188b5942baadfee375a1303169f795
Parents: e4b51bb
Author: Guangxu Cheng <gu...@gmail.com>
Authored: Thu Apr 12 10:18:46 2018 +0800
Committer: tedyu <yu...@gmail.com>
Committed: Wed Apr 11 19:33:56 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |  11 ++
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |  10 ++
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    |   6 ++
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  15 +++
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java |  65 +++++++++++-
 .../hadoop/hbase/HBaseTestingUtility.java       |  13 +++
 .../apache/hadoop/hbase/client/TestAdmin1.java  |  93 +++++++++++++++++
 .../hbase/client/TestAsyncTableAdminApi.java    |  87 ++++++++++++++++
 hbase-shell/src/main/ruby/hbase/admin.rb        |  10 ++
 hbase-shell/src/main/ruby/shell.rb              |   1 +
 hbase-shell/src/main/ruby/shell/commands.rb     |   6 ++
 .../ruby/shell/commands/clone_table_schema.rb   |  42 ++++++++
 hbase-shell/src/test/ruby/hbase/admin_test.rb   | 101 +++++++++++++++++--
 13 files changed, 452 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5a69465e/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 a6a4224..331f2d1 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
@@ -2727,4 +2727,15 @@ public interface Admin extends Abortable, Closeable {
    * @return List of servers that are not cleared
    */
   List<ServerName> clearDeadServers(final List<ServerName> servers) throws IOException;
+
+  /**
+   * Create a new table by cloning the existent table schema.
+   *
+   * @param tableName name of the table to be cloned
+   * @param newTableName name of the new table where the table will be created
+   * @param preserveSplits True if the splits should be preserved
+   * @throws IOException if a remote or network exception occurs
+   */
+  void cloneTableSchema(final TableName tableName, final TableName newTableName,
+      final boolean preserveSplits) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5a69465e/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index 35cdd3f..8141e74 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -1230,4 +1230,14 @@ public interface AsyncAdmin {
    * @return CacheEvictionStats related to the eviction wrapped by a {@link CompletableFuture}.
    */
   CompletableFuture<CacheEvictionStats> clearBlockCache(final TableName tableName);
+
+  /**
+   * Create a new table by cloning the existent table schema.
+   *
+   * @param tableName name of the table to be cloned
+   * @param newTableName name of the new table where the table will be created
+   * @param preserveSplits True if the splits should be preserved
+   */
+  CompletableFuture<Void>  cloneTableSchema(final TableName tableName,
+      final TableName newTableName, final boolean preserveSplits);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5a69465e/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index 9b2390c..5b22668 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -746,4 +746,10 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   public CompletableFuture<CacheEvictionStats> clearBlockCache(TableName tableName) {
     return wrap(rawAdmin.clearBlockCache(tableName));
   }
+
+  @Override
+  public CompletableFuture<Void> cloneTableSchema(TableName tableName, TableName newTableName,
+      boolean preserveSplits) {
+    return wrap(rawAdmin.cloneTableSchema(tableName, newTableName, preserveSplits));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5a69465e/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 8685984..172db5b 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
@@ -4227,4 +4227,19 @@ public class HBaseAdmin implements Admin {
       }
     });
   }
+
+  @Override
+  public void cloneTableSchema(final TableName tableName, final TableName newTableName,
+      final boolean preserveSplits) throws IOException {
+    checkTableExists(tableName);
+    if (tableExists(newTableName)) {
+      throw new TableExistsException(newTableName);
+    }
+    TableDescriptor htd = TableDescriptorBuilder.copy(newTableName, getTableDescriptor(tableName));
+    if (preserveSplits) {
+      createTable(htd, getTableSplits(tableName));
+    } else {
+      createTable(htd);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5a69465e/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 050bfe2..0fd0e59 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -414,7 +414,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   /**
-   * {@link #listTables(boolean)}
+   * {@link #listTableDescriptors(boolean)}
    */
   @Override
   public CompletableFuture<List<TableDescriptor>> listTableDescriptors(Pattern pattern,
@@ -3468,6 +3468,69 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     return future;
   }
 
+  @Override
+  public CompletableFuture<Void> cloneTableSchema(TableName tableName, TableName newTableName,
+      boolean preserveSplits) {
+    CompletableFuture<Void> future = new CompletableFuture<>();
+    tableExists(tableName).whenComplete(
+      (exist, err) -> {
+        if (err != null) {
+          future.completeExceptionally(err);
+          return;
+        }
+        if (!exist) {
+          future.completeExceptionally(new TableNotFoundException(tableName));
+          return;
+        }
+        tableExists(newTableName).whenComplete(
+          (exist1, err1) -> {
+            if (err1 != null) {
+              future.completeExceptionally(err1);
+              return;
+            }
+            if (exist1) {
+              future.completeExceptionally(new TableExistsException(newTableName));
+              return;
+            }
+            getDescriptor(tableName).whenComplete(
+              (tableDesc, err2) -> {
+                if (err2 != null) {
+                  future.completeExceptionally(err2);
+                  return;
+                }
+                TableDescriptor newTableDesc
+                    = TableDescriptorBuilder.copy(newTableName, tableDesc);
+                if (preserveSplits) {
+                  getTableSplits(tableName).whenComplete((splits, err3) -> {
+                    if (err3 != null) {
+                      future.completeExceptionally(err3);
+                    } else {
+                      createTable(newTableDesc, splits).whenComplete(
+                        (result, err4) -> {
+                          if (err4 != null) {
+                            future.completeExceptionally(err4);
+                          } else {
+                            future.complete(result);
+                          }
+                        });
+                    }
+                  });
+                } else {
+                  createTable(newTableDesc).whenComplete(
+                    (result, err5) -> {
+                      if (err5 != null) {
+                        future.completeExceptionally(err5);
+                      } else {
+                        future.complete(result);
+                      }
+                    });
+                }
+              });
+          });
+      });
+    return future;
+  }
+
   private CompletableFuture<CacheEvictionStats> clearBlockCache(ServerName serverName,
       List<RegionInfo> hris) {
     return this.<CacheEvictionStats> newAdminCaller().action((controller, stub) -> this

http://git-wip-us.apache.org/repos/asf/hbase/blob/5a69465e/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 3c3cb0a..8faeb5d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -41,6 +41,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
@@ -4138,4 +4139,16 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     }
     return numHFiles;
   }
+
+  public void verifyTableDescriptorIgnoreTableName(TableDescriptor ltd, TableDescriptor rtd) {
+    assertEquals(ltd.getValues().hashCode(), rtd.getValues().hashCode());
+    Collection<ColumnFamilyDescriptor> ltdFamilies = Arrays.asList(ltd.getColumnFamilies());
+    Collection<ColumnFamilyDescriptor> rtdFamilies = Arrays.asList(rtd.getColumnFamilies());
+    assertEquals(ltdFamilies.size(), rtdFamilies.size());
+    for (Iterator<ColumnFamilyDescriptor> it = ltdFamilies.iterator(), it2 =
+         rtdFamilies.iterator(); it.hasNext();) {
+      assertEquals(0,
+          ColumnFamilyDescriptor.COMPARATOR.compare(it.next(), it2.next()));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5a69465e/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
index 8ac2ddaf..e0eef20 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotEnabledException;
@@ -1421,4 +1422,96 @@ public class TestAdmin1 {
         this.admin.getConnection(), tableName, true);
     assertEquals(1, allRegions.size());
   }
+
+  @Test
+  public void testCloneTableSchema() throws Exception {
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName newTableName = TableName.valueOf(tableName.getNameAsString() + "_new");
+    testCloneTableSchema(tableName, newTableName, false);
+  }
+
+  @Test
+  public void testCloneTableSchemaPreservingSplits() throws Exception {
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName newTableName = TableName.valueOf(tableName.getNameAsString() + "_new");
+    testCloneTableSchema(tableName, newTableName, true);
+  }
+
+  private void testCloneTableSchema(final TableName tableName,
+      final TableName newTableName, boolean preserveSplits) throws Exception {
+    byte[] FAMILY_0 = Bytes.toBytes("cf0");
+    byte[] FAMILY_1 = Bytes.toBytes("cf1");
+    byte[][] splitKeys = new byte[2][];
+    splitKeys[0] = Bytes.toBytes(4);
+    splitKeys[1] = Bytes.toBytes(8);
+    int NUM_FAMILYS = 2;
+    int NUM_REGIONS = 3;
+    int BLOCK_SIZE = 1024;
+    int TTL = 86400;
+    boolean BLOCK_CACHE = false;
+
+    // Create the table
+    TableDescriptor tableDesc = TableDescriptorBuilder
+        .newBuilder(tableName)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_0))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder
+            .newBuilder(FAMILY_1)
+            .setBlocksize(BLOCK_SIZE)
+            .setBlockCacheEnabled(BLOCK_CACHE)
+            .setTimeToLive(TTL)
+            .build()
+        ).build();
+    admin.createTable(tableDesc, splitKeys);
+
+    assertEquals(NUM_REGIONS, TEST_UTIL.getHBaseCluster().getRegions(tableName).size());
+    assertTrue("Table should be created with splitKyes + 1 rows in META",
+        admin.isTableAvailable(tableName, splitKeys));
+
+    // clone & Verify
+    admin.cloneTableSchema(tableName, newTableName, preserveSplits);
+    TableDescriptor newTableDesc = admin.getDescriptor(newTableName);
+
+    assertEquals(NUM_FAMILYS, newTableDesc.getColumnFamilyCount());
+    assertEquals(BLOCK_SIZE, newTableDesc.getColumnFamily(FAMILY_1).getBlocksize());
+    assertEquals(BLOCK_CACHE, newTableDesc.getColumnFamily(FAMILY_1).isBlockCacheEnabled());
+    assertEquals(TTL, newTableDesc.getColumnFamily(FAMILY_1).getTimeToLive());
+    TEST_UTIL.verifyTableDescriptorIgnoreTableName(tableDesc, newTableDesc);
+
+    if (preserveSplits) {
+      assertEquals(NUM_REGIONS, TEST_UTIL.getHBaseCluster().getRegions(newTableName).size());
+      assertTrue("New table should be created with splitKyes + 1 rows in META",
+          admin.isTableAvailable(newTableName, splitKeys));
+    } else {
+      assertEquals(1, TEST_UTIL.getHBaseCluster().getRegions(newTableName).size());
+    }
+  }
+
+  @Test
+  public void testCloneTableSchemaWithNonExistentSourceTable() throws Exception {
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName newTableName = TableName.valueOf(tableName.getNameAsString() + "_new");
+    // test for non-existent source table
+    try {
+      admin.cloneTableSchema(tableName, newTableName, false);
+      fail("Should have failed to create a new table by cloning non-existent source table.");
+    } catch (TableNotFoundException ex) {
+      // expected
+    }
+  }
+
+  @Test
+  public void testCloneTableSchemaWithExistentDestinationTable() throws Exception {
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName newTableName = TableName.valueOf(tableName.getNameAsString() + "_new");
+    byte[] FAMILY_0 = Bytes.toBytes("cf0");
+    TEST_UTIL.createTable(tableName, FAMILY_0);
+    TEST_UTIL.createTable(newTableName, FAMILY_0);
+    // test for existent destination table
+    try {
+      admin.cloneTableSchema(tableName, newTableName, false);
+      fail("Should have failed to create a existent table.");
+    } catch (TableExistsException ex) {
+      // expected
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5a69465e/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
index 2c948dd..1750926 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
@@ -36,7 +36,9 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -366,4 +368,89 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
       assertEquals(1, TEST_UTIL.getHBaseCluster().getRegions(tableName).size());
     }
   }
+
+  @Test
+  public void testCloneTableSchema() throws Exception {
+    final TableName newTableName = TableName.valueOf(tableName.getNameAsString() + "_new");
+    testCloneTableSchema(tableName, newTableName, false);
+  }
+
+  @Test
+  public void testCloneTableSchemaPreservingSplits() throws Exception {
+    final TableName newTableName = TableName.valueOf(tableName.getNameAsString() + "_new");
+    testCloneTableSchema(tableName, newTableName, true);
+  }
+
+  private void testCloneTableSchema(final TableName tableName,
+      final TableName newTableName, boolean preserveSplits) throws Exception {
+    byte[][] splitKeys = new byte[2][];
+    splitKeys[0] = Bytes.toBytes(4);
+    splitKeys[1] = Bytes.toBytes(8);
+    int NUM_FAMILYS = 2;
+    int NUM_REGIONS = 3;
+    int BLOCK_SIZE = 1024;
+    int TTL = 86400;
+    boolean BLOCK_CACHE = false;
+
+    // Create the table
+    TableDescriptor tableDesc = TableDescriptorBuilder
+        .newBuilder(tableName)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_0))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder
+            .newBuilder(FAMILY_1)
+            .setBlocksize(BLOCK_SIZE)
+            .setBlockCacheEnabled(BLOCK_CACHE)
+            .setTimeToLive(TTL)
+            .build()).build();
+    admin.createTable(tableDesc, splitKeys).join();
+
+    assertEquals(NUM_REGIONS, TEST_UTIL.getHBaseCluster().getRegions(tableName).size());
+    assertTrue("Table should be created with splitKyes + 1 rows in META",
+        admin.isTableAvailable(tableName, splitKeys).get());
+
+    // Clone & Verify
+    admin.cloneTableSchema(tableName, newTableName, preserveSplits).join();
+    TableDescriptor newTableDesc = admin.getDescriptor(newTableName).get();
+
+    assertEquals(NUM_FAMILYS, newTableDesc.getColumnFamilyCount());
+    assertEquals(BLOCK_SIZE, newTableDesc.getColumnFamily(FAMILY_1).getBlocksize());
+    assertEquals(BLOCK_CACHE, newTableDesc.getColumnFamily(FAMILY_1).isBlockCacheEnabled());
+    assertEquals(TTL, newTableDesc.getColumnFamily(FAMILY_1).getTimeToLive());
+    TEST_UTIL.verifyTableDescriptorIgnoreTableName(tableDesc, newTableDesc);
+
+    if (preserveSplits) {
+      assertEquals(NUM_REGIONS, TEST_UTIL.getHBaseCluster().getRegions(newTableName).size());
+      assertTrue("New table should be created with splitKyes + 1 rows in META",
+          admin.isTableAvailable(newTableName, splitKeys).get());
+    } else {
+      assertEquals(1, TEST_UTIL.getHBaseCluster().getRegions(newTableName).size());
+    }
+  }
+
+  @Test
+  public void testCloneTableSchemaWithNonExistentSourceTable() throws Exception {
+    final TableName newTableName = TableName.valueOf(tableName.getNameAsString() + "_new");
+    // test for non-existent source table
+    try {
+      admin.cloneTableSchema(tableName, newTableName, false).join();
+      fail("Should have failed when source table doesn't exist.");
+    } catch (CompletionException e) {
+      assertTrue(e.getCause() instanceof TableNotFoundException);
+    }
+  }
+
+  @Test
+  public void testCloneTableSchemaWithExistentDestinationTable() throws Exception {
+    final TableName newTableName = TableName.valueOf(tableName.getNameAsString() + "_new");
+    byte[] FAMILY_0 = Bytes.toBytes("cf0");
+    TEST_UTIL.createTable(tableName, FAMILY_0);
+    TEST_UTIL.createTable(newTableName, FAMILY_0);
+    // test for existent destination table
+    try {
+      admin.cloneTableSchema(tableName, newTableName, false).join();
+      fail("Should have failed when destination table exists.");
+    } catch (CompletionException e) {
+      assertTrue(e.getCause() instanceof TableExistsException);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5a69465e/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 e409e39..cb57c4c 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -29,6 +29,7 @@ java_import org.apache.hadoop.hbase.TableName
 # Wrapper for org.apache.hadoop.hbase.client.HBaseAdmin
 
 module Hbase
+  # rubocop:disable Metrics/ClassLength
   class Admin
     include HBaseConstants
 
@@ -1300,5 +1301,14 @@ module Hbase
     def list_liveservers
       @admin.getClusterStatus.getServers.to_a
     end
+
+    #---------------------------------------------------------------------------
+    # create a new table by cloning the existent table schema.
+    def clone_table_schema(table_name, new_table_name, preserve_splits = true)
+      @admin.cloneTableSchema(TableName.valueOf(table_name),
+                              TableName.valueOf(new_table_name),
+                              preserve_splits)
+    end
   end
+  # rubocop:enable Metrics/ClassLength
 end

http://git-wip-us.apache.org/repos/asf/hbase/blob/5a69465e/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 2e228f5..ab07a79 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -280,6 +280,7 @@ Shell.load_command_group(
     get_table
     locate_region
     list_regions
+    clone_table_schema
   ],
   aliases: {
     'describe' => ['desc']

http://git-wip-us.apache.org/repos/asf/hbase/blob/5a69465e/hbase-shell/src/main/ruby/shell/commands.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands.rb b/hbase-shell/src/main/ruby/shell/commands.rb
index f63128f..85a742e 100644
--- a/hbase-shell/src/main/ruby/shell/commands.rb
+++ b/hbase-shell/src/main/ruby/shell/commands.rb
@@ -104,6 +104,8 @@ module Shell
         @formatter = formatter
       end
 
+      # rubocop:disable Metrics/AbcSize, Metrics/CyclomaticComplexity
+      # rubocop:disable Metrics/MethodLength, Metrics/PerceivedComplexity
       def translate_hbase_exceptions(*args)
         yield
       rescue => cause
@@ -142,6 +144,8 @@ module Shell
           end
         end
         if cause.is_a?(org.apache.hadoop.hbase.TableExistsException)
+          strs = cause.to_s.split(' ')
+          raise "Table already exists: #{strs[0]}!" if strs.size == 1
           raise "Table already exists: #{args.first}!"
         end
         # To be safe, here only AccessDeniedException is considered. In future
@@ -157,6 +161,8 @@ module Shell
         # Throw the other exception which hasn't been handled above
         raise cause
       end
+      # rubocop:enable Metrics/AbcSize, Metrics/CyclomaticComplexity
+      # rubocop:enable Metrics/MethodLength, Metrics/PerceivedComplexity
     end
   end
 end

http://git-wip-us.apache.org/repos/asf/hbase/blob/5a69465e/hbase-shell/src/main/ruby/shell/commands/clone_table_schema.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/clone_table_schema.rb b/hbase-shell/src/main/ruby/shell/commands/clone_table_schema.rb
new file mode 100644
index 0000000..a92e862
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/clone_table_schema.rb
@@ -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.
+#
+
+module Shell
+  module Commands
+    # create a new table by cloning the existent table schema.
+    class CloneTableSchema < Command
+      def help
+        <<-HELP
+          Create a new table by cloning the existent table schema.
+          There're no copies of data involved.
+          Just copy the table descriptor and split keys.
+
+          Passing 'false' as the optional third parameter will
+          not preserve split keys.
+          Examples:
+            hbase> clone_table_schema 'table_name', 'new_table_name'
+            hbase> clone_table_schema 'table_name', 'new_table_name', false
+        HELP
+      end
+
+      def command(table_name, new_table_name, preserve_splits = true)
+        admin.clone_table_schema(table_name, new_table_name, preserve_splits)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/5a69465e/hbase-shell/src/test/ruby/hbase/admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/admin_test.rb b/hbase-shell/src/test/ruby/hbase/admin_test.rb
index a3bf0f4..69dde45 100644
--- a/hbase-shell/src/test/ruby/hbase/admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/admin_test.rb
@@ -334,6 +334,89 @@ module Hbase
   end
 
   # Simple administration methods tests
+  class AdminCloneTableSchemaTest < Test::Unit::TestCase
+    include TestHelpers
+    def setup
+      setup_hbase
+      # Create table test table name
+      @source_table_name = 'hbase_shell_tests_source_table_name'
+      @destination_table_name = 'hbase_shell_tests_destination_table_name'
+    end
+
+    def teardown
+      shutdown
+    end
+
+    define_test "clone_table_schema should create a new table by cloning the
+                 existent table schema." do
+      drop_test_table(@source_table_name)
+      drop_test_table(@destination_table_name)
+      command(:create,
+              @source_table_name,
+              NAME => 'a',
+              CACHE_BLOOMS_ON_WRITE => 'TRUE',
+              CACHE_INDEX_ON_WRITE => 'TRUE',
+              EVICT_BLOCKS_ON_CLOSE => 'TRUE',
+              COMPRESSION_COMPACT => 'GZ')
+      command(:clone_table_schema,
+              @source_table_name,
+              @destination_table_name,
+              false)
+      assert_equal(['a:'],
+                   table(@source_table_name).get_all_columns.sort)
+      assert_match(/CACHE_BLOOMS_ON_WRITE/,
+                   admin.describe(@destination_table_name))
+      assert_match(/CACHE_INDEX_ON_WRITE/,
+                   admin.describe(@destination_table_name))
+      assert_match(/EVICT_BLOCKS_ON_CLOSE/,
+                   admin.describe(@destination_table_name))
+      assert_match(/GZ/,
+                   admin.describe(@destination_table_name))
+    end
+
+    define_test "clone_table_schema should maintain the source table's region
+                 boundaries when preserve_splits set to true" do
+      drop_test_table(@source_table_name)
+      drop_test_table(@destination_table_name)
+      command(:create,
+              @source_table_name,
+              'a',
+              NUMREGIONS => 10,
+              SPLITALGO => 'HexStringSplit')
+      splits = table(@source_table_name)._get_splits_internal
+      command(:clone_table_schema,
+              @source_table_name,
+              @destination_table_name,
+              true)
+      assert_equal(splits, table(@destination_table_name)._get_splits_internal)
+    end
+
+    define_test "clone_table_schema should have failed when source table
+                 doesn't exist." do
+      drop_test_table(@source_table_name)
+      drop_test_table(@destination_table_name)
+      assert_raise(RuntimeError) do
+        command(:clone_table_schema,
+                @source_table_name,
+                @destination_table_name)
+      end
+    end
+
+    define_test "clone_table_schema should have failed when destination
+                 table exists." do
+      drop_test_table(@source_table_name)
+      drop_test_table(@destination_table_name)
+      command(:create, @source_table_name, 'a')
+      command(:create, @destination_table_name, 'a')
+      assert_raise(RuntimeError) do
+        command(:clone_table_schema,
+                @source_table_name,
+                @destination_table_name)
+      end
+    end
+  end
+
+  # Simple administration methods tests
   class AdminRegionTest < Test::Unit::TestCase
     include TestHelpers
     def setup
@@ -362,6 +445,7 @@ module Hbase
   end
 
   # Simple administration methods tests
+  # rubocop:disable Metrics/ClassLength
   class AdminAlterTableTest < Test::Unit::TestCase
     include TestHelpers
 
@@ -417,19 +501,21 @@ module Hbase
       assert_equal(['x:', 'y:', 'z:'], table(@test_name).get_all_columns.sort)
     end
 
-    define_test "alter should support more than one alteration in one call" do
+    define_test 'alter should support more than one alteration in one call' do
       assert_equal(['x:', 'y:'], table(@test_name).get_all_columns.sort)
-      alterOutput = capture_stdout {
-        command(:alter, @test_name, { NAME => 'z' }, { METHOD => 'delete', NAME => 'y' },
-                'MAX_FILESIZE' => 12345678) }
+      alter_out_put = capture_stdout do
+        command(:alter, @test_name, { NAME => 'z' },
+                { METHOD => 'delete', NAME => 'y' },
+                'MAX_FILESIZE' => 12_345_678)
+      end
       command(:enable, @test_name)
-      assert_equal(1, /Updating all regions/.match(alterOutput).size,
-        "HBASE-15641 - Should only perform one table modification per alter.")
+      assert_equal(1, /Updating all regions/.match(alter_out_put).size,
+                   "HBASE-15641 - Should only perform one table
+                   modification per alter.")
       assert_equal(['x:', 'z:'], table(@test_name).get_all_columns.sort)
       assert_match(/12345678/, admin.describe(@test_name))
     end
 
-
     define_test 'alter should support shortcut DELETE alter specs' do
       assert_equal(['x:', 'y:'], table(@test_name).get_all_columns.sort)
       command(:alter, @test_name, 'delete' => 'y')
@@ -532,6 +618,7 @@ module Hbase
       table.close
     end
   end
+  # rubocop:enable Metrics/ClassLength
 
   # Tests for the `status` shell command
   class StatusTest < Test::Unit::TestCase