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.master.normalizer;
019
020import static org.apache.hadoop.hbase.master.normalizer.RegionNormalizerWorker.CUMULATIVE_SIZE_LIMIT_MB_KEY;
021import static org.apache.hadoop.hbase.master.normalizer.RegionNormalizerWorker.DEFAULT_CUMULATIVE_SIZE_LIMIT_MB;
022import static org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils.isEmpty;
023
024import java.time.Instant;
025import java.time.Period;
026import java.util.ArrayList;
027import java.util.Collections;
028import java.util.LinkedList;
029import java.util.List;
030import java.util.Objects;
031import java.util.function.BooleanSupplier;
032import java.util.function.Function;
033import org.apache.hadoop.conf.Configuration;
034import org.apache.hadoop.hbase.HBaseInterfaceAudience;
035import org.apache.hadoop.hbase.RegionMetrics;
036import org.apache.hadoop.hbase.ServerMetrics;
037import org.apache.hadoop.hbase.ServerName;
038import org.apache.hadoop.hbase.Size;
039import org.apache.hadoop.hbase.TableName;
040import org.apache.hadoop.hbase.client.MasterSwitchType;
041import org.apache.hadoop.hbase.client.RegionInfo;
042import org.apache.hadoop.hbase.client.TableDescriptor;
043import org.apache.hadoop.hbase.conf.ConfigurationObserver;
044import org.apache.hadoop.hbase.master.MasterServices;
045import org.apache.hadoop.hbase.master.RegionState;
046import org.apache.hadoop.hbase.master.assignment.RegionStates;
047import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
048import org.apache.yetus.audience.InterfaceAudience;
049import org.slf4j.Logger;
050import org.slf4j.LoggerFactory;
051
052/**
053 * Simple implementation of region normalizer. Logic in use:
054 * <ol>
055 * <li>Get all regions of a given table</li>
056 * <li>Get avg size S of the regions in the table (by total size of store files reported in
057 * RegionMetrics)</li>
058 * <li>For each region R0, if R0 is bigger than S * 2, it is kindly requested to split.</li>
059 * <li>Otherwise, for the next region in the chain R1, if R0 + R1 is smaller then S, R0 and R1 are
060 * kindly requested to merge.</li>
061 * </ol>
062 */
063@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
064class SimpleRegionNormalizer implements RegionNormalizer, ConfigurationObserver {
065  private static final Logger LOG = LoggerFactory.getLogger(SimpleRegionNormalizer.class);
066
067  static final String SPLIT_ENABLED_KEY = "hbase.normalizer.split.enabled";
068  static final boolean DEFAULT_SPLIT_ENABLED = true;
069  static final String MERGE_ENABLED_KEY = "hbase.normalizer.merge.enabled";
070  static final boolean DEFAULT_MERGE_ENABLED = true;
071  /**
072   * @deprecated since 2.5.0 and will be removed in 4.0.0. Use
073   *             {@link SimpleRegionNormalizer#MERGE_MIN_REGION_COUNT_KEY} instead.
074   * @see <a href="https://issues.apache.org/jira/browse/HBASE-25745">HBASE-25745</a>
075   */
076  @Deprecated
077  static final String MIN_REGION_COUNT_KEY = "hbase.normalizer.min.region.count";
078  static final String MERGE_MIN_REGION_COUNT_KEY = "hbase.normalizer.merge.min.region.count";
079  static final int DEFAULT_MERGE_MIN_REGION_COUNT = 3;
080  static final String MERGE_MIN_REGION_AGE_DAYS_KEY = "hbase.normalizer.merge.min_region_age.days";
081  static final int DEFAULT_MERGE_MIN_REGION_AGE_DAYS = 3;
082  static final String MERGE_MIN_REGION_SIZE_MB_KEY = "hbase.normalizer.merge.min_region_size.mb";
083  static final int DEFAULT_MERGE_MIN_REGION_SIZE_MB = 0;
084  static final String MERGE_REQUEST_MAX_NUMBER_OF_REGIONS_COUNT_KEY =
085    "hbase.normalizer.merge.merge_request_max_number_of_regions";
086  static final long DEFAULT_MERGE_REQUEST_MAX_NUMBER_OF_REGIONS_COUNT = 100;
087
088  private MasterServices masterServices;
089  private NormalizerConfiguration normalizerConfiguration;
090
091  public SimpleRegionNormalizer() {
092    masterServices = null;
093    normalizerConfiguration = new NormalizerConfiguration();
094  }
095
096  @Override
097  public Configuration getConf() {
098    return normalizerConfiguration.getConf();
099  }
100
101  @Override
102  public void setConf(final Configuration conf) {
103    if (conf == null) {
104      return;
105    }
106    normalizerConfiguration = new NormalizerConfiguration(conf, normalizerConfiguration);
107  }
108
109  @Override
110  public void onConfigurationChange(Configuration conf) {
111    LOG.debug("Updating configuration parameters according to new configuration instance.");
112    setConf(conf);
113  }
114
115  private static int parseMergeMinRegionCount(final Configuration conf) {
116    final int parsedValue = conf.getInt(MERGE_MIN_REGION_COUNT_KEY, DEFAULT_MERGE_MIN_REGION_COUNT);
117    final int settledValue = Math.max(1, parsedValue);
118    if (parsedValue != settledValue) {
119      warnInvalidValue(MERGE_MIN_REGION_COUNT_KEY, parsedValue, settledValue);
120    }
121    return settledValue;
122  }
123
124  private static Period parseMergeMinRegionAge(final Configuration conf) {
125    final int parsedValue =
126      conf.getInt(MERGE_MIN_REGION_AGE_DAYS_KEY, DEFAULT_MERGE_MIN_REGION_AGE_DAYS);
127    final int settledValue = Math.max(0, parsedValue);
128    if (parsedValue != settledValue) {
129      warnInvalidValue(MERGE_MIN_REGION_AGE_DAYS_KEY, parsedValue, settledValue);
130    }
131    return Period.ofDays(settledValue);
132  }
133
134  private static long parseMergeMinRegionSizeMb(final Configuration conf) {
135    final long parsedValue =
136      conf.getLong(MERGE_MIN_REGION_SIZE_MB_KEY, DEFAULT_MERGE_MIN_REGION_SIZE_MB);
137    final long settledValue = Math.max(0, parsedValue);
138    if (parsedValue != settledValue) {
139      warnInvalidValue(MERGE_MIN_REGION_SIZE_MB_KEY, parsedValue, settledValue);
140    }
141    return settledValue;
142  }
143
144  private static long parseMergeRequestMaxNumberOfRegionsCount(final Configuration conf) {
145    final long parsedValue = conf.getLong(MERGE_REQUEST_MAX_NUMBER_OF_REGIONS_COUNT_KEY,
146      DEFAULT_MERGE_REQUEST_MAX_NUMBER_OF_REGIONS_COUNT);
147    final long settledValue = Math.max(2, parsedValue);
148    if (parsedValue != settledValue) {
149      warnInvalidValue(MERGE_REQUEST_MAX_NUMBER_OF_REGIONS_COUNT_KEY, parsedValue, settledValue);
150    }
151    return settledValue;
152  }
153
154  private static <T> void warnInvalidValue(final String key, final T parsedValue,
155    final T settledValue) {
156    LOG.warn("Configured value {}={} is invalid. Setting value to {}.", key, parsedValue,
157      settledValue);
158  }
159
160  private static <T> void logConfigurationUpdated(final String key, final T oldValue,
161    final T newValue) {
162    if (!Objects.equals(oldValue, newValue)) {
163      LOG.info("Updated configuration for key '{}' from {} to {}", key, oldValue, newValue);
164    }
165  }
166
167  /**
168   * Return this instance's configured value for {@value #SPLIT_ENABLED_KEY}.
169   */
170  public boolean isSplitEnabled() {
171    return normalizerConfiguration.isSplitEnabled();
172  }
173
174  /**
175   * Return this instance's configured value for {@value #MERGE_ENABLED_KEY}.
176   */
177  public boolean isMergeEnabled() {
178    return normalizerConfiguration.isMergeEnabled();
179  }
180
181  /**
182   * Return this instance's configured value for {@value #MERGE_MIN_REGION_COUNT_KEY}.
183   */
184  public int getMergeMinRegionCount() {
185    return normalizerConfiguration.getMergeMinRegionCount();
186  }
187
188  /**
189   * Return this instance's configured value for {@value #MERGE_MIN_REGION_AGE_DAYS_KEY}.
190   */
191  public Period getMergeMinRegionAge() {
192    return normalizerConfiguration.getMergeMinRegionAge();
193  }
194
195  /**
196   * Return this instance's configured value for {@value #MERGE_MIN_REGION_SIZE_MB_KEY}.
197   */
198  public long getMergeMinRegionSizeMb() {
199    return normalizerConfiguration.getMergeMinRegionSizeMb();
200  }
201
202  public long getMergeRequestMaxNumberOfRegionsCount() {
203    return normalizerConfiguration.getMergeRequestMaxNumberOfRegionsCount();
204  }
205
206  @Override
207  public void setMasterServices(final MasterServices masterServices) {
208    this.masterServices = masterServices;
209  }
210
211  @Override
212  public List<NormalizationPlan> computePlansForTable(final TableDescriptor tableDescriptor) {
213    if (tableDescriptor == null) {
214      return Collections.emptyList();
215    }
216    TableName table = tableDescriptor.getTableName();
217    if (table.isSystemTable()) {
218      LOG.debug("Normalization of system table {} isn't allowed", table);
219      return Collections.emptyList();
220    }
221
222    final boolean proceedWithSplitPlanning = proceedWithSplitPlanning(tableDescriptor);
223    final boolean proceedWithMergePlanning = proceedWithMergePlanning(tableDescriptor);
224    if (!proceedWithMergePlanning && !proceedWithSplitPlanning) {
225      LOG.debug("Both split and merge are disabled. Skipping normalization of table: {}", table);
226      return Collections.emptyList();
227    }
228
229    final NormalizeContext ctx = new NormalizeContext(tableDescriptor);
230    if (isEmpty(ctx.getTableRegions())) {
231      return Collections.emptyList();
232    }
233
234    LOG.debug("Computing normalization plan for table:  {}, number of regions: {}", table,
235      ctx.getTableRegions().size());
236
237    final List<NormalizationPlan> plans = new ArrayList<>();
238    int splitPlansCount = 0;
239    if (proceedWithSplitPlanning) {
240      List<NormalizationPlan> splitPlans = computeSplitNormalizationPlans(ctx);
241      splitPlansCount = splitPlans.size();
242      plans.addAll(splitPlans);
243    }
244    int mergePlansCount = 0;
245    if (proceedWithMergePlanning) {
246      List<NormalizationPlan> mergePlans = computeMergeNormalizationPlans(ctx);
247      mergePlansCount = mergePlans.size();
248      plans.addAll(mergePlans);
249    }
250
251    if (
252      normalizerConfiguration.getCumulativePlansSizeLimitMb() != DEFAULT_CUMULATIVE_SIZE_LIMIT_MB
253    ) {
254      // If we are going to truncate our list of plans, shuffle the split and merge plans together
255      // so that the merge plans, which are listed last, are not starved out.
256      shuffleNormalizationPlans(plans);
257    }
258
259    LOG.debug("Computed normalization plans for table {}. Total plans: {}, split plans: {}, "
260      + "merge plans: {}", table, plans.size(), splitPlansCount, mergePlansCount);
261    return plans;
262  }
263
264  /** Returns size of region in MB and if region is not found than -1 */
265  private long getRegionSizeMB(RegionInfo hri) {
266    ServerName sn =
267      masterServices.getAssignmentManager().getRegionStates().getRegionServerOfRegion(hri);
268    if (sn == null) {
269      LOG.debug("{} region was not found on any Server", hri.getRegionNameAsString());
270      return -1;
271    }
272    ServerMetrics serverMetrics = masterServices.getServerManager().getLoad(sn);
273    if (serverMetrics == null) {
274      LOG.debug("server {} was not found in ServerManager", sn.getServerName());
275      return -1;
276    }
277    RegionMetrics regionLoad = serverMetrics.getRegionMetrics().get(hri.getRegionName());
278    if (regionLoad == null) {
279      LOG.debug("{} was not found in RegionsLoad", hri.getRegionNameAsString());
280      return -1;
281    }
282    return (long) regionLoad.getStoreFileSize().get(Size.Unit.MEGABYTE);
283  }
284
285  private boolean isMasterSwitchEnabled(final MasterSwitchType masterSwitchType) {
286    return masterServices.isSplitOrMergeEnabled(masterSwitchType);
287  }
288
289  private boolean proceedWithSplitPlanning(TableDescriptor tableDescriptor) {
290    String value = tableDescriptor.getValue(SPLIT_ENABLED_KEY);
291    return (value == null ? isSplitEnabled() : Boolean.parseBoolean(value))
292      && isMasterSwitchEnabled(MasterSwitchType.SPLIT);
293  }
294
295  private boolean proceedWithMergePlanning(TableDescriptor tableDescriptor) {
296    String value = tableDescriptor.getValue(MERGE_ENABLED_KEY);
297    return (value == null ? isMergeEnabled() : Boolean.parseBoolean(value))
298      && isMasterSwitchEnabled(MasterSwitchType.MERGE);
299  }
300
301  /**
302   * Also make sure tableRegions contains regions of the same table
303   * @param tableRegions    regions of table to normalize
304   * @param tableDescriptor the TableDescriptor
305   * @return average region size depending on
306   * @see TableDescriptor#getNormalizerTargetRegionCount()
307   */
308  private double getAverageRegionSizeMb(final List<RegionInfo> tableRegions,
309    final TableDescriptor tableDescriptor) {
310    if (isEmpty(tableRegions)) {
311      throw new IllegalStateException(
312        "Cannot calculate average size of a table without any regions.");
313    }
314    TableName table = tableDescriptor.getTableName();
315    double avgRegionSize;
316    int targetRegionCount = tableDescriptor.getNormalizerTargetRegionCount();
317    long targetRegionSize = tableDescriptor.getNormalizerTargetRegionSize();
318    LOG.debug("Table {} configured with target region count {}, target region size {} MB", table,
319      targetRegionCount, targetRegionSize);
320
321    if (targetRegionSize > 0) {
322      avgRegionSize = targetRegionSize;
323    } else {
324      final int regionCount = tableRegions.size();
325      final long totalSizeMb = tableRegions.stream().mapToLong(this::getRegionSizeMB).sum();
326      if (targetRegionCount > 0) {
327        avgRegionSize = totalSizeMb / (double) targetRegionCount;
328      } else {
329        avgRegionSize = totalSizeMb / (double) regionCount;
330      }
331      LOG.debug("Table {}, total aggregated regions size: {} MB and average region size {} MB",
332        table, totalSizeMb, String.format("%.3f", avgRegionSize));
333    }
334
335    return avgRegionSize;
336  }
337
338  /**
339   * Determine if a {@link RegionInfo} should be considered for a merge operation.
340   * </p>
341   * Callers beware: for safe concurrency, be sure to pass in the local instance of
342   * {@link NormalizerConfiguration}, don't use {@code this}'s instance.
343   */
344  private boolean skipForMerge(final NormalizerConfiguration normalizerConfiguration,
345    final NormalizeContext ctx, final RegionInfo regionInfo) {
346    final RegionState state = ctx.getRegionStates().getRegionState(regionInfo);
347    final String name = regionInfo.getEncodedName();
348    return logTraceReason(() -> state == null,
349      "skipping merge of region {} because no state information is available.", name)
350      || logTraceReason(() -> !Objects.equals(state.getState(), RegionState.State.OPEN),
351        "skipping merge of region {} because it is not open.", name)
352      || logTraceReason(() -> !isOldEnoughForMerge(normalizerConfiguration, ctx, regionInfo),
353        "skipping merge of region {} because it is not old enough.", name)
354      || logTraceReason(() -> !isLargeEnoughForMerge(normalizerConfiguration, ctx, regionInfo),
355        "skipping merge region {} because it is not large enough.", name);
356  }
357
358  /**
359   * Computes the merge plans that should be executed for this table to converge average region
360   * towards target average or target region count.
361   */
362  private List<NormalizationPlan> computeMergeNormalizationPlans(final NormalizeContext ctx) {
363    final NormalizerConfiguration configuration = normalizerConfiguration;
364    if (ctx.getTableRegions().size() < configuration.getMergeMinRegionCount(ctx)) {
365      LOG.debug(
366        "Table {} has {} regions, required min number of regions for normalizer to run"
367          + " is {}, not computing merge plans.",
368        ctx.getTableName(), ctx.getTableRegions().size(), configuration.getMergeMinRegionCount());
369      return Collections.emptyList();
370    }
371
372    final long avgRegionSizeMb = (long) ctx.getAverageRegionSizeMb();
373    if (avgRegionSizeMb < configuration.getMergeMinRegionSizeMb(ctx)) {
374      return Collections.emptyList();
375    }
376    LOG.debug("Computing normalization plan for table {}. average region size: {} MB, number of"
377      + " regions: {}.", ctx.getTableName(), avgRegionSizeMb, ctx.getTableRegions().size());
378
379    // this nested loop walks the table's region chain once, looking for contiguous sequences of
380    // regions that meet the criteria for merge. The outer loop tracks the starting point of the
381    // next sequence, the inner loop looks for the end of that sequence. A single sequence becomes
382    // an instance of MergeNormalizationPlan.
383
384    final List<NormalizationPlan> plans = new LinkedList<>();
385    final List<NormalizationTarget> rangeMembers = new LinkedList<>();
386    long sumRangeMembersSizeMb;
387    int current = 0;
388    for (int rangeStart = 0; rangeStart < ctx.getTableRegions().size() - 1
389      && current < ctx.getTableRegions().size();) {
390      // walk the region chain looking for contiguous sequences of regions that can be merged.
391      rangeMembers.clear();
392      sumRangeMembersSizeMb = 0;
393      for (current = rangeStart; current < ctx.getTableRegions().size(); current++) {
394        final RegionInfo regionInfo = ctx.getTableRegions().get(current);
395        final long regionSizeMb = getRegionSizeMB(regionInfo);
396        if (skipForMerge(configuration, ctx, regionInfo)) {
397          // this region cannot participate in a range. resume the outer loop.
398          rangeStart = Math.max(current, rangeStart + 1);
399          break;
400        }
401        if (
402          // when there are no range members, seed the range with whatever we have. this way we're
403          // prepared in case the next region is 0-size.
404          rangeMembers.isEmpty()
405            // when there is only one region and the size is 0, seed the range with whatever we
406            // have.
407            || (rangeMembers.size() == 1 && sumRangeMembersSizeMb == 0)
408            // add an empty region to the current range only if it doesn't exceed max merge request
409            // region count
410            || (regionSizeMb == 0 && rangeMembers.size() < getMergeRequestMaxNumberOfRegionsCount())
411            // add region if current range region size is less than avg region size of table
412            // and current range doesn't exceed max merge request region count
413            || ((regionSizeMb + sumRangeMembersSizeMb <= avgRegionSizeMb)
414              && (rangeMembers.size() < getMergeRequestMaxNumberOfRegionsCount()))
415        ) {
416          // add the current region to the range when there's capacity remaining.
417          rangeMembers.add(new NormalizationTarget(regionInfo, regionSizeMb));
418          sumRangeMembersSizeMb += regionSizeMb;
419          continue;
420        }
421        // we have accumulated enough regions to fill a range. resume the outer loop.
422        rangeStart = Math.max(current, rangeStart + 1);
423        break;
424      }
425      if (rangeMembers.size() > 1) {
426        plans.add(new MergeNormalizationPlan.Builder().setTargets(rangeMembers).build());
427      }
428    }
429    return plans;
430  }
431
432  /**
433   * Determine if a region in {@link RegionState} should be considered for a split operation.
434   */
435  private static boolean skipForSplit(final RegionState state, final RegionInfo regionInfo) {
436    final String name = regionInfo.getEncodedName();
437    return logTraceReason(() -> state == null,
438      "skipping split of region {} because no state information is available.", name)
439      || logTraceReason(() -> !Objects.equals(state.getState(), RegionState.State.OPEN),
440        "skipping merge of region {} because it is not open.", name);
441  }
442
443  /**
444   * Computes the split plans that should be executed for this table to converge average region size
445   * towards target average or target region count. <br />
446   * if the region is > 2 times larger than average, we split it. split is more high priority
447   * normalization action than merge.
448   */
449  private List<NormalizationPlan> computeSplitNormalizationPlans(final NormalizeContext ctx) {
450    final double avgRegionSize = ctx.getAverageRegionSizeMb();
451    LOG.debug("Table {}, average region size: {} MB", ctx.getTableName(),
452      String.format("%.3f", avgRegionSize));
453
454    final List<NormalizationPlan> plans = new ArrayList<>();
455    for (final RegionInfo hri : ctx.getTableRegions()) {
456      if (skipForSplit(ctx.getRegionStates().getRegionState(hri), hri)) {
457        continue;
458      }
459      final long regionSizeMb = getRegionSizeMB(hri);
460      if (regionSizeMb > 2 * avgRegionSize) {
461        LOG.info(
462          "Table {}, large region {} has size {} MB, more than twice avg size {} MB, "
463            + "splitting",
464          ctx.getTableName(), hri.getRegionNameAsString(), regionSizeMb,
465          String.format("%.3f", avgRegionSize));
466        plans.add(new SplitNormalizationPlan(hri, regionSizeMb));
467      }
468    }
469    return plans;
470  }
471
472  /**
473   * Return {@code true} when {@code regionInfo} has a creation date that is old enough to be
474   * considered for a merge operation, {@code false} otherwise.
475   */
476  private static boolean isOldEnoughForMerge(final NormalizerConfiguration normalizerConfiguration,
477    final NormalizeContext ctx, final RegionInfo regionInfo) {
478    final Instant currentTime = Instant.ofEpochMilli(EnvironmentEdgeManager.currentTime());
479    final Instant regionCreateTime = Instant.ofEpochMilli(regionInfo.getRegionId());
480    return currentTime
481      .isAfter(regionCreateTime.plus(normalizerConfiguration.getMergeMinRegionAge(ctx)));
482  }
483
484  /**
485   * Return {@code true} when {@code regionInfo} has a size that is sufficient to be considered for
486   * a merge operation, {@code false} otherwise.
487   * </p>
488   * Callers beware: for safe concurrency, be sure to pass in the local instance of
489   * {@link NormalizerConfiguration}, don't use {@code this}'s instance.
490   */
491  private boolean isLargeEnoughForMerge(final NormalizerConfiguration normalizerConfiguration,
492    final NormalizeContext ctx, final RegionInfo regionInfo) {
493    return getRegionSizeMB(regionInfo) >= normalizerConfiguration.getMergeMinRegionSizeMb(ctx);
494  }
495
496  /**
497   * This very simple method exists so we can verify it was called in a unit test. Visible for
498   * testing.
499   */
500  void shuffleNormalizationPlans(List<NormalizationPlan> plans) {
501    Collections.shuffle(plans);
502  }
503
504  private static boolean logTraceReason(final BooleanSupplier predicate, final String fmtWhenTrue,
505    final Object... args) {
506    final boolean value = predicate.getAsBoolean();
507    if (value) {
508      LOG.trace(fmtWhenTrue, args);
509    }
510    return value;
511  }
512
513  /**
514   * Holds the configuration values read from {@link Configuration}. Encapsulation in a POJO enables
515   * atomic hot-reloading of configs without locks.
516   */
517  private static final class NormalizerConfiguration {
518    private final Configuration conf;
519    private final boolean splitEnabled;
520    private final boolean mergeEnabled;
521    private final int mergeMinRegionCount;
522    private final Period mergeMinRegionAge;
523    private final long mergeMinRegionSizeMb;
524    private final long mergeRequestMaxNumberOfRegionsCount;
525    private final long cumulativePlansSizeLimitMb;
526
527    private NormalizerConfiguration() {
528      conf = null;
529      splitEnabled = DEFAULT_SPLIT_ENABLED;
530      mergeEnabled = DEFAULT_MERGE_ENABLED;
531      mergeMinRegionCount = DEFAULT_MERGE_MIN_REGION_COUNT;
532      mergeMinRegionAge = Period.ofDays(DEFAULT_MERGE_MIN_REGION_AGE_DAYS);
533      mergeMinRegionSizeMb = DEFAULT_MERGE_MIN_REGION_SIZE_MB;
534      mergeRequestMaxNumberOfRegionsCount = DEFAULT_MERGE_REQUEST_MAX_NUMBER_OF_REGIONS_COUNT;
535      cumulativePlansSizeLimitMb = DEFAULT_CUMULATIVE_SIZE_LIMIT_MB;
536    }
537
538    private NormalizerConfiguration(final Configuration conf,
539      final NormalizerConfiguration currentConfiguration) {
540      this.conf = conf;
541      splitEnabled = conf.getBoolean(SPLIT_ENABLED_KEY, DEFAULT_SPLIT_ENABLED);
542      mergeEnabled = conf.getBoolean(MERGE_ENABLED_KEY, DEFAULT_MERGE_ENABLED);
543      mergeMinRegionCount = parseMergeMinRegionCount(conf);
544      mergeMinRegionAge = parseMergeMinRegionAge(conf);
545      mergeMinRegionSizeMb = parseMergeMinRegionSizeMb(conf);
546      mergeRequestMaxNumberOfRegionsCount = parseMergeRequestMaxNumberOfRegionsCount(conf);
547      cumulativePlansSizeLimitMb =
548        conf.getLong(CUMULATIVE_SIZE_LIMIT_MB_KEY, DEFAULT_CUMULATIVE_SIZE_LIMIT_MB);
549      logConfigurationUpdated(SPLIT_ENABLED_KEY, currentConfiguration.isSplitEnabled(),
550        splitEnabled);
551      logConfigurationUpdated(MERGE_ENABLED_KEY, currentConfiguration.isMergeEnabled(),
552        mergeEnabled);
553      logConfigurationUpdated(MERGE_MIN_REGION_COUNT_KEY,
554        currentConfiguration.getMergeMinRegionCount(), mergeMinRegionCount);
555      logConfigurationUpdated(MERGE_MIN_REGION_AGE_DAYS_KEY,
556        currentConfiguration.getMergeMinRegionAge(), mergeMinRegionAge);
557      logConfigurationUpdated(MERGE_MIN_REGION_SIZE_MB_KEY,
558        currentConfiguration.getMergeMinRegionSizeMb(), mergeMinRegionSizeMb);
559      logConfigurationUpdated(MERGE_REQUEST_MAX_NUMBER_OF_REGIONS_COUNT_KEY,
560        currentConfiguration.getMergeRequestMaxNumberOfRegionsCount(),
561        mergeRequestMaxNumberOfRegionsCount);
562    }
563
564    public Configuration getConf() {
565      return conf;
566    }
567
568    public boolean isSplitEnabled() {
569      return splitEnabled;
570    }
571
572    public boolean isMergeEnabled() {
573      return mergeEnabled;
574    }
575
576    public int getMergeMinRegionCount() {
577      return mergeMinRegionCount;
578    }
579
580    public int getMergeMinRegionCount(NormalizeContext context) {
581      String stringValue =
582        context.getOrDefault(MERGE_MIN_REGION_COUNT_KEY, Function.identity(), null);
583      if (stringValue == null) {
584        stringValue = context.getOrDefault(MIN_REGION_COUNT_KEY, Function.identity(), null);
585        if (stringValue != null) {
586          LOG.debug(
587            "The config key {} in table descriptor is deprecated. Instead please use {}. "
588              + "In future release we will remove the deprecated config.",
589            MIN_REGION_COUNT_KEY, MERGE_MIN_REGION_COUNT_KEY);
590        }
591      }
592      final int mergeMinRegionCount = stringValue == null ? 0 : Integer.parseInt(stringValue);
593      if (mergeMinRegionCount <= 0) {
594        return getMergeMinRegionCount();
595      }
596      return mergeMinRegionCount;
597    }
598
599    public Period getMergeMinRegionAge() {
600      return mergeMinRegionAge;
601    }
602
603    public Period getMergeMinRegionAge(NormalizeContext context) {
604      final int mergeMinRegionAge =
605        context.getOrDefault(MERGE_MIN_REGION_AGE_DAYS_KEY, Integer::parseInt, -1);
606      if (mergeMinRegionAge < 0) {
607        return getMergeMinRegionAge();
608      }
609      return Period.ofDays(mergeMinRegionAge);
610    }
611
612    public long getMergeMinRegionSizeMb() {
613      return mergeMinRegionSizeMb;
614    }
615
616    public long getMergeMinRegionSizeMb(NormalizeContext context) {
617      final long mergeMinRegionSizeMb =
618        context.getOrDefault(MERGE_MIN_REGION_SIZE_MB_KEY, Long::parseLong, (long) -1);
619      if (mergeMinRegionSizeMb < 0) {
620        return getMergeMinRegionSizeMb();
621      }
622      return mergeMinRegionSizeMb;
623    }
624
625    public long getMergeRequestMaxNumberOfRegionsCount() {
626      return mergeRequestMaxNumberOfRegionsCount;
627    }
628
629    private long getCumulativePlansSizeLimitMb() {
630      return cumulativePlansSizeLimitMb;
631    }
632  }
633
634  /**
635   * Inner class caries the state necessary to perform a single invocation of
636   * {@link #computePlansForTable(TableDescriptor)}. Grabbing this data from the assignment manager
637   * up-front allows any computed values to be realized just once.
638   */
639  private class NormalizeContext {
640    private final TableName tableName;
641    private final RegionStates regionStates;
642    private final List<RegionInfo> tableRegions;
643    private final double averageRegionSizeMb;
644    private final TableDescriptor tableDescriptor;
645
646    public NormalizeContext(final TableDescriptor tableDescriptor) {
647      this.tableDescriptor = tableDescriptor;
648      tableName = tableDescriptor.getTableName();
649      regionStates =
650        SimpleRegionNormalizer.this.masterServices.getAssignmentManager().getRegionStates();
651      tableRegions = regionStates.getRegionsOfTable(tableName);
652      // The list of regionInfo from getRegionsOfTable() is ordered by regionName.
653      // regionName does not necessary guarantee the order by STARTKEY (let's say 'aa1', 'aa1!',
654      // in order by regionName, it will be 'aa1!' followed by 'aa1').
655      // This could result in normalizer merging non-adjacent regions into one and creates overlaps.
656      // In order to avoid that, sort the list by RegionInfo.COMPARATOR.
657      // See HBASE-24376
658      tableRegions.sort(RegionInfo.COMPARATOR);
659      averageRegionSizeMb =
660        SimpleRegionNormalizer.this.getAverageRegionSizeMb(this.tableRegions, this.tableDescriptor);
661    }
662
663    public TableName getTableName() {
664      return tableName;
665    }
666
667    public RegionStates getRegionStates() {
668      return regionStates;
669    }
670
671    public List<RegionInfo> getTableRegions() {
672      return tableRegions;
673    }
674
675    public double getAverageRegionSizeMb() {
676      return averageRegionSizeMb;
677    }
678
679    public <T> T getOrDefault(String key, Function<String, T> function, T defaultValue) {
680      String value = tableDescriptor.getValue(key);
681      if (value == null) {
682        return defaultValue;
683      } else {
684        return function.apply(value);
685      }
686    }
687  }
688}