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 2012/06/13 02:19:32 UTC

svn commit: r1349597 - 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/coprocessor/ src/main/java/org/apache/...

Author: apurtell
Date: Wed Jun 13 00:19:31 2012
New Revision: 1349597

URL: http://svn.apache.org/viewvc?rev=1349597&view=rev
Log:
HBASE-6092. Authorize flush, split, compact operations in AccessController (Laxman)

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/coprocessor/BaseRegionObserver.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java

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=1349597&r1=1349596&r2=1349597&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 Wed Jun 13 00:19:31 2012
@@ -53,11 +53,14 @@ import org.apache.hadoop.hbase.ipc.Reque
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.MapMaker;
@@ -366,12 +369,12 @@ public class AccessController extends Ba
    * that means he/she can edit/modify/delete the table.
    * If current user is the table owner, and has CREATE permission,
    * then he/she has table admin permission. otherwise ADMIN rights are checked.
-   * @param e Master coprocessor environment
+   * @param e Coprocessor environment
    * @param tableName Table requested
    * @throws IOException if obtaining the current user fails
    * @throws AccessDeniedException if authorization is denied
    */
-  private void requireTableAdminPermission(MasterCoprocessorEnvironment e, byte[] tableName)
+  private void requireTableAdminPermission(CoprocessorEnvironment e, byte[] tableName)
       throws IOException {
     User user = getActiveUser();
     AuthResult result = null;
@@ -721,6 +724,23 @@ public class AccessController extends Ba
   }
 
   @Override
+  public void preFlush(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException {
+    requireTableAdminPermission(e.getEnvironment(), getTableName(e.getEnvironment()));
+  }
+
+  @Override
+  public void preSplit(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException {
+    requireTableAdminPermission(e.getEnvironment(), getTableName(e.getEnvironment()));
+  }
+
+  @Override
+  public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e,
+      final Store store, final InternalScanner scanner) throws IOException {
+    requireTableAdminPermission(e.getEnvironment(), getTableName(e.getEnvironment()));
+    return scanner;
+  }
+
+  @Override
   public void preGetClosestRowBefore(final ObserverContext<RegionCoprocessorEnvironment> c,
       final byte [] row, final byte [] family, final Result result)
       throws IOException {
@@ -1088,14 +1108,13 @@ public class AccessController extends Ba
     return tableName;
   }
 
-  private String getTableOwner(MasterCoprocessorEnvironment e,
-      byte[] tableName) throws IOException {
+  private String getTableOwner(CoprocessorEnvironment e, byte[] tableName) throws IOException {
     HTableDescriptor htd = e.getTable(tableName).getTableDescriptor();
     return htd.getOwnerString();
   }
 
-  private boolean isActiveUserTableOwner(MasterCoprocessorEnvironment e,
-      byte[] tableName) throws IOException {
+  private boolean isActiveUserTableOwner(CoprocessorEnvironment e, byte[] tableName)
+      throws IOException {
     String activeUser = getActiveUser().getShortName();
     return activeUser.equals(getTableOwner(e, tableName));
   }

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=1349597&r1=1349596&r2=1349597&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 Wed Jun 13 00:19:31 2012
@@ -31,6 +31,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -51,7 +52,10 @@ import org.apache.hadoop.hbase.client.Sc
 import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -89,6 +93,7 @@ public class TestAccessController {
   private static byte[] TEST_FAMILY = Bytes.toBytes("f1");
 
   private static MasterCoprocessorEnvironment CP_ENV;
+  private static RegionCoprocessorEnvironment RCP_ENV;
   private static AccessController ACCESS_CONTROLLER;
 
   @BeforeClass
@@ -126,6 +131,11 @@ public class TestAccessController {
     AccessControllerProtocol protocol =
         meta.coprocessorProxy(AccessControllerProtocol.class, TEST_TABLE);
 
+    HRegion region = TEST_UTIL.getHBaseCluster().getRegions(TEST_TABLE).get(0);
+    RegionCoprocessorHost rcpHost = region.getCoprocessorHost();
+    RCP_ENV = rcpHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER,
+        Coprocessor.PRIORITY_HIGHEST, 1, conf);
+
     protocol.grant(new UserPermission(Bytes.toBytes(USER_ADMIN.getShortName()),
                       Permission.Action.ADMIN, Permission.Action.CREATE,
                       Permission.Action.READ, Permission.Action.WRITE));
@@ -542,6 +552,54 @@ public class TestAccessController {
     verifyAllowed(USER_RW, action);
   }
 
+  @Test
+  public void testSplit() throws Exception {
+    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preSplit(ObserverContext.createAndPrepare(RCP_ENV, null));
+        return null;
+      }
+    };
+
+    // verify that superuser and admin only can split
+    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_TBLADM);
+
+    // all others should be denied
+    verifyDenied(action, USER_OWNER, USER_RW, USER_RO, USER_NONE);
+  }
+
+  @Test
+  public void testFlush() throws Exception {
+    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preFlush(ObserverContext.createAndPrepare(RCP_ENV, null));
+        return null;
+      }
+    };
+
+    // verify that superuser and admin only can flush
+    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_TBLADM);
+
+    // all others should be denied
+    verifyDenied(action, USER_OWNER, USER_RW, USER_RO, USER_NONE);
+  }
+
+  @Test
+  public void testCompact() throws Exception {
+    PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preCompact(ObserverContext.createAndPrepare(RCP_ENV, null), null, null);
+        return null;
+      }
+    };
+
+    // verify that superuser and admin only can compact
+    verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_TBLADM);
+
+    // all others should be denied
+    verifyDenied(action, USER_OWNER, USER_RW, USER_RO, USER_NONE);
+  }
+
   private void verifyRead(PrivilegedExceptionAction action) throws Exception {
     // should be denied
     verifyDenied(USER_NONE, action);

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java?rev=1349597&r1=1349596&r2=1349597&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java Wed Jun 13 00:19:31 2012
@@ -69,17 +69,21 @@ public abstract class BaseRegionObserver
       boolean abortRequested) { }
 
   @Override
