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.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_OPENED_VALUE; 021 022import java.io.IOException; 023import java.util.concurrent.CountDownLatch; 024import java.util.concurrent.Future; 025import java.util.concurrent.atomic.AtomicReference; 026import org.apache.hadoop.conf.Configuration; 027import org.apache.hadoop.hbase.HBaseClassTestRule; 028import org.apache.hadoop.hbase.HBaseTestingUtility; 029import org.apache.hadoop.hbase.HConstants; 030import org.apache.hadoop.hbase.PleaseHoldException; 031import org.apache.hadoop.hbase.TableName; 032import org.apache.hadoop.hbase.client.Put; 033import org.apache.hadoop.hbase.client.RegionInfo; 034import org.apache.hadoop.hbase.client.Table; 035import org.apache.hadoop.hbase.master.HMaster; 036import org.apache.hadoop.hbase.master.MasterServices; 037import org.apache.hadoop.hbase.master.RegionPlan; 038import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; 039import org.apache.hadoop.hbase.master.region.MasterRegion; 040import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; 041import org.apache.hadoop.hbase.testclassification.MasterTests; 042import org.apache.hadoop.hbase.testclassification.MediumTests; 043import org.apache.hadoop.hbase.util.Bytes; 044import org.junit.AfterClass; 045import org.junit.BeforeClass; 046import org.junit.ClassRule; 047import org.junit.Test; 048import org.junit.experimental.categories.Category; 049 050import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest; 051import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse; 052 053@Category({ MasterTests.class, MediumTests.class }) 054public class TestReportRegionStateTransitionRetry { 055 056 @ClassRule 057 public static final HBaseClassTestRule CLASS_RULE = 058 HBaseClassTestRule.forClass(TestReportRegionStateTransitionRetry.class); 059 060 private static final AtomicReference<CountDownLatch> RESUME_AND_FAIL = new AtomicReference<>(); 061 062 private static final class AssignmentManagerForTest extends AssignmentManager { 063 064 public AssignmentManagerForTest(MasterServices master, MasterRegion masterRegion) { 065 super(master, masterRegion); 066 } 067 068 @Override 069 public ReportRegionStateTransitionResponse reportRegionStateTransition( 070 ReportRegionStateTransitionRequest req) throws PleaseHoldException { 071 ReportRegionStateTransitionResponse resp = super.reportRegionStateTransition(req); 072 CountDownLatch latch = RESUME_AND_FAIL.getAndSet(null); 073 if (latch != null) { 074 try { 075 latch.await(); 076 } catch (InterruptedException e) { 077 throw new RuntimeException(e); 078 } 079 throw new PleaseHoldException("Inject error"); 080 } 081 return resp; 082 } 083 } 084 085 public static final class HMasterForTest extends HMaster { 086 087 public HMasterForTest(Configuration conf) throws IOException { 088 super(conf); 089 } 090 091 @Override 092 protected AssignmentManager createAssignmentManager(MasterServices master, 093 MasterRegion masterRegion) { 094 return new AssignmentManagerForTest(master, masterRegion); 095 } 096 } 097 098 private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); 099 100 private static TableName NAME = TableName.valueOf("Retry"); 101 102 private static byte[] CF = Bytes.toBytes("cf"); 103 104 @BeforeClass 105 public static void setUp() throws Exception { 106 UTIL.getConfiguration().setClass(HConstants.MASTER_IMPL, HMasterForTest.class, HMaster.class); 107 UTIL.startMiniCluster(1); 108 UTIL.createTable(NAME, CF); 109 UTIL.waitTableAvailable(NAME); 110 } 111 112 @AfterClass 113 public static void tearDown() throws Exception { 114 UTIL.shutdownMiniCluster(); 115 } 116 117 @Test 118 public void testRetryOnClose() throws Exception { 119 RegionInfo region = UTIL.getMiniHBaseCluster().getRegions(NAME).get(0).getRegionInfo(); 120 ProcedureExecutor<MasterProcedureEnv> procExec = 121 UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor(); 122 AssignmentManager am = UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager(); 123 RegionStateNode rsn = am.getRegionStates().getRegionStateNode(region); 124 125 CountDownLatch latch = new CountDownLatch(1); 126 RESUME_AND_FAIL.set(latch); 127 Future<byte[]> future = 128 am.moveAsync(new RegionPlan(region, rsn.getRegionLocation(), rsn.getRegionLocation())); 129 TransitRegionStateProcedure proc = 130 procExec.getProcedures().stream().filter(p -> p instanceof TransitRegionStateProcedure) 131 .filter(p -> !p.isFinished()).map(p -> (TransitRegionStateProcedure) p).findAny().get(); 132 133 // wait until we schedule the OpenRegionProcedure 134 UTIL.waitFor(10000, 135 () -> proc.getCurrentStateId() == REGION_STATE_TRANSITION_CONFIRM_OPENED_VALUE); 136 // Fail the reportRegionStateTransition for closing 137 latch.countDown(); 138 future.get(); 139 140 // confirm that the region can still be write 141 try (Table table = UTIL.getConnection().getTableBuilder(NAME, null).setWriteRpcTimeout(1000) 142 .setOperationTimeout(2000).build()) { 143 table.put( 144 new Put(Bytes.toBytes("key")).addColumn(CF, Bytes.toBytes("cq"), Bytes.toBytes("val"))); 145 } 146 } 147}