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 2 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_DEFAULT;
import static org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure.REOPEN_BATCH_SIZE_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,12 @@ 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 batchSize = conf.getInt(REOPEN_BATCH_SIZE_KEY, REOPEN_BATCH_SIZE_DEFAULT);
addChildProcedure(
new ReopenTableRegionsProcedure(getTableName(), backoffMillis, batchSize));
}
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,12 @@ 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_KEY = "hbase.table.regions.reopen.batch.size";
public static final int REOPEN_BATCH_SIZE_DEFAULT = Integer.MAX_VALUE;

private TableName tableName;

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

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

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

private RetryCounter retryCounter;

private final long reopenBatchBackoffMillis;
rmdmattingly marked this conversation as resolved.
Show resolved Hide resolved
private final int reopenBatchSize;

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_DEFAULT);
}

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

public ReopenTableRegionsProcedure(final TableName tableName, final List<byte[]> regionNames,
long reopenBatchBackoffMillis, int reopenBatchSize) {
this.tableName = tableName;
this.regionNames = regionNames;
this.reopenBatchBackoffMillis = reopenBatchBackoffMillis;
this.reopenBatchSize = reopenBatchSize;
}

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

@RestrictedApi(explanation = "Should only be called in tests", link = "",
allowedOnPath = ".*/src/test/.*")
public List<HRegionLocation> getCurrentRegionBatch() {
return new ArrayList<>(currentRegionBatch);
}

private boolean canSchedule(MasterProcedureEnv env, HRegionLocation loc) {
if (loc.getSeqNum() < 0) {
return false;
Expand Down Expand Up @@ -114,7 +144,8 @@ 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) {
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 @@ -139,33 +170,57 @@ protected Flow executeFromState(MasterProcedureEnv env, ReopenTableRegionsState
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);
if (reopenBatchBackoffMillis > 0) {
backoff(reopenBatchBackoffMillis);
}
return Flow.HAS_MORE_STATE;
}
}
if (regions.stream().anyMatch(loc -> canSchedule(env, loc))) {
if (currentRegionBatch.stream().anyMatch(loc -> canSchedule(env, loc))) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I think the changes in this method are sort of confusing. I think the division of labor should be:

REOPEN_TABLE_REGIONS_REOPEN_REGIONS -- breaks the set of regions into a smaller batch, and reopens that batch. The code you have for that looks ok
REOPEN_TABLE_REGIONS_CONFIRM_REOPENED -- checks if any regions are still needing reopen. If so, go back to REOPEN_REGIONS. This is how it used to work.

If we totally reverted the changes in REOPEN_TABLE_REGIONS_CONFIRM_REOPENED, I think we'd have successfully broken the full region list into batches and the flow would work, just without backoff. So in this method we really just need to add the backoff.

The backoff as you have it below looks ok at first glance. This is just a long winded way of saying we should drop the highlighted code :)

Copy link
Contributor Author

@rmdmattingly rmdmattingly Nov 28, 2023

Choose a reason for hiding this comment

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

I think this complexity is necessary in order to differentiate between are there more regions to reopen vs are regions still reopening in the current batch, the former being a state change and fixed backoff, and the latter being a retry and exponentially increasing backoff

Copy link
Contributor

Choose a reason for hiding this comment

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

Let's look back at the original impl of CONFIRM_REOPEN. It only does the backoff if there are still regions to reopen and none of them are schedulable. Meaning we back off in order to wait for regions to be schedulable.

Now let's look at your new code. I see what you are saying. You are basically doing:

  • If currentRegionBatch are all reopened, exit or process the next batch
  • Otherwise, can any of the remaining be scheduled? If not, exponential backoff. If so, process next batch.

The key thing there is that "process next batch". You'd think that would be "re-schedule existing batch". But as is, when you go to process next batch, the first thing REOPEN_REGIONS does is create a new currentRegionBatch. So lets say for the first batch of 50, only 10 reopen. With the current design, you'd expect it to try again to reopen those remaining 40. But it will actually reopen another 50 (40 of which are likely to have been in the last batch).

TBH I don't think there's a huge problem with that approach. But given that's how it works, we can simplify this code as I described before -- we don't really care about currentRegionBatch here, just that if any regions are left to reopen and schedulable.

If we wanted to do what I think you are intending to do, we probably need to complicate things a bit further. Maybe we update REOPEN_REGIONS to only create currentRegionBatch if it's currently null. Then in CONFIRM_REOPEN, null it out after confirming that they've been reopened. That way, in the above example when we kickoff REOPEN_REGIONS again we'll only open the remaining 40.

Does that make sense? Between these 2 options I might opt for the simpler one where we keep REOPEN_REGIONS as is and remove most of the diff from CONFIRM_REOPEN.

