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 2013/06/25 00:21:42 UTC

svn commit: r1496240 - in /hbase/branches/0.94: security/src/main/java/org/apache/hadoop/hbase/security/access/ security/src/test/java/org/apache/hadoop/hbase/security/access/ src/main/java/org/apache/hadoop/hbase/client/ src/main/java/org/apache/hadoo...

Author: apurtell
Date: Mon Jun 24 22:21:41 2013
New Revision: 1496240

URL: http://svn.apache.org/r1496240
Log:
HBASE-8692. [AccessController] Restrict HTableDescriptor enumeration

Modified:
    hbase/branches/0.94/security/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
    hbase/branches/0.94/security/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
    hbase/branches/0.94/src/main/ruby/hbase/admin.rb
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
    hbase/branches/0.94/src/test/ruby/hbase/admin_test.rb

Modified: hbase/branches/0.94/security/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/security/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java?rev=1496240&r1=1496239&r2=1496240&view=diff
==============================================================================
--- hbase/branches/0.94/security/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java (original)
+++ hbase/branches/0.94/security/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java Mon Jun 24 22:21:41 2013
@@ -35,6 +35,8 @@ import org.apache.hadoop.hbase.HRegionIn
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableNotDisabledException;
+import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
@@ -56,6 +58,7 @@ import org.apache.hadoop.hbase.filter.Wr
 import org.apache.hadoop.hbase.ipc.HBaseRPC;
 import org.apache.hadoop.hbase.ipc.ProtocolSignature;
 import org.apache.hadoop.hbase.ipc.RequestContext;
+import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
@@ -1330,4 +1333,38 @@ public class AccessController extends Ba
       throws IOException {
     requirePermission("stop", Permission.Action.ADMIN);
   }
+
+  @Override
+  public void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      List<String> tableNamesList, List<HTableDescriptor> descriptors) throws IOException {
+    // If the list is empty, this is a request for all table descriptors and requires GLOBAL
+    // ADMIN privs.
+    if (tableNamesList == null || tableNamesList.isEmpty()) {
+      requirePermission("getTableDescriptors", Permission.Action.ADMIN);
+    }
+    // Otherwise, if the requestor has ADMIN or CREATE privs for all listed tables, the
+    // request can be granted.
+    else {
+      MasterServices masterServices = ctx.getEnvironment().getMasterServices();
+      for (String tableName: tableNamesList) {
+        // Do not deny if the table does not exist
+        byte[] nameAsBytes = Bytes.toBytes(tableName);
+        try {
+          masterServices.checkTableModifiable(nameAsBytes);
+        } catch (TableNotFoundException ex) {
+          // Skip checks for a table that does not exist
+          continue;
+        } catch (TableNotDisabledException ex) {
+          // We don't care about this
+        }
+        requirePermission("getTableDescriptors", nameAsBytes, null, null,
+          Permission.Action.ADMIN, Permission.Action.CREATE);
+      }
+    }
+  }
+
+  @Override
+  public void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      List<HTableDescriptor> descriptors) throws IOException {
+  }
 }

Modified: hbase/branches/0.94/security/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/security/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java?rev=1496240&r1=1496239&r2=1496240&view=diff
==============================================================================
--- hbase/branches/0.94/security/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java (original)
+++ hbase/branches/0.94/security/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java Mon Jun 24 22:21:41 2013
@@ -79,6 +79,7 @@ import org.apache.hadoop.hbase.security.
 import org.apache.hadoop.hbase.security.access.Permission.Action;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.apache.hadoop.ipc.RemoteException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -238,6 +239,9 @@ public class TestAccessController {
           }
         }
         else {
+          if (e instanceof RemoteException) {
+            e = ((RemoteException)e).unwrapRemoteException();
+          }
           // For doBulkLoad calls AccessDeniedException
           // is buried in the stack trace
           Throwable ex = e;
@@ -1869,4 +1873,91 @@ public class TestAccessController {
     }
   }
 
