001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.regionserver; 019 020import java.io.IOException; 021import java.util.Optional; 022import org.apache.hadoop.hbase.HBaseClassTestRule; 023import org.apache.hadoop.hbase.HBaseTestingUtil; 024import org.apache.hadoop.hbase.HConstants; 025import org.apache.hadoop.hbase.TableName; 026import org.apache.hadoop.hbase.client.Delete; 027import org.apache.hadoop.hbase.client.Get; 028import org.apache.hadoop.hbase.client.Increment; 029import org.apache.hadoop.hbase.client.Result; 030import org.apache.hadoop.hbase.client.Table; 031import org.apache.hadoop.hbase.client.TableDescriptor; 032import org.apache.hadoop.hbase.coprocessor.ObserverContext; 033import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; 034import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; 035import org.apache.hadoop.hbase.coprocessor.RegionObserver; 036import org.apache.hadoop.hbase.testclassification.MediumTests; 037import org.apache.hadoop.hbase.util.Bytes; 038import org.apache.hadoop.hbase.util.Threads; 039import org.junit.AfterClass; 040import org.junit.Assert; 041import org.junit.BeforeClass; 042import org.junit.ClassRule; 043import org.junit.Rule; 044import org.junit.Test; 045import org.junit.experimental.categories.Category; 046import org.junit.rules.TestName; 047 048@Category({ MediumTests.class }) 049public class TestSettingTimeoutOnBlockingPoint { 050 051 @ClassRule 052 public static final HBaseClassTestRule CLASS_RULE = 053 HBaseClassTestRule.forClass(TestSettingTimeoutOnBlockingPoint.class); 054 055 private final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); 056 private static final byte[] FAM = Bytes.toBytes("f"); 057 private static final byte[] ROW1 = Bytes.toBytes("row1"); 058 private static final byte[] ROW2 = Bytes.toBytes("row2"); 059 060 @Rule 061 public TestName testName = new TestName(); 062 063 @BeforeClass 064 public static void setUpBeforeClass() throws Exception { 065 TEST_UTIL.getConfiguration().setBoolean(HConstants.STATUS_PUBLISHED, true); 066 TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); 067 // simulate queue blocking 068 TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 2); 069 TEST_UTIL.startMiniCluster(2); 070 } 071 072 @AfterClass 073 public static void setUpAfterClass() throws Exception { 074 TEST_UTIL.shutdownMiniCluster(); 075 } 076 077 public static class SleepCoprocessor implements RegionCoprocessor, RegionObserver { 078 public static final int SLEEP_TIME = 10000; 079 080 @Override 081 public Optional<RegionObserver> getRegionObserver() { 082 return Optional.of(this); 083 } 084 085 @Override 086 public Result preIncrementAfterRowLock( 087 final ObserverContext<? extends RegionCoprocessorEnvironment> e, final Increment increment) 088 throws IOException { 089 Threads.sleep(SLEEP_TIME); 090 return null; 091 } 092 } 093 094 @Test 095 public void testRowLock() throws IOException { 096 TableDescriptor hdt = TEST_UTIL.createModifyableTableDescriptor(testName.getMethodName()) 097 .setCoprocessor(SleepCoprocessor.class.getName()).build(); 098 TEST_UTIL.createTable(hdt, new byte[][] { FAM }, TEST_UTIL.getConfiguration()); 099 TableName tableName = hdt.getTableName(); 100 Thread incrementThread = new Thread(() -> { 101 try { 102 try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { 103 table.incrementColumnValue(ROW1, FAM, FAM, 1); 104 } 105 } catch (IOException e) { 106 Assert.fail(e.getMessage()); 107 } 108 }); 109 Thread getThread = new Thread(() -> { 110 try (Table table = 111 TEST_UTIL.getConnection().getTableBuilder(tableName, null).setRpcTimeout(1000).build()) { 112 Delete delete = new Delete(ROW1); 113 table.delete(delete); 114 } catch (IOException e) { 115 Assert.fail(e.getMessage()); 116 } 117 }); 118 119 incrementThread.start(); 120 Threads.sleep(1000); 121 getThread.start(); 122 Threads.sleep(2000); 123 try (Table table = 124 TEST_UTIL.getConnection().getTableBuilder(tableName, null).setRpcTimeout(1000).build()) { 125 // We have only two handlers. The first thread will get a write lock for row1 and occupy 126 // the first handler. The second thread need a read lock for row1, it should quit after 1000 127 // ms and give back the handler because it can not get the lock in time. 128 // So we can get the value using the second handler. 129 table.get(new Get(ROW2)); // Will throw exception if the timeout checking is failed 130 } finally { 131 incrementThread.interrupt(); 132 getThread.interrupt(); 133 } 134 } 135}