You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2016/11/01 19:51:14 UTC

[20/50] hbase git commit: HBASE-15224 Undo "hbase.increment.fast.but.narrow.consistency" option; it is not necessary since HBASE-15213

HBASE-15224 Undo "hbase.increment.fast.but.narrow.consistency" option; it is not necessary since HBASE-15213


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

Branch: refs/heads/branch-1.0
Commit: b9c3419d483493a29b4980cc2b38ca629d80a598
Parents: eb9af81
Author: stack <st...@apache.org>
Authored: Mon Feb 8 08:43:11 2016 -0800
Committer: stack <st...@apache.org>
Committed: Mon Feb 8 10:14:57 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      | 143 +------------------
 ...tIncrementFromClientSideWithCoprocessor.java |   5 -
 .../client/TestIncrementsFromClientSide.java    |  95 ++----------
 .../hbase/regionserver/TestAtomicOperation.java |  62 +++-----
 .../hbase/regionserver/TestRegionIncrement.java |  24 +---
 5 files changed, 30 insertions(+), 299 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b9c3419d/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 0c5dfe7..8aed3a6 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
@@ -221,16 +221,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
       "hbase.hregion.scan.loadColumnFamiliesOnDemand";
 
   /**
-   * Set region to take the fast increment path. Constraint is that caller can only access the
-   * Cell via Increment; intermixing Increment with other Mutations will give indeterminate
-   * results. A Get with {@link IsolationLevel#READ_UNCOMMITTED} will get the latest increment
-   * or an Increment of zero will do the same.
-   */
-  public static final String INCREMENT_FAST_BUT_NARROW_CONSISTENCY_KEY =
-      "hbase.increment.fast.but.narrow.consistency";
-  private final boolean incrementFastButNarrowConsistency;
-
-  /**
    * This is the global default value for durability. All tables/mutations not
    * defining a durability or using USE_DEFAULT will default to this value.
    */
@@ -712,10 +702,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
           false :
           conf.getBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE,
               HConstants.DEFAULT_ENABLE_CLIENT_BACKPRESSURE);
-
-    // See #INCREMENT_FAST_BUT_NARROW_CONSISTENCY_KEY for what this flag is about.
-    this.incrementFastButNarrowConsistency =
-      this.conf.getBoolean(INCREMENT_FAST_BUT_NARROW_CONSISTENCY_KEY, false);
   }
 
   void setHTableSpecificConf() {
@@ -5840,139 +5826,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
     startRegionOperation(Operation.INCREMENT);
     this.writeRequestsCount.increment();
     try {
-      // Which Increment is it? Narrow increment-only consistency or slow (default) and general
-      // row-wide consistency.
-
-      // So, difference between fastAndNarrowConsistencyIncrement and slowButConsistentIncrement is
-      // that the former holds the row lock until the sync completes; this allows us to reason that
-      // there are no other writers afoot when we read the current increment value. The row lock
-      // means that we do not need to wait on mvcc reads to catch up to writes before we proceed
-      // with the read, the root of the slowdown seen in HBASE-14460. The fast-path also does not
-      // wait on mvcc to complete before returning to the client. We also reorder the write so that
-      // the update of memstore happens AFTER sync returns; i.e. the write pipeline does less
-      // zigzagging now.
-      //
-      // See the comment on INCREMENT_FAST_BUT_NARROW_CONSISTENCY_KEY
-      // for the constraints that apply when you take this code path; it is correct but only if
-      // Increments are used mutating an Increment Cell; mixing concurrent Put+Delete and Increment
-      // will yield indeterminate results.
-      return this.incrementFastButNarrowConsistency?
-        fastAndNarrowConsistencyIncrement(increment, nonceGroup, nonce):
-        slowButConsistentIncrement(increment, nonceGroup, nonce);
+      return doIncrement(increment, nonceGroup, nonce);
     } finally {
       if (this.metricsRegion != null) this.metricsRegion.updateIncrement();
       closeRegionOperation(Operation.INCREMENT);
     }
   }
 