+  @Test
+  public void testTableDescriptorsEnumeration() throws Exception {
+    User TABLE_ADMIN = User.createUserForTesting(conf, "UserA", new String[0]);
+
+    // Grant TABLE ADMIN privs on test table to UserA
+    HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      AccessControllerProtocol protocol = acl.coprocessorProxy(
+          AccessControllerProtocol.class, TEST_TABLE);
+      protocol.grant(new UserPermission(Bytes.toBytes(TABLE_ADMIN.getShortName()),
+        TEST_TABLE, null, Permission.Action.ADMIN));
+    } finally {
+      acl.close();
+    }
+
+    PrivilegedExceptionAction listTablesAction = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
+        try {
+          admin.listTables();
+        } finally {
+          admin.close();
+        }
+        return null;
+      }
+    };
+
+    PrivilegedExceptionAction getTableDescAction = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
+        try {
+          admin.getTableDescriptor(TEST_TABLE);
+        } finally {
+          admin.close();
+        }
+        return null;
+      }
+    };
+
+    verifyAllowed(listTablesAction, SUPERUSER, USER_ADMIN);
+    verifyDenied(listTablesAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, TABLE_ADMIN);
+
+    verifyAllowed(getTableDescAction, SUPERUSER, USER_ADMIN, USER_CREATE, TABLE_ADMIN);
+    verifyDenied(getTableDescAction, USER_RW, USER_RO, USER_NONE);
+  }
+
+  @Test
+  public void testTableDeletion() throws Exception {
+    final User tableAdmin = User.createUserForTesting(conf, "TestUser", new String[0]);
+
+    // We need to create a new table here because we will be testing what
+    // happens when it is deleted
+    final byte[] tableName = Bytes.toBytes("testTableDeletion");
+    HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+    HTableDescriptor htd = new HTableDescriptor(tableName);
+    htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
+    admin.createTable(htd);
+    TEST_UTIL.waitTableEnabled(tableName, 5000);
+
+    // Grant TABLE ADMIN privs
+    HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+    try {
+      AccessControllerProtocol protocol = acl.coprocessorProxy(
+          AccessControllerProtocol.class, tableName);
+      protocol.grant(new UserPermission(Bytes.toBytes(tableAdmin.getShortName()),
+        tableName, null, Permission.Action.ADMIN));
+    } finally {
+      acl.close();
+    }
+
+    PrivilegedExceptionAction deleteTableAction = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
+        try {
+          admin.disableTable(tableName);
+          admin.deleteTable(tableName);
+        } finally {
+          admin.close();
+        }
+        return null;
+      }
+    };
+
+    verifyDenied(deleteTableAction, USER_RW, USER_RO, USER_NONE);
+    verifyAllowed(deleteTableAction, tableAdmin);
+  }
+
 }

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java?rev=1496240&r1=1496239&r2=1496240&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java Mon Jun 24 22:21:41 2013
@@ -25,6 +25,7 @@ import java.io.InterruptedIOException;
 import java.lang.reflect.Proxy;
 import java.lang.reflect.UndeclaredThrowableException;
 import java.net.SocketTimeoutException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
@@ -564,6 +565,8 @@ public class HBaseAdmin implements Abort
     // Wait until all regions deleted
     HRegionInterface server =
       connection.getHRegionConnection(firstMetaServer.getHostname(), firstMetaServer.getPort());
