Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HBASE-28215: region reopen procedure batching/throttling #5534

Merged
merged 9 commits into from
Dec 4, 2023
Merged
Show file tree
Hide file tree
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,11 @@
*/
package org.apache.hadoop.hbase.master.procedure;

import static org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure.REOPEN_BATCH_BACKOFF_MILLIS_DEFAULT;
import static org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure.REOPEN_BATCH_BACKOFF_MILLIS_KEY;
import static org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure.REOPEN_BATCH_SIZE_MAX_DEFAULT;
import static org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure.REOPEN_BATCH_SIZE_MAX_KEY;

import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
Expand All @@ -25,6 +30,7 @@
import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ConcurrentTableModificationException;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseIOException;
Expand Down Expand Up @@ -147,7 +153,13 @@ protected Flow executeFromState(final MasterProcedureEnv env, final ModifyTableS
break;
case MODIFY_TABLE_REOPEN_ALL_REGIONS:
if (isTableEnabled(env)) {
addChildProcedure(new ReopenTableRegionsProcedure(getTableName()));
Configuration conf = env.getMasterConfiguration();
long backoffMillis =
conf.getLong(REOPEN_BATCH_BACKOFF_MILLIS_KEY, REOPEN_BATCH_BACKOFF_MILLIS_DEFAULT);
int batchSizeMax =
conf.getInt(REOPEN_BATCH_SIZE_MAX_KEY, REOPEN_BATCH_SIZE_MAX_DEFAULT);
addChildProcedure(
new ReopenTableRegionsProcedure(getTableName(), backoffMillis, batchSizeMax));
}
setNextState(ModifyTableState.MODIFY_TABLE_ASSIGN_NEW_REPLICAS);
break;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,12 @@
*/
package org.apache.hadoop.hbase.master.procedure;

import com.google.errorprone.annotations.RestrictedApi;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.TableName;
Expand Down Expand Up @@ -53,6 +55,16 @@ public class ReopenTableRegionsProcedure

private static final Logger LOG = LoggerFactory.getLogger(ReopenTableRegionsProcedure.class);

public static final String REOPEN_BATCH_BACKOFF_MILLIS_KEY =
"hbase.table.regions.reopen.batch.backoff.ms";
public static final long REOPEN_BATCH_BACKOFF_MILLIS_DEFAULT = 0L;
public static final String REOPEN_BATCH_SIZE_MAX_KEY =
"hbase.table.regions.reopen.batch.size.max";
public static final int REOPEN_BATCH_SIZE_MAX_DEFAULT = Integer.MAX_VALUE;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nitpick: we might want to have these names be hbase.reopen.table.regions.progressive.batch.size.max and backoff.ms. It's more of a progressive reopen system now, and i reordered the table.regions.reopen to align more with the procedure name.


// this minimum prevents a max which would break this procedure
private static final int MINIMUM_BATCH_SIZE_MAX = 1;

private TableName tableName;

// Specify specific regions of a table to reopen.
Expand All @@ -61,20 +73,41 @@ public class ReopenTableRegionsProcedure

private List<HRegionLocation> regions = Collections.emptyList();

private List<HRegionLocation> currentRegionBatch = Collections.emptyList();

private RetryCounter retryCounter;

private long reopenBatchBackoffMillis;
private int reopenBatchSize;
private int reopenBatchSizeMax;
private long regionsReopened = 0;
private long batchesProcessed = 0;

public ReopenTableRegionsProcedure() {
regionNames = Collections.emptyList();
this(null);
}

public ReopenTableRegionsProcedure(TableName tableName) {
this.tableName = tableName;
this.regionNames = Collections.emptyList();
this(tableName, Collections.emptyList());
}

public ReopenTableRegionsProcedure(final TableName tableName, final List<byte[]> regionNames) {
this(tableName, regionNames, REOPEN_BATCH_BACKOFF_MILLIS_DEFAULT,
REOPEN_BATCH_SIZE_MAX_DEFAULT);
}

public ReopenTableRegionsProcedure(final TableName tableName, long reopenBatchBackoffMillis,
int reopenBatchSizeMax) {
this(tableName, Collections.emptyList(), reopenBatchBackoffMillis, reopenBatchSizeMax);
}

public ReopenTableRegionsProcedure(final TableName tableName, final List<byte[]> regionNames,
long reopenBatchBackoffMillis, int reopenBatchSizeMax) {
this.tableName = tableName;
this.regionNames = regionNames;
this.reopenBatchBackoffMillis = reopenBatchBackoffMillis;
this.reopenBatchSize = 1;
this.reopenBatchSizeMax = Math.max(reopenBatchSizeMax, MINIMUM_BATCH_SIZE_MAX);
bbeaudreault marked this conversation as resolved.
Show resolved Hide resolved
}

@Override
Expand All @@ -87,6 +120,18 @@ public TableOperationType getTableOperationType() {
return TableOperationType.REGION_EDIT;
}

@RestrictedApi(explanation = "Should only be called in tests", link = "",
allowedOnPath = ".*/src/test/.*")
public long getRegionsReopened() {
return regionsReopened;
}

@RestrictedApi(explanation = "Should only be called in tests", link = "",
allowedOnPath = ".*/src/test/.*")
public long getBatchesProcessed() {
return batchesProcessed;
}

private boolean canSchedule(MasterProcedureEnv env, HRegionLocation loc) {
if (loc.getSeqNum() < 0) {
return false;
Expand Down Expand Up @@ -114,7 +159,11 @@ protected Flow executeFromState(MasterProcedureEnv env, ReopenTableRegionsState
setNextState(ReopenTableRegionsState.REOPEN_TABLE_REGIONS_REOPEN_REGIONS);
return Flow.HAS_MORE_STATE;
case REOPEN_TABLE_REGIONS_REOPEN_REGIONS:
for (HRegionLocation loc : regions) {
if (!regions.isEmpty()) {
batchesProcessed++;
}
currentRegionBatch = regions.stream().limit(reopenBatchSize).collect(Collectors.toList());
for (HRegionLocation loc : currentRegionBatch) {
RegionStateNode regionNode =
env.getAssignmentManager().getRegionStates().getRegionStateNode(loc.getRegion());
// this possible, maybe the region has already been merged or split, see HBASE-20921
Expand All @@ -133,39 +182,66 @@ protected Flow executeFromState(MasterProcedureEnv env, ReopenTableRegionsState
regionNode.unlock();
}
addChildProcedure(proc);
regionsReopened++;
}
setNextState(ReopenTableRegionsState.REOPEN_TABLE_REGIONS_CONFIRM_REOPENED);
return Flow.HAS_MORE_STATE;
case REOPEN_TABLE_REGIONS_CONFIRM_REOPENED:
regions = regions.stream().map(env.getAssignmentManager().getRegionStates()::checkReopened)
.filter(l -> l != null).collect(Collectors.toList());
if (regions.isEmpty()) {
return Flow.NO_MORE_STATE;
// we need to create a set of region names because the HRegionLocation hashcode is only
// based
// on the server name
Set<byte[]> currentRegionBatchNames = currentRegionBatch.stream()
.map(r -> r.getRegion().getRegionName()).collect(Collectors.toSet());
currentRegionBatch = regions.stream()
.filter(r -> currentRegionBatchNames.contains(r.getRegion().getRegionName()))
.collect(Collectors.toList());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Are you sure we need to do this? What if we did something like this:

  case REOPEN_TABLE_REGIONS_CONFIRM_REOPENED:
    // update region lists based on what's been reopened
    regions = filterReopened(regions);
    currentRegionBatch = filterReopened(currentRegionBatch)

    // existing batch didn't fully reopen, so try to resolve that first.
    // since this is a retry, don't do the batch backoff
    if (!currentRegionBatch.isEmpty()) {
      return reopenIfSchedulable(currentRegionBatch, false);
    }

   if (regions.isEmpty()) {
     return Flow.NO_MORE_STATE;
   }

    // no batching or batch is finished, schedule more from main regions
    return reopenIfSchedulable(regions, true);
  default:
    throw new UnsupportedOperationException("unhandled state=" + state);
}

...

private List<HRegionLocation> filterReopened(List<HRegionLocation> regionsToCheck) {
  return regionsToCheck.stream().map(env.getAssignmentManager().getRegionStates()::checkReopened)
          .filter(l -> l != null).collect(Collectors.toList());
}

private void reopenIfSchedulable(List<HRegionLocations> regionsToReopen, boolean shouldBatchBackoff) throws ProcedureSuspendedException {
  if (regionsToReopen.stream().anyMatch(loc -> canSchedule(env, loc))) {
    retryCounter = null;
    setNextState(ReopenTableRegionsState.REOPEN_TABLE_REGIONS_REOPEN_REGIONS);
    if (shouldBatchBackoff && reopenBatchBackoffMillis > 0) {
      reopenBatchSize = Math.min(reopenBatchSizeMax, 2 * reopenBatchSize);
      setBackoffStateAndSuspend(reopenBatchBackoffMillis);
    } else {
      return Flow.HAS_MORE_STATE;
    }
  }

  // We can not schedule TRSP for all the regions need to reopen, wait for a while and retry
  // again.
  if (retryCounter == null) {
    retryCounter = ProcedureUtil.createRetryCounter(env.getMasterConfiguration());
  }
  long backoffMillis = retryCounter.getBackoffTimeAndIncrementAttempts();
  LOG.info(
    "There are still {} region(s) which need to be reopened for table {}. {} are in "
      + "OPENING state, suspend {}secs and try again later",
    regions.size(), tableName, regionsToCheck.size(), backoffMillis / 1000);
  setBackoffStateAndSuspend(backoffMillis);
}

It feels cleaner to read and understand what's happening. As mentioned in another comment, I'd recommend updating REOPEN_REGIONS to:

// if we didn't finish reopening the last batch yet, let's keep trying until we do.
// at that point, the batch will be empty and we can generate a new batch
if (currentRegionBatch.isEmpty()) {
  currentRegionBatch = regions.stream().limit(reopenBatchSize).collect(Collectors.toList());
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agreed all around about the feedback here, just pushed a simplification that follows this advice almost exactly. The tests are all still passing of course

if (currentRegionBatch.isEmpty()) {
if (regions.isEmpty()) {
return Flow.NO_MORE_STATE;
} else {
setNextState(ReopenTableRegionsState.REOPEN_TABLE_REGIONS_REOPEN_REGIONS);
reopenBatchSize = Math.min(reopenBatchSizeMax, 2 * reopenBatchSize);
if (reopenBatchBackoffMillis > 0) {
setBackoffStateAndSuspend(reopenBatchBackoffMillis);
} else {
return Flow.HAS_MORE_STATE;
}
}
}
if (regions.stream().anyMatch(loc -> canSchedule(env, loc))) {
if (currentRegionBatch.stream().anyMatch(loc -> canSchedule(env, loc))) {
retryCounter = null;
setNextState(ReopenTableRegionsState.REOPEN_TABLE_REGIONS_REOPEN_REGIONS);
return Flow.HAS_MORE_STATE;
if (reopenBatchBackoffMillis > 0) {
setBackoffStateAndSuspend(reopenBatchBackoffMillis);
} else {
return Flow.HAS_MORE_STATE;
}
}
// We can not schedule TRSP for all the regions need to reopen, wait for a while and retry
// again.
if (retryCounter == null) {
retryCounter = ProcedureUtil.createRetryCounter(env.getMasterConfiguration());
}
long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
long backoffMillis = retryCounter.getBackoffTimeAndIncrementAttempts();
LOG.info(
"There are still {} region(s) which need to be reopened for table {} are in "
"There are still {} region(s) which need to be reopened for table {}. {} are in "
+ "OPENING state, suspend {}secs and try again later",
regions.size(), tableName, backoff / 1000);
setTimeout(Math.toIntExact(backoff));
setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT);
skipPersistence();
throw new ProcedureSuspendedException();
regions.size(), tableName, currentRegionBatch.size(), backoffMillis / 1000);
setBackoffStateAndSuspend(backoffMillis);
default:
throw new UnsupportedOperationException("unhandled state=" + state);
}
}

private void setBackoffStateAndSuspend(long millis) throws ProcedureSuspendedException {
setTimeout(Math.toIntExact(millis));
setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT);
skipPersistence();
bbeaudreault marked this conversation as resolved.
Show resolved Hide resolved
throw new ProcedureSuspendedException();
}

private List<HRegionLocation>
getRegionLocationsForReopen(List<HRegionLocation> tableRegionsForReopen) {

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,103 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.procedure;

import static org.junit.Assert.assertTrue;

import java.io.IOException;
import java.time.Duration;
import java.time.Instant;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;

/**
* Confirm that we will rate limit reopen batches when reopening all table regions. This can avoid
* the pain associated with reopening too many regions at once.
*/
@Category({ MasterTests.class, MediumTests.class })
public class TestReopenTableRegionsProcedureBatchBackoff {

@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestReopenTableRegionsProcedureBatchBackoff.class);

private static final HBaseTestingUtil UTIL = new HBaseTestingUtil();

private static TableName TABLE_NAME = TableName.valueOf("BatchBackoff");
private static final int BACKOFF_MILLIS_PER_RS = 3_000;
private static final int REOPEN_BATCH_SIZE = 1;

private static byte[] CF = Bytes.toBytes("cf");

@BeforeClass
public static void setUp() throws Exception {
Configuration conf = UTIL.getConfiguration();
conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
UTIL.startMiniCluster(1);
UTIL.createMultiRegionTable(TABLE_NAME, CF, 10);
}

@AfterClass
public static void tearDown() throws Exception {
UTIL.shutdownMiniCluster();
}

@Test
public void testRegionBatchBackoff() throws IOException {
ProcedureExecutor<MasterProcedureEnv> procExec =
UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
List<RegionInfo> regions = UTIL.getAdmin().getRegions(TABLE_NAME);
assertTrue(10 <= regions.size());
ReopenTableRegionsProcedure proc =
new ReopenTableRegionsProcedure(TABLE_NAME, BACKOFF_MILLIS_PER_RS, REOPEN_BATCH_SIZE);
procExec.submitProcedure(proc);
Instant startedAt = Instant.now();
ProcedureSyncWait.waitForProcedureToComplete(procExec, proc, 60_000);
Instant stoppedAt = Instant.now();
assertTrue(Duration.between(startedAt, stoppedAt).toMillis()
> (long) regions.size() * BACKOFF_MILLIS_PER_RS);
}

@Test
public void testRegionBatchNoBackoff() throws IOException {
ProcedureExecutor<MasterProcedureEnv> procExec =
UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
List<RegionInfo> regions = UTIL.getAdmin().getRegions(TABLE_NAME);
assertTrue(10 <= regions.size());
int noBackoffMillis = 0;
ReopenTableRegionsProcedure proc =
new ReopenTableRegionsProcedure(TABLE_NAME, noBackoffMillis, REOPEN_BATCH_SIZE);
procExec.submitProcedure(proc);
ProcedureSyncWait.waitForProcedureToComplete(procExec, proc,
(long) regions.size() * BACKOFF_MILLIS_PER_RS);
}
}
Loading