-  public void preFlush(ObserverContext<RegionCoprocessorEnvironment> e) { }
+  public void preFlush(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException {
+  }
 
   @Override
-  public void postFlush(ObserverContext<RegionCoprocessorEnvironment> e) { }
+  public void postFlush(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException {
+  }
 
   @Override
-  public void preSplit(ObserverContext<RegionCoprocessorEnvironment> e) { }
+  public void preSplit(ObserverContext<RegionCoprocessorEnvironment> e) throws IOException {
+  }
 
   @Override
-  public void postSplit(ObserverContext<RegionCoprocessorEnvironment> e,
-      HRegion l, HRegion r) { }
+  public void postSplit(ObserverContext<RegionCoprocessorEnvironment> e, HRegion l, HRegion r)
+      throws IOException {
+  }
 
   @Override
   public void preCompactSelection(final ObserverContext<RegionCoprocessorEnvironment> c,
@@ -91,13 +95,14 @@ public abstract class BaseRegionObserver
 
   @Override
   public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e,
-      final Store store, final InternalScanner scanner) {
+      final Store store, final InternalScanner scanner) throws IOException {
     return scanner;
   }
 
   @Override
-  public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e,
-      final Store store, final StoreFile resultFile) { }
+  public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e, final Store store,
+      final StoreFile resultFile) throws IOException {
+  }
 
   @Override
   public void preGetClosestRowBefore(final ObserverContext<RegionCoprocessorEnvironment> e,

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java?rev=1349597&r1=1349596&r2=1349597&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java Wed Jun 13 00:19:31 2012
@@ -62,14 +62,16 @@ public interface RegionObserver extends 
   /**
    * Called before the memstore is flushed to disk.
    * @param c the environment provided by the region server
+   * @throws IOException if an error occurred on the coprocessor
    */
-  void preFlush(final ObserverContext<RegionCoprocessorEnvironment> c);
+  void preFlush(final ObserverContext<RegionCoprocessorEnvironment> c) throws IOException;
 
   /**
    * Called after the memstore is flushed to disk.
    * @param c the environment provided by the region server
+   * @throws IOException if an error occurred on the coprocessor
    */
-  void postFlush(final ObserverContext<RegionCoprocessorEnvironment> c);
+  void postFlush(final ObserverContext<RegionCoprocessorEnvironment> c) throws IOException;
 
   /**
    * Called prior to selecting the {@link StoreFile}s to compact from the list
@@ -114,9 +116,10 @@ public interface RegionObserver extends 
    * rewriting
    * @return the scanner to use during compaction.  Should not be {@code null}
    * unless the implementation is writing new store files on its own.
+   * @throws IOException if an error occurred on the coprocessor
    */
   InternalScanner preCompact(final ObserverContext<RegionCoprocessorEnvironment> c,
-    final Store store, final InternalScanner scanner);
+      final Store store, final InternalScanner scanner) throws IOException;
 
   /**
    * Called after compaction has completed and the new store file has been
@@ -124,16 +127,18 @@ public interface RegionObserver extends 
    * @param c the environment provided by the region server
    * @param store the store being compacted
    * @param resultFile the new store file written out during compaction
+   * @throws IOException if an error occurred on the coprocessor
    */
-  void postCompact(final ObserverContext<RegionCoprocessorEnvironment> c,
-    final Store store, StoreFile resultFile);
+  void postCompact(final ObserverContext<RegionCoprocessorEnvironment> c, final Store store,
+      StoreFile resultFile) throws IOException;
 
   /**
    * Called before the region is split.
    * @param c the environment provided by the region server
    * (e.getRegion() returns the parent region)
+   * @throws IOException if an error occurred on the coprocessor
    */
