You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zg...@apache.org on 2018/10/26 09:55:26 UTC

hbase git commit: HBASE-21365 Throw exception when user put data with skip wal to a table which may be replicated

Repository: hbase
Updated Branches:
  refs/heads/master 3a7412d0a -> 23b7510ae


HBASE-21365 Throw exception when user put data with skip wal to a table which may be replicated


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

Branch: refs/heads/master
Commit: 23b7510ae22933f91d8c5b55b5bda4241cf86723
Parents: 3a7412d
Author: Guanghao Zhang <zg...@apache.org>
Authored: Wed Oct 24 14:27:31 2018 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Fri Oct 26 17:43:38 2018 +0800

----------------------------------------------------------------------
 .../InvalidMutationDurabilityException.java     |  44 ++++++
 .../hadoop/hbase/regionserver/HRegion.java      |  49 +++---
 .../TestInvalidMutationDurabilityException.java | 156 +++++++++++++++++++
 3 files changed, 226 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/23b7510a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/InvalidMutationDurabilityException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/InvalidMutationDurabilityException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/InvalidMutationDurabilityException.java
new file mode 100644
index 0000000..d694f37
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/InvalidMutationDurabilityException.java
@@ -0,0 +1,44 @@
+/**
+ *
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Thrown if Mutation's {@link Durability} is skip wal but table need replication.
+ */
+@InterfaceAudience.Public
+public class InvalidMutationDurabilityException extends DoNotRetryIOException {
+
+  /**
+   * default constructor
+   */
+  public InvalidMutationDurabilityException() {
+    super();
+  }
+
+  /**
+   * @param message exception message
+   */
+  public InvalidMutationDurabilityException(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/23b7510a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index fc33512..c10a824 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -2949,7 +2949,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         if(family == null) {
           throw new NoSuchColumnFamilyException("Empty family is invalid");
         }
-        checkFamily(family);
+        checkFamily(family, delete.getDurability());
       }
     }
   }
@@ -3561,7 +3561,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
     @Override
     public void checkAndPreparePut(Put p) throws IOException {
-      region.checkFamilies(p.getFamilyCellMap().keySet());
+      region.checkFamilies(p.getFamilyCellMap().keySet(), p.getDurability());
     }
 
     @Override
@@ -4454,14 +4454,30 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     store.add(cell, memstoreAccounting);
   }
 
-  /**
-   * Check the collection of families for validity.
-   * @param families
-   * @throws NoSuchColumnFamilyException
-   */
-  public void checkFamilies(Collection<byte[]> families) throws NoSuchColumnFamilyException {
+  private void checkFamilies(Collection<byte[]> families, Durability durability)
+      throws NoSuchColumnFamilyException, InvalidMutationDurabilityException {
     for (byte[] family : families) {
-      checkFamily(family);
+      checkFamily(family, durability);
+    }
+  }
+
+  private void checkFamily(final byte[] family, Durability durability)
+      throws NoSuchColumnFamilyException, InvalidMutationDurabilityException {
+    checkFamily(family);
+    if (durability.equals(Durability.SKIP_WAL)
+        && htableDescriptor.getColumnFamily(family).getScope()
+        != HConstants.REPLICATION_SCOPE_LOCAL) {
+      throw new InvalidMutationDurabilityException(
+          "Mutation's durability is SKIP_WAL but table's column family " + Bytes.toString(family)
+              + " need replication");
+    }
+  }
+
+  void checkFamily(final byte[] family) throws NoSuchColumnFamilyException {
+    if (!this.htableDescriptor.hasColumnFamily(family)) {
+      throw new NoSuchColumnFamilyException(
+          "Column family " + Bytes.toString(family) + " does not exist in region " + this
+              + " in table " + this.htableDescriptor);
     }
   }
 
@@ -7762,7 +7778,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     checkReadOnly();
     checkResources();
     checkRow(mutation.getRow(), op.toString());
-    checkFamilies(mutation.getFamilyCellMap().keySet());
+    checkFamilies(mutation.getFamilyCellMap().keySet(), mutation.getDurability());
     this.writeRequestsCount.increment();
     WriteEntry writeEntry = null;
     startRegionOperation(op);
@@ -8112,19 +8128,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return cells;
   }
 
