You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2017/05/03 21:03:13 UTC

hbase git commit: HBASE-17978 Ensure superusers can circumvent actions restricted by space quota violations

Repository: hbase
Updated Branches:
  refs/heads/HBASE-16961 ecccca14f -> 06f71477c


HBASE-17978 Ensure superusers can circumvent actions restricted by space quota violations


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

Branch: refs/heads/HBASE-16961
Commit: 06f71477c087b7c695786042e353769578dfd74c
Parents: ecccca1
Author: Josh Elser <el...@apache.org>
Authored: Wed May 3 12:10:50 2017 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Wed May 3 17:01:56 2017 -0400

----------------------------------------------------------------------
 .../hbase/regionserver/RSRpcServices.java       |   4 +
 .../hbase/quotas/SpaceQuotaHelperForTests.java  |  27 +-
 .../quotas/TestSuperUserQuotaPermissions.java   | 300 +++++++++++++++++++
 3 files changed, 329 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/06f71477/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 679bba2..56fccce 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -112,6 +112,7 @@ import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
 import org.apache.hadoop.hbase.regionserver.handler.OpenPriorityRegionHandler;
 import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message;
@@ -1471,7 +1472,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       checkOpen();
       requestCount.increment();
       Region region = getRegion(request.getRegion());
+      // Quota support is enabled, the requesting user is not system/super user
+      // and a quota policy is enforced that disables compactions.
       if (QuotaUtil.isQuotaEnabled(getConfiguration()) &&
+          !Superusers.isSuperUser(RpcServer.getRequestUser()) &&
           this.regionServer.getRegionServerSpaceQuotaManager().areCompactionsDisabled(
               region.getTableDesc().getTableName())) {
         throw new DoNotRetryIOException("Compactions on this region are "

http://git-wip-us.apache.org/repos/asf/hbase/blob/06f71477/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java
index b7c51a2..1e2235a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java
@@ -110,6 +110,18 @@ public class SpaceQuotaHelperForTests {
     }
   }
 
+  QuotaSettings getTableSpaceQuota(Connection conn, TableName tn) throws IOException {
+    try (QuotaRetriever scanner = QuotaRetriever.open(
+        conn.getConfiguration(), new QuotaFilter().setTableFilter(tn.getNameAsString()))) {
+      for (QuotaSettings setting : scanner) {
+        if (setting.getTableName().equals(tn) && setting.getQuotaType() == QuotaType.SPACE) {
+          return setting;
+        }
+      }
+      return null;
+    }
+  }
+
   /**
    * Waits 30seconds for the HBase quota table to exist.
    */
@@ -130,7 +142,10 @@ public class SpaceQuotaHelperForTests {
   }
 
   void writeData(TableName tn, long sizeInBytes) throws IOException {
-    final Connection conn = testUtil.getConnection();
+    writeData(testUtil.getConnection(), tn, sizeInBytes);
+  }
+
+  void writeData(Connection conn, TableName tn, long sizeInBytes) throws IOException {
     final Table table = conn.getTable(tn);
     try {
       List<Put> updates = new ArrayList<>();
@@ -226,8 +241,16 @@ public class SpaceQuotaHelperForTests {
     return createTableWithRegions(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR, numRegions);
   }
 
+  TableName createTableWithRegions(Admin admin, int numRegions) throws Exception {
+    return createTableWithRegions(
+        testUtil.getAdmin(), NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR, numRegions);
+  }
+
   TableName createTableWithRegions(String namespace, int numRegions) throws Exception {
-    final Admin admin = testUtil.getAdmin();
+    return createTableWithRegions(testUtil.getAdmin(), namespace, numRegions);
+  }
+
+  TableName createTableWithRegions(Admin admin, String namespace, int numRegions) throws Exception {
     final TableName tn = TableName.valueOf(
         namespace, testName.getMethodName() + counter.getAndIncrement());
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/06f71477/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSuperUserQuotaPermissions.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSuperUserQuotaPermissions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSuperUserQuotaPermissions.java
new file mode 100644
index 0000000..812e7e0
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSuperUserQuotaPermissions.java
@@ -0,0 +1,300 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.quotas;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.Waiter.Predicate;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.security.access.AccessControlClient;
+import org.apache.hadoop.hbase.security.access.AccessController;
+import org.apache.hadoop.hbase.security.access.Permission.Action;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Test class to verify that the HBase superuser can override quotas.
+ */
+@Category(MediumTests.class)
+public class TestSuperUserQuotaPermissions {
+  private static final Log LOG = LogFactory.getLog(TestSuperUserQuotaPermissions.class);
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  // Default to the user running the tests
+  private static final String SUPERUSER_NAME = System.getProperty("user.name");
+  private static final UserGroupInformation SUPERUSER_UGI =
+      UserGroupInformation.createUserForTesting(SUPERUSER_NAME, new String[0]);
+  private static final String REGULARUSER_NAME = "quota_regularuser";
+  private static final UserGroupInformation REGULARUSER_UGI =
+      UserGroupInformation.createUserForTesting(REGULARUSER_NAME, new String[0]);
+  private static final AtomicLong COUNTER = new AtomicLong(0);
+
+  @Rule
+  public TestName testName = new TestName();
+  private SpaceQuotaHelperForTests helper;
+
+  @BeforeClass
+  public static void setupMiniCluster() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    // Increase the frequency of some of the chores for responsiveness of the test
+    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_DELAY_KEY, 1000);
+    conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_PERIOD_KEY, 1000);
+    conf.setInt(QuotaObserverChore.QUOTA_OBSERVER_CHORE_DELAY_KEY, 1000);
+    conf.setInt(QuotaObserverChore.QUOTA_OBSERVER_CHORE_PERIOD_KEY, 1000);
+    conf.setInt(SpaceQuotaRefresherChore.POLICY_REFRESHER_CHORE_DELAY_KEY, 1000);
+    conf.setInt(SpaceQuotaRefresherChore.POLICY_REFRESHER_CHORE_PERIOD_KEY, 1000);
+    conf.setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
+
+    conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, AccessController.class.getName());
+    conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, AccessController.class.getName());
+    conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY, AccessController.class.getName());
+    conf.setBoolean("hbase.security.exec.permission.checks", true);
+    conf.setBoolean("hbase.security.authorization", true);
+    conf.set("hbase.superuser", SUPERUSER_NAME);
+
+    TEST_UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void removeAllQuotas() throws Exception {
+    final Connection conn = TEST_UTIL.getConnection();
+    if (helper == null) {
+      helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, COUNTER);
+    }
+    // Wait for the quota table to be created
+    if (!conn.getAdmin().tableExists(QuotaUtil.QUOTA_TABLE_NAME)) {
+      helper.waitForQuotaTable(conn);
+    } else {
+      // Or, clean up any quotas from previous test runs.
+      helper.removeAllQuotas(conn);
+      assertEquals(0, helper.listNumDefinedQuotas(conn));
+    }
+  }
+
+  @Test
+  public void testSuperUserCanStillCompact() throws Exception {
+    // Create a table and write enough data to push it into quota violation
+    final TableName tn = doAsSuperUser(new Callable<TableName>() {
+      @Override
+      public TableName call() throws Exception {
+        try (Connection conn = getConnection()) {
+          Admin admin = conn.getAdmin();
+          final TableName tn = helper.createTableWithRegions(admin, 5);
+          final long sizeLimit = 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
+          QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
+              tn, sizeLimit, SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
+          admin.setQuota(settings);
+          // Grant the normal user permissions
+          try {
+            AccessControlClient.grant(
+                conn, tn, REGULARUSER_NAME, null, null, Action.READ, Action.WRITE);
+          } catch (Throwable t) {
+            if (t instanceof Exception) {
+              throw (Exception) t;
+            }
+            throw new Exception(t);
+          }
+          return tn;
+        }
+      }
+    });
+
+    // Write a bunch of data as our end-user
+    doAsRegularUser(new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        try (Connection conn = getConnection()) {
+          helper.writeData(tn, 3L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
+          return null;
+        }
+      }
+    });
+
+    waitForTableToEnterQuotaViolation(tn);
+
+    // Should throw an exception, unprivileged users cannot compact due to the quota
+    try {
+      doAsRegularUser(new Callable<Void>() {
+        @Override
+        public Void call() throws Exception {
+          try (Connection conn = getConnection()) {
+            conn.getAdmin().majorCompact(tn);
+            return null;
+          }
+        }
+      });
+      fail("Expected an exception trying to compact a table with a quota violation");
+    } catch (DoNotRetryIOException e) {
+      // Expected
+    }
+
+    // Should not throw an exception (superuser can do anything)
+    doAsSuperUser(new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        try (Connection conn = getConnection()) {
+          conn.getAdmin().majorCompact(tn);
+          return null;
+        }
+      }
+    });
+  }
+
+  @Test
+  public void testSuperuserCanRemoveQuota() throws Exception {
+    // Create a table and write enough data to push it into quota violation
+    final TableName tn = doAsSuperUser(new Callable<TableName>() {
+      @Override
+      public TableName call() throws Exception {
+        try (Connection conn = getConnection()) {
+          final Admin admin = conn.getAdmin();
+          final TableName tn = helper.createTableWithRegions(admin, 5);
+          final long sizeLimit = 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
+          QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
+              tn, sizeLimit, SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
+          admin.setQuota(settings);
+          // Grant the normal user permission to create a table and set a quota
+          try {
+            AccessControlClient.grant(
+                conn, tn, REGULARUSER_NAME, null, null, Action.READ, Action.WRITE);
+          } catch (Throwable t) {
+            if (t instanceof Exception) {
+              throw (Exception) t;
+            }
+            throw new Exception(t);
+          }
+          return tn;
+        }
+      }
+    });
+
+    // Write a bunch of data as our end-user
+    doAsRegularUser(new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        try (Connection conn = getConnection()) {
+          helper.writeData(tn, 3L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
+          return null;
+        }
+      }
+    });
+
+    // Wait for the table to hit quota violation
+    waitForTableToEnterQuotaViolation(tn);
+
+    // Try to be "bad" and remove the quota as the end user (we want to write more data!)
+    doAsRegularUser(new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        try (Connection conn = getConnection()) {
+          final Admin admin = conn.getAdmin();
+          QuotaSettings qs = QuotaSettingsFactory.removeTableSpaceLimit(tn);
+          try {
+            admin.setQuota(qs);
+            fail("Expected that an unprivileged user should not be allowed to remove a quota");
+          } catch (Exception e) {
+            // pass
+          }
+          return null;
+        }
+      }
+    });
+
+    // Verify that the superuser can remove the quota
+    doAsSuperUser(new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        try (Connection conn = getConnection()) {
+          final Admin admin = conn.getAdmin();
+          QuotaSettings qs = QuotaSettingsFactory.removeTableSpaceLimit(tn);
+          admin.setQuota(qs);
+          assertNull(helper.getTableSpaceQuota(conn, tn));
+          return null;
+        }
+      }
+    });
+  }
+
+  private Connection getConnection() throws IOException {
+    return ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
+  }
+
+  private <T> T doAsSuperUser(Callable<T> task) throws Exception {
+    return doAsUser(SUPERUSER_UGI, task);
+  }
+
+  private <T> T doAsRegularUser(Callable<T> task) throws Exception {
+    return doAsUser(REGULARUSER_UGI, task);
+  }
+
+  private <T> T doAsUser(UserGroupInformation ugi, Callable<T> task) throws Exception {
+    return ugi.doAs(new PrivilegedExceptionAction<T>() {
+      public T run() throws Exception {
+        return task.call();
+      }
+    });
+  }
+
+  private void waitForTableToEnterQuotaViolation(TableName tn) throws Exception {
+    // Verify that the RegionServer has the quota in violation
+    final HRegionServer rs = TEST_UTIL.getHBaseCluster().getRegionServer(0);
+    Waiter.waitFor(TEST_UTIL.getConfiguration(), 30 * 1000, 1000, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        Map<TableName,SpaceQuotaSnapshot> snapshots =
+            rs.getRegionServerSpaceQuotaManager().copyQuotaSnapshots();
+        SpaceQuotaSnapshot snapshot = snapshots.get(tn);
+        if (snapshot == null) {
+          LOG.info("Found no snapshot for " + tn);
+          return false;
+        }
+        LOG.info("Found snapshot " + snapshot);
+        return snapshot.getQuotaStatus().isInViolation();
+      }
+    });
+  }
+}