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.regionserver;
019
020import org.apache.hadoop.conf.Configuration;
021import org.apache.hadoop.hbase.HBaseInterfaceAudience;
022import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
023import org.apache.yetus.audience.InterfaceAudience;
024import org.slf4j.Logger;
025import org.slf4j.LoggerFactory;
026
027/**
028 * This class represents a split policy which makes the split decision based on how busy a region
029 * is. The metric that is used here is the fraction of total write requests that are blocked due to
030 * high memstore utilization. This fractional rate is calculated over a running window of
031 * "hbase.busy.policy.aggWindow" milliseconds. The rate is a time-weighted aggregated average of the
032 * rate in the current window and the true average rate in the previous window.
033 */
034
035@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
036public class BusyRegionSplitPolicy extends IncreasingToUpperBoundRegionSplitPolicy {
037
038  private static final Logger LOG = LoggerFactory.getLogger(BusyRegionSplitPolicy.class);
039
040  // Maximum fraction blocked write requests before region is considered for split
041  private float maxBlockedRequests;
042  public static final float DEFAULT_MAX_BLOCKED_REQUESTS = 0.2f;
043
044  // Minimum age of the region in milliseconds before it is considered for split
045  private long minAge = -1;
046  public static final long DEFAULT_MIN_AGE_MS = 600000; // 10 minutes
047
048  // The window time in milliseconds over which the blocked requests rate is calculated
049  private long aggregationWindow;
050  public static final long DEFAULT_AGGREGATION_WINDOW = 300000; // 5 minutes
051
052  private HRegion region;
053  private long prevTime;
054  private long startTime;
055  private long writeRequestCount;
056  private long blockedRequestCount;
057  private float blockedRate;
058
059  @Override
060  public String toString() {
061    return "BusyRegionSplitPolicy{" + "maxBlockedRequests=" + maxBlockedRequests + ", minAge="
062      + minAge + ", aggregationWindow=" + aggregationWindow + ", " + super.toString() + '}';
063  }
064
065  @Override
066  protected void configureForRegion(final HRegion region) {
067    super.configureForRegion(region);
068    this.region = region;
069    Configuration conf = getConf();
070
071    maxBlockedRequests =
072      conf.getFloat("hbase.busy.policy.blockedRequests", DEFAULT_MAX_BLOCKED_REQUESTS);
073    minAge = conf.getLong("hbase.busy.policy.minAge", DEFAULT_MIN_AGE_MS);
074    aggregationWindow = conf.getLong("hbase.busy.policy.aggWindow", DEFAULT_AGGREGATION_WINDOW);
075
076    if (maxBlockedRequests < 0.00001f || maxBlockedRequests > 0.99999f) {
077      LOG.warn("Threshold for maximum blocked requests is set too low or too high, "
078        + " resetting to default of " + DEFAULT_MAX_BLOCKED_REQUESTS);
079      maxBlockedRequests = DEFAULT_MAX_BLOCKED_REQUESTS;
080    }
081
082    if (aggregationWindow <= 0) {
083      LOG.warn("Aggregation window size is too low: " + aggregationWindow
084        + ". Resetting it to default of " + DEFAULT_AGGREGATION_WINDOW);
085      aggregationWindow = DEFAULT_AGGREGATION_WINDOW;
086    }
087
088    init();
089  }
090
091  private synchronized void init() {
092    startTime = EnvironmentEdgeManager.currentTime();
093    prevTime = startTime;
094    blockedRequestCount = region.getBlockedRequestsCount();
095    writeRequestCount = region.getWriteRequestsCount();
096  }
097
098  @Override
099  protected boolean shouldSplit() {
100    float blockedReqRate = updateRate();
101    if (super.shouldSplit()) {
102      return true;
103    }
104
105    if (EnvironmentEdgeManager.currentTime() < startTime + minAge) {
106      return false;
107    }
108
109    for (HStore store : region.getStores()) {
110      if (!store.canSplit()) {
111        return false;
112      }
113    }
114
115    if (blockedReqRate >= maxBlockedRequests) {
116      if (LOG.isDebugEnabled()) {
117        LOG.debug("Going to split region " + region.getRegionInfo().getRegionNameAsString()
118          + " because it's too busy. Blocked Request rate: " + blockedReqRate);
119      }
120      return true;
121    }
122
123    return false;
124  }
125
126  /**
127   * Update the blocked request rate based on number of blocked and total write requests in the last
128   * aggregation window, or since last call to this method, whichever is farthest in time. Uses
129   * weighted rate calculation based on the previous rate and new data.
130   * @return Updated blocked request rate.
131   */
132  private synchronized float updateRate() {
133    float aggBlockedRate;
134    long curTime = EnvironmentEdgeManager.currentTime();
135
136    long newBlockedReqs = region.getBlockedRequestsCount();
137    long newWriteReqs = region.getWriteRequestsCount();
138
139    aggBlockedRate =
140      (newBlockedReqs - blockedRequestCount) / (newWriteReqs - writeRequestCount + 0.00001f);
141
142    if (curTime - prevTime >= aggregationWindow) {
143      blockedRate = aggBlockedRate;
144      prevTime = curTime;
145      blockedRequestCount = newBlockedReqs;
146      writeRequestCount = newWriteReqs;
147    } else if (curTime - startTime >= aggregationWindow) {
148      // Calculate the aggregate blocked rate as the weighted sum of
149      // previous window's average blocked rate and blocked rate in this window so far.
150      float timeSlice = (curTime - prevTime) / (aggregationWindow + 0.0f);
151      aggBlockedRate = (1 - timeSlice) * blockedRate + timeSlice * aggBlockedRate;
152    } else {
153      aggBlockedRate = 0.0f;
154    }
155    return aggBlockedRate;
156  }
157}