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.procedure; 019 020import static org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure.PROGRESSIVE_BATCH_BACKOFF_MILLIS_DEFAULT; 021import static org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure.PROGRESSIVE_BATCH_BACKOFF_MILLIS_KEY; 022import static org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure.PROGRESSIVE_BATCH_SIZE_MAX_KEY; 023import static org.junit.jupiter.api.Assertions.assertEquals; 024import static org.junit.jupiter.api.Assertions.assertTrue; 025 026import java.io.IOException; 027import java.util.List; 028import java.util.Set; 029import java.util.stream.Collectors; 030import org.apache.hadoop.conf.Configuration; 031import org.apache.hadoop.hbase.HBaseConfiguration; 032import org.apache.hadoop.hbase.HBaseTestingUtil; 033import org.apache.hadoop.hbase.TableName; 034import org.apache.hadoop.hbase.client.RegionInfo; 035import org.apache.hadoop.hbase.client.TableDescriptorBuilder; 036import org.apache.hadoop.hbase.master.RegionState.State; 037import org.apache.hadoop.hbase.master.ServerManager; 038import org.apache.hadoop.hbase.master.assignment.AssignmentManager; 039import org.apache.hadoop.hbase.master.assignment.RegionStateNode; 040import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure; 041import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; 042import org.apache.hadoop.hbase.testclassification.MasterTests; 043import org.apache.hadoop.hbase.testclassification.MediumTests; 044import org.apache.hadoop.hbase.util.Bytes; 045import org.junit.jupiter.api.AfterAll; 046import org.junit.jupiter.api.BeforeAll; 047import org.junit.jupiter.api.Tag; 048import org.junit.jupiter.api.Test; 049 050import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState; 051 052/** 053 * Confirm that we will batch region reopens when reopening all table regions. This can avoid the 054 * pain associated with reopening too many regions at once. 055 */ 056@Tag(MasterTests.TAG) 057@Tag(MediumTests.TAG) 058public class TestReopenTableRegionsProcedureBatching { 059 060 private static final HBaseTestingUtil UTIL = new HBaseTestingUtil(); 061 private static final int BACKOFF_MILLIS_PER_RS = 0; 062 private static final int REOPEN_BATCH_SIZE_MAX = 1; 063 064 private static TableName TABLE_NAME = TableName.valueOf("Batching"); 065 066 private static byte[] CF = Bytes.toBytes("cf"); 067 068 @BeforeAll 069 public static void setUp() throws Exception { 070 Configuration conf = UTIL.getConfiguration(); 071 conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1); 072 UTIL.startMiniCluster(1); 073 UTIL.createMultiRegionTable(TABLE_NAME, CF); 074 } 075 076 @AfterAll 077 public static void tearDown() throws Exception { 078 UTIL.shutdownMiniCluster(); 079 } 080 081 @Test 082 public void testSmallMaxBatchSize() throws IOException { 083 AssignmentManager am = UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager(); 084 ProcedureExecutor<MasterProcedureEnv> procExec = 085 UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor(); 086 List<RegionInfo> regions = UTIL.getAdmin().getRegions(TABLE_NAME); 087 assertTrue(2 <= regions.size()); 088 Set<StuckRegion> stuckRegions = 089 regions.stream().map(r -> stickRegion(am, procExec, r)).collect(Collectors.toSet()); 090 ReopenTableRegionsProcedure proc = 091 new ReopenTableRegionsProcedure(TABLE_NAME, BACKOFF_MILLIS_PER_RS, REOPEN_BATCH_SIZE_MAX); 092 procExec.submitProcedure(proc); 093 UTIL.waitFor(10000, () -> proc.getState() == ProcedureState.WAITING_TIMEOUT); 094 095 // the first batch should be small 096 confirmBatchSize(1, stuckRegions, proc); 097 ProcedureSyncWait.waitForProcedureToComplete(procExec, proc, 60_000); 098 099 // other batches should also be small 100 assertTrue(proc.getBatchesProcessed() >= regions.size()); 101 102 // all regions should only be opened once 103 assertEquals(proc.getRegionsReopened(), regions.size()); 104 } 105 106 @Test 107 public void testDefaultMaxBatchSize() throws IOException { 108 AssignmentManager am = UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager(); 109 ProcedureExecutor<MasterProcedureEnv> procExec = 110 UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor(); 111 List<RegionInfo> regions = UTIL.getAdmin().getRegions(TABLE_NAME); 112 assertTrue(2 <= regions.size()); 113 Set<StuckRegion> stuckRegions = 114 regions.stream().map(r -> stickRegion(am, procExec, r)).collect(Collectors.toSet()); 115 ReopenTableRegionsProcedure proc = new ReopenTableRegionsProcedure(TABLE_NAME); 116 procExec.submitProcedure(proc); 117 UTIL.waitFor(10000, () -> proc.getState() == ProcedureState.WAITING_TIMEOUT); 118 119 // the first batch should be large 120 confirmBatchSize(regions.size(), stuckRegions, proc); 121 ProcedureSyncWait.waitForProcedureToComplete(procExec, proc, 60_000); 122 123 // all regions should only be opened once 124 assertEquals(proc.getRegionsReopened(), regions.size()); 125 } 126 127 @Test 128 public void testNegativeBatchSizeDoesNotBreak() throws IOException { 129 AssignmentManager am = UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager(); 130 ProcedureExecutor<MasterProcedureEnv> procExec = 131 UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor(); 132 List<RegionInfo> regions = UTIL.getAdmin().getRegions(TABLE_NAME); 133 assertTrue(2 <= regions.size()); 134 Set<StuckRegion> stuckRegions = 135 regions.stream().map(r -> stickRegion(am, procExec, r)).collect(Collectors.toSet()); 136 ReopenTableRegionsProcedure proc = 137 new ReopenTableRegionsProcedure(TABLE_NAME, BACKOFF_MILLIS_PER_RS, -100); 138 procExec.submitProcedure(proc); 139 UTIL.waitFor(10000, () -> proc.getState() == ProcedureState.WAITING_TIMEOUT); 140 141 // the first batch should be small 142 confirmBatchSize(1, stuckRegions, proc); 143 ProcedureSyncWait.waitForProcedureToComplete(procExec, proc, 60_000); 144 145 // other batches should also be small 146 assertTrue(proc.getBatchesProcessed() >= regions.size()); 147 148 // all regions should only be opened once 149 assertEquals(proc.getRegionsReopened(), regions.size()); 150 } 151 152 @Test 153 public void testBatchSizeDoesNotOverflow() { 154 ReopenTableRegionsProcedure proc = 155 new ReopenTableRegionsProcedure(TABLE_NAME, BACKOFF_MILLIS_PER_RS, Integer.MAX_VALUE); 156 int currentBatchSize = 1; 157 while (currentBatchSize < Integer.MAX_VALUE) { 158 currentBatchSize = proc.progressBatchSize(); 159 assertTrue(currentBatchSize > 0); 160 } 161 } 162 163 @Test 164 public void testBackoffConfigurationFromTableDescriptor() { 165 Configuration conf = HBaseConfiguration.create(); 166 TableDescriptorBuilder tbd = TableDescriptorBuilder.newBuilder(TABLE_NAME); 167 168 // Default (no batching, no backoff) 169 ReopenTableRegionsProcedure proc = ReopenTableRegionsProcedure.throttled(conf, tbd.build()); 170 assertEquals(PROGRESSIVE_BATCH_BACKOFF_MILLIS_DEFAULT, proc.getReopenBatchBackoffMillis()); 171 assertEquals(Integer.MAX_VALUE, proc.progressBatchSize()); 172 173 // From Configuration (backoff: 100ms, max: 6) 174 conf.setLong(PROGRESSIVE_BATCH_BACKOFF_MILLIS_KEY, 100); 175 conf.setInt(PROGRESSIVE_BATCH_SIZE_MAX_KEY, 6); 176 proc = ReopenTableRegionsProcedure.throttled(conf, tbd.build()); 177 assertEquals(100, proc.getReopenBatchBackoffMillis()); 178 assertEquals(2, proc.progressBatchSize()); 179 assertEquals(4, proc.progressBatchSize()); 180 assertEquals(6, proc.progressBatchSize()); 181 assertEquals(6, proc.progressBatchSize()); 182 183 // From TableDescriptor (backoff: 200ms, max: 7) 184 tbd.setValue(PROGRESSIVE_BATCH_BACKOFF_MILLIS_KEY, "200"); 185 tbd.setValue(PROGRESSIVE_BATCH_SIZE_MAX_KEY, "7"); 186 proc = ReopenTableRegionsProcedure.throttled(conf, tbd.build()); 187 assertEquals(200, proc.getReopenBatchBackoffMillis()); 188 assertEquals(2, proc.progressBatchSize()); 189 assertEquals(4, proc.progressBatchSize()); 190 assertEquals(7, proc.progressBatchSize()); 191 assertEquals(7, proc.progressBatchSize()); 192 } 193 194 private void confirmBatchSize(int expectedBatchSize, Set<StuckRegion> stuckRegions, 195 ReopenTableRegionsProcedure proc) { 196 while (true) { 197 if (proc.getBatchesProcessed() == 0) { 198 continue; 199 } 200 stuckRegions.forEach(this::unstickRegion); 201 UTIL.waitFor(5000, () -> expectedBatchSize == proc.getRegionsReopened()); 202 break; 203 } 204 } 205 206 static class StuckRegion { 207 final TransitRegionStateProcedure trsp; 208 final RegionStateNode regionNode; 209 final long openSeqNum; 210 211 public StuckRegion(TransitRegionStateProcedure trsp, RegionStateNode regionNode, 212 long openSeqNum) { 213 this.trsp = trsp; 214 this.regionNode = regionNode; 215 this.openSeqNum = openSeqNum; 216 } 217 } 218 219 private StuckRegion stickRegion(AssignmentManager am, 220 ProcedureExecutor<MasterProcedureEnv> procExec, RegionInfo regionInfo) { 221 RegionStateNode regionNode = am.getRegionStates().getRegionStateNode(regionInfo); 222 TransitRegionStateProcedure trsp = 223 TransitRegionStateProcedure.unassign(procExec.getEnvironment(), regionInfo); 224 regionNode.lock(); 225 long openSeqNum; 226 try { 227 openSeqNum = regionNode.getOpenSeqNum(); 228 regionNode.setState(State.OPENING); 229 regionNode.setOpenSeqNum(-1L); 230 regionNode.setProcedure(trsp); 231 } finally { 232 regionNode.unlock(); 233 } 234 return new StuckRegion(trsp, regionNode, openSeqNum); 235 } 236 237 private void unstickRegion(StuckRegion stuckRegion) { 238 stuckRegion.regionNode.lock(); 239 try { 240 stuckRegion.regionNode.setState(State.OPEN); 241 stuckRegion.regionNode.setOpenSeqNum(stuckRegion.openSeqNum); 242 stuckRegion.regionNode.unsetProcedure(stuckRegion.trsp); 243 } finally { 244 stuckRegion.regionNode.unlock(); 245 } 246 } 247}