You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2017/12/17 15:06:46 UTC

hbase git commit: HBASE-18775 Add a Read-Only property to turn off all writes for the cluster

Repository: hbase
Updated Branches:
  refs/heads/HBASE-18477 9e78c0eb8 -> 0e9c1dc22


HBASE-18775 Add a Read-Only property to turn off all writes for the cluster

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/HBASE-18477
Commit: 0e9c1dc2235c31813aaeae27266711d8b280be0a
Parents: 9e78c0e
Author: Zach York <zy...@amazon.com>
Authored: Wed May 3 17:09:29 2017 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Sun Dec 17 07:06:31 2017 -0800

----------------------------------------------------------------------
 .../TestReadReplicaClustersTableNameUtil.java   |   3 +
 .../org/apache/hadoop/hbase/master/HMaster.java |  28 ++++
 .../hadoop/hbase/regionserver/HRegion.java      |  10 +-
 .../hbase/regionserver/HRegionServer.java       |   5 +
 .../hadoop/hbase/master/TestReadOnly.java       | 146 +++++++++++++++++++
 .../hadoop/hbase/regionserver/TestHRegion.java  |  65 ++++++++-
 6 files changed, 250 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0e9c1dc2/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestReadReplicaClustersTableNameUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestReadReplicaClustersTableNameUtil.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestReadReplicaClustersTableNameUtil.java
