You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/06/01 22:29:29 UTC

[GitHub] [hbase] ndimiduk commented on a change in pull request #1786: HBASE-24418 Consolidate Normalizer implementations

ndimiduk commented on a change in pull request #1786:
URL: https://github.com/apache/hbase/pull/1786#discussion_r433523464



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -18,126 +17,436 @@
  */
 package org.apache.hadoop.hbase.master.normalizer;
 
+import java.io.IOException;
+import java.time.Instant;
+import java.time.Period;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
-
-import org.apache.hadoop.hbase.HBaseIOException;
+import java.util.Objects;
+import java.util.function.BooleanSupplier;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.RegionMetrics;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.Size;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 
 /**
  * Simple implementation of region normalizer. Logic in use:
  * <ol>
- * <li>Get all regions of a given table
- * <li>Get avg size S of each region (by total size of store files reported in RegionMetrics)
- * <li>Seek every single region one by one. If a region R0 is bigger than S * 2, it is kindly
- * requested to split. Thereon evaluate the next region R1
- * <li>Otherwise, if R0 + R1 is smaller than S, R0 and R1 are kindly requested to merge. Thereon
- * evaluate the next region R2
- * <li>Otherwise, R1 is evaluated
+ *   <li>Get all regions of a given table</li>
+ *   <li>Get avg size S of the regions in the table (by total size of store files reported in
+ *     RegionMetrics)</li>
+ *   <li>For each region R0, if R0 is bigger than S * 2, it is kindly requested to split.</li>
+ *   <li>Otherwise, for the next region in the chain R1, if R0 + R1 is smaller then S, R0 and R1
+ *     are kindly requested to merge.</li>
+ * </ol>
+ * <p>
+ * The following parameters are configurable:
+ * <ol>
+ *   <li>Whether to split a region as part of normalization. Configuration:
+ *     {@value SPLIT_ENABLED_KEY}, default: {@value DEFAULT_SPLIT_ENABLED}.</li>
+ *   <li>Whether to merge a region as part of normalization. Configuration:
+ *     {@value MERGE_ENABLED_KEY}, default: {@value DEFAULT_MERGE_ENABLED}.</li>
+ *   <li>The minimum number of regions in a table to consider it for normalization. Configuration:
+ *     {@value MIN_REGION_COUNT_KEY}, default: {@value DEFAULT_MIN_REGION_COUNT}.</li>
+ *   <li>The minimum age for a region to be considered for a merge, in days. Configuration:
+ *     {@value MERGE_MIN_REGION_AGE_DAYS_KEY}, default:
+ *     {@value DEFAULT_MERGE_MIN_REGION_AGE_DAYS}.</li>
+ *   <li>The minimum size for a region to be considered for a merge, in whole MBs. Configuration:
+ *     {@value MERGE_MIN_REGION_SIZE_MB_KEY}, default:
+ *     {@value DEFAULT_MERGE_MIN_REGION_SIZE_MB}.</li>
  * </ol>
  * <p>
- * Region sizes are coarse and approximate on the order of megabytes. Additionally, "empty" regions
- * (less than 1MB, with the previous note) are not merged away. This is by design to prevent
- * normalization from undoing the pre-splitting of a table.
+ * To see detailed logging of the application of these configuration values, set the log level for
+ * this class to `TRACE`.
  */
