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.junit.Assert.assertEquals;
021import static org.junit.Assert.assertTrue;
022
023import java.io.IOException;
024import java.util.List;
025import java.util.Set;
026import java.util.stream.Collectors;
027import org.apache.hadoop.conf.Configuration;
028import org.apache.hadoop.hbase.HBaseClassTestRule;
029import org.apache.hadoop.hbase.HBaseTestingUtil;
030import org.apache.hadoop.hbase.TableName;
031import org.apache.hadoop.hbase.client.RegionInfo;
032import org.apache.hadoop.hbase.master.RegionState.State;
033import org.apache.hadoop.hbase.master.ServerManager;
034import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
035import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
036import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
037import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
038import org.apache.hadoop.hbase.testclassification.MasterTests;
039import org.apache.hadoop.hbase.testclassification.MediumTests;
040import org.apache.hadoop.hbase.util.Bytes;
041import org.junit.AfterClass;
042import org.junit.BeforeClass;
043import org.junit.ClassRule;
044import org.junit.Test;
045import org.junit.experimental.categories.Category;
046
047import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
048
049/**
050 * Confirm that we will batch region reopens when reopening all table regions. This can avoid the
051 * pain associated with reopening too many regions at once.
052 */
053@Category({ MasterTests.class, MediumTests.class })
054public class TestReopenTableRegionsProcedureBatching {
055
056  @ClassRule
057  public static final HBaseClassTestRule CLASS_RULE =
058    HBaseClassTestRule.forClass(TestReopenTableRegionsProcedureBatching.class);
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  @BeforeClass
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  @AfterClass
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  private void confirmBatchSize(int expectedBatchSize, Set<StuckRegion> stuckRegions,
164    ReopenTableRegionsProcedure proc) {
165    while (true) {
166      if (proc.getBatchesProcessed() == 0) {
167        continue;
168      }
169      stuckRegions.forEach(this::unstickRegion);
170      UTIL.waitFor(5000, () -> expectedBatchSize == proc.getRegionsReopened());
171      break;
172    }
173  }
174
175  static class StuckRegion {
176    final TransitRegionStateProcedure trsp;
177    final RegionStateNode regionNode;
178    final long openSeqNum;
179
180    public StuckRegion(TransitRegionStateProcedure trsp, RegionStateNode regionNode,
181      long openSeqNum) {
182      this.trsp = trsp;
183      this.regionNode = regionNode;
184      this.openSeqNum = openSeqNum;
185    }
186  }
187
188  private StuckRegion stickRegion(AssignmentManager am,
189    ProcedureExecutor<MasterProcedureEnv> procExec, RegionInfo regionInfo) {
190    RegionStateNode regionNode = am.getRegionStates().getRegionStateNode(regionInfo);
191    TransitRegionStateProcedure trsp =
192      TransitRegionStateProcedure.unassign(procExec.getEnvironment(), regionInfo);
193    regionNode.lock();
194    long openSeqNum;
195    try {
196      openSeqNum = regionNode.getOpenSeqNum();
197      regionNode.setState(State.OPENING);
198      regionNode.setOpenSeqNum(-1L);
199      regionNode.setProcedure(trsp);
200    } finally {
201      regionNode.unlock();
202    }
203    return new StuckRegion(trsp, regionNode, openSeqNum);
204  }
205
206  private void unstickRegion(StuckRegion stuckRegion) {
207    stuckRegion.regionNode.lock();
208    try {
209      stuckRegion.regionNode.setState(State.OPEN);
210      stuckRegion.regionNode.setOpenSeqNum(stuckRegion.openSeqNum);
211      stuckRegion.regionNode.unsetProcedure(stuckRegion.trsp);
212    } finally {
213      stuckRegion.regionNode.unlock();
214    }
215  }
216}