+    List<String> tableNameAsList = new ArrayList<String>(1);
+    tableNameAsList.add(Bytes.toString(tableName));
     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
       long scannerId = -1L;
       try {
@@ -580,15 +583,8 @@ public class HBaseAdmin implements Abort
         // HMaster removes the table from its HTableDescriptors
         if (values == null) {
           tableExists = false;
-          HTableDescriptor[] htds = getMaster().getHTableDescriptors();
-          if (htds != null && htds.length > 0) {
-            for (HTableDescriptor htd: htds) {
-              if (Bytes.equals(tableName, htd.getName())) {
-                tableExists = true;
-                break;
-              }
-            }
-          }
+          HTableDescriptor[] htds = getMaster().getHTableDescriptors(tableNameAsList);
+          tableExists = (htds != null && htds.length > 0);
           if (!tableExists) {
             break;
           }

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=1496240&r1=1496239&r2=1496240&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java Mon Jun 24 22:21:41 2013
@@ -1889,17 +1889,14 @@ public class HConnectionManager {
       if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
         return HTableDescriptor.META_TABLEDESC;
       }
+      List<String> tableNameList = new ArrayList<String>(1);
+      tableNameList.add(Bytes.toString(tableName));
       if (this.master == null) {
         this.master = getMaster();
       }
-
-      HTableDescriptor[] htds = master.getHTableDescriptors();
+      HTableDescriptor[] htds = master.getHTableDescriptors(tableNameList);
       if (htds != null && htds.length > 0) {
-        for (HTableDescriptor htd: htds) {
-          if (Bytes.equals(tableName, htd.getName())) {
-            return htd;
-          }
-        }
+        return htds[0];
       }
       throw new TableNotFoundException(Bytes.toString(tableName));
     }

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java?rev=1496240&r1=1496239&r2=1496240&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java Mon Jun 24 22:21:41 2013
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.UnknownRe
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 
 import java.io.IOException;
+import java.util.List;
 
 public class BaseMasterObserver implements MasterObserver {
   @Override
@@ -232,4 +233,14 @@ public class BaseMasterObserver implemen
   public void postDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final SnapshotDescription snapshot) throws IOException {
   }
+
+  @Override
+  public void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      List<String> tableNamesList, List<HTableDescriptor> descriptors) throws IOException {
+  }
+
+  @Override
+  public void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      List<HTableDescriptor> descriptors) throws IOException {
+  }
 }

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java?rev=1496240&r1=1496239&r2=1496240&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java Mon Jun 24 22:21:41 2013
@@ -24,6 +24,7 @@ import org.apache.hadoop.hbase.*;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 
 import java.io.IOException;
+import java.util.List;
 
 /**
  * Defines coprocessor hooks for interacting with operations on the
@@ -386,4 +387,23 @@ public interface MasterObserver extends 
    */
   void postDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final SnapshotDescription snapshot) throws IOException;
+
+  /**
+   * Called before a getTableDescriptors request has been processed.
+   * @param ctx the environment to interact with the framework and master
+   * @param tableNamesList the list of table names, or null if querying for all
+   * @param descriptors an empty list, can be filled with what to return if bypassing
+   * @throws IOException
+   */
+  void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      List<String> tableNamesList, List<HTableDescriptor> descriptors) throws IOException;
+
+  /**
+   * Called after a getTableDescriptors request has been processed.
+   * @param ctx the environment to interact with the framework and master
+   * @param descriptors the list of descriptors about to be returned
+   * @throws IOException
+   */
+  void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      List<HTableDescriptor> descriptors) throws IOException;
 }

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java?rev=1496240&r1=1496239&r2=1496240&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java Mon Jun 24 22:21:41 2013
@@ -166,10 +166,10 @@ public interface HMasterInterface extend
   /**
    * Return cluster status.
    * @return status object
+   * @throws IOException
    */
   public ClusterStatus getClusterStatus();
 
-
   /**
    * Move the region <code>r</code> to <code>dest</code>.
    * @param encodedRegionName The encoded region name; i.e. the hash that makes
@@ -259,16 +259,29 @@ public interface HMasterInterface extend
 
   /**
    * Get array of all HTDs.
+   * <p>
+   * NOTE: This interface up to and including 0.94.8 included a getHTableDescriptors which
+   * did not throw IOE. Adding and deleting checked exceptions declared as thrown by a method
+   * does not break binary compatibility, so rolling restart scenarios will work; however,
+   * it will break contract compatibility possibly requiring source code changes upon
+   * next recompilation.
    * @return array of HTableDescriptor
    */
-  public HTableDescriptor[] getHTableDescriptors();
+  public HTableDescriptor[] getHTableDescriptors() throws IOException;
 
   /**
    * Get array of HTDs for requested tables.
+   * <p>
+   * NOTE: This interface up to and including 0.94.8 included a getHTableDescriptors which
+   * did not throw IOE. Adding and deleting checked exceptions declared as thrown by a method
+   * does not break binary compatibility, so rolling restart scenarios will work; however,
+   * it will break contract compatibility possibly requiring source code changes upon
+   * next recompilation.
    * @param tableNames
    * @return array of HTableDescriptor
+   * @throws IOException 
    */
-  public HTableDescriptor[] getHTableDescriptors(List<String> tableNames);
+  public HTableDescriptor[] getHTableDescriptors(List<String> tableNames) throws IOException;
 
   /**
    * Executes a single {@link org.apache.hadoop.hbase.ipc.CoprocessorProtocol}

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1496240&r1=1496239&r2=1496240&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Mon Jun 24 22:21:41 2013
@@ -1923,20 +1923,34 @@ Server {
    * @param tableNames
    * @return HTableDescriptor[]
    */
