Return-Path: X-Original-To: apmail-hbase-commits-archive@www.apache.org Delivered-To: apmail-hbase-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 4FAF118910 for ; Mon, 8 Feb 2016 18:09:37 +0000 (UTC) Received: (qmail 83443 invoked by uid 500); 8 Feb 2016 18:09:37 -0000 Delivered-To: apmail-hbase-commits-archive@hbase.apache.org Received: (qmail 83404 invoked by uid 500); 8 Feb 2016 18:09:37 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 83394 invoked by uid 99); 8 Feb 2016 18:09:37 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 08 Feb 2016 18:09:37 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id EFE18E00A0; Mon, 8 Feb 2016 18:09:36 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: stack@apache.org To: commits@hbase.apache.org Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: hbase git commit: HBASE-15224 Undo "hbase.increment.fast.but.narrow.consistency" option; it is not necessary since HBASE-15213 Date: Mon, 8 Feb 2016 18:09:36 +0000 (UTC) Repository: hbase Updated Branches: refs/heads/branch-1.1 7fc757681 -> 28a3fdd09 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/28a3fdd0 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/28a3fdd0 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/28a3fdd0 Branch: refs/heads/branch-1.1 Commit: 28a3fdd09af248e8bf17df2a44e665cc36e7f187 Parents: 7fc7576 Author: stack Authored: Mon Feb 8 08:43:11 2016 -0800 Committer: stack Committed: Mon Feb 8 10:09:28 2016 -0800 ---------------------------------------------------------------------- .../hadoop/hbase/regionserver/HRegion.java | 143 +------------------ ...tIncrementFromClientSideWithCoprocessor.java | 5 - .../client/TestIncrementsFromClientSide.java | 95 ++---------- .../hbase/regionserver/TestAtomicOperation.java | 63 +++----- .../hbase/regionserver/TestRegionIncrement.java | 24 +--- 5 files changed, 32 insertions(+), 298 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/28a3fdd0/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 86d9b3c..6fd88b8 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 @@ -216,16 +216,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi private static final int DEFAULT_MAX_WAIT_FOR_SEQ_ID = 30000; /** - * 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. */ @@ -767,10 +757,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi 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() { @@ -7110,139 +7096,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi 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 allKVs = new ArrayList(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> forMemStore = new HashMap>(); - for (Map.Entry> entry: increment.getFamilyCellMap().entrySet()) { - byte [] columnFamilyName = entry.getKey(); - List 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 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> entry: forMemStore.entrySet()) { - Store store = entry.getKey(); - List 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 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/28a3fdd0/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/28a3fdd0/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 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/28a3fdd0/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 3b97721..51a7bd6 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(); @@ -540,7 +513,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 @@ -552,11 +525,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 @@ -572,7 +545,7 @@ 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, HConstants.NO_NONCE, HConstants.NO_NONCE); MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext(conf); @@ -647,7 +620,7 @@ public class TestAtomicOperation { } return new WrappedRowLock(super.getRowLockInternal(row, waitForLock)); } - + public class WrappedRowLock extends RowLockImpl { private WrappedRowLock(RowLock rowLock) { http://git-wip-us.apache.org/repos/asf/hbase/blob/28a3fdd0/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 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();