1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.regionserver;
19  
20  import static org.junit.Assert.assertEquals;
21  
22  import java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.List;
25  import java.util.concurrent.CountDownLatch;
26  
27  import org.apache.hadoop.conf.Configuration;
28  import org.apache.hadoop.fs.FileSystem;
29  import org.apache.hadoop.fs.Path;
30  import org.apache.hadoop.hbase.HBaseConfiguration;
31  import org.apache.hadoop.hbase.HConstants;
32  import org.apache.hadoop.hbase.HRegionInfo;
33  import org.apache.hadoop.hbase.HTableDescriptor;
34  import org.apache.hadoop.hbase.KeyValue;
35  import org.apache.hadoop.hbase.SmallTests;
36  import org.apache.hadoop.hbase.MultithreadedTestUtil;
37  import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext;
38  import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
39  import org.apache.hadoop.hbase.client.Mutation;
40  import org.apache.hadoop.hbase.client.Put;
41  import org.apache.hadoop.hbase.client.Scan;
42  import org.apache.hadoop.hbase.filter.BinaryComparator;
43  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
44  import org.apache.hadoop.hbase.io.HeapSize;
45  import org.apache.hadoop.hbase.regionserver.wal.HLog;
46  import org.apache.hadoop.hbase.util.Bytes;
47  import org.apache.hadoop.hbase.util.Pair;
48  import org.junit.Test;
49  import org.junit.experimental.categories.Category;
50  
51  import com.google.common.collect.Lists;
52  
53  /**
54   * Test of HBASE-7051; that checkAndPuts and puts behave atomically with respect to each other.
55   * Rather than perform a bunch of trials to verify atomicity, this test recreates a race condition
56   * that causes the test to fail if checkAndPut doesn't wait for outstanding put transactions
57   * to complete.  It does this by invasively overriding HRegion function to affect the timing of
58   * the operations.
59   */
60  @Category(SmallTests.class)
61  public class TestHBase7051 {
62  
63    private static CountDownLatch latch = new CountDownLatch(1);
64    private enum TestStep {
65      INIT,                  // initial put of 10 to set value of the cell
66      PUT_STARTED,           // began doing a put of 50 to cell
67      PUT_COMPLETED,         // put complete (released RowLock, but may not have advanced MVCC).
68      CHECKANDPUT_STARTED,   // began checkAndPut: if 10 -> 11
69      CHECKANDPUT_COMPLETED  // completed checkAndPut
70      // NOTE: at the end of these steps, the value of the cell should be 50, not 11!
71    }
72    private static volatile TestStep testStep = TestStep.INIT;
73    private final String family = "f1";
74    	 
75    @Test
76    public void testPutAndCheckAndPutInParallel() throws Exception {
77  
78      final String tableName = "testPutAndCheckAndPut";
79      Configuration conf = HBaseConfiguration.create();
80      conf.setClass(HConstants.REGION_IMPL, MockHRegion.class, HeapSize.class);
81      final MockHRegion region = (MockHRegion) TestHRegion.initHRegion(Bytes.toBytes(tableName),
82          tableName, conf, Bytes.toBytes(family));
83  
84      List<Pair<Mutation, Integer>> putsAndLocks = Lists.newArrayList();
85      Put[] puts = new Put[1];
86      Put put = new Put(Bytes.toBytes("r1"));
87      put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("10"));
88      puts[0] = put;
89      Pair<Mutation, Integer> pair = new Pair<Mutation, Integer>(puts[0], null);
90  
91      putsAndLocks.add(pair);
92  
93      region.batchMutate(putsAndLocks.toArray(new Pair[0]));
94      MultithreadedTestUtil.TestContext ctx =
95        new MultithreadedTestUtil.TestContext(conf);
96      ctx.addThread(new PutThread(ctx, region));
97      ctx.addThread(new CheckAndPutThread(ctx, region));
98      ctx.startThreads();
99      while (testStep != TestStep.CHECKANDPUT_COMPLETED) {
100       Thread.sleep(100);
101     }
102     ctx.stop();
103     Scan s = new Scan();
104     RegionScanner scanner = region.getScanner(s);
105     List<KeyValue> results = new ArrayList<KeyValue>();
106     scanner.next(results, 2);
107     for (KeyValue keyValue : results) {
108       assertEquals("50",Bytes.toString(keyValue.getValue()));
109     }
110 
111   }
112 
113   private class PutThread extends TestThread {
114     private MockHRegion region;
115     PutThread(TestContext ctx, MockHRegion region) {
116       super(ctx);
117       this.region = region;
118     }
119 
120     public void doWork() throws Exception {
121       List<Pair<Mutation, Integer>> putsAndLocks = Lists.newArrayList();
122       Put[] puts = new Put[1];
123       Put put = new Put(Bytes.toBytes("r1"));
124       put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("50"));
125       puts[0] = put;
126       Pair<Mutation, Integer> pair = new Pair<Mutation, Integer>(puts[0], null);
127       putsAndLocks.add(pair);
128       testStep = TestStep.PUT_STARTED;
129       region.batchMutate(putsAndLocks.toArray(new Pair[0]));
130     }
131   }
132 
133   private class CheckAndPutThread extends TestThread {
134     private MockHRegion region;
135     CheckAndPutThread(TestContext ctx, MockHRegion region) {
136       super(ctx);
137       this.region = region;
138    }
139 
140     public void doWork() throws Exception {
141       Put[] puts = new Put[1];
142       Put put = new Put(Bytes.toBytes("r1"));
143       put.add(Bytes.toBytes(family), Bytes.toBytes("q1"), Bytes.toBytes("11"));
144       puts[0] = put;
145       while (testStep != TestStep.PUT_COMPLETED) {
146         Thread.sleep(100);
147       }
148       testStep = TestStep.CHECKANDPUT_STARTED;
149       region.checkAndMutate(Bytes.toBytes("r1"), Bytes.toBytes(family), Bytes.toBytes("q1"),
150         CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("10")), put, null, true);
151       testStep = TestStep.CHECKANDPUT_COMPLETED;
152     }
153   }
154 
155   public static class MockHRegion extends HRegion {
156 
157     public MockHRegion(Path tableDir, HLog log, FileSystem fs, Configuration conf,
158         final HRegionInfo regionInfo, final HTableDescriptor htd, RegionServerServices rsServices) {
159       super(tableDir, log, fs, conf, regionInfo, htd, rsServices);
160     }
161 
162     @Override
163     public void releaseRowLock(Integer lockId) {
164       if (testStep == TestStep.INIT) {
165         super.releaseRowLock(lockId);
166         return;
167       }
168 
169       if (testStep == TestStep.PUT_STARTED) {
170         try {
171           testStep = TestStep.PUT_COMPLETED;
172           super.releaseRowLock(lockId);
173           // put has been written to the memstore and the row lock has been released, but the
174           // MVCC has not been advanced.  Prior to fixing HBASE-7051, the following order of
175           // operations would cause the non-atomicity to show up:
176           // 1) Put releases row lock (where we are now)
177           // 2) CheckAndPut grabs row lock and reads the value prior to the put (10)
178           //    because the MVCC has not advanced
179           // 3) Put advances MVCC
180           // So, in order to recreate this order, we wait for the checkAndPut to grab the rowLock
181           // (see below), and then wait some more to give the checkAndPut time to read the old
182           // value.
183           latch.await();
184           Thread.sleep(1000);
185         } catch (InterruptedException e) {
186           Thread.currentThread().interrupt();
187         }
188       }
189       else if (testStep == TestStep.CHECKANDPUT_STARTED) {
190         super.releaseRowLock(lockId);
191       }
192     }
193 
194     @Override
195     public Integer getLock(Integer lockid, byte[] row, boolean waitForLock) throws IOException {
196       if (testStep == TestStep.CHECKANDPUT_STARTED) {
197         latch.countDown();
198       }
199       return super.getLock(lockid, row, waitForLock);
200     }
201 
202   }
203 
204 }