You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2018/01/27 06:39:48 UTC

[06/14] hbase git commit: HBASE-19841 LocalHTU to not enforce stream capabilities

HBASE-19841 LocalHTU to not enforce stream capabilities


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

Branch: refs/heads/HBASE-19064
Commit: 9ea152d2351cfb2607a5a861358470969ce24548
Parents: 94a1c05
Author: Mike Drob <md...@apache.org>
Authored: Wed Jan 24 00:42:32 2018 -0600
Committer: Mike Drob <md...@apache.org>
Committed: Fri Jan 26 10:50:16 2018 -0600

----------------------------------------------------------------------
 .../apache/hadoop/hbase/util/CommonFSUtils.java |  8 +++++++-
 .../org/apache/hadoop/hbase/HBaseTestCase.java  |  1 +
 .../hadoop/hbase/HBaseTestingUtility.java       | 10 ++++++++++
 .../hbase/client/TestIntraRowPagination.java    |  2 +-
 .../coprocessor/TestRegionObserverStacking.java |  7 ++-----
 .../hbase/filter/TestColumnPrefixFilter.java    |  3 +--
 .../hbase/filter/TestDependentColumnFilter.java |  2 +-
 .../apache/hadoop/hbase/filter/TestFilter.java  |  2 +-
 .../hbase/filter/TestFilterFromRegionSide.java  |  2 +-
 .../filter/TestInvocationRecordFilter.java      |  2 +-
 .../filter/TestMultipleColumnPrefixFilter.java  |  3 +--
 .../hbase/io/asyncfs/TestLocalAsyncOutput.java  |  3 ++-
 .../encoding/TestSeekBeforeWithReverseScan.java |  2 +-
 .../hbase/regionserver/TestBlocksScanned.java   |  2 +-
 .../regionserver/TestCompactingMemStore.java    |  6 ++++--
 .../regionserver/TestCompactionPolicy.java      |  2 +-
 .../hbase/regionserver/TestDefaultMemStore.java | 20 ++++++++------------
 .../hbase/regionserver/TestHRegionInfo.java     |  2 +-
 .../hadoop/hbase/regionserver/TestHStore.java   |  6 +++---
 .../regionserver/TestRegionInfoBuilder.java     |  2 +-
 .../regionserver/TestResettingCounters.java     |  3 +--
 .../TestStoreFileRefresherChore.java            |  2 +-
 .../TestWALMonotonicallyIncreasingSeqId.java    |  2 +-
 .../wal/TestWALActionsListener.java             |  2 +-
 .../apache/hadoop/hbase/wal/TestWALMethods.java |  2 +-
 hbase-server/src/test/resources/hbase-site.xml  |  9 ---------
 26 files changed, 54 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java