-  void preSplit(final ObserverContext<RegionCoprocessorEnvironment> c);
+  void preSplit(final ObserverContext<RegionCoprocessorEnvironment> c) throws IOException;
 
   /**
    * Called after the region is split.
@@ -141,9 +146,10 @@ public interface RegionObserver extends 
    * (e.getRegion() returns the parent region)
    * @param l the left daughter region
    * @param r the right daughter region
+   * @throws IOException if an error occurred on the coprocessor
    */
   void postSplit(final ObserverContext<RegionCoprocessorEnvironment> c, final HRegion l,
-    final HRegion r);
+      final HRegion r) throws IOException;
 
   /**
    * Called before the region is reported as closed to the master.

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java?rev=1349597&r1=1349596&r2=1349597&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java Wed Jun 13 00:19:31 2012
@@ -355,8 +355,9 @@ public class RegionCoprocessorHost
    * Called prior to rewriting the store files selected for compaction
    * @param store the store being compacted
    * @param scanner the scanner used to read store data during compaction
+   * @throws IOException 
    */
-  public InternalScanner preCompact(Store store, InternalScanner scanner) {
+  public InternalScanner preCompact(Store store, InternalScanner scanner) throws IOException {
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     boolean bypass = false;
     for (RegionEnvironment env: coprocessors) {
@@ -366,7 +367,7 @@ public class RegionCoprocessorHost
           scanner = ((RegionObserver)env.getInstance()).preCompact(
               ctx, store, scanner);
         } catch (Throwable e) {
-          handleCoprocessorThrowableNoRethrow(env,e);
+          handleCoprocessorThrowable(env,e);
         }
         bypass |= ctx.shouldBypass();
         if (ctx.shouldComplete()) {
@@ -381,8 +382,9 @@ public class RegionCoprocessorHost
    * Called after the store compaction has completed.
    * @param store the store being compacted
    * @param resultFile the new store file written during compaction
+   * @throws IOException 
    */
-  public void postCompact(Store store, StoreFile resultFile) {
+  public void postCompact(Store store, StoreFile resultFile) throws IOException {
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
@@ -390,7 +392,7 @@ public class RegionCoprocessorHost
         try {
           ((RegionObserver)env.getInstance()).postCompact(ctx, store, resultFile);
         } catch (Throwable e) {
-          handleCoprocessorThrowableNoRethrow(env, e);
+          handleCoprocessorThrowable(env, e);
         }
         if (ctx.shouldComplete()) {
           break;
@@ -401,8 +403,9 @@ public class RegionCoprocessorHost
 
   /**
    * Invoked before a memstore flush
+   * @throws IOException 
    */
-  public void preFlush() {
+  public void preFlush() throws IOException {
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
@@ -410,7 +413,7 @@ public class RegionCoprocessorHost
         try {
           ((RegionObserver)env.getInstance()).preFlush(ctx);
         } catch (Throwable e) {
-          handleCoprocessorThrowableNoRethrow(env, e);
+          handleCoprocessorThrowable(env, e);
         }
         if (ctx.shouldComplete()) {
           break;
@@ -421,8 +424,9 @@ public class RegionCoprocessorHost
 
   /**
    * Invoked after a memstore flush
+   * @throws IOException 
    */
-  public void postFlush() {
+  public void postFlush() throws IOException {
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
@@ -430,7 +434,7 @@ public class RegionCoprocessorHost
         try {
           ((RegionObserver)env.getInstance()).postFlush(ctx);
         } catch (Throwable e) {
-          handleCoprocessorThrowableNoRethrow(env, e);
+          handleCoprocessorThrowable(env, e);
         }
         if (ctx.shouldComplete()) {
           break;
@@ -441,8 +445,9 @@ public class RegionCoprocessorHost
 
   /**
    * Invoked just before a split
+   * @throws IOException 
    */
-  public void preSplit() {
+  public void preSplit() throws IOException {
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
@@ -450,7 +455,7 @@ public class RegionCoprocessorHost
         try {
           ((RegionObserver)env.getInstance()).preSplit(ctx);
         } catch (Throwable e) {
-          handleCoprocessorThrowableNoRethrow(env, e);
+          handleCoprocessorThrowable(env, e);
         }
         if (ctx.shouldComplete()) {
           break;
@@ -463,8 +468,9 @@ public class RegionCoprocessorHost
    * Invoked just after a split
    * @param l the new left-hand daughter region
    * @param r the new right-hand daughter region
+   * @throws IOException 
    */
-  public void postSplit(HRegion l, HRegion r) {
+  public void postSplit(HRegion l, HRegion r) throws IOException {
     ObserverContext<RegionCoprocessorEnvironment> ctx = null;
     for (RegionEnvironment env: coprocessors) {
       if (env.getInstance() instanceof RegionObserver) {
@@ -472,7 +478,7 @@ public class RegionCoprocessorHost
         try {
           ((RegionObserver)env.getInstance()).postSplit(ctx, l, r);
         } catch (Throwable e) {
-          handleCoprocessorThrowableNoRethrow(env, e);
+          handleCoprocessorThrowable(env, e);
         }
         if (ctx.shouldComplete()) {
           break;