You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2021/07/26 18:54:40 UTC

[hbase] branch branch-2.3 updated: Revert "HBASE-26001 When turn on access control, the cell level TTL of Increment and Append operations is invalid (#3403)"

This is an automated email from the ASF dual-hosted git repository.

stack pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.3 by this push:
     new 86320d6  Revert "HBASE-26001 When turn on access control, the cell level TTL of Increment and Append operations is invalid (#3403)"
86320d6 is described below

commit 86320d6b0498a5679f25ea9717adae8453d4b50b
Author: stack <st...@apache.org>
AuthorDate: Mon Jul 26 10:57:20 2021 -0700

    Revert "HBASE-26001 When turn on access control, the cell level TTL of Increment and Append operations is invalid (#3403)"
    
    This reverts commit 864ac503a008dbcf6b8c9ec246bbf2f7b913c39d.
---
 .../hbase/security/access/AccessController.java    |  45 +++++--
 .../TestPostIncrementAndAppendBeforeWAL.java       | 139 +--------------------
 2 files changed, 37 insertions(+), 147 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index a643d74..6b7a532 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -135,6 +135,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
 import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet;
 import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
@@ -1784,7 +1785,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
       List<Pair<Cell, Cell>> cellPairs) throws IOException {
     // If the HFile version is insufficient to persist tags, we won't have any
     // work to do here
-    if (!cellFeaturesEnabled || mutation.getACL() == null) {
+    if (!cellFeaturesEnabled) {
       return cellPairs;
     }
     return cellPairs.stream().map(pair -> new Pair<>(pair.getFirst(),
@@ -1798,7 +1799,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
       List<Pair<Cell, Cell>> cellPairs) throws IOException {
     // If the HFile version is insufficient to persist tags, we won't have any
     // work to do here
-    if (!cellFeaturesEnabled || mutation.getACL() == null) {
+    if (!cellFeaturesEnabled) {
       return cellPairs;
     }
     return cellPairs.stream().map(pair -> new Pair<>(pair.getFirst(),
@@ -1807,28 +1808,50 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
   }
 
   private Cell createNewCellWithTags(Mutation mutation, Cell oldCell, Cell newCell) {
-    // As Increment and Append operations have already copied the tags of oldCell to the newCell,
-    // there is no need to rewrite them again. Just extract non-acl tags of newCell if we need to
-    // add a new acl tag for the cell. Actually, oldCell is useless here.
+    // Collect any ACLs from the old cell
     List<Tag> tags = Lists.newArrayList();
-    if (newCell != null) {
-      Iterator<Tag> tagIterator = PrivateCellUtil.tagsIterator(newCell);
+    List<Tag> aclTags = Lists.newArrayList();
+    ListMultimap<String,Permission> perms = ArrayListMultimap.create();
+    if (oldCell != null) {
+      Iterator<Tag> tagIterator = PrivateCellUtil.tagsIterator(oldCell);
       while (tagIterator.hasNext()) {
         Tag tag = tagIterator.next();
         if (tag.getType() != PermissionStorage.ACL_TAG_TYPE) {
           // Not an ACL tag, just carry it through
           if (LOG.isTraceEnabled()) {
-            LOG.trace("Carrying forward tag from " + newCell + ": type " + tag.getType()
+            LOG.trace("Carrying forward tag from " + oldCell + ": type " + tag.getType()
                 + " length " + tag.getValueLength());
           }
           tags.add(tag);
+        } else {
+          aclTags.add(tag);
+        }
+      }
+    }
+
+    // Do we have an ACL on the operation?
+    byte[] aclBytes = mutation.getACL();
+    if (aclBytes != null) {
+      // Yes, use it
+      tags.add(new ArrayBackedTag(PermissionStorage.ACL_TAG_TYPE, aclBytes));
+    } else {
+      // No, use what we carried forward
+      if (perms != null) {
+        // TODO: If we collected ACLs from more than one tag we may have a
+        // List<Permission> of size > 1, this can be collapsed into a single
+        // Permission
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Carrying forward ACLs from " + oldCell + ": " + perms);
         }
+        tags.addAll(aclTags);
       }
     }
 
-    // We have checked the ACL tag of mutation is not null.
-    // So that the tags could not be empty.
-    tags.add(new ArrayBackedTag(PermissionStorage.ACL_TAG_TYPE, mutation.getACL()));
+    // If we have no tags to add, just return
+    if (tags.isEmpty()) {
+      return newCell;
+    }
+
     return PrivateCellUtil.createCell(newCell, tags);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestPostIncrementAndAppendBeforeWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestPostIncrementAndAppendBeforeWAL.java
index a845826..031960b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestPostIncrementAndAppendBeforeWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestPostIncrementAndAppendBeforeWAL.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Optional;
 import java.util.stream.Collectors;
@@ -30,15 +29,10 @@ import java.util.stream.Collectors;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellBuilderType;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Tag;
-import org.apache.hadoop.hbase.TagBuilderFactory;
-import org.apache.hadoop.hbase.TagType;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
@@ -51,8 +45,6 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.client.TestFromClientSide;
 import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
-import org.apache.hadoop.hbase.security.access.AccessController;
-import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -72,14 +64,14 @@ import org.slf4j.LoggerFactory;
  * {@link RegionObserver#postIncrementBeforeWAL(ObserverContext, Mutation, List)} and
  * {@link RegionObserver#postAppendBeforeWAL(ObserverContext, Mutation, List)}. These methods may
  * change the cells which will be applied to memstore and WAL. So add unit test for the case which
- * change the cell's column family and tags.
+ * change the cell's column family.
  */
 @Category({CoprocessorTests.class, MediumTests.class})
 public class TestPostIncrementAndAppendBeforeWAL {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestPostIncrementAndAppendBeforeWAL.class);
+      HBaseClassTestRule.forClass(TestPostIncrementAndAppendBeforeWAL.class);
 
   @Rule
   public TestName name = new TestName();
@@ -100,10 +92,6 @@ public class TestPostIncrementAndAppendBeforeWAL {
   private static final byte[] CQ1 = Bytes.toBytes("cq1");
   private static final byte[] CQ2 = Bytes.toBytes("cq2");
   private static final byte[] VALUE = Bytes.toBytes("value");
-  private static final byte[] VALUE2 = Bytes.toBytes("valuevalue");
-  private static final String USER = "User";
-  private static final Permission PERMS =
-    Permission.newBuilder().withActions(Permission.Action.READ).build();
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
@@ -173,94 +161,6 @@ public class TestPostIncrementAndAppendBeforeWAL {
     }
   }
 
-  @Test
-  public void testIncrementTTLWithACLTag() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
-    createTableWithCoprocessor(tableName, ChangeCellWithACLTagObserver.class.getName());
-    try (Table table = connection.getTable(tableName)) {
-      // Increment without TTL
-      Increment firstIncrement = new Increment(ROW).addColumn(CF1_BYTES, CQ1, 1)
-        .setACL(USER, PERMS);
-      Result result = table.increment(firstIncrement);
-      assertEquals(1, result.size());
-      assertEquals(1, Bytes.toLong(result.getValue(CF1_BYTES, CQ1)));
-
-      // Check if the new cell can be read
-      Get get = new Get(ROW).addColumn(CF1_BYTES, CQ1);
-      result = table.get(get);
-      assertEquals(1, result.size());
-      assertEquals(1, Bytes.toLong(result.getValue(CF1_BYTES, CQ1)));
-
-      // Increment with TTL
-      Increment secondIncrement = new Increment(ROW).addColumn(CF1_BYTES, CQ1, 1).setTTL(1000)
-        .setACL(USER, PERMS);
-      result = table.increment(secondIncrement);
-
-      // We should get value 2 here
-      assertEquals(1, result.size());
-      assertEquals(2, Bytes.toLong(result.getValue(CF1_BYTES, CQ1)));
-
-      // Wait 4s to let the second increment expire
-      Thread.sleep(4000);
-      get = new Get(ROW).addColumn(CF1_BYTES, CQ1);
-      result = table.get(get);
-
-      // The value should revert to 1
-      assertEquals(1, result.size());
-      assertEquals(1, Bytes.toLong(result.getValue(CF1_BYTES, CQ1)));
-    }
-  }
-
-  @Test
-  public void testAppendTTLWithACLTag() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
-    createTableWithCoprocessor(tableName, ChangeCellWithACLTagObserver.class.getName());
-    try (Table table = connection.getTable(tableName)) {
-      // Append without TTL
-      Append firstAppend = new Append(ROW).addColumn(CF1_BYTES, CQ2, VALUE).setACL(USER, PERMS);
-      Result result = table.append(firstAppend);
-      assertEquals(1, result.size());
-      assertTrue(Bytes.equals(VALUE, result.getValue(CF1_BYTES, CQ2)));
-
-      // Check if the new cell can be read
-      Get get = new Get(ROW).addColumn(CF1_BYTES, CQ2);
-      result = table.get(get);
-      assertEquals(1, result.size());
-      assertTrue(Bytes.equals(VALUE, result.getValue(CF1_BYTES, CQ2)));
-
-      // Append with TTL
-      Append secondAppend = new Append(ROW).addColumn(CF1_BYTES, CQ2, VALUE).setTTL(1000)
-        .setACL(USER, PERMS);
-      result = table.append(secondAppend);
-
-      // We should get "valuevalue""
-      assertEquals(1, result.size());
-      assertTrue(Bytes.equals(VALUE2, result.getValue(CF1_BYTES, CQ2)));
-
-      // Wait 4s to let the second append expire
-      Thread.sleep(4000);
-      get = new Get(ROW).addColumn(CF1_BYTES, CQ2);
-      result = table.get(get);
-
-      // The value should revert to "value"
-      assertEquals(1, result.size());
-      assertTrue(Bytes.equals(VALUE, result.getValue(CF1_BYTES, CQ2)));
-    }
-  }
-
-  private static boolean checkAclTag(byte[] acl, Cell cell) {
-    Iterator<Tag> iter = PrivateCellUtil.tagsIterator(cell);
-    while (iter.hasNext()) {
-      Tag tag = iter.next();
-      if (tag.getType() == TagType.ACL_TAG_TYPE) {
-        Tag temp = TagBuilderFactory.create().
-          setTagType(TagType.ACL_TAG_TYPE).setTagValue(acl).build();
-        return Tag.matchingValue(tag, temp);
-      }
-    }
-    return false;
-  }
-
   public static class ChangeCellWithDifferntColumnFamilyObserver
       implements RegionCoprocessor, RegionObserver {
     @Override
@@ -332,37 +232,4 @@ public class TestPostIncrementAndAppendBeforeWAL {
           .collect(Collectors.toList());
     }
   }
-
-  public static class ChangeCellWithACLTagObserver extends AccessController {
-    @Override
-    public Optional<RegionObserver> getRegionObserver() {
-      return Optional.of(this);
-    }
-
-    @Override
-    public List<Pair<Cell, Cell>> postIncrementBeforeWAL(
-        ObserverContext<RegionCoprocessorEnvironment> ctx, Mutation mutation,
-        List<Pair<Cell, Cell>> cellPairs) throws IOException {
-      List<Pair<Cell, Cell>> result = super.postIncrementBeforeWAL(ctx, mutation, cellPairs);
-      for (Pair<Cell, Cell> pair : result) {
-        if (mutation.getACL() != null && !checkAclTag(mutation.getACL(), pair.getSecond())) {
-          throw new DoNotRetryIOException("Unmatched ACL tag.");
-        }
-      }
-      return result;
-    }
-
-    @Override
-    public List<Pair<Cell, Cell>> postAppendBeforeWAL(
-        ObserverContext<RegionCoprocessorEnvironment> ctx, Mutation mutation,
-        List<Pair<Cell, Cell>> cellPairs) throws IOException {
-      List<Pair<Cell, Cell>> result = super.postAppendBeforeWAL(ctx, mutation, cellPairs);
-      for (Pair<Cell, Cell> pair : result) {
-        if (mutation.getACL() != null && !checkAclTag(mutation.getACL(), pair.getSecond())) {
-          throw new DoNotRetryIOException("Unmatched ACL tag.");
-        }
-      }
-      return result;
-    }
-  }
-}
+}
\ No newline at end of file