-  /**
-   * The bulk of this method is a bulk-and-paste of the slowButConsistentIncrement but with some
-   * reordering to enable the fast increment (reordering allows us to also drop some state
-   * carrying Lists and variables so the flow here is more straight-forward). We copy-and-paste
-   * because cannot break down the method further into smaller pieces. Too much state. Will redo
-   * in trunk and tip of branch-1 to undo duplication here and in append, checkAnd*, etc. For why
-   * this route is 'faster' than the alternative slowButConsistentIncrement path, see the comment
-   * in calling method.
-   * @return Resulting increment
-   * @throws IOException
-   */
-  private Result fastAndNarrowConsistencyIncrement(Increment increment, long nonceGroup,
-      long nonce)
-  throws IOException {
-    long accumulatedResultSize = 0;
-    RowLock rowLock = null;
-    WALKey walKey = null;
-    // This is all kvs accumulated during this increment processing. Includes increments where the
-    // increment is zero: i.e. client just wants to get current state of the increment w/o
-    // changing it. These latter increments by zero are NOT added to the WAL.
-    List<Cell> allKVs = new ArrayList<Cell>(increment.size());
-    Durability effectiveDurability = getEffectiveDurability(increment.getDurability());
-    long txid = 0;
-    rowLock = getRowLock(increment.getRow());
-    try {
-      lock(this.updatesLock.readLock());
-      try {
-        if (this.coprocessorHost != null) {
-          Result r = this.coprocessorHost.preIncrementAfterRowLock(increment);
-          if (r != null) return increment.isReturnResults() ? r : null;
-        }
-        // Process increments a Store/family at a time.
-        long now = EnvironmentEdgeManager.currentTime();
-        final boolean writeToWAL = effectiveDurability != Durability.SKIP_WAL;
-        WALEdit walEdits = null;
-        // Accumulate edits for memstore to add later after we've added to WAL.
-        Map<Store, List<Cell>> forMemStore = new HashMap<Store, List<Cell>>();
-        for (Map.Entry<byte [], List<Cell>> entry: increment.getFamilyCellMap().entrySet()) {
-          byte [] columnFamilyName = entry.getKey();
-          List<Cell> increments = entry.getValue();
-          Store store = this.stores.get(columnFamilyName);
-          // Do increment for this store; be sure to 'sort' the increments first so increments
-          // match order in which we get back current Cells when we get.
-          List<Cell> results = applyIncrementsToColumnFamily(increment, columnFamilyName,
-              sort(increments, store.getComparator()), now,
-              MultiVersionConsistencyControl.NO_WRITE_NUMBER, allKVs,
-              IsolationLevel.READ_UNCOMMITTED);
-          if (!results.isEmpty()) {
-            forMemStore.put(store, results);
-            // Prepare WAL updates
-            if (writeToWAL) {
-              if (walEdits == null) walEdits = new WALEdit();
-              walEdits.getCells().addAll(results);
-            }
-          }
-        }
-
-        // Actually write to WAL now. If walEdits is non-empty, we write the WAL.
-        if (walEdits != null && !walEdits.isEmpty()) {
-          // Using default cluster id, as this can only happen in the originating cluster.
-          // A slave cluster receives the final value (not the delta) as a Put. We use HLogKey
-          // here instead of WALKey directly to support legacy coprocessors.
-          walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
-            this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, nonceGroup, nonce);
-          txid = this.wal.append(this.htableDescriptor, this.getRegionInfo(),
-             walKey, walEdits, getSequenceId(), true, null/*walEdits has the List to apply*/);
-        } else {
-          // Append a faked WALEdit in order for SKIP_WAL updates to get mvccNum assigned
-          walKey = this.appendEmptyEdit(this.wal, null/*walEdits has the List to apply*/);
-        }
-
-        if (txid != 0) syncOrDefer(txid, effectiveDurability);
-
-        // Tell MVCC about the new sequenceid.
-        WriteEntry we = mvcc.beginMemstoreInsertWithSeqNum(walKey.getSequenceId());
-
-        // Now write to memstore.
-        for (Map.Entry<Store, List<Cell>> entry: forMemStore.entrySet()) {
-          Store store = entry.getKey();
-          List<Cell> results = entry.getValue();
-          if (store.getFamily().getMaxVersions() == 1) {
-            // Upsert if VERSIONS for this CF == 1. Use write sequence id rather than read point
-            // when doing fast increment.
-            accumulatedResultSize += store.upsert(results, walKey.getSequenceId());
-          } else {
-            // Otherwise keep older versions around
-            for (Cell cell: results) {
-              Pair<Long, Cell> ret = store.add(cell);
-              accumulatedResultSize += ret.getFirst();
-            }
-          }
-        }
-
-        // Tell mvcc this write is complete.
-        this.mvcc.advanceMemstore(we);
-      } finally {
-        this.updatesLock.readLock().unlock();
-      }
-    } finally {
-      rowLock.release();
-    }
-    // Request a cache flush.  Do it outside update lock.
-    if (isFlushSize(this.addAndGetGlobalMemstoreSize(accumulatedResultSize))) requestFlush();
-    return increment.isReturnResults() ? Result.create(allKVs) : null;
-  }
-
-  private Result slowButConsistentIncrement(Increment increment, long nonceGroup, long nonce)
-  throws IOException {
+  private Result doIncrement(Increment increment, long nonceGroup, long nonce) throws IOException {
     RowLock rowLock = null;
     WriteEntry writeEntry = null;
     WALKey walKey = null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b9c3419d/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementFromClientSideWithCoprocessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementFromClientSideWithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementFromClientSideWithCoprocessor.java
index a67cc45..c9bc7c2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementFromClientSideWithCoprocessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementFromClientSideWithCoprocessor.java
@@ -34,16 +34,11 @@ import org.junit.experimental.categories.Category;
  */
 @Category(LargeTests.class)
 public class TestIncrementFromClientSideWithCoprocessor extends TestIncrementsFromClientSide {
-  public TestIncrementFromClientSideWithCoprocessor(final boolean fast) {
-    super(fast);
-  }
-
   @Before
   public void before() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
         MultiRowMutationEndpoint.class.getName(), NoOpScanPolicyObserver.class.getName());
     conf.setBoolean("hbase.table.sanity.checks", true); // enable for below tests
-    super.before();
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/b9c3419d/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java
index 77cebbd..188fb66 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java
@@ -18,47 +18,37 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
-import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.After;
-import org.junit.Before;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.junit.Assert.assertTrue;
 
 /**
  * Run Increment tests that use the HBase clients; {@link HTable}.
- * 
+ *
  * Test is parameterized to run the slow and fast increment code paths. If fast, in the @before, we
  * do a rolling restart of the single regionserver so that it can pick up the go fast configuration.
  * Doing it this way should be faster than starting/stopping a cluster per test.
- * 
+ *
  * Test takes a long time because spin up a cluster between each run -- ugh.
  */
-@RunWith(Parameterized.class)
 @Category(LargeTests.class)
 @SuppressWarnings ("deprecation")
 public class TestIncrementsFromClientSide {
@@ -69,47 +59,18 @@ public class TestIncrementsFromClientSide {
   // This test depends on there being only one slave running at at a time. See the @Before
   // method where we do rolling restart.
   protected static int SLAVES = 1;
-  private String oldINCREMENT_FAST_BUT_NARROW_CONSISTENCY_KEY;
   @Rule public TestName name = new TestName();
-  @Parameters(name = "fast={0}")
   public static Collection<Object []> data() {
     return Arrays.asList(new Object[] {Boolean.FALSE}, new Object [] {Boolean.TRUE});
   }
-  private final boolean fast;
-
-  public TestIncrementsFromClientSide(final boolean fast) {
-    this.fast = fast;
-  }
 
   @BeforeClass
   public static void beforeClass() throws Exception {
-    Configuration conf = TEST_UTIL.getConfiguration();
-    conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
-        MultiRowMutationEndpoint.class.getName());
-    conf.setBoolean("hbase.table.sanity.checks", true); // enable for below tests
+    TEST_UTIL.startMiniCluster();
   }
 
-  @Before
-  public void before() throws Exception {
-    Configuration conf = TEST_UTIL.getConfiguration();
-    if (this.fast) {
-      this.oldINCREMENT_FAST_BUT_NARROW_CONSISTENCY_KEY =
-          conf.get(HRegion.INCREMENT_FAST_BUT_NARROW_CONSISTENCY_KEY);
-      conf.setBoolean(HRegion.INCREMENT_FAST_BUT_NARROW_CONSISTENCY_KEY, this.fast);
-    }
-    // We need more than one region server in this test
-    TEST_UTIL.startMiniCluster(SLAVES);
-  }
-
-  @After
-  public void after() throws Exception {
-    Configuration conf = TEST_UTIL.getConfiguration();
-    if (this.fast) {
-      if (this.oldINCREMENT_FAST_BUT_NARROW_CONSISTENCY_KEY != null) {
-        conf.set(HRegion.INCREMENT_FAST_BUT_NARROW_CONSISTENCY_KEY,
-            this.oldINCREMENT_FAST_BUT_NARROW_CONSISTENCY_KEY);
-      }
-    }
+  @AfterClass
+  public static void afterClass() throws Exception {
     TEST_UTIL.shutdownMiniCluster();
   }
 
@@ -130,7 +91,6 @@ public class TestIncrementsFromClientSide {
     ht.incrementColumnValue(ROW, FAMILY, COLUMN, 5);
 
     Get get = new Get(ROW);
-    if (this.fast) get.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED);
     Result r = ht.get(get);
     assertEquals(1, r.size());
     assertEquals(5, Bytes.toLong(r.getValue(FAMILY, COLUMN)));
@@ -164,35 +124,6 @@ public class TestIncrementsFromClientSide {
   }
 
   @Test
-  public void testIncrementReturnValue() throws Exception {
-    LOG.info("Starting " + this.name.getMethodName());
-    final TableName TABLENAME =
-      TableName.valueOf(filterStringSoTableNameSafe(this.name.getMethodName()));
-    Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
-    final byte[] COLUMN = Bytes.toBytes("column");
-    Put p = new Put(ROW);
-    p.add(FAMILY, COLUMN, Bytes.toBytes(5L));
-    ht.put(p);
-
-    Increment inc = new Increment(ROW);
-    inc.addColumn(FAMILY, COLUMN, 5L);
-
-    Result r = ht.increment(inc);
-    long result = Bytes.toLong(r.getValue(FAMILY, COLUMN));
-    assertEquals(10, result);
-
-    if (this.fast) {
-      inc = new Increment(ROW);
-      inc.addColumn(FAMILY, COLUMN, 5L);
-      inc.setReturnResults(false);
-      r = ht.increment(inc);
-      assertTrue(r.getExists() == null);
-    }
-
-  }
-
-
-  @Test
   public void testIncrementInvalidArguments() throws Exception {
     LOG.info("Starting " + this.name.getMethodName());
     final TableName TABLENAME =
@@ -267,7 +198,6 @@ public class TestIncrementsFromClientSide {
 
     // Verify expected results
     Get get = new Get(ROW);
-    if (this.fast) get.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED);
     Result r = ht.get(get);
     Cell [] kvs = r.rawCells();
     assertEquals(3, kvs.length);
@@ -309,7 +239,6 @@ public class TestIncrementsFromClientSide {
 
     // Verify expected results
     Get get = new Get(ROW);
-    if (this.fast) get.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED);
     Result r = ht.get(get);
     Cell[] kvs = r.rawCells();
     assertEquals(3, kvs.length);
@@ -332,7 +261,7 @@ public class TestIncrementsFromClientSide {
     assertIncrementKey(kvs[0], ROW, FAMILY, QUALIFIERS[0], 2);
     assertIncrementKey(kvs[1], ROW, FAMILY, QUALIFIERS[1], 2);
     assertIncrementKey(kvs[2], ROW, FAMILY, QUALIFIERS[2], 2);
-    
+
     ht.close();
   }
 
@@ -371,7 +300,6 @@ public class TestIncrementsFromClientSide {
 
     // Verify expected results
     Get get = new Get(ROW);
-    if (this.fast) get.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED);
     Result r = ht.get(get);
     Cell [] kvs = r.rawCells();
     assertEquals(5, kvs.length);
@@ -389,7 +317,6 @@ public class TestIncrementsFromClientSide {
     ht.increment(inc);
     // Verify
     get = new Get(ROWS[0]);
-    if (this.fast) get.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED);
     r = ht.get(get);
     kvs = r.rawCells();
     assertEquals(QUALIFIERS.length, kvs.length);
@@ -438,4 +365,4 @@ public class TestIncrementsFromClientSide {
   public static String filterStringSoTableNameSafe(final String str) {
     return str.replaceAll("\\[fast\\=(.*)\\]", ".FAST.is.$1");
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b9c3419d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
index f2ea717..74d8cba 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
@@ -52,7 +52,6 @@ import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
-import org.apache.hadoop.hbase.client.IsolationLevel;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
@@ -93,11 +92,11 @@ public class TestAtomicOperation {
   static final byte [] row = Bytes.toBytes("rowA");
   static final byte [] row2 = Bytes.toBytes("rowB");
 
-  @Before 
+  @Before
   public void setup() {
     tableName = Bytes.toBytes(name.getMethodName());
   }
-  
+
   @After
   public void teardown() throws IOException {
     if (region != null) {
@@ -107,7 +106,7 @@ public class TestAtomicOperation {
   }
   //////////////////////////////////////////////////////////////////////////////
   // New tests that doesn't spin up a mini cluster but rather just test the
-  // individual code pieces in the HRegion. 
+  // individual code pieces in the HRegion.
   //////////////////////////////////////////////////////////////////////////////
 
   /**
@@ -135,29 +134,10 @@ public class TestAtomicOperation {
   }
 
   /**
-   * Test multi-threaded increments. Take the fast but narrow consistency path through HRegion.
-   */
-  @Test
-  public void testIncrementMultiThreadsFastPath() throws IOException {
-    Configuration conf = TEST_UTIL.getConfiguration();
-    String oldValue = conf.get(HRegion.INCREMENT_FAST_BUT_NARROW_CONSISTENCY_KEY);
-    conf.setBoolean(HRegion.INCREMENT_FAST_BUT_NARROW_CONSISTENCY_KEY, true);
-    try {
-      testIncrementMultiThreads(true);
-    } finally {
-      if (oldValue != null) conf.set(HRegion.INCREMENT_FAST_BUT_NARROW_CONSISTENCY_KEY, oldValue);
-    }
-  }
-
-  /**
-   * Test multi-threaded increments. Take the slow but consistent path through HRegion.
+   * Test multi-threaded increments.
    */
   @Test
-  public void testIncrementMultiThreadsSlowPath() throws IOException {
-    testIncrementMultiThreads(false);
-  }
-
-  private void testIncrementMultiThreads(final boolean fast) throws IOException {
+  public void testIncrementMultiThreads() throws IOException {
     LOG.info("Starting test testIncrementMultiThreads");
     // run a with mixed column families (1 and 3 versions)
     initHRegion(tableName, name.getMethodName(), new int[] {1,3}, fam1, fam2);
@@ -187,23 +167,19 @@ public class TestAtomicOperation {
       } catch (InterruptedException e) {
       }
     }
-
-    assertICV(row, fam1, qual1, expectedTotal, fast);
-    assertICV(row, fam1, qual2, expectedTotal*2, fast);
-    assertICV(row, fam2, qual3, expectedTotal*3, fast);
-    LOG.info("testIncrementMultiThreads successfully verified that total is " +
-             expectedTotal);
+    assertICV(row, fam1, qual1, expectedTotal);
+    assertICV(row, fam1, qual2, expectedTotal*2);
+    assertICV(row, fam2, qual3, expectedTotal*3);
+    LOG.info("testIncrementMultiThreads successfully verified that total is " + expectedTotal);
   }
 
 
   private void assertICV(byte [] row,
                          byte [] familiy,
                          byte[] qualifier,
-                         long amount,
-                         boolean fast) throws IOException {
+                         long amount) throws IOException {
     // run a get and see?
     Get get = new Get(row);
-    if (fast) get.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED);
     get.addColumn(familiy, qualifier);
     Result result = region.get(get);
     assertEquals(1, result.size());
@@ -242,7 +218,6 @@ public class TestAtomicOperation {
     private final HRegion region;
     private final int numIncrements;
     private final int amount;
-    private final boolean fast;
 
 
     public Incrementer(HRegion region, int threadNumber, int amount, int numIncrements) {
@@ -250,8 +225,6 @@ public class TestAtomicOperation {
       this.region = region;
       this.numIncrements = numIncrements;
       this.amount = amount;
-      this.fast = region.getBaseConf().
-          getBoolean(HRegion.INCREMENT_FAST_BUT_NARROW_CONSISTENCY_KEY, false);
       setDaemon(true);
     }
 
@@ -306,8 +279,8 @@ public class TestAtomicOperation {
 
               Get g = new Get(row);
               Result result = region.get(g);
-              assertEquals(result.getValue(fam1, qual1).length, result.getValue(fam1, qual2).length); 
-              assertEquals(result.getValue(fam1, qual1).length, result.getValue(fam2, qual3).length); 
+              assertEquals(result.getValue(fam1, qual1).length, result.getValue(fam1, qual2).length);
+              assertEquals(result.getValue(fam1, qual1).length, result.getValue(fam2, qual3).length);
             } catch (IOException e) {
               e.printStackTrace();
               failures.incrementAndGet();
@@ -539,7 +512,7 @@ public class TestAtomicOperation {
       this.failures = failures;
     }
   }
-  
+
   private static CountDownLatch latch = new CountDownLatch(1);
   private enum TestStep {
     INIT,                  // initial put of 10 to set value of the cell
@@ -551,11 +524,11 @@ public class TestAtomicOperation {
   }
   private static volatile TestStep testStep = TestStep.INIT;
   private final String family = "f1";
-     
+
   /**
    * Test written as a verifier for HBASE-7051, CheckAndPut should properly read
-   * MVCC. 
-   * 
+   * MVCC.
+   *
    * Moved into TestAtomicOperation from its original location, TestHBase7051
    */
   @Test
@@ -571,7 +544,6 @@ public class TestAtomicOperation {
     Put put = new Put(Bytes.toBytes("r1"));
     put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("10"));
     puts[0] = put;
-    
     region.batchMutate(puts);
     MultithreadedTestUtil.TestContext ctx =
       new MultithreadedTestUtil.TestContext(conf);
@@ -645,9 +617,7 @@ public class TestAtomicOperation {
       }
       return new WrappedRowLock(super.getRowLockInternal(row, waitForLock));
     }
-    
     public class WrappedRowLock extends RowLock {
-
       private WrappedRowLock(RowLock rowLock) {
         super(rowLock.context);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b9c3419d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
index 92285a8..c9cb90f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
@@ -43,15 +43,11 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.After;
-import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.junit.rules.TestRule;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
 
 
 /**
@@ -64,38 +60,20 @@ import org.junit.runners.Parameterized.Parameters;
  * prove atomicity on row.
  */
 @Category(MediumTests.class)
-@RunWith(Parameterized.class)
 public class TestRegionIncrement {
   private static final Log LOG = LogFactory.getLog(TestRegionIncrement.class);
   @Rule public TestName name = new TestName();
   @Rule public final TestRule timeout =
       CategoryBasedTimeout.builder().withTimeout(this.getClass()).
         withLookingForStuckThread(true).build();
-  private static HBaseTestingUtility TEST_UTIL;
+  private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private final static byte [] INCREMENT_BYTES = Bytes.toBytes("increment");
   private static final int THREAD_COUNT = 10;
   private static final int INCREMENT_COUNT = 10000;
-
-  @Parameters(name = "fast={0}")
   public static Collection<Object []> data() {
     return Arrays.asList(new Object[] {Boolean.FALSE}, new Object [] {Boolean.TRUE});
   }
 
-  private final boolean fast;
-
-  public TestRegionIncrement(final boolean fast) {
-    this.fast = fast;
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    TEST_UTIL = HBaseTestingUtility.createLocalHTU();
-    if (this.fast) {
-      TEST_UTIL.getConfiguration().
-        setBoolean(HRegion.INCREMENT_FAST_BUT_NARROW_CONSISTENCY_KEY, this.fast);
-    }
-  }
-
   @After
   public void tearDown() throws Exception {
     TEST_UTIL.cleanupTestDir();