index bde0202..2c48f6f 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestReadReplicaClustersTableNameUtil.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestReadReplicaClustersTableNameUtil.java
@@ -18,11 +18,14 @@
 package org.apache.hadoop.hbase.util;
 
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+@Category(MediumTests.class)
 public class TestReadReplicaClustersTableNameUtil {
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/0e9c1dc2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index ad304ae..112f53d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -373,6 +373,8 @@ public class HMaster extends HRegionServer implements MasterServices {
   // the key is table name, the value is the number of compactions in that table.
   private Map<TableName, AtomicInteger> mobCompactionStates = Maps.newConcurrentMap();
 
+  private final boolean readOnly;
+
   MasterCoprocessorHost cpHost;
 
   private final boolean preLoadTableDescriptors;
@@ -535,6 +537,8 @@ public class HMaster extends HRegionServer implements MasterServices {
     } else {
       activeMasterManager = null;
     }
+
+    this.readOnly = conf.getBoolean(READ_ONLY_ENABLED_KEY, false);
   }
 
   // Main run loop. Calls through to the regionserver run loop.
@@ -1733,6 +1737,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       final long nonceGroup,
       final long nonce) throws IOException {
     checkInitialized();
+    checkReadOnly();
 
     String namespace = tableDescriptor.getTableName().getNamespaceAsString();
     this.clusterSchemaService.getNamespace(namespace);
@@ -2100,6 +2105,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       final long nonceGroup,
       final long nonce) throws IOException {
     checkInitialized();
+    checkReadOnly();
 
     return MasterProcedureUtil.submitProcedure(
         new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
@@ -2132,6 +2138,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       final long nonceGroup,
       final long nonce) throws IOException {
     checkInitialized();
+    checkReadOnly();
 
     return MasterProcedureUtil.submitProcedure(
         new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
@@ -2163,6 +2170,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       final long nonce)
       throws IOException {
     checkInitialized();
+    checkReadOnly();
     checkCompression(columnDescriptor);
     checkEncryption(conf, columnDescriptor);
     checkReplicationScope(columnDescriptor);
@@ -2199,6 +2207,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       final long nonce)
       throws IOException {
     checkInitialized();
+    checkReadOnly();
     checkCompression(descriptor);
     checkEncryption(conf, descriptor);
     checkReplicationScope(descriptor);
@@ -2238,6 +2247,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       final long nonce)
       throws IOException {
     checkInitialized();
+    checkReadOnly();
 
     return MasterProcedureUtil.submitProcedure(
         new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
@@ -2269,6 +2279,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   public long enableTable(final TableName tableName, final long nonceGroup, final long nonce)
       throws IOException {
     checkInitialized();
+    checkReadOnly();
 
     return MasterProcedureUtil.submitProcedure(
         new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
@@ -2322,6 +2333,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   public long disableTable(final TableName tableName, final long nonceGroup, final long nonce)
       throws IOException {
     checkInitialized();
+    checkReadOnly();
 
     return MasterProcedureUtil.submitProcedure(
         new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
@@ -2391,6 +2403,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   public long modifyTable(final TableName tableName, final TableDescriptor descriptor,
       final long nonceGroup, final long nonce) throws IOException {
     checkInitialized();
+    checkReadOnly();
     sanityCheckTableDescriptor(descriptor);
 
     return MasterProcedureUtil.submitProcedure(
@@ -2444,6 +2457,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   @Override
   public void checkTableModifiable(final TableName tableName)
       throws IOException, TableNotFoundException, TableNotDisabledException {
+    checkReadOnly();
     if (isCatalogTable(tableName)) {
       throw new IOException("Can't modify catalog tables");
     }
@@ -2901,11 +2915,16 @@ public class HMaster extends HRegionServer implements MasterServices {
   long createNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
       final long nonce) throws IOException {
     checkInitialized();
+    if (!namespaceDescriptor.getName().equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR) &&
+        !namespaceDescriptor.getName().equals(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR)) {
+      checkReadOnly();
+    }
 
     TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
 
     return MasterProcedureUtil.submitProcedure(
         new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
+
       @Override
       protected void run() throws IOException {
         if (getMaster().getMasterCoprocessorHost().preCreateNamespace(namespaceDescriptor)) {
@@ -2935,6 +2954,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   long modifyNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
       final long nonce) throws IOException {
     checkInitialized();
+    checkReadOnly();
 
     TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
 
@@ -2969,6 +2989,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   long deleteNamespace(final String name, final long nonceGroup, final long nonce)
       throws IOException {
     checkInitialized();
+    checkReadOnly();
 
     return MasterProcedureUtil.submitProcedure(
         new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
@@ -3314,6 +3335,13 @@ public class HMaster extends HRegionServer implements MasterServices {
         .getDefaultLoadBalancerClass().getName());
   }
 
+  private void checkReadOnly() throws DoNotRetryIOException {
+    if (readOnly) {
+      throw new AccessDeniedException("No modification is allowed when " +
+          READ_ONLY_ENABLED_KEY + " is set to true");
+    }
+  }
+
   /**
    * @return RegionNormalizerTracker instance
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/0e9c1dc2/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 823ef39..47308e9 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
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
+import static org.apache.hadoop.hbase.regionserver.HRegionServer.READ_ONLY_ENABLED_KEY;
 import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
@@ -654,6 +655,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   // whether to unassign region if we hit FNFE
   private final RegionUnassigner regionUnassigner;
+
+  // Whether the user has specified this region as read-only via READ_ONLY_ENABLED_KEY
+  private boolean readOnly;
+
   /**
    * HRegion constructor. This constructor should only be used for testing and
    * extensions.  Instances of HRegion should be instantiated with the
@@ -821,6 +826,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     } else {
       this.regionUnassigner = null;
     }
+
+    this.readOnly = !getRegionInfo().isSystemTable() &&
+        conf.getBoolean(READ_ONLY_ENABLED_KEY, false);
   }
 
   void setHTableSpecificConf() {
@@ -922,7 +930,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
     this.lastReplayedOpenRegionSeqId = maxSeqId;
 
-    this.writestate.setReadOnly(ServerRegionReplicaUtil.isReadOnly(this));
+    this.writestate.setReadOnly(readOnly || ServerRegionReplicaUtil.isReadOnly(this));
     this.writestate.flushRequested = false;
     this.writestate.compacting.set(0);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/0e9c1dc2/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 9b7eef3..d795642 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -459,6 +459,11 @@ public class HRegionServer extends HasThread implements
     "hbase.regionserver.hostname.disable.master.reversedns";
 
   /**
+   * boolean value that disables modifications when enabled.
+   */
+  public static final String READ_ONLY_ENABLED_KEY = "hbase.readonly";
+
+  /**
    * This servers startcode.
    */
   protected final long startcode;

http://git-wip-us.apache.org/repos/asf/hbase/blob/0e9c1dc2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestReadOnly.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestReadOnly.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestReadOnly.java
new file mode 100644
index 0000000..e238a44
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestReadOnly.java
@@ -0,0 +1,146 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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.master;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.hbase.regionserver.HRegionServer.READ_ONLY_ENABLED_KEY;
+
+@Category(MediumTests.class)
+public class TestReadOnly {
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final ColumnFamilyDescriptor CFD = ColumnFamilyDescriptorBuilder.of("f1");
+  private static final NamespaceDescriptor NSD = NamespaceDescriptor.DEFAULT_NAMESPACE;
+  private static final NamespaceDescriptor NON_DEFAULT_NSD = NamespaceDescriptor.create("newNS").build();
+  private static final HTableDescriptor NON_DEFAULT_TABLE = new HTableDescriptor(
+      TableName.valueOf(NON_DEFAULT_NSD.getName(), "TestReadOnly"));
+
+  private static HMaster master;
+  private static Configuration config;
+  private static TableName tableName;
+  private static TableDescriptor tableDescriptor;
+
+  @ClassRule
+  public static TestName name = new TestName();
+
+  @BeforeClass
+  public static void beforeAllTests() throws Exception {
+    config = TEST_UTIL.getConfiguration();
+    config.setBoolean(READ_ONLY_ENABLED_KEY, true);
+    TEST_UTIL.startMiniCluster();
+    master = TEST_UTIL.getHBaseCluster().getMaster();
+    tableName = TableName.valueOf(name.getClass().getName());
+    tableDescriptor = TableDescriptorBuilder.newBuilder(tableName).build();
+  }
+
+  @AfterClass
+  public static void afterAllTests() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test (expected = AccessDeniedException.class)
+  public void testCreateTableReadOnly() throws IOException {
+    master.createTable(tableDescriptor, null, 0, 0);
+  }
+
+  @Test (expected = AccessDeniedException.class)
+  public void testCreateNonDefaultNSTableReadOnly() throws IOException {
+    master.createTable(NON_DEFAULT_TABLE, null, 0, 0);
+  }
+
+  @Test (expected = AccessDeniedException.class)
+  public void testDeleteTableReadOnly() throws IOException {
+    master.deleteTable(tableName, 0, 0);
+  }
+
+  @Test (expected = AccessDeniedException.class)
+  public void testTruncateTableReadOnly() throws IOException {
+    master.truncateTable(tableName, false, 0, 0);
+  }
+
+  @Test (expected = AccessDeniedException.class)
+  public void testAddColumnReadOnly() throws IOException {
+    master.addColumn(tableName, CFD, 0, 0);
+  }
+
+  @Test (expected = AccessDeniedException.class)
+  public void testModifyColumnReadOnly() throws IOException {
+    master.modifyColumn(tableName, CFD, 0, 0);
+  }
+
+  @Test (expected = AccessDeniedException.class)
+  public void testDeleteColumnReadOnly() throws IOException {
+    master.deleteColumn(tableName, CFD.getName(), 0, 0);
+  }
+
+  @Test (expected = AccessDeniedException.class)
+  public void testEnableTableReadOnly() throws IOException {
+    master.enableTable(tableName, 0, 0);
+  }
+
+  @Test (expected = AccessDeniedException.class)
+  public void testDisableTableReadOnly() throws IOException {
+    master.disableTable(tableName, 0, 0);
+  }
+
+  @Test (expected = AccessDeniedException.class)
+  public void testModifyTableReadOnly() throws IOException {
+    master.modifyTable(tableName, tableDescriptor, 0, 0);
+  }
+
+  @Test (expected = AccessDeniedException.class)
+  public void testCheckTableModifiableReadOnly() throws IOException {
+    master.checkTableModifiable(tableName);
+  }
+
+  @Test (expected = AccessDeniedException.class)
+  public void testCreateNamespaceReadOnly() throws IOException {
+    master.createNamespace(NON_DEFAULT_NSD, 0, 0);
+  }
+
+  @Test (expected = AccessDeniedException.class)
+  public void testModifyNamespaceReadOnly() throws IOException {
+    master.modifyNamespace(NSD, 0, 0);
+  }
+
+  @Test (expected = AccessDeniedException.class)
+  public void testDeleteNamespaceReadOnly() throws IOException {
+    master.deleteNamespace(NSD.getName(), 0, 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0e9c1dc2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index d567513..d1de1e5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.HBaseTestingUtility.COLUMNS;
 import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
 import static org.apache.hadoop.hbase.HBaseTestingUtility.fam2;
 import static org.apache.hadoop.hbase.HBaseTestingUtility.fam3;
+import static org.apache.hadoop.hbase.regionserver.HRegionServer.READ_ONLY_ENABLED_KEY;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -40,6 +41,7 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 
@@ -199,6 +201,7 @@ public class TestHRegion {
   public static final TestRule timeout =
       CategoryBasedTimeout.forClass(TestHRegion.class);
 
+  private static final byte[] READ_ONLY_TABLE = Bytes.toBytes("readOnlyTable");
   private static final String COLUMN_FAMILY = "MyCF";
   private static final byte [] COLUMN_FAMILY_BYTES = Bytes.toBytes(COLUMN_FAMILY);
 
@@ -1384,7 +1387,6 @@ public class TestHRegion {
 
   @Test
   public void testAppendWithReadOnlyTable() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
     this.region = initHRegion(tableName, method, CONF, true, Bytes.toBytes("somefamily"));
     boolean exceptionCaught = false;
     Append append = new Append(Bytes.toBytes("somerow"));
@@ -1393,18 +1395,17 @@ public class TestHRegion {
         Bytes.toBytes("somevalue"));
     try {
       region.append(append);
-    } catch (IOException e) {
+    } catch (DoNotRetryIOException e) {
       exceptionCaught = true;
     } finally {
       HBaseTestingUtility.closeRegionAndWAL(this.region);
       this.region = null;
     }
-    assertTrue(exceptionCaught == true);
+    assertTrue(exceptionCaught);
   }
 
   @Test
   public void testIncrWithReadOnlyTable() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
     this.region = initHRegion(tableName, method, CONF, true, Bytes.toBytes("somefamily"));
     boolean exceptionCaught = false;
     Increment inc = new Increment(Bytes.toBytes("somerow"));
@@ -1412,13 +1413,65 @@ public class TestHRegion {
     inc.addColumn(Bytes.toBytes("somefamily"), Bytes.toBytes("somequalifier"), 1L);
     try {
       region.increment(inc);
-    } catch (IOException e) {
+    } catch (DoNotRetryIOException e) {
+      exceptionCaught = true;
+    } finally {
+      HBaseTestingUtility.closeRegionAndWAL(this.region);
+      this.region = null;
+    }
+    assertTrue(exceptionCaught);
+  }
+
+  @Test
+  public void testPutWithReadOnlyTable() throws Exception {
+    this.region = initHRegion(tableName, method, CONF, true, Bytes.toBytes("somefamily"));
+    boolean exceptionCaught = false;
+    Put put = new Put(Bytes.toBytes("somerow"));
+    put.setDurability(Durability.SKIP_WAL);
+    put.addColumn(Bytes.toBytes("somefamily"), Bytes.toBytes("somequalifier"), Bytes.toBytes("value"));
+    try {
+      region.put(put);
+    } catch (DoNotRetryIOException e) {
       exceptionCaught = true;
     } finally {
       HBaseTestingUtility.closeRegionAndWAL(this.region);
       this.region = null;
     }
-    assertTrue(exceptionCaught == true);
+    assertTrue(exceptionCaught);
+  }
+
+  @Test
+  public void testDeleteWithReadOnlyTable() throws Exception {
+    this.region = initHRegion(tableName, method, CONF, true, Bytes.toBytes("somefamily"));
+    boolean exceptionCaught = false;
+    Delete delete = new Delete(Bytes.toBytes("somerow"));
+    delete.setDurability(Durability.SKIP_WAL);
+    delete.addColumn(Bytes.toBytes("somefamily"), Bytes.toBytes("somequalifier"), 1L);
+    try {
+      region.delete(delete);
+    } catch (DoNotRetryIOException e) {
+      exceptionCaught = true;
+    } finally {
+      HBaseTestingUtility.closeRegionAndWAL(this.region);
+      this.region = null;
+    }
+    assertTrue(exceptionCaught);
+  }
+
+  @Test
+  public void testGlobalReadOnly() throws IOException {
+    TableName writableTable = TableName.valueOf("writableTable");
+    this.region = initHRegion(writableTable, method, CONF, false, Bytes.toBytes("somefamily"));
+    assertFalse(this.region.isReadOnly());
+
+    CONF.setBoolean(READ_ONLY_ENABLED_KEY, true);
+    //Don't set the table attribute to ReadOnly
+    this.region = initHRegion(writableTable, method, CONF, false, Bytes.toBytes("somefamily"));
+    assertTrue(this.region.isReadOnly());
+
+    this.region = initHRegion(TableName.META_TABLE_NAME, null, CONF, Bytes.toBytes("somefamily"));
+    assertFalse(this.region.isReadOnly());
+    assertTrue(this.region.getTableDescriptor().getTableName().isMeta());
   }
 
   private void deleteColumns(HRegion r, String value, String keyPrefix) throws IOException {