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.master.assignment; 019 020import static org.junit.Assert.assertEquals; 021 022import java.io.IOException; 023import java.net.ConnectException; 024import java.util.List; 025import java.util.concurrent.CopyOnWriteArrayList; 026import java.util.concurrent.CountDownLatch; 027import org.apache.hadoop.conf.Configuration; 028import org.apache.hadoop.hbase.HBaseClassTestRule; 029import org.apache.hadoop.hbase.HBaseTestingUtility; 030import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer; 031import org.apache.hadoop.hbase.PleaseHoldException; 032import org.apache.hadoop.hbase.ServerName; 033import org.apache.hadoop.hbase.StartMiniClusterOption; 034import org.apache.hadoop.hbase.TableName; 035import org.apache.hadoop.hbase.client.RegionInfo; 036import org.apache.hadoop.hbase.master.HMaster; 037import org.apache.hadoop.hbase.master.MasterServices; 038import org.apache.hadoop.hbase.master.RegionPlan; 039import org.apache.hadoop.hbase.master.RegionServerList; 040import org.apache.hadoop.hbase.master.RegionState; 041import org.apache.hadoop.hbase.master.ServerManager; 042import org.apache.hadoop.hbase.master.region.MasterRegion; 043import org.apache.hadoop.hbase.regionserver.HRegionServer; 044import org.apache.hadoop.hbase.regionserver.RSRpcServices; 045import org.apache.hadoop.hbase.testclassification.LargeTests; 046import org.apache.hadoop.hbase.testclassification.MasterTests; 047import org.apache.hadoop.hbase.util.Bytes; 048import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; 049import org.junit.AfterClass; 050import org.junit.BeforeClass; 051import org.junit.ClassRule; 052import org.junit.Test; 053import org.junit.experimental.categories.Category; 054import org.slf4j.Logger; 055import org.slf4j.LoggerFactory; 056 057import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; 058import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; 059 060import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 061import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest; 062import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse; 063import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition; 064import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; 065import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest; 066import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse; 067 068/** 069 * Testcase for HBASE-21811. 070 */ 071@Category({ MasterTests.class, LargeTests.class }) 072public class TestWakeUpUnexpectedProcedure { 073 074 @ClassRule 075 public static final HBaseClassTestRule CLASS_RULE = 076 HBaseClassTestRule.forClass(TestWakeUpUnexpectedProcedure.class); 077 078 private static final Logger LOG = LoggerFactory.getLogger(TestWakeUpUnexpectedProcedure.class); 079 080 private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); 081 082 private static TableName NAME = TableName.valueOf("Assign"); 083 084 private static final List<ServerName> EXCLUDE_SERVERS = new CopyOnWriteArrayList<>(); 085 086 private static byte[] CF = Bytes.toBytes("cf"); 087 088 private static volatile ServerName SERVER_TO_KILL; 089 090 private static volatile CountDownLatch ARRIVE_EXEC_PROC; 091 092 private static volatile CountDownLatch RESUME_EXEC_PROC; 093 094 private static volatile CountDownLatch RESUME_IS_SERVER_ONLINE; 095 096 private static volatile CountDownLatch ARRIVE_REPORT; 097 098 private static volatile CountDownLatch RESUME_REPORT; 099 100 private static final class RSRpcServicesForTest extends RSRpcServices { 101 102 public RSRpcServicesForTest(HRegionServer rs) throws IOException { 103 super(rs); 104 } 105 106 @Override 107 public ExecuteProceduresResponse executeProcedures(RpcController controller, 108 ExecuteProceduresRequest request) throws ServiceException { 109 if (request.getOpenRegionCount() > 0) { 110 if (ARRIVE_EXEC_PROC != null) { 111 SERVER_TO_KILL = regionServer.getServerName(); 112 ARRIVE_EXEC_PROC.countDown(); 113 ARRIVE_EXEC_PROC = null; 114 try { 115 RESUME_EXEC_PROC.await(); 116 } catch (InterruptedException e) { 117 throw new RuntimeException(e); 118 } 119 throw new ServiceException(new ConnectException("Inject error")); 120 } 121 } 122 return super.executeProcedures(controller, request); 123 } 124 } 125 126 public static final class RSForTest extends MiniHBaseClusterRegionServer { 127 128 public RSForTest(Configuration conf) throws IOException, InterruptedException { 129 super(conf); 130 } 131 132 @Override 133 protected RSRpcServices createRpcServices() throws IOException { 134 return new RSRpcServicesForTest(this); 135 } 136 } 137 138 private static final class AMForTest extends AssignmentManager { 139 140 public AMForTest(MasterServices master, MasterRegion masterRegion) { 141 super(master, masterRegion); 142 } 143 144 @Override 145 public ReportRegionStateTransitionResponse reportRegionStateTransition( 146 ReportRegionStateTransitionRequest req) throws PleaseHoldException { 147 RegionStateTransition rst = req.getTransition(0); 148 if ( 149 rst.getTransitionCode() == TransitionCode.OPENED 150 && ProtobufUtil.toTableName(rst.getRegionInfo(0).getTableName()).equals(NAME) 151 ) { 152 CountDownLatch arrive = ARRIVE_REPORT; 153 if (ARRIVE_REPORT != null) { 154 ARRIVE_REPORT = null; 155 arrive.countDown(); 156 // so we will choose another rs next time 157 EXCLUDE_SERVERS.add(ProtobufUtil.toServerName(req.getServer())); 158 try { 159 RESUME_REPORT.await(); 160 } catch (InterruptedException e) { 161 throw new RuntimeException(); 162 } 163 } 164 } 165 return super.reportRegionStateTransition(req); 166 } 167 } 168 169 private static final class SMForTest extends ServerManager { 170 171 public SMForTest(MasterServices master, RegionServerList storage) { 172 super(master, storage); 173 } 174 175 @Override 176 public boolean isServerOnline(ServerName serverName) { 177 ServerName toKill = SERVER_TO_KILL; 178 if (toKill != null && toKill.equals(serverName)) { 179 for (StackTraceElement ele : new Exception().getStackTrace()) { 180 // halt it is called from RSProcedureDispatcher, to delay the remoteCallFailed. 181 if ("scheduleForRetry".equals(ele.getMethodName())) { 182 if (RESUME_IS_SERVER_ONLINE != null) { 183 try { 184 RESUME_IS_SERVER_ONLINE.await(); 185 } catch (InterruptedException e) { 186 throw new RuntimeException(e); 187 } 188 } 189 break; 190 } 191 } 192 } 193 return super.isServerOnline(serverName); 194 } 195 196 @Override 197 public List<ServerName> createDestinationServersList() { 198 return super.createDestinationServersList(EXCLUDE_SERVERS); 199 } 200 } 201 202 public static final class HMasterForTest extends HMaster { 203 204 public HMasterForTest(Configuration conf) throws IOException { 205 super(conf); 206 } 207 208 @Override 209 protected AssignmentManager createAssignmentManager(MasterServices master, 210 MasterRegion masterRegion) { 211 return new AMForTest(master, masterRegion); 212 } 213 214 @Override 215 protected ServerManager createServerManager(MasterServices master, RegionServerList storage) 216 throws IOException { 217 setupClusterConnection(); 218 return new SMForTest(master, storage); 219 } 220 } 221 222 @BeforeClass 223 public static void setUp() throws Exception { 224 UTIL.startMiniCluster(StartMiniClusterOption.builder().numMasters(1) 225 .masterClass(HMasterForTest.class).numRegionServers(3).rsClass(RSForTest.class).build()); 226 UTIL.createTable(NAME, CF); 227 // Here the test region must not be hosted on the same rs with meta region. 228 // We have 3 RSes and only two regions(meta and the test region), so they will not likely to be 229 // hosted on the same RS. 230 UTIL.waitTableAvailable(NAME); 231 UTIL.getAdmin().balancerSwitch(false, true); 232 } 233 234 @AfterClass 235 public static void tearDown() throws Exception { 236 UTIL.shutdownMiniCluster(); 237 } 238 239 @SuppressWarnings("FutureReturnValueIgnored") 240 @Test 241 public void test() throws Exception { 242 RegionInfo region = UTIL.getMiniHBaseCluster().getRegions(NAME).get(0).getRegionInfo(); 243 AssignmentManager am = UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager(); 244 RegionStateNode rsn = am.getRegionStates().getRegionStateNode(region); 245 246 ServerName sn = rsn.getRegionLocation(); 247 RESUME_EXEC_PROC = new CountDownLatch(1); 248 ARRIVE_EXEC_PROC = new CountDownLatch(1); 249 RESUME_IS_SERVER_ONLINE = new CountDownLatch(1); 250 251 // reopen the region, and halt the executeProcedures method at RS side 252 am.moveAsync(new RegionPlan(region, sn, sn)); 253 ARRIVE_EXEC_PROC.await(); 254 255 RESUME_REPORT = new CountDownLatch(1); 256 ARRIVE_REPORT = new CountDownLatch(1); 257 258 // kill the region server 259 ServerName serverToKill = SERVER_TO_KILL; 260 UTIL.getMiniHBaseCluster().stopRegionServer(serverToKill); 261 RESUME_EXEC_PROC.countDown(); 262 263 // wait until we are going to open the region on a new rs 264 ARRIVE_REPORT.await(); 265 266 // resume the isServerOnline check, to let the rs procedure 267 RESUME_IS_SERVER_ONLINE.countDown(); 268 269 // before HBASE-20811 the state could become OPEN, and this is why later the region will be 270 // assigned to two regionservers. 271 for (int i = 0; i < 15; i++) { 272 if (rsn.getState() == RegionState.State.OPEN) { 273 break; 274 } 275 Thread.sleep(1000); 276 } 277 278 // resume the old report 279 RESUME_REPORT.countDown(); 280 281 // wait a bit to let the region to be online, it is not easy to write a condition for this so 282 // just sleep a while. 283 Thread.sleep(10000); 284 285 // confirm that the region is only on one rs 286 int count = 0; 287 for (RegionServerThread t : UTIL.getMiniHBaseCluster().getRegionServerThreads()) { 288 if (!t.getRegionServer().getRegions(NAME).isEmpty()) { 289 LOG.info("{} is on {}", region, t.getRegionServer().getServerName()); 290 count++; 291 } 292 } 293 assertEquals(1, count); 294 } 295}