-@InterfaceAudience.Private
-public class SimpleRegionNormalizer extends AbstractRegionNormalizer {
-
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class SimpleRegionNormalizer implements RegionNormalizer {
   private static final Logger LOG = LoggerFactory.getLogger(SimpleRegionNormalizer.class);
-  private static long[] skippedCount = new long[NormalizationPlan.PlanType.values().length];
+
+  static final String SPLIT_ENABLED_KEY = "hbase.normalizer.split.enabled";
+  static final boolean DEFAULT_SPLIT_ENABLED = true;
+  static final String MERGE_ENABLED_KEY = "hbase.normalizer.merge.enabled";
+  static final boolean DEFAULT_MERGE_ENABLED = true;
+  // TODO: after HBASE-24416, `min.region.count` only applies to merge plans; should
+  //  deprecate/rename the configuration key.
+  static final String MIN_REGION_COUNT_KEY = "hbase.normalizer.min.region.count";
+  static final int DEFAULT_MIN_REGION_COUNT = 3;
+  static final String MERGE_MIN_REGION_AGE_DAYS_KEY = "hbase.normalizer.merge.min_region_age.days";
+  static final int DEFAULT_MERGE_MIN_REGION_AGE_DAYS = 3;
+  static final String MERGE_MIN_REGION_SIZE_MB_KEY = "hbase.normalizer.merge.min_region_size.mb";
+  static final int DEFAULT_MERGE_MIN_REGION_SIZE_MB = 1;
+
+  private final long[] skippedCount = new long[NormalizationPlan.PlanType.values().length];
+  private final Comparator<NormalizationPlan> planComparator = new SplitPlanFirstComparator();
+
+  private Configuration conf;
+  private MasterServices masterServices;
+  private boolean splitEnabled;
+  private boolean mergeEnabled;
+  private int minRegionCount;
+  private Period mergeMinRegionAge;
+  private int mergeMinRegionSizeMb;
+
+  public SimpleRegionNormalizer() {
+    splitEnabled = DEFAULT_SPLIT_ENABLED;
+    mergeEnabled = DEFAULT_MERGE_ENABLED;
+    minRegionCount = DEFAULT_MIN_REGION_COUNT;
+    mergeMinRegionAge = Period.ofDays(DEFAULT_MERGE_MIN_REGION_AGE_DAYS);
+    mergeMinRegionSizeMb = DEFAULT_MERGE_MIN_REGION_SIZE_MB;
+  }
 
   @Override
-  public void planSkipped(RegionInfo hri, PlanType type) {
-    skippedCount[type.ordinal()]++;
+  public Configuration getConf() {
+    return conf;
   }
 
   @Override
-  public long getSkippedCount(NormalizationPlan.PlanType type) {
-    return skippedCount[type.ordinal()];
+  public void setConf(final Configuration conf) {
+    if (conf == null) {
+      return;
+    }
+    this.conf = conf;
+    splitEnabled = conf.getBoolean(SPLIT_ENABLED_KEY, DEFAULT_SPLIT_ENABLED);
+    mergeEnabled = conf.getBoolean(MERGE_ENABLED_KEY, DEFAULT_MERGE_ENABLED);
+    minRegionCount = parseMinRegionCount(conf);
+    mergeMinRegionAge = parseMergeMinRegionAge(conf);
+    mergeMinRegionSizeMb = parseMergeMinRegionSizeMb(conf);
+  }
+
+  private static int parseMinRegionCount(final Configuration conf) {
+    final int parsedValue = conf.getInt(MIN_REGION_COUNT_KEY, DEFAULT_MIN_REGION_COUNT);
+    final int settledValue = Math.max(1, parsedValue);
+    if (parsedValue != settledValue) {
+      warnInvalidValue(MIN_REGION_COUNT_KEY, parsedValue, settledValue);
+    }
+    return settledValue;
+  }
+
+  private static Period parseMergeMinRegionAge(final Configuration conf) {
+    final int parsedValue =
+      conf.getInt(MERGE_MIN_REGION_AGE_DAYS_KEY, DEFAULT_MERGE_MIN_REGION_AGE_DAYS);
+    final int settledValue = Math.max(0, parsedValue);
+    if (parsedValue != settledValue) {
+      warnInvalidValue(MERGE_MIN_REGION_AGE_DAYS_KEY, parsedValue, settledValue);
+    }
+    return Period.ofDays(settledValue);
+  }
+
+  private static int parseMergeMinRegionSizeMb(final Configuration conf) {
+    final int parsedValue =
+      conf.getInt(MERGE_MIN_REGION_SIZE_MB_KEY, DEFAULT_MERGE_MIN_REGION_SIZE_MB);
+    final int settledValue = Math.max(0, parsedValue);
+    if (parsedValue != settledValue) {
+      warnInvalidValue(MERGE_MIN_REGION_SIZE_MB_KEY, parsedValue, settledValue);
+    }
+    return settledValue;
+  }
+
+  private static <T> void warnInvalidValue(final String key, final T parsedValue,
+    final T settledValue) {
+    LOG.warn("Configured value {}={} is invalid. Setting value to {}.",
+      key, parsedValue, settledValue);
   }
 
   /**
-   * Comparator class that gives higher priority to region Split plan.
+   * Return this instance's configured value for {@value SPLIT_ENABLED_KEY}.
    */
-  static class PlanComparator implements Comparator<NormalizationPlan> {
-    @Override
-    public int compare(NormalizationPlan plan1, NormalizationPlan plan2) {
-      boolean plan1IsSplit = plan1 instanceof SplitNormalizationPlan;
-      boolean plan2IsSplit = plan2 instanceof SplitNormalizationPlan;
-      if (plan1IsSplit && plan2IsSplit) {
-        return 0;
-      } else if (plan1IsSplit) {
-        return -1;
-      } else if (plan2IsSplit) {
-        return 1;
-      } else {
-        return 0;
-      }
-    }
+  public boolean isSplitEnabled() {
+    return splitEnabled;
   }
 
-  private Comparator<NormalizationPlan> planComparator = new PlanComparator();
+  /**
+   * Return this instance's configured value for {@value MERGE_ENABLED_KEY}.
+   */
+  public boolean isMergeEnabled() {
+    return mergeEnabled;
+  }
 
   /**
-   * Computes next most "urgent" normalization action on the table. Action may be either a split, or
-   * a merge, or no action.
-   * @param table table to normalize
-   * @return normalization plan to execute
+   * Return this instance's configured value for {@value MIN_REGION_COUNT_KEY}.
+   */
+  public int getMinRegionCount() {
+    return minRegionCount;
+  }
+
+  /**
+   * Return this instance's configured value for {@value MERGE_MIN_REGION_AGE_DAYS_KEY}.
    */
+  public Period getMergeMinRegionAge() {
+    return mergeMinRegionAge;
+  }
+
+  /**
+   * Return this instance's configured value for {@value MERGE_MIN_REGION_SIZE_MB_KEY}.
+   */
+  public int getMergeMinRegionSizeMb() {
+    return mergeMinRegionSizeMb;
+  }
+
   @Override
-  public List<NormalizationPlan> computePlanForTable(TableName table) throws HBaseIOException {
+  public void setMasterServices(final MasterServices masterServices) {
+    this.masterServices = masterServices;
+  }
+
+  @Override
+  public void planSkipped(final RegionInfo hri, final PlanType type) {
+    skippedCount[type.ordinal()]++;
+  }
+
+  @Override
+  public long getSkippedCount(NormalizationPlan.PlanType type) {
+    return skippedCount[type.ordinal()];
+  }
+
+  @Override
+  public List<NormalizationPlan> computePlansForTable(TableName table) {
     if (table == null || table.isSystemTable()) {
       LOG.debug("Normalization of system table {} isn't allowed", table);
-      return null;
+      return Collections.emptyList();
     }
-    boolean splitEnabled = isSplitEnabled();
-    boolean mergeEnabled = isMergeEnabled();
+    boolean splitEnabled = isSplitEnabled() && isMasterSwitchEnabled(MasterSwitchType.SPLIT);
+    boolean mergeEnabled = isMergeEnabled() && isMasterSwitchEnabled(MasterSwitchType.MERGE);
     if (!mergeEnabled && !splitEnabled) {
-      LOG.debug("Both split and merge are disabled for table: {}", table);
-      return null;
+      LOG.debug("Both split and merge are disabled. Skipping normalization of table: {}", table);
+      return Collections.emptyList();
     }
+
     List<NormalizationPlan> plans = new ArrayList<>();
-    List<RegionInfo> tableRegions =
-        masterServices.getAssignmentManager().getRegionStates().getRegionsOfTable(table);
+    List<RegionInfo> tableRegions = masterServices.getAssignmentManager()
+      .getRegionStates()
+      .getRegionsOfTable(table);

Review comment:
       I've pulled out the common state into an inner class; it's not so bad after all. PTAL.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org