-  public HTableDescriptor[] getHTableDescriptors(List<String> tableNames) {
-    List<HTableDescriptor> list =
+  public HTableDescriptor[] getHTableDescriptors(List<String> tableNames)
+      throws IOException {
+    List<HTableDescriptor> descriptors =
       new ArrayList<HTableDescriptor>(tableNames.size());
-    for (String s: tableNames) {
-      HTableDescriptor htd = null;
-      try {
-        htd = this.tableDescriptors.get(s);
-      } catch (IOException e) {
-        LOG.warn("Failed getting descriptor for " + s, e);
+    
+    boolean bypass = false;
+    if (this.cpHost != null) {
+      bypass = this.cpHost.preGetTableDescriptors(tableNames, descriptors);
+    }
+
+    if (!bypass) {
+      for (String s: tableNames) {
+        HTableDescriptor htd = null;
+        try {
+          htd = this.tableDescriptors.get(s);
+        } catch (IOException e) {
+          LOG.warn("Failed getting descriptor for " + s, e);
+        }
+        if (htd == null) continue;
+        descriptors.add(htd);
       }
-      if (htd == null) continue;
-      list.add(htd);
     }
-    return list.toArray(new HTableDescriptor [] {});
+
+    if (this.cpHost != null) {
+      this.cpHost.postGetTableDescriptors(descriptors);
+    }
+
+    return descriptors.toArray(new HTableDescriptor [] {});
   }
 
   @Override
@@ -2014,16 +2028,21 @@ Server {
   /**
    * Get all table descriptors
    * @return All descriptors or null if none.
+   * @throws IOException
    */
-  public HTableDescriptor [] getHTableDescriptors() {
-    Map<String, HTableDescriptor> descriptors = null;
-    try {
-      descriptors = this.tableDescriptors.getAll();
-    } catch (IOException e) {
-      LOG.warn("Failed getting all descriptors", e);
+  public HTableDescriptor [] getHTableDescriptors() throws IOException {
+    List<HTableDescriptor> descriptors = new ArrayList<HTableDescriptor>();
+    boolean bypass = false;
+    if (this.cpHost != null) {
+      bypass = this.cpHost.preGetTableDescriptors(null, descriptors);
+    }
+    if (!bypass) {
+      descriptors.addAll(this.tableDescriptors.getAll().values());
+    }
+    if (this.cpHost != null) {
+      this.cpHost.postGetTableDescriptors(descriptors);
     }
-    return descriptors == null?
-      null: descriptors.values().toArray(new HTableDescriptor [] {});
+    return descriptors.toArray(new HTableDescriptor [] {});
   }
 
   /**

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java?rev=1496240&r1=1496239&r2=1496240&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java Mon Jun 24 22:21:41 2013
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.ipc.Copro
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 
 import java.io.IOException;
+import java.util.List;
 
 /**
  * Provides the coprocessor framework and environment for master oriented
@@ -772,4 +773,44 @@ public class MasterCoprocessorHost
       }
     }
   }
+
+  public boolean preGetTableDescriptors(final List<String> tableNamesList,
+      final List<HTableDescriptor> descriptors) throws IOException {
+    boolean bypass = false;
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
+    for (MasterEnvironment env : coprocessors) {
+      if (env.getInstance() instanceof MasterObserver) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        try {
+          ((MasterObserver) env.getInstance()).preGetTableDescriptors(ctx,
+            tableNamesList, descriptors);
+        } catch (Throwable e) {
+          handleCoprocessorThrowable(env, e);
+        }
+        bypass |= ctx.shouldBypass();
+        if (ctx.shouldComplete()) {
+          break;
+        }
+      }
+    }
+    return bypass;
+  }
+
+  public void postGetTableDescriptors(List<HTableDescriptor> descriptors) throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
+    for (MasterEnvironment env: coprocessors) {
+      if (env.getInstance() instanceof MasterObserver) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        try {
+          ((MasterObserver)env.getInstance()).postGetTableDescriptors(ctx, descriptors);
+        } catch (Throwable e) {
+          handleCoprocessorThrowable(env, e);
+        }
+        if (ctx.shouldComplete()) {
+          break;
+        }
+      }
+    }
+  }
+
 }

Modified: hbase/branches/0.94/src/main/ruby/hbase/admin.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/ruby/hbase/admin.rb?rev=1496240&r1=1496239&r2=1496240&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/ruby/hbase/admin.rb (original)
+++ hbase/branches/0.94/src/main/ruby/hbase/admin.rb Mon Jun 24 22:21:41 2013
@@ -293,16 +293,7 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Returns table's structure description
     def describe(table_name)
-      tables = @admin.listTables.to_a
-      tables << org.apache.hadoop.hbase.HTableDescriptor::META_TABLEDESC
-      tables << org.apache.hadoop.hbase.HTableDescriptor::ROOT_TABLEDESC
-
-      tables.each do |t|
-        # Found the table
-        return t.to_s if t.getNameAsString == table_name
-      end
-
-      raise(ArgumentError, "Failed to find table named #{table_name}")
+      @admin.getTableDescriptor(table_name.to_java_bytes).to_s
     end
 
     #----------------------------------------------------------------------------------------------

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java?rev=1496240&r1=1496239&r2=1496240&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java Mon Jun 24 22:21:41 2013
@@ -31,8 +31,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
 
-import junit.framework.Assert;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -102,6 +100,8 @@ public class TestMasterObserver {
     private boolean postRestoreSnapshotCalled;
     private boolean preDeleteSnapshotCalled;
     private boolean postDeleteSnapshotCalled;
+    private boolean preGetTableDescriptorsCalled;
+    private boolean postGetTableDescriptorsCalled;
 
     public void enableBypass(boolean bypass) {
       this.bypass = bypass;
@@ -142,6 +142,8 @@ public class TestMasterObserver {
       postRestoreSnapshotCalled = false;
       preDeleteSnapshotCalled = false;
       postDeleteSnapshotCalled = false;
+      preGetTableDescriptorsCalled = false;
+      postGetTableDescriptorsCalled = false;
     }
 
     @Override
@@ -551,6 +553,22 @@ public class TestMasterObserver {
     public boolean wasDeleteSnapshotCalled() {
       return preDeleteSnapshotCalled && postDeleteSnapshotCalled;
     }
+
+    @Override
+    public void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
+        List<String> tableNamesList, List<HTableDescriptor> descriptors) throws IOException {
+      preGetTableDescriptorsCalled = true;
+    }
+
+    @Override
+    public void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
+        List<HTableDescriptor> descriptors) throws IOException {
+      postGetTableDescriptorsCalled = true;
+    }
+
+    public boolean wasGetTableDescriptorsCalled() {
+      return preGetTableDescriptorsCalled && postGetTableDescriptorsCalled;
+    }
   }
 
   private static HBaseTestingUtility UTIL = new HBaseTestingUtility();
@@ -878,6 +896,22 @@ public class TestMasterObserver {
     }
   }
 
+  @Test
+  public void testTableDescriptorsEnumeration() throws Exception {
+    MiniHBaseCluster cluster = UTIL.getHBaseCluster();
+
+    HMaster master = cluster.getMaster();
+    MasterCoprocessorHost host = master.getCoprocessorHost();
+    CPMasterObserver cp = (CPMasterObserver)host.findCoprocessor(
+        CPMasterObserver.class.getName());
+    cp.resetStates();
+
+    master.getHTableDescriptors();
+
+    assertTrue("Coprocessor should be called on table descriptors request",
+      cp.wasGetTableDescriptorsCalled());
+  }
+
   @org.junit.Rule
   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();

Modified: hbase/branches/0.94/src/test/ruby/hbase/admin_test.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/ruby/hbase/admin_test.rb?rev=1496240&r1=1496239&r2=1496240&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/ruby/hbase/admin_test.rb (original)
+++ hbase/branches/0.94/src/test/ruby/hbase/admin_test.rb Mon Jun 24 22:21:41 2013
@@ -161,7 +161,7 @@ module Hbase
     #-------------------------------------------------------------------------------
 
     define_test "describe should fail for non-existent tables" do
-      assert_raise(ArgumentError) do
+      assert_raise(NativeException) do
         admin.describe('.NOT.EXISTS.')
       end
     end