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 compaction pressure is greater than 1.0, no limitation.</li>
33   * <li>In off peak hours, use a fixed throughput limitation
34   * {@value #HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_OFFPEAK}</li>
35   * <li>In normal hours, the max throughput is tuned between
36   * {@value #HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_LOWER_BOUND} and
37   * {@value #HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_HIGHER_BOUND}, using the formula &quot;lower +
38   * (higer - lower) * compactionPressure&quot;, where compactionPressure is in range [0.0, 1.0]</li>
39   * </ul>
40   * @see org.apache.hadoop.hbase.regionserver.Store#getCompactionPressure()
41   */
42  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
43  public class PressureAwareCompactionThroughputController extends PressureAwareThroughputController {
44  
45    private final static Log LOG = LogFactory
46        .getLog(PressureAwareCompactionThroughputController.class);
47  
48    public static final String HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_HIGHER_BOUND =
49        "hbase.hstore.compaction.throughput.higher.bound";
50  
51    private static final long DEFAULT_HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_HIGHER_BOUND =
52        20L * 1024 * 1024;
53  
54    public static final String HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_LOWER_BOUND =
55        "hbase.hstore.compaction.throughput.lower.bound";
56  
57    private static final long DEFAULT_HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_LOWER_BOUND =
58        10L * 1024 * 1024;
59  
60    public static final String HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_OFFPEAK =
61        "hbase.hstore.compaction.throughput.offpeak";
62  
63    private static final long DEFAULT_HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_OFFPEAK = Long.MAX_VALUE;
64  
65    public static final String HBASE_HSTORE_COMPACTION_THROUGHPUT_TUNE_PERIOD =
66        "hbase.hstore.compaction.throughput.tune.period";
67  
68    private static final int DEFAULT_HSTORE_COMPACTION_THROUGHPUT_TUNE_PERIOD = 60 * 1000;
69  
70    // check compaction throughput every this size
71    private static final String HBASE_HSTORE_COMPACTION_THROUGHPUT_CONTROL_CHECK_INTERVAL =
72      "hbase.hstore.compaction.throughput.control.check.interval";
73  
74    private long maxThroughputOffpeak;
75  
76    @Override
77    public void setup(final RegionServerServices server) {
78      server.getChoreService().scheduleChore(
79        new ScheduledChore("CompactionThroughputTuner", this, tuningPeriod) {
80  
81          @Override
82          protected void chore() {
83            tune(server.getCompactionPressure());
84          }
85        });
86    }
87  
88    private void tune(double compactionPressure) {
89      double maxThroughputToSet;
90      if (compactionPressure > 1.0) {
91        // set to unlimited if some stores already reach the blocking store file count
92        maxThroughputToSet = Double.MAX_VALUE;
93      } else if (offPeakHours.isOffPeakHour()) {
94        maxThroughputToSet = maxThroughputOffpeak;
95      } else {
96        // compactionPressure is between 0.0 and 1.0, we use a simple linear formula to
97        // calculate the throughput limitation.
98        maxThroughputToSet =
99            maxThroughputLowerBound + (maxThroughputUpperBound - maxThroughputLowerBound)
100               * compactionPressure;
101     }
102     if (LOG.isDebugEnabled()) {
103       if (Math.abs(maxThroughputToSet - getMaxThroughput()) < .0000001) {
104         LOG.debug("CompactionPressure is " + compactionPressure + ", tune throughput to "
105             + throughputDesc(maxThroughputToSet));
106       } else if (LOG.isTraceEnabled()) {
107         LOG.trace("CompactionPressure is " + compactionPressure + ", keep throughput throttling to "
108             + throughputDesc(maxThroughputToSet));
109       }
110     }
111     this.setMaxThroughput(maxThroughputToSet);
112   }
113 
114   @Override
115   public void setConf(Configuration conf) {
116     super.setConf(conf);
117     if (conf == null) {
118       return;
119     }
120     this.maxThroughputUpperBound =
121         conf.getLong(HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_HIGHER_BOUND,
122           DEFAULT_HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_HIGHER_BOUND);
123     this.maxThroughputLowerBound =
124         conf.getLong(HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_LOWER_BOUND,
125           DEFAULT_HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_LOWER_BOUND);
126     this.maxThroughputOffpeak =
127         conf.getLong(HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_OFFPEAK,
128           DEFAULT_HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_OFFPEAK);
129     this.offPeakHours = OffPeakHours.getInstance(conf);
130     this.controlPerSize =
131         conf.getLong(HBASE_HSTORE_COMPACTION_THROUGHPUT_CONTROL_CHECK_INTERVAL,
132           this.maxThroughputLowerBound);
133     this.setMaxThroughput(this.maxThroughputLowerBound);
134     this.tuningPeriod =
135         getConf().getInt(HBASE_HSTORE_COMPACTION_THROUGHPUT_TUNE_PERIOD,
136           DEFAULT_HSTORE_COMPACTION_THROUGHPUT_TUNE_PERIOD);
137     LOG.info("Compaction throughput configurations, higher bound: "
138         + throughputDesc(maxThroughputUpperBound) + ", lower bound "
139         + throughputDesc(maxThroughputLowerBound) + ", off peak: "
140         + throughputDesc(maxThroughputOffpeak) + ", tuning period: " + tuningPeriod + " ms");
141   }
142 
143   @Override
144   public String toString() {
145     return "DefaultCompactionThroughputController [maxThroughput="
146         + throughputDesc(getMaxThroughput()) + ", activeCompactions=" + activeOperations.size()
147         + "]";
148   }
149 }