-  //
-  // New HBASE-880 Helpers
-  //
-
-  void checkFamily(final byte [] family)
-  throws NoSuchColumnFamilyException {
-    if (!this.htableDescriptor.hasColumnFamily(family)) {
-      throw new NoSuchColumnFamilyException("Column family " +
-          Bytes.toString(family) + " does not exist in region " + this
-          + " in table " + this.htableDescriptor);
-    }
-  }
-
   public static final long FIXED_OVERHEAD = ClassSize.align(
       ClassSize.OBJECT +
       ClassSize.ARRAY +

http://git-wip-us.apache.org/repos/asf/hbase/blob/23b7510a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestInvalidMutationDurabilityException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestInvalidMutationDurabilityException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestInvalidMutationDurabilityException.java
new file mode 100644
index 0000000..e22f6dc
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestInvalidMutationDurabilityException.java
@@ -0,0 +1,156 @@
+/**
+ *
+ * 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.client;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.InvalidMutationDurabilityException;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestInvalidMutationDurabilityException {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestInvalidMutationDurabilityException.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static TableName TABLE_NOT_REPLICATE = TableName.valueOf("TableNotReplicate");
+
+  private static TableName TABLE_NEED_REPLICATE = TableName.valueOf("TableNeedReplicate");
+
+  private static byte[] CF = Bytes.toBytes("cf");
+
+  private static byte[] CQ = Bytes.toBytes("cq");
+
+  private static Table tableNotReplicate;
+
+  private static Table tableNeedReplicate;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.startMiniCluster();
+    UTIL.getAdmin().createTable(TableDescriptorBuilder.newBuilder(TABLE_NOT_REPLICATE)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(CF).build()).build());
+    UTIL.getAdmin().createTable(TableDescriptorBuilder.newBuilder(TABLE_NEED_REPLICATE)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(CF)
+            .setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build());
+    tableNotReplicate = UTIL.getConnection().getTable(TABLE_NOT_REPLICATE);
+    tableNeedReplicate = UTIL.getConnection().getTable(TABLE_NEED_REPLICATE);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.getAdmin().disableTable(TABLE_NOT_REPLICATE);
+    UTIL.getAdmin().disableTable(TABLE_NEED_REPLICATE);
+    UTIL.getAdmin().deleteTable(TABLE_NOT_REPLICATE);
+    UTIL.getAdmin().deleteTable(TABLE_NEED_REPLICATE);
+    UTIL.shutdownMiniCluster();
+  }
+
+  private Put newPutWithSkipWAL() {
+    Put put = new Put(Bytes.toBytes("row"));
+    put.addColumn(CF, CQ, Bytes.toBytes("value"));
+    put.setDurability(Durability.SKIP_WAL);
+    return put;
+  }
+
+  @Test
+  public void testPutToTableNotReplicate() throws Exception {
+    tableNotReplicate.put(newPutWithSkipWAL());
+  }
+
+  @Test(expected = InvalidMutationDurabilityException.class)
+  public void testPutToTableNeedReplicate() throws Exception {
+    tableNeedReplicate.put(newPutWithSkipWAL());
+  }
+
+  private Delete newDeleteWithSkipWAL() {
+    Delete delete = new Delete(Bytes.toBytes("row"));
+    delete.addColumn(CF, CQ);
+    delete.setDurability(Durability.SKIP_WAL);
+    return delete;
+  }
+
+  @Test
+  public void testDeleteToTableNotReplicate() throws Exception {
+    tableNotReplicate.delete(newDeleteWithSkipWAL());
+  }
+
+  @Test(expected = InvalidMutationDurabilityException.class)
+  public void testDeleteToTableNeedReplicate() throws Exception {
+    tableNeedReplicate.delete(newDeleteWithSkipWAL());
+  }
+
+  private Append newAppendWithSkipWAL() {
+    Append append = new Append(Bytes.toBytes("row"));
+    append.addColumn(CF, CQ, Bytes.toBytes("value"));
+    append.setDurability(Durability.SKIP_WAL);
+    return append;
+  }
+
+  @Test
+  public void testAppendToTableNotReplicate() throws Exception {
+    tableNotReplicate.append(newAppendWithSkipWAL());
+  }
+
+  @Test(expected = InvalidMutationDurabilityException.class)
+  public void testAppendToTableNeedReplicate() throws Exception {
+    tableNeedReplicate.append(newAppendWithSkipWAL());
+  }
+
+  private Increment newIncrementWithSkipWAL() {
+    Increment increment = new Increment(Bytes.toBytes("row"));
+    increment.addColumn(CF, CQ, 1);
+    increment.setDurability(Durability.SKIP_WAL);
+    return increment;
+  }
+
+  @Test
+  public void testIncrementToTableNotReplicate() throws Exception {
+    tableNotReplicate.increment(newIncrementWithSkipWAL());
+  }
+
+  @Test(expected = InvalidMutationDurabilityException.class)
+  public void testIncrementToTableNeedReplicate() throws Exception {
+    tableNeedReplicate.increment(newIncrementWithSkipWAL());
+  }
+
+  @Test
+  public void testCheckWithMutateToTableNotReplicate() throws Exception {
+    tableNotReplicate.checkAndMutate(Bytes.toBytes("row"), CF).qualifier(CQ).ifNotExists()
+        .thenPut(newPutWithSkipWAL());
+  }
+
+  @Test(expected = InvalidMutationDurabilityException.class)
+  public void testCheckWithMutateToTableNeedReplicate() throws Exception {
+    tableNeedReplicate.checkAndMutate(Bytes.toBytes("row"), CF).qualifier(CQ).ifNotExists()
+        .thenPut(newPutWithSkipWAL());
+  }
+}