Copy link
Contributor

Choose a reason for hiding this comment

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

On second thought, after a discussion on https://issues.apache.org/jira/browse/HBASE-25549, i wonder if we should take the safer approach where require each batch to finish before scheduling more. This feature could serve a dual purpose of progressive rollout and rate limiting.

We could also trivially update this code to do an actual progressive deploy -- first batch size 1, then 2, then 4, etc up to the current batch size config you added. At that point it stays at that max concurrency until completion.

Thoughts on adding that? I think we just need one more field for currentBatchSize which we increment after each batch up to reopenBatchSize. It might make sense to persist this new field in the proto.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Now let's look at your new code. I see what you are saying. You are basically doing:
If currentRegionBatch are all reopened, exit or process the next batch
Otherwise, can any of the remaining be scheduled? If not, exponential backoff. If so, process next batch.

I think that's missing a few distinct outcomes of CONFIRM_REOPENED, and that's where this complexity comes into play. Here are the possibilities:

  1. The current batch has reopened successfully and there are no more regions to produce a next batch, so finish
  2. The current batch has reopened successfully and there are more regions to produce a next batch, so set the next state to REOPEN_REGIONS and suspend. This will process the next batch
  3. The current batch has not reopened successfully, and some of the current batch can still be scheduled. In this case return to REOPEN_REGIONS to avoid an infinite loop due to a bypassed proc or something. This could technically reprocess the next batch, or redefine what the "current batch" looks like, but I think that's ok given the exceptional nature of this case
  4. The current batch has not reopened successfully, and none of the current batch can still be scheduled (because they have already been scheduled). In this case we're just waiting on TRSPs to complete, so retain current state of CONFIRM_REOPENED and retry w/ increasing backoff

If I just revert this diff as described then this procedure will just get stuck in infinite loops (that I think could theoretically be exited if TRSP finishes quickly enough?) and all unit tests fail

We could also trivially update this code to do an actual progressive deploy -- first batch size 1, then 2, then 4, etc up to the current batch size config you added. At that point it stays at that max concurrency until completion.

I like this idea, and I think it would be trivial to add in the current state of this PR. We'd just need to store a maxReopenBatchSize and increase the reopenBatchSize to Math.min(2*reopenBatchSize, maxReopenBatchSize) each time we enter state 2 above

Copy link
Contributor

@bbeaudreault bbeaudreault Nov 29, 2023

Choose a reason for hiding this comment

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

Ok lets table the missing outcomes/simplification discussion in that case, since we'll want to keep the current logic for the progressive reopen. We'll also want to do as i said, where in CONFIRM we update currentRegionBatch to filter out the reopened regions, and then in REOPEN we should only create a new currentRegionBatch if the existing isEmpty. That way if there's some problematic region in the batch, we won't move on to the next batch until its been reopened. Right now I think we'd just move on and create a new batch including the problematic region + more?

Once we have that in there, I need to take another look at the logic here. I think it could be a bit clearer, but don't want to make any suggestions until we have the complete impl in place.

In terms of progressive, it'd be nice to start with 1 region but otherwise agree with your Math.min idea.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sounds good, I just pushed a change which does the following:

  1. each reopen proc will start with a batch size of 1
  2. the batch size will double after each confirmed batch (until we hit our max)
  3. our tests confirm the intended batching behavior, and that each region is only opened once

retryCounter = null;
setNextState(ReopenTableRegionsState.REOPEN_TABLE_REGIONS_REOPEN_REGIONS);
if (reopenBatchBackoffMillis > 0) {
backoff(reopenBatchBackoffMillis);
Copy link
Contributor

@bbeaudreault bbeaudreault Nov 28, 2023

Choose a reason for hiding this comment

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

This is a little confusing to read. Specifically, the way to backoff is by throwing ProcedureSuspendedException but below you are returning Flow.HAS_MORE_STATE. It would be nice to indicate that the return only happens if we don't back-off. One way to do that would be to add a comment below. But it might be better to rename backoff to setBackoffState() and throw a ProcedureSuspendedException here. You're already throwing it below (which is redundant given backoff throws it now)

Copy link
Contributor

Choose a reason for hiding this comment

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

One other thing to note -- since we do skipPersistence(), we won't persist the setNextState change above. I think this should be fine. The implication is that in the case of a crash/restart while the procedure is suspended, we might backoff twice which is not the end of the world.

}
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();
regions.size(), tableName, currentRegionBatch.size(), backoffMillis / 1000);
backoff(backoffMillis);
throw new ProcedureSuspendedException();
default:
throw new UnsupportedOperationException("unhandled state=" + state);
}
}

private void backoff(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