index 9efec07..bb98c40 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java
@@ -394,7 +394,13 @@ public abstract class CommonFSUtils {
 
   public static FileSystem getWALFileSystem(final Configuration c) throws IOException {
     Path p = getWALRootDir(c);
-    return p.getFileSystem(c);
+    FileSystem fs = p.getFileSystem(c);
+    // Need to copy this to the new filesystem we are returning in case it is localFS
+    String enforceStreamCapabilities = c.get(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE);
+    if (enforceStreamCapabilities != null) {
+      fs.getConf().set(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE, enforceStreamCapabilities);
+    }
+    return fs;
   }
 
   private static boolean isValidWALRootDir(Path walDir, final Configuration c) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
index 5f2ffb2..c9e6f7e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
@@ -113,6 +113,7 @@ public abstract class HBaseTestCase extends TestCase {
         if (fs.exists(testDir)) {
           fs.delete(testDir, true);
         }
+        conf.setBoolean("hbase.unsafe.stream.capability.enforce",false);
       } else {
         testDir = FSUtils.getRootDir(conf);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 75abd5e..97dbdf5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase;
 
+import static org.apache.hadoop.hbase.util.CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -298,10 +299,18 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
   public static final Collection<Object[]> BLOOM_AND_COMPRESSION_COMBINATIONS =
       bloomAndCompressionCombinations();
 
+  /**
+   * Creates a new HBaseTestingUtility for starting a mini-cluster.
+   * If you are not using the startMini* methods, consider {@link #createLocalHTU()} instead.
+   */
   public HBaseTestingUtility() {
     this(HBaseConfiguration.create());
   }
 
+  /**
+   * Creates a new HBaseTestingUtility for starting a mini-cluster.
+   * If you are not using the startMini* methods, consider {@link #createLocalHTU()} instead.
+   */
   public HBaseTestingUtility(Configuration conf) {
     super(conf);
 
@@ -347,6 +356,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     htu.getConfiguration().set("fs.defaultFS","file:///");
     htu.getConfiguration().set(HConstants.HBASE_DIR, "file://" + dataTestDir);
     LOG.debug("Setting " + HConstants.HBASE_DIR + " to " + dataTestDir);
+    htu.getConfiguration().setBoolean(UNSAFE_STREAM_CAPABILITY_ENFORCE,false);
     htu.localMode = true;
     return htu;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java
index 43a2e77..ff11b4c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java
@@ -40,7 +40,7 @@ import org.junit.experimental.categories.Category;
 @Category({SmallTests.class, ClientTests.class})
 public class TestIntraRowPagination {
 
-  private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
 
   /**
    * Test from client side for scan with maxResultPerCF set

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
index 8470116..177af69 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
@@ -27,12 +27,10 @@ import junit.framework.TestCase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Coprocessor;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MockRegionServerServices;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Durability;
@@ -50,8 +48,7 @@ import org.mockito.Mockito;
 
 @Category({CoprocessorTests.class, SmallTests.class})
 public class TestRegionObserverStacking extends TestCase {
-  private static HBaseTestingUtility TEST_UTIL
-    = new HBaseTestingUtility();
+  private static HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
   static final Path DIR = TEST_UTIL.getDataTestDir();
 
   public static class ObserverA implements RegionCoprocessor, RegionObserver {
@@ -143,7 +140,7 @@ public class TestRegionObserverStacking extends TestCase {
     byte[] A = Bytes.toBytes("A");
     byte[][] FAMILIES = new byte[][] { A } ;
 
-    Configuration conf = HBaseConfiguration.create();
+    Configuration conf = TEST_UTIL.getConfiguration();
     HRegion region = initHRegion(TABLE, getClass().getName(),
       conf, FAMILIES);
     RegionCoprocessorHost h = region.getCoprocessorHost();

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
index 9b71d45..55abd1d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
@@ -51,8 +51,7 @@ import org.junit.rules.TestName;
 @Category({FilterTests.class, SmallTests.class})
 public class TestColumnPrefixFilter {
 
-  private final static HBaseTestingUtility TEST_UTIL = new
-      HBaseTestingUtility();
+  private final static HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
 
   @Rule
   public TestName name = new TestName();

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java
index ae90c63..c560f20 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java
@@ -71,7 +71,7 @@ public class TestDependentColumnFilter {
     Bytes.toBytes("bad1"), Bytes.toBytes("bad2"), Bytes.toBytes("bad3")
   };
   private static final byte[] MATCH_VAL = Bytes.toBytes("match");
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private final static HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
 
   List<KeyValue> testVals;
   private HRegion region;

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
index ec11ce0..eb7adf8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
@@ -71,7 +71,7 @@ import org.slf4j.LoggerFactory;
 public class TestFilter {
   private final static Logger LOG = LoggerFactory.getLogger(TestFilter.class);
   private HRegion region;
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private final static HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
 
   @Rule
   public TestName name = new TestName();

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java
index ad5ee99..40b3686 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java
@@ -50,7 +50,7 @@ import java.util.List;
 @Category(SmallTests.class)
 public class TestFilterFromRegionSide {
 
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private final static HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
   private static HRegion REGION;
 
   private static TableName TABLE_NAME = TableName.valueOf("TestFilterFromRegionSide");

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java
index 768ab7a..5cf168f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java
@@ -59,7 +59,7 @@ public class TestInvocationRecordFilter {
   private static final String QUALIFIER_PREFIX = "qualifier";
   private static final String VALUE_PREFIX = "value";
 
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private final static HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
   private HRegion region;
 
   @Before

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java
index d30cb37..3af7a38 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultipleColumnPrefixFilter.java
@@ -51,8 +51,7 @@ import org.junit.rules.TestName;
 @Category({FilterTests.class, SmallTests.class})
 public class TestMultipleColumnPrefixFilter {
 
-  private final static HBaseTestingUtility TEST_UTIL = new
-      HBaseTestingUtility();
+  private final static HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
 
   @Rule
   public TestName name = new TestName();

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java
index 6909027..833311f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java
@@ -23,6 +23,7 @@ import java.util.concurrent.ExecutionException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -42,7 +43,7 @@ public class TestLocalAsyncOutput {
 
   private static Class<? extends Channel> CHANNEL_CLASS = NioSocketChannel.class;
 
-  private static final HBaseCommonTestingUtility TEST_UTIL = new HBaseCommonTestingUtility();
+  private static final HBaseCommonTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
 
   @AfterClass
   public static void tearDownAfterClass() throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java
index d304e74..91b4082 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java
@@ -46,7 +46,7 @@ import org.junit.experimental.categories.Category;
 
 @Category({ IOTests.class, SmallTests.class })
 public class TestSeekBeforeWithReverseScan {
-  private final HBaseTestingUtility testUtil = new HBaseTestingUtility();
+  private final HBaseTestingUtility testUtil = HBaseTestingUtility.createLocalHTU();
 
   private HRegion region;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java
index a0babe8..1d1e585 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java
@@ -55,7 +55,7 @@ public class TestBlocksScanned extends HBaseTestCase {
   public void setUp() throws Exception {
     super.setUp();
 
-    TEST_UTIL = new HBaseTestingUtility();
+    TEST_UTIL = HBaseTestingUtility.createLocalHTU();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
index 87e4aff..1290cb1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
@@ -95,11 +95,13 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
 
   protected void compactingSetUp() throws Exception {
     super.internalSetUp();
-    Configuration conf = new Configuration();
+
+    HBaseTestingUtility hbaseUtility = HBaseTestingUtility.createLocalHTU();
+    Configuration conf = hbaseUtility.getConfiguration();
+
     conf.setBoolean(MemStoreLAB.USEMSLAB_KEY, true);
     conf.setFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
     conf.setInt(HRegion.MEMSTORE_PERIODIC_FLUSH_INTERVAL, 1000);
-    HBaseTestingUtility hbaseUtility = HBaseTestingUtility.createLocalHTU(conf);
     HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("foobar"));
     htd.addFamily(hcd);

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
index ca4b227..d8102df 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
@@ -47,7 +47,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 public class TestCompactionPolicy {
   private final static Logger LOG = LoggerFactory.getLogger(TestCompactionPolicy.class);
-  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected final static HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
 
   protected Configuration conf;
   protected HStore store;

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
index 53e04e0..2369dc4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeepDeletedCells;
@@ -92,6 +91,9 @@ public class TestDefaultMemStore {
     return this.name.getMethodName();
   }
 
+  private HBaseTestingUtility util;
+  private Configuration conf;
+
   @Before
   public void setUp() throws Exception {
     internalSetUp();
@@ -107,6 +109,8 @@ public class TestDefaultMemStore {
   }
 
   protected void internalSetUp() throws Exception {
+    this.util = HBaseTestingUtility.createLocalHTU();
+    this.conf = util.getConfiguration();
     this.mvcc = new MultiVersionConcurrencyControl();
   }
 
@@ -162,7 +166,6 @@ public class TestDefaultMemStore {
     List<KeyValueScanner> memstorescanners = this.memstore.getScanners(0);
     Scan scan = new Scan();
     List<Cell> result = new ArrayList<>();
-    Configuration conf = HBaseConfiguration.create();
     ScanInfo scanInfo = new ScanInfo(conf, null, 0, 1, HConstants.LATEST_TIMESTAMP,
         KeepDeletedCells.FALSE, HConstants.DEFAULT_BLOCKSIZE, 0, this.memstore.getComparator(), false);
     int count = 0;
@@ -536,7 +539,7 @@ public class TestDefaultMemStore {
 
   @Test
   public void testMultipleVersionsSimple() throws Exception {
-    DefaultMemStore m = new DefaultMemStore(new Configuration(), CellComparatorImpl.COMPARATOR);
+    DefaultMemStore m = new DefaultMemStore(conf, CellComparatorImpl.COMPARATOR);
     byte [] row = Bytes.toBytes("testRow");
     byte [] family = Bytes.toBytes("testFamily");
     byte [] qf = Bytes.toBytes("testQualifier");
@@ -582,7 +585,6 @@ public class TestDefaultMemStore {
       }
     }
     //starting from each row, validate results should contain the starting row
-    Configuration conf = HBaseConfiguration.create();
     for (int startRowId = 0; startRowId < ROW_COUNT; startRowId++) {
       ScanInfo scanInfo =
           new ScanInfo(conf, FAMILY, 0, 1, Integer.MAX_VALUE, KeepDeletedCells.FALSE,
@@ -818,7 +820,6 @@ public class TestDefaultMemStore {
    */
   @Test
   public void testUpsertMemstoreSize() throws Exception {
-    Configuration conf = HBaseConfiguration.create();
     memstore = new DefaultMemStore(conf, CellComparatorImpl.COMPARATOR);
     MemStoreSize oldSize = memstore.size();
 
@@ -901,7 +902,6 @@ public class TestDefaultMemStore {
    */
   @Test
   public void testShouldFlush() throws Exception {
-    Configuration conf = new Configuration();
     conf.setInt(HRegion.MEMSTORE_PERIODIC_FLUSH_INTERVAL, 1000);
     checkShouldFlush(conf, true);
     // test disable flush
@@ -913,10 +913,8 @@ public class TestDefaultMemStore {
     try {
       EnvironmentEdgeForMemstoreTest edge = new EnvironmentEdgeForMemstoreTest();
       EnvironmentEdgeManager.injectEdge(edge);
-      HBaseTestingUtility hbaseUtility = HBaseTestingUtility.createLocalHTU(conf);
       String cf = "foo";
-      HRegion region =
-          hbaseUtility.createTestRegion("foobar", ColumnFamilyDescriptorBuilder.of(cf));
+      HRegion region = util.createTestRegion("foobar", ColumnFamilyDescriptorBuilder.of(cf));
 
       edge.setCurrentTimeMillis(1234);
       Put p = new Put(Bytes.toBytes("r"));
@@ -937,10 +935,8 @@ public class TestDefaultMemStore {
     // write an edit in the META and ensure the shouldFlush (that the periodic memstore
     // flusher invokes) returns true after SYSTEM_CACHE_FLUSH_INTERVAL (even though
     // the MEMSTORE_PERIODIC_FLUSH_INTERVAL is set to a higher value)
-    Configuration conf = new Configuration();
     conf.setInt(HRegion.MEMSTORE_PERIODIC_FLUSH_INTERVAL, HRegion.SYSTEM_CACHE_FLUSH_INTERVAL * 10);
-    HBaseTestingUtility hbaseUtility = HBaseTestingUtility.createLocalHTU(conf);
-    Path testDir = hbaseUtility.getDataTestDir();
+    Path testDir = util.getDataTestDir();
     EnvironmentEdgeForMemstoreTest edge = new EnvironmentEdgeForMemstoreTest();
     EnvironmentEdgeManager.injectEdge(edge);
     edge.setCurrentTimeMillis(1234);

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
index ae54505..87f321c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
@@ -65,7 +65,7 @@ public class TestHRegionInfo {
 
   @Test
   public void testReadAndWriteHRegionInfoFile() throws IOException, InterruptedException {
-    HBaseTestingUtility htu = new HBaseTestingUtility();
+    HBaseTestingUtility htu = HBaseTestingUtility.createLocalHTU();
     HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
     Path basedir = htu.getDataTestDir();
     // Create a region.  That'll write the .regioninfo file.

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
index 288333b..3bde43f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
@@ -146,7 +146,7 @@ public class TestHStore {
   long id = System.currentTimeMillis();
   Get get = new Get(row);
 
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
   private static final String DIR = TEST_UTIL.getDataTestDir("TestStore").toString();
 
 
@@ -234,7 +234,7 @@ public class TestHStore {
   public void testFlushSizeSizing() throws Exception {
     LOG.info("Setting up a faulty file system that cannot write in " +
       this.name.getMethodName());
-    final Configuration conf = HBaseConfiguration.create();
+    final Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
     // Only retry once.
     conf.setInt("hbase.hstore.flush.retries.number", 1);
     User user = User.createUserForTesting(conf, this.name.getMethodName(),
@@ -661,7 +661,7 @@ public class TestHStore {
   public void testHandleErrorsInFlush() throws Exception {
     LOG.info("Setting up a faulty file system that cannot write");
 
-    final Configuration conf = HBaseConfiguration.create();
+    final Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
     User user = User.createUserForTesting(conf,
         "testhandleerrorsinflush", new String[]{"foo"});
     // Inject our faulty LocalFileSystem

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java
index ab4f890..17d2d2c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java
@@ -91,7 +91,7 @@ public class TestRegionInfoBuilder {
 
   @Test
   public void testReadAndWriteRegionInfoFile() throws IOException, InterruptedException {
-    HBaseTestingUtility htu = new HBaseTestingUtility();
+    HBaseTestingUtility htu = HBaseTestingUtility.createLocalHTU();
     RegionInfo ri = RegionInfoBuilder.FIRST_META_REGIONINFO;
     Path basedir = htu.getDataTestDir();
     // Create a region.  That'll write the .regioninfo file.

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java
index 570422e..114bf85 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java
@@ -45,8 +45,7 @@ public class TestResettingCounters {
 
   @Test
   public void testResettingCounters() throws Exception {
-
-    HBaseTestingUtility htu = new HBaseTestingUtility();
+    HBaseTestingUtility htu = HBaseTestingUtility.createLocalHTU();
     Configuration conf = htu.getConfiguration();
     FileSystem fs = FileSystem.get(conf);
     byte [] table = Bytes.toBytes(name.getMethodName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
index 35c744f..cb55986 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
@@ -70,7 +70,7 @@ public class TestStoreFileRefresherChore {
 
   @Before
   public void setUp() throws IOException {
-    TEST_UTIL = new HBaseTestingUtility();
+    TEST_UTIL = HBaseTestingUtility.createLocalHTU();
     testDir = TEST_UTIL.getDataTestDir("TestStoreFileRefresherChore");
     FSUtils.setRootDir(TEST_UTIL.getConfiguration(), testDir);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
index a5148b3..09bc4eb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
@@ -77,7 +77,7 @@ import org.slf4j.LoggerFactory;
 @Category({ RegionServerTests.class, SmallTests.class })
 public class TestWALMonotonicallyIncreasingSeqId {
   private final Logger LOG = LoggerFactory.getLogger(getClass());
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private final static HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
   private static Path testDir = TEST_UTIL.getDataTestDir("TestWALMonotonicallyIncreasingSeqId");
   private WALFactory wals;
   private FileSystem fileSystem;

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
index 380ea74..1fe9c75 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
@@ -55,7 +55,7 @@ import org.junit.experimental.categories.Category;
 public class TestWALActionsListener {
 
   private final static HBaseTestingUtility TEST_UTIL =
-      new HBaseTestingUtility();
+      HBaseTestingUtility.createLocalHTU();
 
   private final static byte[] SOME_BYTES =  Bytes.toBytes("t");
   private static Configuration conf;

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java
index 5364504..4a8656f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java
@@ -56,7 +56,7 @@ public class TestWALMethods {
   private static final TableName TEST_TABLE =
       TableName.valueOf("test_table");
 
-  private final HBaseTestingUtility util = new HBaseTestingUtility();
+  private final HBaseTestingUtility util = HBaseTestingUtility.createLocalHTU();
 
   @Test
   public void testServerNameFromWAL() throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ea152d2/hbase-server/src/test/resources/hbase-site.xml
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/resources/hbase-site.xml b/hbase-server/src/test/resources/hbase-site.xml
index dbdf776..64a1964 100644
--- a/hbase-server/src/test/resources/hbase-site.xml
+++ b/hbase-server/src/test/resources/hbase-site.xml
@@ -158,13 +158,4 @@
     <name>hbase.hconnection.threads.keepalivetime</name>
     <value>3</value>
   </property>
-  <property>
-    <name>hbase.unsafe.stream.capability.enforce</name>
-    <value>false</value>
-    <description>
-      Controls whether HBase will check for stream capabilities (hflush/hsync).
-      Disable this if you intend to run on LocalFileSystem.
-      WARNING: Doing so may expose you to additional risk of data loss!
-    </description>
-  </property>
 </configuration>