1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.regionserver;
20
21 import java.io.IOException;
22
23 import org.apache.hadoop.hbase.HBaseTestingUtility;
24 import org.apache.hadoop.hbase.HConstants;
25 import org.apache.hadoop.hbase.HTableDescriptor;
26 import org.apache.hadoop.hbase.TableName;
27 import org.apache.hadoop.hbase.client.Delete;
28 import org.apache.hadoop.hbase.client.Get;
29 import org.apache.hadoop.hbase.client.Increment;
30 import org.apache.hadoop.hbase.client.Result;
31 import org.apache.hadoop.hbase.client.Table;
32 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
33 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
34 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
35 import org.apache.hadoop.hbase.testclassification.LargeTests;
36 import org.apache.hadoop.hbase.util.Bytes;
37 import org.apache.hadoop.hbase.util.Threads;
38 import org.junit.AfterClass;
39 import org.junit.Assert;
40 import org.junit.BeforeClass;
41 import org.junit.Rule;
42 import org.junit.Test;
43 import org.junit.experimental.categories.Category;
44 import org.junit.rules.TestName;
45
46 @Category({LargeTests.class})
47 public class TestSettingTimeoutOnBlockingPoint {
48
49 private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
50 private static final byte[] FAM = Bytes.toBytes("f");
51 private static final byte[] ROW1 = Bytes.toBytes("row1");
52 private static final byte[] ROW2 = Bytes.toBytes("row2");
53
54 @Rule
55 public TestName testName = new TestName();
56
57 @BeforeClass
58 public static void setUpBeforeClass() throws Exception {
59 TEST_UTIL.getConfiguration().setBoolean(HConstants.STATUS_PUBLISHED, true);
60 TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
61
62 TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 2);
63 TEST_UTIL.startMiniCluster(2);
64 }
65
66 @AfterClass
67 public static void setUpAfterClass() throws Exception {
68 TEST_UTIL.shutdownMiniCluster();
69 }
70
71 public static class SleepCoprocessor extends BaseRegionObserver {
72 public static final int SLEEP_TIME = 10000;
73
74 @Override
75 public Result preIncrementAfterRowLock(final ObserverContext<RegionCoprocessorEnvironment> e,
76 final Increment increment) throws IOException {
77 Threads.sleep(SLEEP_TIME);
78 return super.preIncrementAfterRowLock(e, increment);
79 }
80 }
81
82 @Test
83 public void testRowLock() throws IOException {
84 final TableName tableName = TableName.valueOf(testName.getMethodName());
85 HTableDescriptor hdt = TEST_UTIL.createTableDescriptor(tableName.getNameAsString());
86 hdt.addCoprocessor(SleepCoprocessor.class.getName());
87 TEST_UTIL.createTable(hdt, new byte[][]{FAM}, TEST_UTIL.getConfiguration());
88
89 Thread incrementThread = new Thread(new Runnable() {
90 @Override
91 public void run() {
92 try {
93 try(Table table = TEST_UTIL.getConnection().getTable(tableName)) {
94 table.incrementColumnValue(ROW1, FAM, FAM, 1);
95 }
96 } catch (IOException e) {
97 Assert.fail(e.getMessage());
98 }
99 }
100 });
101 Thread getThread = new Thread(new Runnable() {
102 @Override
103 public void run() {
104 try {
105 try(Table table = TEST_UTIL.getConnection().getTable(tableName)) {
106 table.setRpcTimeout(1000);
107 Delete delete = new Delete(ROW1);
108 table.delete(delete);
109 }
110 } catch (IOException e) {
111 Assert.fail(e.getMessage());
112 }
113 }
114 });
115
116 incrementThread.start();
117 Threads.sleep(1000);
118 getThread.start();
119 Threads.sleep(2000);
120 try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
121
122
123
124
125 table.setRpcTimeout(1000);
126 table.get(new Get(ROW2));
127 } finally {
128 incrementThread.interrupt();
129 getThread.interrupt();
130 }
131 }
132 }