View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.regionserver.throttle;
19  
20  import org.apache.commons.logging.Log;
21  import org.apache.commons.logging.LogFactory;
22  import org.apache.hadoop.conf.Configuration;
23  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
24  import org.apache.hadoop.hbase.ScheduledChore;
25  import org.apache.hadoop.hbase.classification.InterfaceAudience;
26  import org.apache.hadoop.hbase.regionserver.RegionServerServices;
27  import org.apache.hadoop.hbase.regionserver.compactions.OffPeakHours;
28  
29  /**
30   * A throughput controller which uses the follow schema to limit throughput
31   * <ul>
32   * <li>If flush pressure is greater than or equal to 1.0, no limitation.</li>
33   * <li>In normal case, the max throughput is tuned between
34   * {@value #HBASE_HSTORE_FLUSH_MAX_THROUGHPUT_LOWER_BOUND} and
35   * {@value #HBASE_HSTORE_FLUSH_MAX_THROUGHPUT_UPPER_BOUND}, using the formula &quot;lower +
36   * (upper - lower) * flushPressure&quot;, where flushPressure is in range [0.0, 1.0)</li>
37   * </ul>
38   * @see org.apache.hadoop.hbase.regionserver.HRegionServer#getFlushPressure()
39   */
40  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
41  public class PressureAwareFlushThroughputController extends PressureAwareThroughputController {
42  
43    private static final Log LOG = LogFactory.getLog(PressureAwareFlushThroughputController.class);
44  
45    public static final String HBASE_HSTORE_FLUSH_MAX_THROUGHPUT_UPPER_BOUND =
46        "hbase.hstore.flush.throughput.upper.bound";
47  
48    private static final long DEFAULT_HBASE_HSTORE_FLUSH_MAX_THROUGHPUT_UPPER_BOUND =
49        200L * 1024 * 1024;
50  
51    public static final String HBASE_HSTORE_FLUSH_MAX_THROUGHPUT_LOWER_BOUND =
52        "hbase.hstore.flush.throughput.lower.bound";
53  
54    private static final long DEFAULT_HBASE_HSTORE_FLUSH_MAX_THROUGHPUT_LOWER_BOUND =
55        100L * 1024 * 1024;
56  
57    public static final String HBASE_HSTORE_FLUSH_THROUGHPUT_TUNE_PERIOD =
58        "hbase.hstore.flush.throughput.tune.period";
59  
60    private static final int DEFAULT_HSTORE_FLUSH_THROUGHPUT_TUNE_PERIOD = 20 * 1000;
61  
62    // check flush throughput every this size
63    public static final String HBASE_HSTORE_FLUSH_THROUGHPUT_CONTROL_CHECK_INTERVAL =
64        "hbase.hstore.flush.throughput.control.check.interval";
65  
66    private static final long DEFAULT_HBASE_HSTORE_FLUSH_THROUGHPUT_CONTROL_CHECK_INTERVAL =
67        10L * 1024 * 1024;// 10MB
68  
69    @Override
70    public void setup(final RegionServerServices server) {
71      server.getChoreService().scheduleChore(
72        new ScheduledChore("FlushThroughputTuner", this, tuningPeriod, this.tuningPeriod) {
73  
74          @Override
75          protected void chore() {
76            tune(server.getFlushPressure());
77          }
78        });
79    }
80  
81    private void tune(double flushPressure) {
82      double maxThroughputToSet;
83      if (flushPressure >= 1.0) {
84        // set to unlimited if global memstore size already exceeds lower limit
85        maxThroughputToSet = Double.MAX_VALUE;
86      } else {
87        // flushPressure is between 0.0 and 1.0, we use a simple linear formula to
88        // calculate the throughput limitation.
89        maxThroughputToSet =
90            maxThroughputLowerBound + (maxThroughputUpperBound - maxThroughputLowerBound)
91                * flushPressure;
92      }
93      if (LOG.isDebugEnabled()) {
94        LOG.debug("flushPressure is " + flushPressure + ", tune flush throughput to "
95            + throughputDesc(maxThroughputToSet));
96      }
97      this.setMaxThroughput(maxThroughputToSet);
98    }
99  
100   @Override
101   public void setConf(Configuration conf) {
102     super.setConf(conf);
103     if (conf == null) {
104       return;
105     }
106     this.maxThroughputUpperBound =
107         conf.getLong(HBASE_HSTORE_FLUSH_MAX_THROUGHPUT_UPPER_BOUND,
108           DEFAULT_HBASE_HSTORE_FLUSH_MAX_THROUGHPUT_UPPER_BOUND);
109     this.maxThroughputLowerBound =
110         conf.getLong(HBASE_HSTORE_FLUSH_MAX_THROUGHPUT_LOWER_BOUND,
111           DEFAULT_HBASE_HSTORE_FLUSH_MAX_THROUGHPUT_LOWER_BOUND);
112     this.offPeakHours = OffPeakHours.getInstance(conf);
113     this.controlPerSize =
114         conf.getLong(HBASE_HSTORE_FLUSH_THROUGHPUT_CONTROL_CHECK_INTERVAL,
115           DEFAULT_HBASE_HSTORE_FLUSH_THROUGHPUT_CONTROL_CHECK_INTERVAL);
116     this.setMaxThroughput(this.maxThroughputLowerBound);
117     this.tuningPeriod =
118         getConf().getInt(HBASE_HSTORE_FLUSH_THROUGHPUT_TUNE_PERIOD,
119           DEFAULT_HSTORE_FLUSH_THROUGHPUT_TUNE_PERIOD);
120     LOG.info("Flush throughput configurations, upper bound: "
121         + throughputDesc(maxThroughputUpperBound) + ", lower bound "
122         + throughputDesc(maxThroughputLowerBound) + ", tuning period: " + tuningPeriod + " ms");
123   }
124 
125   @Override
126   public String toString() {
127     return "DefaultFlushController [maxThroughput=" + throughputDesc(getMaxThroughput())
128         + ", activeFlushNumber=" + activeOperations.size() + "]";
129   }
130 }