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/09/24 23:10:10 UTC

[GitHub] [hbase] ndimiduk opened a new pull request #2454: HBASE-24628 WIP Normalizer needs a throttle

ndimiduk opened a new pull request #2454:
URL: https://github.com/apache/hbase/pull/2454


   Implement a rate limiter for the normalizer. Implemented in terms of
   MB/sec of affacted region size (the same metrics used to make
   normalization decisions). Uses Guava `RateLimiter` to perform the
   resource accounting. `RateLimiter` works by blocking (uninterruptible
   😖) the calling thread. Thus, the whole construction of the normalizer
   subsystem needed refactoring. See the provided `package-info.java` for
   an overview of this new structure.


----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r500199740



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -371,7 +358,11 @@ private boolean skipForMerge(final RegionStates regionStates, final RegionInfo r
       final long nextSizeMb = getRegionSizeMB(next);
       // always merge away empty regions when they present themselves.
       if (currentSizeMb == 0 || nextSizeMb == 0 || currentSizeMb + nextSizeMb < avgRegionSizeMb) {
-        plans.add(new MergeNormalizationPlan(current, next));
+        final MergeNormalizationPlan plan = new MergeNormalizationPlan.Builder()
+          .addTarget(current, currentSizeMb)
+          .addTarget(next, nextSizeMb)

Review comment:
       Sure this is not normalizer's job to validate but by keeping target regions as list, we might keep odd no of regions a while just to abandon them later by `MergeTableRegionsProcedure#checkRegionsToMerge` if we keep it unbounded list.
   How about `List<Pair<NormalizerRegionInfo, NormalizerRegionInfo>>` as target regions in `MergeNormalizationPlan`? 
   `NormalizerRegionInfo` can just contain `regionInfo` and `regionSize`.




----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-701818539


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   4m 15s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 31s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  8s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m 36s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 41s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 56s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  0s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  0s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m 24s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 39s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 146m  6s |  hbase-server in the patch failed.  |
   |  |   | 179m 21s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/4/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 28289da50c64 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ee02e673b7 |
   | Default Java | 1.8.0_232 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/4/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/4/testReport/ |
   | Max. process+thread count | 4482 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/4/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-703987864


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 24s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m  6s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   1m 54s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 22s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m  7s |  hbase-server: The patch generated 0 new + 145 unchanged - 7 fixed = 145 total (was 152)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  11m 19s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | -1 :x: |  spotbugs  |   2m 15s |  hbase-server generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 15s |  The patch does not generate ASF License warnings.  |
   |  |   |  32m 40s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-server |
   |  |  Possible doublecheck on org.apache.hadoop.hbase.master.normalizer.RegionNormalizerManager.started in org.apache.hadoop.hbase.master.normalizer.RegionNormalizerManager.start()  At RegionNormalizerManager.java:org.apache.hadoop.hbase.master.normalizer.RegionNormalizerManager.start()  At RegionNormalizerManager.java:[lines 70-74] |
   |  |  Possible doublecheck on org.apache.hadoop.hbase.master.normalizer.RegionNormalizerManager.stopped in org.apache.hadoop.hbase.master.normalizer.RegionNormalizerManager.stop()  At RegionNormalizerManager.java:org.apache.hadoop.hbase.master.normalizer.RegionNormalizerManager.stop()  At RegionNormalizerManager.java:[lines 90-94] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/8/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 66b9ea3c8a1b 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9f238bd79e |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/8/artifact/yetus-general-check/output/new-spotbugs-hbase-server.html |
   | Max. process+thread count | 94 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/8/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-702591347


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 44s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 11s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m 24s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 20s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 14s |  hbase-server: The patch generated 0 new + 145 unchanged - 7 fixed = 145 total (was 152)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  13m 20s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   2m 23s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 14s |  The patch does not generate ASF License warnings.  |
   |  |   |  37m  5s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/6/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 092a9bd98932 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 3b91a15183 |
   | Max. process+thread count | 94 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/6/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] huaxiangsun commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
huaxiangsun commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r499116161



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorker.java
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.RateLimiter;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+
+/**
+ * Consumes normalization request targets ({@link TableName}s) off the
+ * {@link RegionNormalizerWorkQueue}, dispatches them to the {@link RegionNormalizer},
+ * and executes the resulting {@link NormalizationPlan}s.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorker implements Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerWorker.class);
+  private static final String RATE_LIMIT_BYTES_PER_SEC_KEY =
+    "hbase.normalizer.throughput.max_bytes_per_sec";
+  private static final long RATE_UNLIMITED_BYTES = 1_000_000_000_000L; // 1TB/sec
+
+  private final MasterServices masterServices;
+  private final RegionNormalizer regionNormalizer;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RateLimiter rateLimiter;
+
+  private final long[] skippedCount;
+  private long splitPlanCount;
+  private long mergePlanCount;
+
+  public RegionNormalizerWorker(
+    final Configuration configuration,
+    final MasterServices masterServices,
+    final RegionNormalizer regionNormalizer,
+    final RegionNormalizerWorkQueue<TableName> workQueue
+  ) {
+    this.masterServices = masterServices;
+    this.regionNormalizer = regionNormalizer;
+    this.workQueue = workQueue;
+    this.skippedCount = new long[NormalizationPlan.PlanType.values().length];
+    this.splitPlanCount = 0;
+    this.mergePlanCount = 0;
+    this.rateLimiter = loadRateLimiter(configuration);
+  }
+
+  private static RateLimiter loadRateLimiter(final Configuration configuration) {
+    long rateLimitBytes =
+      configuration.getLongBytes(RATE_LIMIT_BYTES_PER_SEC_KEY, RATE_UNLIMITED_BYTES);
+    long rateLimitMbs = rateLimitBytes / 1_000_000L;
+    if (rateLimitMbs <= 0) {
+      LOG.warn("Configured value {}={} is <= 1MB. Falling back to default.",
+        RATE_LIMIT_BYTES_PER_SEC_KEY, rateLimitBytes);
+      rateLimitBytes = RATE_UNLIMITED_BYTES;
+      rateLimitMbs = RATE_UNLIMITED_BYTES / 1_000_000L;
+    }
+    LOG.info("Normalizer rate limit set to {}",
+      rateLimitBytes == RATE_UNLIMITED_BYTES ? "unlimited" : rateLimitMbs + " MB/sec");
+    return RateLimiter.create(rateLimitMbs);
+  }
+
+  /**
+   * @see RegionNormalizerManager#planSkipped(NormalizationPlan.PlanType)
+   */
+  void planSkipped(NormalizationPlan.PlanType type) {
+    synchronized (skippedCount) {
+      // updates come here via procedure threads, so synchronize access to this counter.
+      skippedCount[type.ordinal()]++;
+    }
+  }
+
+  /**
+   * @see RegionNormalizerManager#getSkippedCount(NormalizationPlan.PlanType)
+   */
+  long getSkippedCount(NormalizationPlan.PlanType type) {
+    return skippedCount[type.ordinal()];
+  }
+
+  /**
+   * @see HMaster#getSplitPlanCount()
+   */
+  long getSplitPlanCount() {
+    return splitPlanCount;
+  }
+
+  /**
+   * @see HMaster#getMergePlanCount()
+   */
+  long getMergePlanCount() {
+    return mergePlanCount;
+  }
+
+  @Override
+  public void run() {
+    while (true) {
+      if (Thread.interrupted()) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+      final TableName tableName;
+      try {
+        tableName = workQueue.take();
+      } catch (InterruptedException e) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+
+      final List<NormalizationPlan> plans = calculatePlans(tableName);
+      submitPlans(plans);
+    }
+  }
+
+  private List<NormalizationPlan> calculatePlans(final TableName tableName) {
+    try {
+      final TableDescriptor tblDesc = masterServices.getTableDescriptors().get(tableName);
+      if (tblDesc != null && !tblDesc.isNormalizationEnabled()) {
+        LOG.debug("Skipping table {} because normalization is disabled in its table properties.",
+          tableName);
+        return Collections.emptyList();
+      }
+    } catch (IOException e) {
+      LOG.debug("Skipping table {} because unable to access its table descriptor.", tableName, e);
+      return Collections.emptyList();
+    }
+
+    if (masterServices.skipRegionManagementAction("region normalizer")) {
+      return Collections.emptyList();
+    }
+
+    final List<NormalizationPlan> plans = regionNormalizer.computePlansForTable(tableName);
+    if (CollectionUtils.isEmpty(plans)) {
+      LOG.debug("No normalization required for table {}.", tableName);
+      return Collections.emptyList();
+    }
+    return plans;
+  }
+
+  private void submitPlans(final List<NormalizationPlan> plans) {
+    // as of this writing, `plan.submit()` is non-blocking and uses Async Admin APIs to submit
+    // task, so there's no artificial rate-limiting of merge/split requests due to this serial loop.
+    for (NormalizationPlan plan : plans) {
+      switch (plan.getType()) {
+        case MERGE: {
+          submitMergePlan((MergeNormalizationPlan) plan);
+          break;
+        }
+        case SPLIT: {
+          submitSplitPlan((SplitNormalizationPlan) plan);
+          break;
+        }
+        case NONE:
+          LOG.debug("Nothing to do for {} with PlanType=NONE. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+        default:
+          LOG.warn("Plan {} is of an unrecognized PlanType. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+      }
+    }
+  }
+
+  /**
+   * Interacts with {@link MasterServices} in order to execute a plan.
+   */
+  private void submitMergePlan(final MergeNormalizationPlan plan) {
+    final int totalSizeMb;
+    try {
+      final long totalSizeMbLong = plan.getNormalizationTargets()
+        .stream()
+        .mapToLong(NormalizationTarget::getRegionSizeMb)
+        .reduce(0, Math::addExact);
+      totalSizeMb = Math.toIntExact(totalSizeMbLong);
+    } catch (ArithmeticException e) {
+      LOG.debug("Sum of merge request size overflows rate limiter data type. {}", plan);
+      planSkipped(plan.getType());
+      return;
+    }
+
+    final RegionInfo[] infos = plan.getNormalizationTargets()
+      .stream()
+      .map(NormalizationTarget::getRegionInfo)
+      .toArray(RegionInfo[]::new);
+    final long rateLimitedSecs = Math.round(rateLimiter.acquire(Math.max(1, totalSizeMb)));
+    LOG.debug("Rate limiting delayed this operation by {}", Duration.ofSeconds(rateLimitedSecs));

Review comment:
       If I understand this correctly, this ratelimiter.acquire() needs to be moved after the mergeRegions() call. mergeRegions() is an asynchronous call, so while it is being run at the background, rateLimiter.acquire() will blocked for some time before it picks up the next action. 
   




----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r500842594



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -371,7 +358,11 @@ private boolean skipForMerge(final RegionStates regionStates, final RegionInfo r
       final long nextSizeMb = getRegionSizeMB(next);
       // always merge away empty regions when they present themselves.
       if (currentSizeMb == 0 || nextSizeMb == 0 || currentSizeMb + nextSizeMb < avgRegionSizeMb) {
-        plans.add(new MergeNormalizationPlan(current, next));
+        final MergeNormalizationPlan plan = new MergeNormalizationPlan.Builder()
+          .addTarget(current, currentSizeMb)
+          .addTarget(next, nextSizeMb)

Review comment:
       I don't have strong opinion for API change since merge system is already taking care of validation.
   
   You might prefer adding a small javadoc for `Builder.addTarget()` explaining to add even no of regions with the sequence in which client wants to merge them?




----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-703967636


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 51s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 11s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 18s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m 48s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 50s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 46s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 20s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 20s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m  9s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 46s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 140m 59s |  hbase-server in the patch failed.  |
   |  |   | 174m  2s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/7/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux ae55231dcb05 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 16251dbb53 |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/7/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/7/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/7/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/7/testReport/ |
   | Max. process+thread count | 3681 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/7/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r495992554



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java
##########
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is
+ * an ordered collection class that has the following properties:
+ * <ul>
+ *   <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
+ *   <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
+ *   <li>Work is retrieved on a FIFO policy.</li>
+ *   <li>Work retrieval blocks the calling thread until new work is available, as a
+ *     {@link BlockingQueue}.</li>
+ *   <li>Allows a producer to insert an item at the head of the queue, if desired.</li>
+ * </ul>
+ * Assumes low-frequency and low-parallelism concurrent access, so protects state using a
+ * simplistic synchronization strategy.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorkQueue<E> {
+
+  /** Underlying storage structure that gives us the Set behavior and FIFO retrieval policy. */
+  private LinkedHashSet<E> delegate;

Review comment:
       nit: `private Set<E> delegate` ?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorker.java
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.RateLimiter;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+
+/**
+ * Consumes normalization request targets ({@link TableName}s) off the
+ * {@link RegionNormalizerWorkQueue}, dispatches them to the {@link RegionNormalizer},
+ * and executes the resulting {@link NormalizationPlan}s.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorker implements Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerWorker.class);
+  private static final String RATE_LIMIT_BYTES_PER_SEC_KEY =
+    "hbase.normalizer.throughput.max_bytes_per_sec";
+  private static final long RATE_UNLIMITED_BYTES = 1_000_000_000_000L; // 1TB/sec
+
+  private final MasterServices masterServices;
+  private final RegionNormalizer regionNormalizer;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RateLimiter rateLimiter;
+
+  private final long[] skippedCount;
+  private long splitPlanCount;
+  private long mergePlanCount;
+
+  public RegionNormalizerWorker(
+    final Configuration configuration,
+    final MasterServices masterServices,
+    final RegionNormalizer regionNormalizer,
+    final RegionNormalizerWorkQueue<TableName> workQueue
+  ) {
+    this.masterServices = masterServices;
+    this.regionNormalizer = regionNormalizer;
+    this.workQueue = workQueue;
+    this.skippedCount = new long[NormalizationPlan.PlanType.values().length];
+    this.splitPlanCount = 0;
+    this.mergePlanCount = 0;
+    this.rateLimiter = loadRateLimiter(configuration);
+  }
+
+  private static RateLimiter loadRateLimiter(final Configuration configuration) {
+    long rateLimitBytes =
+      configuration.getLongBytes(RATE_LIMIT_BYTES_PER_SEC_KEY, RATE_UNLIMITED_BYTES);
+    long rateLimitMbs = rateLimitBytes / 1_000_000L;
+    if (rateLimitMbs <= 0) {
+      LOG.warn("Configured value {}={} is <= 1MB. Falling back to default.",

Review comment:
       nit: `<= 0MB` ?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java
##########
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is
+ * an ordered collection class that has the following properties:
+ * <ul>
+ *   <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
+ *   <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
+ *   <li>Work is retrieved on a FIFO policy.</li>
+ *   <li>Work retrieval blocks the calling thread until new work is available, as a
+ *     {@link BlockingQueue}.</li>
+ *   <li>Allows a producer to insert an item at the head of the queue, if desired.</li>
+ * </ul>
+ * Assumes low-frequency and low-parallelism concurrent access, so protects state using a
+ * simplistic synchronization strategy.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorkQueue<E> {
+
+  /** Underlying storage structure that gives us the Set behavior and FIFO retrieval policy. */
+  private LinkedHashSet<E> delegate;
+
+  // the locking structure used here follows the example found in LinkedBlockingQueue. The
+  // difference is that our locks guard access to `delegate` rather than the head node.
+
+  /** Lock held by take, poll, etc */
+  private final ReentrantLock takeLock;
+
+  /** Wait queue for waiting takes */
+  private final Condition notEmpty;
+
+  /** Lock held by put, offer, etc */
+  private final ReentrantLock putLock;
+
+  public RegionNormalizerWorkQueue() {
+    delegate = new LinkedHashSet<>();
+    takeLock = new ReentrantLock();
+    notEmpty = takeLock.newCondition();
+    putLock = new ReentrantLock();
+  }
+
+  /**
+   * Signals a waiting take. Called only from put/offer (which do not
+   * otherwise ordinarily lock takeLock.)
+   */
+  private void signalNotEmpty() {
+    final ReentrantLock takeLock = this.takeLock;
+    takeLock.lock();
+    try {
+      notEmpty.signal();
+    } finally {
+      takeLock.unlock();
+    }
+  }
+
+  /**
+   * Locks to prevent both puts and takes.
+   */
+  private void fullyLock() {
+    putLock.lock();
+    takeLock.lock();
+  }
+
+  /**
+   * Unlocks to allow both puts and takes.
+   */
+  private void fullyUnlock() {
+    takeLock.unlock();
+    putLock.unlock();
+  }
+
+  /**
+   * Inserts the specified element at the tail of the queue, if it's not already present.
+   *
+   * @param e the element to add
+   */
+  public void put(E e) {
+    if (e == null) {
+      throw new NullPointerException();
+    }
+
+    putLock.lock();
+    try {
+      delegate.add(e);
+    } finally {
+      putLock.unlock();
+    }
+
+    if (!delegate.isEmpty()) {
+      signalNotEmpty();
+    }
+  }
+
+  /**
+   * Inserts the specified element at the head of the queue.
+   *
+   * @param e the element to add
+   */
+  public void putFirst(E e) {
+    if (e == null) {
+      throw new NullPointerException();
+    }
+    putAllFirst(Collections.singleton(e));
+  }
+
+  /**
+   * Inserts the specified elements at the tail of the queue. Any elements already present in
+   * the queue are ignored.
+   *
+   * @param c the elements to add
+   */
+  public void putAll(Collection<? extends E> c) {
+    if (c == null) {
+      throw new NullPointerException();
+    }
+
+    putLock.lock();
+    try {
+      delegate.addAll(c);
+    } finally {
+      putLock.unlock();
+    }
+
+    if (!delegate.isEmpty()) {
+      signalNotEmpty();
+    }
+  }
+
+  /**
+   * Inserts the specified elements at the head of the queue.
+   *
+   * @param c the elements to add
+   */
+  public void putAllFirst(Collection<? extends E> c) {
+    if (c == null) {
+      throw new NullPointerException();
+    }
+
+    fullyLock();
+    try {
+      final LinkedHashSet<E> copy = new LinkedHashSet<>(c.size() + delegate.size());
+      copy.addAll(c);
+      copy.addAll(delegate);
+      delegate = copy;
+    } finally {
+      fullyUnlock();
+    }
+
+    if (!delegate.isEmpty()) {
+      signalNotEmpty();
+    }
+  }
+
+  /**
+   * Retrieves and removes the head of this queue, waiting if necessary
+   * until an element becomes available.
+   *
+   * @return the head of this queue
+   * @throws InterruptedException if interrupted while waiting
+   */
+  public E take() throws InterruptedException {
+    E x;
+    takeLock.lockInterruptibly();
+    try {
+      while (delegate.isEmpty()) {
+        notEmpty.await();
+      }
+      final Iterator<E> iter = delegate.iterator();
+      x = iter.next();
+      iter.remove();

Review comment:
       wondering if this operation should require `putLock`. 
   Edit: I see we take full lock (take + put) while putting data in front of the queue (LinkedHashSet copy operation in `putAllFirst`). As long as `putAllFirst` is the only way to update front elements of the queue, we are good.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java
##########
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is
+ * an ordered collection class that has the following properties:
+ * <ul>
+ *   <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
+ *   <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
+ *   <li>Work is retrieved on a FIFO policy.</li>
+ *   <li>Work retrieval blocks the calling thread until new work is available, as a
+ *     {@link BlockingQueue}.</li>
+ *   <li>Allows a producer to insert an item at the head of the queue, if desired.</li>
+ * </ul>
+ * Assumes low-frequency and low-parallelism concurrent access, so protects state using a
+ * simplistic synchronization strategy.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorkQueue<E> {
+
+  /** Underlying storage structure that gives us the Set behavior and FIFO retrieval policy. */
+  private LinkedHashSet<E> delegate;
+
+  // the locking structure used here follows the example found in LinkedBlockingQueue. The
+  // difference is that our locks guard access to `delegate` rather than the head node.
+
+  /** Lock held by take, poll, etc */
+  private final ReentrantLock takeLock;
+
+  /** Wait queue for waiting takes */
+  private final Condition notEmpty;
+
+  /** Lock held by put, offer, etc */
+  private final ReentrantLock putLock;
+
+  public RegionNormalizerWorkQueue() {
+    delegate = new LinkedHashSet<>();
+    takeLock = new ReentrantLock();
+    notEmpty = takeLock.newCondition();
+    putLock = new ReentrantLock();
+  }
+
+  /**
+   * Signals a waiting take. Called only from put/offer (which do not
+   * otherwise ordinarily lock takeLock.)
+   */
+  private void signalNotEmpty() {
+    final ReentrantLock takeLock = this.takeLock;

Review comment:
       can we directly use (without local var) `this.takeLock`.lock() and .unlock() ? or assigning to local variable is for any extra level of thread safety purpose?




----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r501326583



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationTarget.java
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A POJO that caries details about a region selected for normalization through the pipeline.
+ */
+@InterfaceAudience.Private
+public class NormalizationTarget {

Review comment:
       I went through all classes/interfaces in the package and left only those necessary at `public`. The comment on the package javadoc also explains this explicitly.




----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-704797742


   Will try to complete another pass by EOD.


----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-705285135


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 48s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 34s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m  5s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m  0s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 38s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m  9s |  hbase-server: The patch generated 14 new + 148 unchanged - 7 fixed = 162 total (was 155)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  22m 14s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  spotbugs  |   2m 26s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 13s |  The patch does not generate ASF License warnings.  |
   |  |   |  45m 52s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 8ca82291e3c1 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / a8c49a6463 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 94 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r499443807



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -371,7 +358,11 @@ private boolean skipForMerge(final RegionStates regionStates, final RegionInfo r
       final long nextSizeMb = getRegionSizeMB(next);
       // always merge away empty regions when they present themselves.
       if (currentSizeMb == 0 || nextSizeMb == 0 || currentSizeMb + nextSizeMb < avgRegionSizeMb) {
-        plans.add(new MergeNormalizationPlan(current, next));
+        final MergeNormalizationPlan plan = new MergeNormalizationPlan.Builder()
+          .addTarget(current, currentSizeMb)
+          .addTarget(next, nextSizeMb)

Review comment:
       > Regardless, maybe we want to validate `list_size() % 2 == 0` in `MergeNormalizationPlan` constructor which initializes `normalizationTargets`?
   
   I still think that this validation is better suitable for this current PR because we are introducing at least the flexibility to submit variable no of target regions for merging.




----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r497866798



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorker.java
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.RateLimiter;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+
+/**
+ * Consumes normalization request targets ({@link TableName}s) off the
+ * {@link RegionNormalizerWorkQueue}, dispatches them to the {@link RegionNormalizer},
+ * and executes the resulting {@link NormalizationPlan}s.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorker implements Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerWorker.class);
+  private static final String RATE_LIMIT_BYTES_PER_SEC_KEY =
+    "hbase.normalizer.throughput.max_bytes_per_sec";
+  private static final long RATE_UNLIMITED_BYTES = 1_000_000_000_000L; // 1TB/sec
+
+  private final MasterServices masterServices;
+  private final RegionNormalizer regionNormalizer;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RateLimiter rateLimiter;
+
+  private final long[] skippedCount;
+  private long splitPlanCount;
+  private long mergePlanCount;
+
+  public RegionNormalizerWorker(
+    final Configuration configuration,
+    final MasterServices masterServices,
+    final RegionNormalizer regionNormalizer,
+    final RegionNormalizerWorkQueue<TableName> workQueue
+  ) {
+    this.masterServices = masterServices;
+    this.regionNormalizer = regionNormalizer;
+    this.workQueue = workQueue;
+    this.skippedCount = new long[NormalizationPlan.PlanType.values().length];
+    this.splitPlanCount = 0;
+    this.mergePlanCount = 0;
+    this.rateLimiter = loadRateLimiter(configuration);
+  }
+
+  private static RateLimiter loadRateLimiter(final Configuration configuration) {
+    long rateLimitBytes =
+      configuration.getLongBytes(RATE_LIMIT_BYTES_PER_SEC_KEY, RATE_UNLIMITED_BYTES);
+    long rateLimitMbs = rateLimitBytes / 1_000_000L;
+    if (rateLimitMbs <= 0) {
+      LOG.warn("Configured value {}={} is <= 1MB. Falling back to default.",
+        RATE_LIMIT_BYTES_PER_SEC_KEY, rateLimitBytes);
+      rateLimitBytes = RATE_UNLIMITED_BYTES;
+      rateLimitMbs = RATE_UNLIMITED_BYTES / 1_000_000L;
+    }
+    LOG.info("Normalizer rate limit set to {}",
+      rateLimitBytes == RATE_UNLIMITED_BYTES ? "unlimited" : rateLimitMbs + " MB/sec");
+    return RateLimiter.create(rateLimitMbs);
+  }
+
+  /**
+   * @see RegionNormalizerManager#planSkipped(NormalizationPlan.PlanType)
+   */
+  void planSkipped(NormalizationPlan.PlanType type) {
+    synchronized (skippedCount) {
+      // updates come here via procedure threads, so synchronize access to this counter.
+      skippedCount[type.ordinal()]++;
+    }
+  }
+
+  /**
+   * @see RegionNormalizerManager#getSkippedCount(NormalizationPlan.PlanType)
+   */
+  long getSkippedCount(NormalizationPlan.PlanType type) {
+    return skippedCount[type.ordinal()];
+  }
+
+  /**
+   * @see HMaster#getSplitPlanCount()
+   */
+  long getSplitPlanCount() {
+    return splitPlanCount;
+  }
+
+  /**
+   * @see HMaster#getMergePlanCount()
+   */
+  long getMergePlanCount() {
+    return mergePlanCount;
+  }
+
+  @Override
+  public void run() {
+    while (true) {
+      if (Thread.interrupted()) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+      final TableName tableName;
+      try {
+        tableName = workQueue.take();
+      } catch (InterruptedException e) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+
+      final List<NormalizationPlan> plans = calculatePlans(tableName);
+      submitPlans(plans);
+    }
+  }
+
+  private List<NormalizationPlan> calculatePlans(final TableName tableName) {
+    try {
+      final TableDescriptor tblDesc = masterServices.getTableDescriptors().get(tableName);
+      if (tblDesc != null && !tblDesc.isNormalizationEnabled()) {
+        LOG.debug("Skipping table {} because normalization is disabled in its table properties.",
+          tableName);
+        return Collections.emptyList();
+      }
+    } catch (IOException e) {
+      LOG.debug("Skipping table {} because unable to access its table descriptor.", tableName, e);
+      return Collections.emptyList();
+    }
+
+    if (masterServices.skipRegionManagementAction("region normalizer")) {
+      return Collections.emptyList();
+    }
+
+    final List<NormalizationPlan> plans = regionNormalizer.computePlansForTable(tableName);
+    if (CollectionUtils.isEmpty(plans)) {
+      LOG.debug("No normalization required for table {}.", tableName);
+      return Collections.emptyList();
+    }
+    return plans;
+  }
+
+  private void submitPlans(final List<NormalizationPlan> plans) {
+    // as of this writing, `plan.submit()` is non-blocking and uses Async Admin APIs to submit
+    // task, so there's no artificial rate-limiting of merge/split requests due to this serial loop.
+    for (NormalizationPlan plan : plans) {
+      switch (plan.getType()) {
+        case MERGE: {
+          submitMergePlan((MergeNormalizationPlan) plan);
+          break;
+        }
+        case SPLIT: {
+          submitSplitPlan((SplitNormalizationPlan) plan);
+          break;
+        }
+        case NONE:
+          LOG.debug("Nothing to do for {} with PlanType=NONE. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+        default:
+          LOG.warn("Plan {} is of an unrecognized PlanType. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+      }
+    }
+  }
+
+  /**
+   * Interacts with {@link MasterServices} in order to execute a plan.
+   */
+  private void submitMergePlan(final MergeNormalizationPlan plan) {
+    final int totalSizeMb;
+    try {
+      final long totalSizeMbLong = plan.getNormalizationTargets()
+        .stream()
+        .mapToLong(NormalizationTarget::getRegionSizeMb)
+        .reduce(0, Math::addExact);
+      totalSizeMb = Math.toIntExact(totalSizeMbLong);
+    } catch (ArithmeticException e) {
+      LOG.debug("Sum of merge request size overflows rate limiter data type. {}", plan);
+      planSkipped(plan.getType());
+      return;
+    }
+
+    final RegionInfo[] infos = plan.getNormalizationTargets()
+      .stream()
+      .map(NormalizationTarget::getRegionInfo)
+      .toArray(RegionInfo[]::new);
+    final long rateLimitedSecs = Math.round(rateLimiter.acquire(Math.max(1, totalSizeMb)));
+    LOG.debug("Rate limiting delayed this operation by {}", Duration.ofSeconds(rateLimitedSecs));

Review comment:
       The plan is logged immediately after. It's all coming from the same thread, so I think it's okay to omit `plan` from this log line.




----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r500199740



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -371,7 +358,11 @@ private boolean skipForMerge(final RegionStates regionStates, final RegionInfo r
       final long nextSizeMb = getRegionSizeMB(next);
       // always merge away empty regions when they present themselves.
       if (currentSizeMb == 0 || nextSizeMb == 0 || currentSizeMb + nextSizeMb < avgRegionSizeMb) {
-        plans.add(new MergeNormalizationPlan(current, next));
+        final MergeNormalizationPlan plan = new MergeNormalizationPlan.Builder()
+          .addTarget(current, currentSizeMb)
+          .addTarget(next, nextSizeMb)

Review comment:
       Sure this is not normalizer's job to validate but by keeping target regions as list, we tend to have a probability of keeping odd no of regions a while just to get them abandoned later by `MergeTableRegionsProcedure#checkRegionsToMerge` if we keep it unbounded list. Not that `SimpleRegionNormalizer` will do it, it's just a probability.
   
   How about `List<Pair<NormalizerRegionInfo, NormalizerRegionInfo>>` as target regions in `MergeNormalizationPlan`? 
   New POJO `NormalizerRegionInfo` can just contain `regionInfo` and `regionSize`. This way, it's a clear indication of sending list of pair of regions to merge.




----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r501290825



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerManager.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * This class encapsulates the details of the {@link RegionNormalizer} subsystem.
+ */
+@InterfaceAudience.Private
+public class RegionNormalizerManager {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerManager.class);
+
+  private final RegionNormalizerTracker regionNormalizerTracker;
+  private final RegionNormalizerChore regionNormalizerChore;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RegionNormalizerWorker worker;
+  private final ExecutorService pool;
+
+  private final Object startStopLock = new Object();
+  private boolean started = false;
+  private boolean stopped = false;
+
+  public RegionNormalizerManager(
+    @NonNull  final RegionNormalizerTracker regionNormalizerTracker,
+    @Nullable final RegionNormalizerChore regionNormalizerChore,
+    @Nullable final RegionNormalizerWorkQueue<TableName> workQueue,
+    @Nullable final RegionNormalizerWorker worker
+  ) {
+    this.regionNormalizerTracker = regionNormalizerTracker;
+    this.regionNormalizerChore = regionNormalizerChore;
+    this.workQueue = workQueue;
+    this.worker = worker;
+    this.pool = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+      .setDaemon(true)
+      .setNameFormat("normalizer-worker-%d")
+      .setUncaughtExceptionHandler(
+        (thread, throwable) ->
+          LOG.error("Uncaught exception, worker thread likely terminated.", throwable))
+      .build());
+  }
+
+  public void start() {
+    if (started) {
+      return;
+    }
+    synchronized (startStopLock) {
+      if (started) {
+        return;
+      }
+      regionNormalizerTracker.start();
+      if (worker != null) {
+        // worker will be null when master is in maintenance mode.
+        pool.submit(worker);
+      }
+      started = true;
+    }
+  }
+
+  public void stop() {
+    if (!started) {
+      throw new IllegalStateException("calling `stop` without first calling `start`.");
+    }
+    if (stopped) {

Review comment:
       Here's a WIP to move over to Guava Service. Let me know if you think it's worth the effort. https://github.com/ndimiduk/hbase/commit/7d7bdb4f2affba2f4ff82871ec48c7d850538a3f




----------------------------------------------------------------
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



[GitHub] [hbase] saintstack commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r501292281



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java
##########
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is
+ * an ordered collection class that has the following properties:
+ * <ul>
+ *   <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
+ *   <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
+ *   <li>Work is retrieved on a FIFO policy.</li>

Review comment:
       ok




----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-704034999


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  3s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 49s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  0s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m 11s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 35s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 49s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 58s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 58s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m 11s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 35s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 192m 31s |  hbase-server in the patch failed.  |
   |  |   | 220m 32s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/8/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux eaa3047605bd 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9f238bd79e |
   | Default Java | 1.8.0_232 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/8/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/8/testReport/ |
   | Max. process+thread count | 3364 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/8/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-698761075


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  2s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   6m  2s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 44s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   3m 19s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 18s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 50s |  hbase-server: The patch generated 1 new + 145 unchanged - 7 fixed = 146 total (was 152)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  17m 40s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   3m 14s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 15s |  The patch does not generate ASF License warnings.  |
   |  |   |  51m 19s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 755c155f241b 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 687e53b7e4 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 94 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/2/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r500048833



##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterChoreScheduled.java
##########
@@ -114,22 +113,27 @@ public void testDefaultScheduledChores() throws Exception {
     hbckChoreTestChoreField.testIfChoreScheduled(hbckChore);
   }
 
-
+  /**
+   * Reflect into the {@link HMaster} instance and find by field name a specified instance
+   * of {@link ScheduledChore}.
+   */
   private static class TestChoreField<E extends ScheduledChore> {
 
-    private E getChoreObj(String fieldName) throws NoSuchFieldException,
-        IllegalAccessException {
-      Field masterField = HMaster.class.getDeclaredField(fieldName);
-      masterField.setAccessible(true);
-      E choreFieldVal = (E) masterField.get(hMaster);
-      return choreFieldVal;
+    @SuppressWarnings("unchecked")
+    private E getChoreObj(String fieldName) {
+      try {
+        Field masterField = HMaster.class.getDeclaredField(fieldName);
+        masterField.setAccessible(true);
+        return (E) masterField.get(hMaster);
+      } catch (Exception e) {
+        throw new AssertionError(
+          "Unable to retrieve field '" + fieldName + "' from HMaster instance.", e);

Review comment:
       Darn it, quite a thing I missed 🤦




----------------------------------------------------------------
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



[GitHub] [hbase] saintstack commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r500018771



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationTarget.java
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A POJO that caries details about a region selected for normalization through the pipeline.
+ */
+@InterfaceAudience.Private
+public class NormalizationTarget {
+  private final RegionInfo regionInfo;
+  private final long regionSizeMb;
+
+  public NormalizationTarget(final RegionInfo regionInfo, final long regionSizeMb) {

Review comment:
       ditto
   

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationTarget.java
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A POJO that caries details about a region selected for normalization through the pipeline.
+ */
+@InterfaceAudience.Private
+public class NormalizationTarget {
+  private final RegionInfo regionInfo;
+  private final long regionSizeMb;
+
+  public NormalizationTarget(final RegionInfo regionInfo, final long regionSizeMb) {
+    this.regionInfo = regionInfo;
+    this.regionSizeMb = regionSizeMb;
+  }
+
+  public RegionInfo getRegionInfo() {
+    return regionInfo;
+  }
+
+  public long getRegionSizeMb() {
+    return regionSizeMb;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    NormalizationTarget that = (NormalizationTarget) o;
+
+    return new EqualsBuilder()
+      .append(regionSizeMb, that.regionSizeMb)
+      .append(regionInfo, that.regionInfo)
+      .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37)
+      .append(regionInfo)
+      .append(regionSizeMb)
+      .toHashCode();
+  }
+
+  @Override public String toString() {
+    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)

Review comment:
       Should we use this everywhere? ToStringBuilder? Seems ok.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationTarget.java
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A POJO that caries details about a region selected for normalization through the pipeline.
+ */
+@InterfaceAudience.Private
+public class NormalizationTarget {

Review comment:
       Used outside this package? If not, remove the public?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
##########
@@ -354,6 +353,13 @@ long splitRegion(
    */
   boolean isInMaintenanceMode();
 
+  /**
+   * Checks master state before initiating action over region topology.
+   * @param action the name of the action under consideration, for logging.
+   * @return {@code true} when the caller should exit early, {@code false} otherwise.
+   */
+  boolean skipRegionManagementAction(final String action);

Review comment:
       ok

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java
##########
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is
+ * an ordered collection class that has the following properties:
+ * <ul>
+ *   <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
+ *   <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
+ *   <li>Work is retrieved on a FIFO policy.</li>
+ *   <li>Work retrieval blocks the calling thread until new work is available, as a
+ *     {@link BlockingQueue}.</li>
+ *   <li>Allows a producer to insert an item at the head of the queue, if desired.</li>
+ * </ul>
+ * Assumes low-frequency and low-parallelism concurrent access, so protects state using a
+ * simplistic synchronization strategy.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorkQueue<E> {
+
+  /** Underlying storage structure that gives us the Set behavior and FIFO retrieval policy. */
+  private LinkedHashSet<E> delegate;
+
+  // the locking structure used here follows the example found in LinkedBlockingQueue. The
+  // difference is that our locks guard access to `delegate` rather than the head node.
+
+  /** Lock held by take, poll, etc */
+  private final ReentrantLock takeLock;
+
+  /** Wait queue for waiting takes */
+  private final Condition notEmpty;
+
+  /** Lock held by put, offer, etc */
+  private final ReentrantLock putLock;
+
+  public RegionNormalizerWorkQueue() {

Review comment:
       Local to this package? If so, remove the public?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java
##########
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is
+ * an ordered collection class that has the following properties:
+ * <ul>
+ *   <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
+ *   <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
+ *   <li>Work is retrieved on a FIFO policy.</li>

Review comment:
       If I put in something already in the queue, there'll be one instance only.  The new put moves to the head of the queue?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerManager.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * This class encapsulates the details of the {@link RegionNormalizer} subsystem.
+ */
+@InterfaceAudience.Private
+public class RegionNormalizerManager {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerManager.class);
+
+  private final RegionNormalizerTracker regionNormalizerTracker;
+  private final RegionNormalizerChore regionNormalizerChore;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RegionNormalizerWorker worker;
+  private final ExecutorService pool;
+
+  private final Object startStopLock = new Object();
+  private boolean started = false;
+  private boolean stopped = false;
+
+  public RegionNormalizerManager(
+    @NonNull  final RegionNormalizerTracker regionNormalizerTracker,
+    @Nullable final RegionNormalizerChore regionNormalizerChore,
+    @Nullable final RegionNormalizerWorkQueue<TableName> workQueue,
+    @Nullable final RegionNormalizerWorker worker
+  ) {
+    this.regionNormalizerTracker = regionNormalizerTracker;
+    this.regionNormalizerChore = regionNormalizerChore;
+    this.workQueue = workQueue;
+    this.worker = worker;
+    this.pool = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+      .setDaemon(true)
+      .setNameFormat("normalizer-worker-%d")
+      .setUncaughtExceptionHandler(
+        (thread, throwable) ->
+          LOG.error("Uncaught exception, worker thread likely terminated.", throwable))
+      .build());
+  }
+
+  public void start() {
+    if (started) {
+      return;
+    }
+    synchronized (startStopLock) {
+      if (started) {
+        return;
+      }
+      regionNormalizerTracker.start();
+      if (worker != null) {
+        // worker will be null when master is in maintenance mode.
+        pool.submit(worker);
+      }
+      started = true;
+    }
+  }
+
+  public void stop() {
+    if (!started) {
+      throw new IllegalStateException("calling `stop` without first calling `start`.");
+    }
+    if (stopped) {

Review comment:
       Ditto on these. 
   
   Or want to use a guava Service? That too much?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerManager.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * This class encapsulates the details of the {@link RegionNormalizer} subsystem.
+ */
+@InterfaceAudience.Private
+public class RegionNormalizerManager {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerManager.class);
+
+  private final RegionNormalizerTracker regionNormalizerTracker;
+  private final RegionNormalizerChore regionNormalizerChore;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RegionNormalizerWorker worker;
+  private final ExecutorService pool;
+
+  private final Object startStopLock = new Object();
+  private boolean started = false;
+  private boolean stopped = false;
+
+  public RegionNormalizerManager(
+    @NonNull  final RegionNormalizerTracker regionNormalizerTracker,
+    @Nullable final RegionNormalizerChore regionNormalizerChore,
+    @Nullable final RegionNormalizerWorkQueue<TableName> workQueue,
+    @Nullable final RegionNormalizerWorker worker
+  ) {
+    this.regionNormalizerTracker = regionNormalizerTracker;
+    this.regionNormalizerChore = regionNormalizerChore;
+    this.workQueue = workQueue;
+    this.worker = worker;
+    this.pool = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+      .setDaemon(true)
+      .setNameFormat("normalizer-worker-%d")
+      .setUncaughtExceptionHandler(
+        (thread, throwable) ->
+          LOG.error("Uncaught exception, worker thread likely terminated.", throwable))
+      .build());
+  }
+
+  public void start() {
+    if (started) {
+      return;

Review comment:
       Good. Remove this check of started that is outside sync block. We don't come here anyways, the lock is narrow... and check of started under synchronized will always be right.




----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-705861790


   Thanks for the detailed reviews everyone!


----------------------------------------------------------------
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



[GitHub] [hbase] huaxiangsun commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
huaxiangsun commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r499726489



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorker.java
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.RateLimiter;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+
+/**
+ * Consumes normalization request targets ({@link TableName}s) off the
+ * {@link RegionNormalizerWorkQueue}, dispatches them to the {@link RegionNormalizer},
+ * and executes the resulting {@link NormalizationPlan}s.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorker implements Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerWorker.class);
+  private static final String RATE_LIMIT_BYTES_PER_SEC_KEY =
+    "hbase.normalizer.throughput.max_bytes_per_sec";
+  private static final long RATE_UNLIMITED_BYTES = 1_000_000_000_000L; // 1TB/sec
+
+  private final MasterServices masterServices;
+  private final RegionNormalizer regionNormalizer;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RateLimiter rateLimiter;
+
+  private final long[] skippedCount;
+  private long splitPlanCount;
+  private long mergePlanCount;
+
+  public RegionNormalizerWorker(
+    final Configuration configuration,
+    final MasterServices masterServices,
+    final RegionNormalizer regionNormalizer,
+    final RegionNormalizerWorkQueue<TableName> workQueue
+  ) {
+    this.masterServices = masterServices;
+    this.regionNormalizer = regionNormalizer;
+    this.workQueue = workQueue;
+    this.skippedCount = new long[NormalizationPlan.PlanType.values().length];
+    this.splitPlanCount = 0;
+    this.mergePlanCount = 0;
+    this.rateLimiter = loadRateLimiter(configuration);
+  }
+
+  private static RateLimiter loadRateLimiter(final Configuration configuration) {
+    long rateLimitBytes =
+      configuration.getLongBytes(RATE_LIMIT_BYTES_PER_SEC_KEY, RATE_UNLIMITED_BYTES);
+    long rateLimitMbs = rateLimitBytes / 1_000_000L;
+    if (rateLimitMbs <= 0) {
+      LOG.warn("Configured value {}={} is <= 1MB. Falling back to default.",
+        RATE_LIMIT_BYTES_PER_SEC_KEY, rateLimitBytes);
+      rateLimitBytes = RATE_UNLIMITED_BYTES;
+      rateLimitMbs = RATE_UNLIMITED_BYTES / 1_000_000L;
+    }
+    LOG.info("Normalizer rate limit set to {}",
+      rateLimitBytes == RATE_UNLIMITED_BYTES ? "unlimited" : rateLimitMbs + " MB/sec");
+    return RateLimiter.create(rateLimitMbs);
+  }
+
+  /**
+   * @see RegionNormalizerManager#planSkipped(NormalizationPlan.PlanType)
+   */
+  void planSkipped(NormalizationPlan.PlanType type) {
+    synchronized (skippedCount) {
+      // updates come here via procedure threads, so synchronize access to this counter.
+      skippedCount[type.ordinal()]++;
+    }
+  }
+
+  /**
+   * @see RegionNormalizerManager#getSkippedCount(NormalizationPlan.PlanType)
+   */
+  long getSkippedCount(NormalizationPlan.PlanType type) {
+    return skippedCount[type.ordinal()];
+  }
+
+  /**
+   * @see HMaster#getSplitPlanCount()
+   */
+  long getSplitPlanCount() {
+    return splitPlanCount;
+  }
+
+  /**
+   * @see HMaster#getMergePlanCount()
+   */
+  long getMergePlanCount() {
+    return mergePlanCount;
+  }
+
+  @Override
+  public void run() {
+    while (true) {
+      if (Thread.interrupted()) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+      final TableName tableName;
+      try {
+        tableName = workQueue.take();
+      } catch (InterruptedException e) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+
+      final List<NormalizationPlan> plans = calculatePlans(tableName);
+      submitPlans(plans);
+    }
+  }
+
+  private List<NormalizationPlan> calculatePlans(final TableName tableName) {
+    try {
+      final TableDescriptor tblDesc = masterServices.getTableDescriptors().get(tableName);
+      if (tblDesc != null && !tblDesc.isNormalizationEnabled()) {
+        LOG.debug("Skipping table {} because normalization is disabled in its table properties.",
+          tableName);
+        return Collections.emptyList();
+      }
+    } catch (IOException e) {
+      LOG.debug("Skipping table {} because unable to access its table descriptor.", tableName, e);
+      return Collections.emptyList();
+    }
+
+    if (masterServices.skipRegionManagementAction("region normalizer")) {
+      return Collections.emptyList();
+    }
+
+    final List<NormalizationPlan> plans = regionNormalizer.computePlansForTable(tableName);
+    if (CollectionUtils.isEmpty(plans)) {
+      LOG.debug("No normalization required for table {}.", tableName);
+      return Collections.emptyList();
+    }
+    return plans;
+  }
+
+  private void submitPlans(final List<NormalizationPlan> plans) {
+    // as of this writing, `plan.submit()` is non-blocking and uses Async Admin APIs to submit
+    // task, so there's no artificial rate-limiting of merge/split requests due to this serial loop.
+    for (NormalizationPlan plan : plans) {
+      switch (plan.getType()) {
+        case MERGE: {
+          submitMergePlan((MergeNormalizationPlan) plan);
+          break;
+        }
+        case SPLIT: {
+          submitSplitPlan((SplitNormalizationPlan) plan);
+          break;
+        }
+        case NONE:
+          LOG.debug("Nothing to do for {} with PlanType=NONE. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+        default:
+          LOG.warn("Plan {} is of an unrecognized PlanType. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+      }
+    }
+  }
+
+  /**
+   * Interacts with {@link MasterServices} in order to execute a plan.
+   */
+  private void submitMergePlan(final MergeNormalizationPlan plan) {
+    final int totalSizeMb;
+    try {
+      final long totalSizeMbLong = plan.getNormalizationTargets()
+        .stream()
+        .mapToLong(NormalizationTarget::getRegionSizeMb)
+        .reduce(0, Math::addExact);
+      totalSizeMb = Math.toIntExact(totalSizeMbLong);
+    } catch (ArithmeticException e) {
+      LOG.debug("Sum of merge request size overflows rate limiter data type. {}", plan);
+      planSkipped(plan.getType());
+      return;
+    }
+
+    final RegionInfo[] infos = plan.getNormalizationTargets()
+      .stream()
+      .map(NormalizationTarget::getRegionInfo)
+      .toArray(RegionInfo[]::new);
+    final long rateLimitedSecs = Math.round(rateLimiter.acquire(Math.max(1, totalSizeMb)));
+    LOG.debug("Rate limiting delayed this operation by {}", Duration.ofSeconds(rateLimitedSecs));

Review comment:
       Yeah, syntax wise, the latency calculated is for the current action, so it needs to block after the current is submitted.




----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r500845391



##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterChoreScheduled.java
##########
@@ -114,22 +113,27 @@ public void testDefaultScheduledChores() throws Exception {
     hbckChoreTestChoreField.testIfChoreScheduled(hbckChore);
   }
 
-
+  /**
+   * Reflect into the {@link HMaster} instance and find by field name a specified instance
+   * of {@link ScheduledChore}.
+   */
   private static class TestChoreField<E extends ScheduledChore> {
 
-    private E getChoreObj(String fieldName) throws NoSuchFieldException,
-        IllegalAccessException {
-      Field masterField = HMaster.class.getDeclaredField(fieldName);
-      masterField.setAccessible(true);
-      E choreFieldVal = (E) masterField.get(hMaster);
-      return choreFieldVal;
+    @SuppressWarnings("unchecked")
+    private E getChoreObj(String fieldName) {
+      try {
+        Field masterField = HMaster.class.getDeclaredField(fieldName);
+        masterField.setAccessible(true);
+        return (E) masterField.get(hMaster);
+      } catch (Exception e) {
+        throw new AssertionError(
+          "Unable to retrieve field '" + fieldName + "' from HMaster instance.", e);

Review comment:
       Given that this test is fragile in the way we use reflection, having this nice message in error clearly points out what the issue is and dev can just focus on refactor part to ensure chore object that is being refactored is somehow covered in this test. Nice error message 👍 




----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r496771793



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorker.java
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.RateLimiter;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+
+/**
+ * Consumes normalization request targets ({@link TableName}s) off the
+ * {@link RegionNormalizerWorkQueue}, dispatches them to the {@link RegionNormalizer},
+ * and executes the resulting {@link NormalizationPlan}s.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorker implements Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerWorker.class);
+  private static final String RATE_LIMIT_BYTES_PER_SEC_KEY =
+    "hbase.normalizer.throughput.max_bytes_per_sec";
+  private static final long RATE_UNLIMITED_BYTES = 1_000_000_000_000L; // 1TB/sec
+
+  private final MasterServices masterServices;
+  private final RegionNormalizer regionNormalizer;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RateLimiter rateLimiter;
+
+  private final long[] skippedCount;
+  private long splitPlanCount;
+  private long mergePlanCount;
+
+  public RegionNormalizerWorker(
+    final Configuration configuration,
+    final MasterServices masterServices,
+    final RegionNormalizer regionNormalizer,
+    final RegionNormalizerWorkQueue<TableName> workQueue
+  ) {
+    this.masterServices = masterServices;
+    this.regionNormalizer = regionNormalizer;
+    this.workQueue = workQueue;
+    this.skippedCount = new long[NormalizationPlan.PlanType.values().length];
+    this.splitPlanCount = 0;
+    this.mergePlanCount = 0;
+    this.rateLimiter = loadRateLimiter(configuration);
+  }
+
+  private static RateLimiter loadRateLimiter(final Configuration configuration) {
+    long rateLimitBytes =
+      configuration.getLongBytes(RATE_LIMIT_BYTES_PER_SEC_KEY, RATE_UNLIMITED_BYTES);
+    long rateLimitMbs = rateLimitBytes / 1_000_000L;
+    if (rateLimitMbs <= 0) {
+      LOG.warn("Configured value {}={} is <= 1MB. Falling back to default.",
+        RATE_LIMIT_BYTES_PER_SEC_KEY, rateLimitBytes);
+      rateLimitBytes = RATE_UNLIMITED_BYTES;
+      rateLimitMbs = RATE_UNLIMITED_BYTES / 1_000_000L;
+    }
+    LOG.info("Normalizer rate limit set to {}",
+      rateLimitBytes == RATE_UNLIMITED_BYTES ? "unlimited" : rateLimitMbs + " MB/sec");
+    return RateLimiter.create(rateLimitMbs);
+  }
+
+  /**
+   * @see RegionNormalizerManager#planSkipped(NormalizationPlan.PlanType)
+   */
+  void planSkipped(NormalizationPlan.PlanType type) {
+    synchronized (skippedCount) {
+      // updates come here via procedure threads, so synchronize access to this counter.
+      skippedCount[type.ordinal()]++;
+    }
+  }
+
+  /**
+   * @see RegionNormalizerManager#getSkippedCount(NormalizationPlan.PlanType)
+   */
+  long getSkippedCount(NormalizationPlan.PlanType type) {
+    return skippedCount[type.ordinal()];
+  }
+
+  /**
+   * @see HMaster#getSplitPlanCount()
+   */
+  long getSplitPlanCount() {
+    return splitPlanCount;
+  }
+
+  /**
+   * @see HMaster#getMergePlanCount()
+   */
+  long getMergePlanCount() {
+    return mergePlanCount;
+  }
+
+  @Override
+  public void run() {
+    while (true) {
+      if (Thread.interrupted()) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+      final TableName tableName;
+      try {
+        tableName = workQueue.take();
+      } catch (InterruptedException e) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+
+      final List<NormalizationPlan> plans = calculatePlans(tableName);
+      submitPlans(plans);
+    }
+  }
+
+  private List<NormalizationPlan> calculatePlans(final TableName tableName) {
+    try {
+      final TableDescriptor tblDesc = masterServices.getTableDescriptors().get(tableName);
+      if (tblDesc != null && !tblDesc.isNormalizationEnabled()) {
+        LOG.debug("Skipping table {} because normalization is disabled in its table properties.",
+          tableName);
+        return Collections.emptyList();
+      }
+    } catch (IOException e) {
+      LOG.debug("Skipping table {} because unable to access its table descriptor.", tableName, e);
+      return Collections.emptyList();
+    }
+
+    if (masterServices.skipRegionManagementAction("region normalizer")) {
+      return Collections.emptyList();
+    }
+
+    final List<NormalizationPlan> plans = regionNormalizer.computePlansForTable(tableName);
+    if (CollectionUtils.isEmpty(plans)) {
+      LOG.debug("No normalization required for table {}.", tableName);
+      return Collections.emptyList();
+    }
+    return plans;
+  }
+
+  private void submitPlans(final List<NormalizationPlan> plans) {
+    // as of this writing, `plan.submit()` is non-blocking and uses Async Admin APIs to submit
+    // task, so there's no artificial rate-limiting of merge/split requests due to this serial loop.
+    for (NormalizationPlan plan : plans) {
+      switch (plan.getType()) {
+        case MERGE: {
+          submitMergePlan((MergeNormalizationPlan) plan);
+          break;
+        }
+        case SPLIT: {
+          submitSplitPlan((SplitNormalizationPlan) plan);
+          break;
+        }
+        case NONE:
+          LOG.debug("Nothing to do for {} with PlanType=NONE. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+        default:
+          LOG.warn("Plan {} is of an unrecognized PlanType. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+      }
+    }
+  }
+
+  /**
+   * Interacts with {@link MasterServices} in order to execute a plan.
+   */
+  private void submitMergePlan(final MergeNormalizationPlan plan) {
+    final int totalSizeMb;
+    try {
+      final long totalSizeMbLong = plan.getNormalizationTargets()
+        .stream()
+        .mapToLong(NormalizationTarget::getRegionSizeMb)
+        .reduce(0, Math::addExact);

Review comment:
       Looks like `reduce(0, Math::addExact)` is preferred here over `sum()` due to long overflow. If it happens for two regions, perhaps we might get their merge request again? (and again.... in subsequent normalizer runs?)




----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-701798825


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 29s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 19s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  7s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 39s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 42s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m  2s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  7s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  7s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 33s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 40s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 137m 22s |  hbase-server in the patch failed.  |
   |  |   | 165m 18s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/4/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux d69ccb57c71b 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ee02e673b7 |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/4/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/4/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/4/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/4/testReport/ |
   | Max. process+thread count | 4767 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/4/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r499935707



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
##########
@@ -354,6 +353,13 @@ long splitRegion(
    */
   boolean isInMaintenanceMode();
 
+  /**
+   * Checks master state before initiating action over region topology.
+   * @param action the name of the action under consideration, for logging.
+   * @return {@code true} when the caller should exit early, {@code false} otherwise.
+   */
+  boolean skipRegionManagementAction(final String action);

Review comment:
       > What is this? Is it substantial enough to be added to this Interface?
   
   @saintstack I asked myself the very same question. It's this or have a `MasterServices` instance and cast it to `HMaster` because we need something kind-of secret.




----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-703906574


   > As for TestMasterChoreScheduled test, it tries to retrieve all default (unconditional) chores of HMaster by retrieving singleton objects of chores using reflection from HMaster.class.
   
   Oh, clever. Fragile, but clever. Thanks for the pointer @virajjasani .


----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-705314172


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   4m 33s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 22s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  7s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 45s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 42s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m  9s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  8s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  8s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m  6s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 41s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 129m 35s |  hbase-server in the patch failed.  |
   |  |   | 163m 28s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux a735b84304a2 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / a8c49a6463 |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/testReport/ |
   | Max. process+thread count | 4165 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk merged pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk merged pull request #2454:
URL: https://github.com/apache/hbase/pull/2454


   


----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-702379334


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 50s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   7m  8s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 55s |  master passed  |
   | +1 :green_heart: |  shadedjars  |  12m 27s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   1m 19s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   6m 48s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 12s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 12s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |  12m 41s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   1m 12s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 201m 14s |  hbase-server in the patch failed.  |
   |  |   | 251m  7s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/5/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 16b2491783b0 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 11a336a74a |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/5/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/5/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/5/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/5/testReport/ |
   | Max. process+thread count | 4021 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/5/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk merged pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk merged pull request #2454:
URL: https://github.com/apache/hbase/pull/2454


   


----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-701723046


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 23s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 25s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m 48s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 13s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 23s |  hbase-server: The patch generated 0 new + 145 unchanged - 7 fixed = 145 total (was 152)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  15m  4s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   2m 27s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 16s |  The patch does not generate ASF License warnings.  |
   |  |   |  40m 34s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/4/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux e76eeb933fe6 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ee02e673b7 |
   | Max. process+thread count | 95 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/4/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-705210325


   `org.apache.hadoop.hbase.master.TestMasterFailover.testSimpleMasterFailover` is failing reliably. Let me investigate.


----------------------------------------------------------------
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



[GitHub] [hbase] huaxiangsun commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
huaxiangsun commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r498993561



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
##########
@@ -1953,20 +1952,27 @@ public SetNormalizerRunningResponse setNormalizerRunning(RpcController controlle
     rpcPreCheck("setNormalizerRunning");
 
     // Sets normalizer on/off flag in ZK.
-    boolean prevValue = master.getRegionNormalizerTracker().isNormalizerOn();
-    boolean newValue = request.getOn();
-    try {
-      master.getRegionNormalizerTracker().setNormalizerOn(newValue);
-    } catch (KeeperException ke) {
-      LOG.warn("Error flipping normalizer switch", ke);
-    }
+    // TODO: this method is totally broken in terms of atomicity of actions and values read.
+    //  1. The contract has this RPC returning the previous value. There isn't a ZKUtil method
+    //     that lets us retrieve the previous value as part of setting a new value, so we simply
+    //     perform a read before issuing the update. Thus we have a data race opportunity, between
+    //     when the `prevValue` is read and whatever is actually overwritten.
+    //  2. Down in `setNormalizerOn`, the call to `createAndWatch` inside of the catch clause can
+    //     itself fail in the event that the znode already exists. Thus, another data race, between
+    //     when the initial `setData` call is notified of the absence of the target znode and the
+    //     subsequent `createAndWatch`, with another client creating said node.
+    //  That said, there's supposed to be only one active master and thus there's supposed to be
+    //  only one process with the authority to modify the value.

Review comment:
       Nice comments!




----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r501324509



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -371,7 +358,11 @@ private boolean skipForMerge(final RegionStates regionStates, final RegionInfo r
       final long nextSizeMb = getRegionSizeMB(next);
       // always merge away empty regions when they present themselves.
       if (currentSizeMb == 0 || nextSizeMb == 0 || currentSizeMb + nextSizeMb < avgRegionSizeMb) {
-        plans.add(new MergeNormalizationPlan(current, next));
+        final MergeNormalizationPlan plan = new MergeNormalizationPlan.Builder()
+          .addTarget(current, currentSizeMb)
+          .addTarget(next, nextSizeMb)

Review comment:
       I extended the class-level javadoc to remind callers about the expectations of the `mergeRegions` method.




----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r496987197



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -371,7 +358,11 @@ private boolean skipForMerge(final RegionStates regionStates, final RegionInfo r
       final long nextSizeMb = getRegionSizeMB(next);
       // always merge away empty regions when they present themselves.
       if (currentSizeMb == 0 || nextSizeMb == 0 || currentSizeMb + nextSizeMb < avgRegionSizeMb) {
-        plans.add(new MergeNormalizationPlan(current, next));
+        final MergeNormalizationPlan plan = new MergeNormalizationPlan.Builder()
+          .addTarget(current, currentSizeMb)
+          .addTarget(next, nextSizeMb)

Review comment:
       Yes, merging more than two regions at a time is coming in [HBASE-24419](https://issues.apache.org/jira/browse/HBASE-24419). I started working on it yesterday. And yes, this constructor API for MergeNormalizationPlan will likely change because of it.




----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r496654064



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java
##########
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is
+ * an ordered collection class that has the following properties:
+ * <ul>
+ *   <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
+ *   <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
+ *   <li>Work is retrieved on a FIFO policy.</li>
+ *   <li>Work retrieval blocks the calling thread until new work is available, as a
+ *     {@link BlockingQueue}.</li>
+ *   <li>Allows a producer to insert an item at the head of the queue, if desired.</li>
+ * </ul>
+ * Assumes low-frequency and low-parallelism concurrent access, so protects state using a
+ * simplistic synchronization strategy.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorkQueue<E> {
+
+  /** Underlying storage structure that gives us the Set behavior and FIFO retrieval policy. */
+  private LinkedHashSet<E> delegate;
+
+  // the locking structure used here follows the example found in LinkedBlockingQueue. The
+  // difference is that our locks guard access to `delegate` rather than the head node.
+
+  /** Lock held by take, poll, etc */
+  private final ReentrantLock takeLock;
+
+  /** Wait queue for waiting takes */
+  private final Condition notEmpty;
+
+  /** Lock held by put, offer, etc */
+  private final ReentrantLock putLock;
+
+  public RegionNormalizerWorkQueue() {
+    delegate = new LinkedHashSet<>();
+    takeLock = new ReentrantLock();
+    notEmpty = takeLock.newCondition();
+    putLock = new ReentrantLock();
+  }
+
+  /**
+   * Signals a waiting take. Called only from put/offer (which do not
+   * otherwise ordinarily lock takeLock.)
+   */
+  private void signalNotEmpty() {
+    final ReentrantLock takeLock = this.takeLock;

Review comment:
       Glad to know this!




----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani edited a comment on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani edited a comment on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-703493303


   As for `TestMasterChoreScheduled` test, it tries to retrieve all default (unconditional) chores of HMaster by retrieving singleton objects of chores using reflection from `HMaster.class`. 
   With the current refactored PR, we should facilitate retrieval of `regionNormalizerChore` from `RegionNormalizerManager.class` using reflection in `TestMasterChoreScheduled` (`class` as well as `targetObj` as args should be helpful) because IMHO, HMaster anyways should not be one place for defining all chores objects :
   
   ```
       private E getChoreObj(String fieldName, Class cls, Object targetObj)
           throws NoSuchFieldException, IllegalAccessException {
         Field masterField = cls.getDeclaredField(fieldName);
         masterField.setAccessible(true);
         E choreFieldVal = (E) masterField.get(targetObj);
         return choreFieldVal;
       }
   ```


----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-702693397


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  8s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 49s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 11s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m 29s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 45s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 45s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 15s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 15s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 38s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 46s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 230m 40s |  hbase-server in the patch failed.  |
   |  |   | 263m 36s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/6/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 07aa4051eeb7 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 3b91a15183 |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/6/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/6/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/6/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/6/testReport/ |
   | Max. process+thread count | 3073 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/6/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r496194646



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java
##########
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is
+ * an ordered collection class that has the following properties:
+ * <ul>
+ *   <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
+ *   <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
+ *   <li>Work is retrieved on a FIFO policy.</li>
+ *   <li>Work retrieval blocks the calling thread until new work is available, as a
+ *     {@link BlockingQueue}.</li>
+ *   <li>Allows a producer to insert an item at the head of the queue, if desired.</li>
+ * </ul>
+ * Assumes low-frequency and low-parallelism concurrent access, so protects state using a
+ * simplistic synchronization strategy.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorkQueue<E> {
+
+  /** Underlying storage structure that gives us the Set behavior and FIFO retrieval policy. */
+  private LinkedHashSet<E> delegate;

Review comment:
       I chose to make the field of a concrete type because we depend on the semantics provided by that type for the semantics defined by this class. I don't want to accidentally construct some other form of `Set`, for example, down in `putAllFirst`.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java
##########
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is
+ * an ordered collection class that has the following properties:
+ * <ul>
+ *   <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
+ *   <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
+ *   <li>Work is retrieved on a FIFO policy.</li>
+ *   <li>Work retrieval blocks the calling thread until new work is available, as a
+ *     {@link BlockingQueue}.</li>
+ *   <li>Allows a producer to insert an item at the head of the queue, if desired.</li>
+ * </ul>
+ * Assumes low-frequency and low-parallelism concurrent access, so protects state using a
+ * simplistic synchronization strategy.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorkQueue<E> {
+
+  /** Underlying storage structure that gives us the Set behavior and FIFO retrieval policy. */
+  private LinkedHashSet<E> delegate;
+
+  // the locking structure used here follows the example found in LinkedBlockingQueue. The
+  // difference is that our locks guard access to `delegate` rather than the head node.
+
+  /** Lock held by take, poll, etc */
+  private final ReentrantLock takeLock;
+
+  /** Wait queue for waiting takes */
+  private final Condition notEmpty;
+
+  /** Lock held by put, offer, etc */
+  private final ReentrantLock putLock;
+
+  public RegionNormalizerWorkQueue() {
+    delegate = new LinkedHashSet<>();
+    takeLock = new ReentrantLock();
+    notEmpty = takeLock.newCondition();
+    putLock = new ReentrantLock();
+  }
+
+  /**
+   * Signals a waiting take. Called only from put/offer (which do not
+   * otherwise ordinarily lock takeLock.)
+   */
+  private void signalNotEmpty() {
+    final ReentrantLock takeLock = this.takeLock;
+    takeLock.lock();
+    try {
+      notEmpty.signal();
+    } finally {
+      takeLock.unlock();
+    }
+  }
+
+  /**
+   * Locks to prevent both puts and takes.
+   */
+  private void fullyLock() {
+    putLock.lock();
+    takeLock.lock();
+  }
+
+  /**
+   * Unlocks to allow both puts and takes.
+   */
+  private void fullyUnlock() {
+    takeLock.unlock();
+    putLock.unlock();
+  }
+
+  /**
+   * Inserts the specified element at the tail of the queue, if it's not already present.
+   *
+   * @param e the element to add
+   */
+  public void put(E e) {
+    if (e == null) {
+      throw new NullPointerException();
+    }
+
+    putLock.lock();
+    try {
+      delegate.add(e);
+    } finally {
+      putLock.unlock();
+    }
+
+    if (!delegate.isEmpty()) {
+      signalNotEmpty();
+    }
+  }
+
+  /**
+   * Inserts the specified element at the head of the queue.
+   *
+   * @param e the element to add
+   */
+  public void putFirst(E e) {
+    if (e == null) {
+      throw new NullPointerException();
+    }
+    putAllFirst(Collections.singleton(e));
+  }
+
+  /**
+   * Inserts the specified elements at the tail of the queue. Any elements already present in
+   * the queue are ignored.
+   *
+   * @param c the elements to add
+   */
+  public void putAll(Collection<? extends E> c) {
+    if (c == null) {
+      throw new NullPointerException();
+    }
+
+    putLock.lock();
+    try {
+      delegate.addAll(c);
+    } finally {
+      putLock.unlock();
+    }
+
+    if (!delegate.isEmpty()) {
+      signalNotEmpty();
+    }
+  }
+
+  /**
+   * Inserts the specified elements at the head of the queue.
+   *
+   * @param c the elements to add
+   */
+  public void putAllFirst(Collection<? extends E> c) {
+    if (c == null) {
+      throw new NullPointerException();
+    }
+
+    fullyLock();
+    try {
+      final LinkedHashSet<E> copy = new LinkedHashSet<>(c.size() + delegate.size());
+      copy.addAll(c);
+      copy.addAll(delegate);
+      delegate = copy;
+    } finally {
+      fullyUnlock();
+    }
+
+    if (!delegate.isEmpty()) {
+      signalNotEmpty();
+    }
+  }
+
+  /**
+   * Retrieves and removes the head of this queue, waiting if necessary
+   * until an element becomes available.
+   *
+   * @return the head of this queue
+   * @throws InterruptedException if interrupted while waiting
+   */
+  public E take() throws InterruptedException {
+    E x;
+    takeLock.lockInterruptibly();
+    try {
+      while (delegate.isEmpty()) {
+        notEmpty.await();
+      }
+      final Iterator<E> iter = delegate.iterator();
+      x = iter.next();
+      iter.remove();

Review comment:
       `putAllFirst` (and thus `putFirst`) and `clear` acquire both locks. `putAllFirst` is the only method that actually modifies the reference pointed to by `delegate`... maybe the `takeLock` isn't strictly needed for `clear`.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java
##########
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is
+ * an ordered collection class that has the following properties:
+ * <ul>
+ *   <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
+ *   <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
+ *   <li>Work is retrieved on a FIFO policy.</li>
+ *   <li>Work retrieval blocks the calling thread until new work is available, as a
+ *     {@link BlockingQueue}.</li>
+ *   <li>Allows a producer to insert an item at the head of the queue, if desired.</li>
+ * </ul>
+ * Assumes low-frequency and low-parallelism concurrent access, so protects state using a
+ * simplistic synchronization strategy.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorkQueue<E> {
+
+  /** Underlying storage structure that gives us the Set behavior and FIFO retrieval policy. */
+  private LinkedHashSet<E> delegate;
+
+  // the locking structure used here follows the example found in LinkedBlockingQueue. The
+  // difference is that our locks guard access to `delegate` rather than the head node.
+
+  /** Lock held by take, poll, etc */
+  private final ReentrantLock takeLock;
+
+  /** Wait queue for waiting takes */
+  private final Condition notEmpty;
+
+  /** Lock held by put, offer, etc */
+  private final ReentrantLock putLock;
+
+  public RegionNormalizerWorkQueue() {
+    delegate = new LinkedHashSet<>();
+    takeLock = new ReentrantLock();
+    notEmpty = takeLock.newCondition();
+    putLock = new ReentrantLock();
+  }
+
+  /**
+   * Signals a waiting take. Called only from put/offer (which do not
+   * otherwise ordinarily lock takeLock.)
+   */
+  private void signalNotEmpty() {
+    final ReentrantLock takeLock = this.takeLock;

Review comment:
       This is a code-style that I copied from the reference implementation in `LinkedBlockingQueue`. I asked some jvm experts about this style and they told me it is done this way, particularly for locks, because the jvm doesn't enforce `final` on non-static fields. Which means that (1) `final` fields can be monkeyed with via reflection, but an inline variable declaration cannot be, and (2) a `final` field cannot be optimized by the compiler, while a `final` inline variable declaration can.
   
   TIL.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorker.java
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.RateLimiter;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+
+/**
+ * Consumes normalization request targets ({@link TableName}s) off the
+ * {@link RegionNormalizerWorkQueue}, dispatches them to the {@link RegionNormalizer},
+ * and executes the resulting {@link NormalizationPlan}s.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorker implements Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerWorker.class);
+  private static final String RATE_LIMIT_BYTES_PER_SEC_KEY =
+    "hbase.normalizer.throughput.max_bytes_per_sec";
+  private static final long RATE_UNLIMITED_BYTES = 1_000_000_000_000L; // 1TB/sec
+
+  private final MasterServices masterServices;
+  private final RegionNormalizer regionNormalizer;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RateLimiter rateLimiter;
+
+  private final long[] skippedCount;
+  private long splitPlanCount;
+  private long mergePlanCount;
+
+  public RegionNormalizerWorker(
+    final Configuration configuration,
+    final MasterServices masterServices,
+    final RegionNormalizer regionNormalizer,
+    final RegionNormalizerWorkQueue<TableName> workQueue
+  ) {
+    this.masterServices = masterServices;
+    this.regionNormalizer = regionNormalizer;
+    this.workQueue = workQueue;
+    this.skippedCount = new long[NormalizationPlan.PlanType.values().length];
+    this.splitPlanCount = 0;
+    this.mergePlanCount = 0;
+    this.rateLimiter = loadRateLimiter(configuration);
+  }
+
+  private static RateLimiter loadRateLimiter(final Configuration configuration) {
+    long rateLimitBytes =
+      configuration.getLongBytes(RATE_LIMIT_BYTES_PER_SEC_KEY, RATE_UNLIMITED_BYTES);
+    long rateLimitMbs = rateLimitBytes / 1_000_000L;
+    if (rateLimitMbs <= 0) {
+      LOG.warn("Configured value {}={} is <= 1MB. Falling back to default.",

Review comment:
       The condition is examining the value in megabytes, while the warning is communicating to the operator in terms of the configuration point, which is in bytes. Thus, "MB" is the correct unit, and after division, `1mb` is the minimum effective allowable value.




----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-702865370


   A couple of recent test failures seem relevant.


----------------------------------------------------------------
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



[GitHub] [hbase] huaxiangsun commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
huaxiangsun commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r499116857



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java
##########
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is
+ * an ordered collection class that has the following properties:
+ * <ul>
+ *   <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
+ *   <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
+ *   <li>Work is retrieved on a FIFO policy.</li>
+ *   <li>Work retrieval blocks the calling thread until new work is available, as a
+ *     {@link BlockingQueue}.</li>
+ *   <li>Allows a producer to insert an item at the head of the queue, if desired.</li>
+ * </ul>
+ * Assumes low-frequency and low-parallelism concurrent access, so protects state using a
+ * simplistic synchronization strategy.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorkQueue<E> {
+
+  /** Underlying storage structure that gives us the Set behavior and FIFO retrieval policy. */
+  private LinkedHashSet<E> delegate;
+
+  // the locking structure used here follows the example found in LinkedBlockingQueue. The
+  // difference is that our locks guard access to `delegate` rather than the head node.
+
+  /** Lock held by take, poll, etc */
+  private final ReentrantLock takeLock;
+
+  /** Wait queue for waiting takes */
+  private final Condition notEmpty;
+
+  /** Lock held by put, offer, etc */
+  private final ReentrantLock putLock;
+
+  public RegionNormalizerWorkQueue() {
+    delegate = new LinkedHashSet<>();
+    takeLock = new ReentrantLock();
+    notEmpty = takeLock.newCondition();
+    putLock = new ReentrantLock();
+  }
+
+  /**
+   * Signals a waiting take. Called only from put/offer (which do not
+   * otherwise ordinarily lock takeLock.)
+   */
+  private void signalNotEmpty() {
+    final ReentrantLock takeLock = this.takeLock;
+    takeLock.lock();
+    try {
+      notEmpty.signal();
+    } finally {
+      takeLock.unlock();
+    }
+  }
+
+  /**
+   * Locks to prevent both puts and takes.
+   */
+  private void fullyLock() {
+    putLock.lock();
+    takeLock.lock();
+  }
+
+  /**
+   * Unlocks to allow both puts and takes.
+   */
+  private void fullyUnlock() {

Review comment:
       Not sure if there is any existing data structure for this part, have to admit that I did not look very closely into this part.




----------------------------------------------------------------
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



[GitHub] [hbase] saintstack commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r501291094



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationTarget.java
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A POJO that caries details about a region selected for normalization through the pipeline.
+ */
+@InterfaceAudience.Private
+public class NormalizationTarget {
+  private final RegionInfo regionInfo;
+  private final long regionSizeMb;
+
+  public NormalizationTarget(final RegionInfo regionInfo, final long regionSizeMb) {
+    this.regionInfo = regionInfo;
+    this.regionSizeMb = regionSizeMb;
+  }
+
+  public RegionInfo getRegionInfo() {
+    return regionInfo;
+  }
+
+  public long getRegionSizeMb() {
+    return regionSizeMb;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    NormalizationTarget that = (NormalizationTarget) o;
+
+    return new EqualsBuilder()
+      .append(regionSizeMb, that.regionSizeMb)
+      .append(regionInfo, that.regionInfo)
+      .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37)
+      .append(regionInfo)
+      .append(regionSizeMb)
+      .toHashCode();
+  }
+
+  @Override public String toString() {
+    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)

Review comment:
       Sounds good




----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-702265555


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 49s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 11s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m 14s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 41s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 19s |  hbase-server: The patch generated 0 new + 145 unchanged - 7 fixed = 145 total (was 152)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  11m 52s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   2m 23s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 13s |  The patch does not generate ASF License warnings.  |
   |  |   |  34m 43s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/5/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 00dc08e7e30b 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 11a336a74a |
   | Max. process+thread count | 94 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/5/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani edited a comment on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani edited a comment on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-703493303


   As for `TestMasterChoreScheduled` test, it tries to retrieve all default (unconditional) chores of HMaster by retrieving singleton objects of chores using reflection from `HMaster.class`. 
   With the current refactored PR, we should facilitate retrieval of `regionNormalizerChore` from `RegionNormalizerManager.class` using reflection in `TestMasterChoreScheduled` (`class` as well as `targetObj` as args should be helpful) because IMHO, HMaster doesn't have to one standard place for defining all chores objects:
   
   ```
       private E getChoreObj(String fieldName, Class cls, Object targetObj)
           throws NoSuchFieldException, IllegalAccessException {
         Field masterField = cls.getDeclaredField(fieldName);
         masterField.setAccessible(true);
         E choreFieldVal = (E) masterField.get(targetObj);
         return choreFieldVal;
       }
   ```


----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r500199740



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -371,7 +358,11 @@ private boolean skipForMerge(final RegionStates regionStates, final RegionInfo r
       final long nextSizeMb = getRegionSizeMB(next);
       // always merge away empty regions when they present themselves.
       if (currentSizeMb == 0 || nextSizeMb == 0 || currentSizeMb + nextSizeMb < avgRegionSizeMb) {
-        plans.add(new MergeNormalizationPlan(current, next));
+        final MergeNormalizationPlan plan = new MergeNormalizationPlan.Builder()
+          .addTarget(current, currentSizeMb)
+          .addTarget(next, nextSizeMb)

Review comment:
       Sure this is not normalizer's job to validate but by keeping target regions as list, we do have a probability of keeping odd no of regions around a while just to get them abandoned later by `MergeTableRegionsProcedure#checkRegionsToMerge` if we keep it unbounded list. Not that `SimpleRegionNormalizer` will add odd no of regions, this is just to be treated as a probability.
   
   How about `List<Pair<NormalizerRegionInfo, NormalizerRegionInfo>>` as target regions in `MergeNormalizationPlan`? 
   New POJO `NormalizerRegionInfo` can just contain `regionInfo` and `regionSize`. 
   
   Or `List<MergeNormalizerRegionInfo>` where `MergeNormalizerRegionInfo` can keep `firstRegion` and `secondRegion` just like how it was before but this time we can have list of that object.
   
   This way, at least we indicate a clear way for clients to send multiple pair of regions to merge. Thought?




----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r501269906



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerManager.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * This class encapsulates the details of the {@link RegionNormalizer} subsystem.
+ */
+@InterfaceAudience.Private
+public class RegionNormalizerManager {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerManager.class);
+
+  private final RegionNormalizerTracker regionNormalizerTracker;
+  private final RegionNormalizerChore regionNormalizerChore;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RegionNormalizerWorker worker;
+  private final ExecutorService pool;
+
+  private final Object startStopLock = new Object();
+  private boolean started = false;
+  private boolean stopped = false;
+
+  public RegionNormalizerManager(
+    @NonNull  final RegionNormalizerTracker regionNormalizerTracker,
+    @Nullable final RegionNormalizerChore regionNormalizerChore,
+    @Nullable final RegionNormalizerWorkQueue<TableName> workQueue,
+    @Nullable final RegionNormalizerWorker worker
+  ) {
+    this.regionNormalizerTracker = regionNormalizerTracker;
+    this.regionNormalizerChore = regionNormalizerChore;
+    this.workQueue = workQueue;
+    this.worker = worker;
+    this.pool = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+      .setDaemon(true)
+      .setNameFormat("normalizer-worker-%d")
+      .setUncaughtExceptionHandler(
+        (thread, throwable) ->
+          LOG.error("Uncaught exception, worker thread likely terminated.", throwable))
+      .build());
+  }
+
+  public void start() {
+    if (started) {
+      return;
+    }
+    synchronized (startStopLock) {
+      if (started) {
+        return;
+      }
+      regionNormalizerTracker.start();
+      if (worker != null) {
+        // worker will be null when master is in maintenance mode.
+        pool.submit(worker);
+      }
+      started = true;
+    }
+  }
+
+  public void stop() {
+    if (!started) {
+      throw new IllegalStateException("calling `stop` without first calling `start`.");
+    }
+    if (stopped) {

Review comment:
       I toyed a bit with guava Service. [ServiceExplained](https://github.com/google/guava/wiki/ServiceExplained) helped. I think it could be done, but it's messy. Mapping this code to Guava Service model, the`RegionNormalizerManager` is itself a `ServiceManager` that is composed of the `RegionNormalizerTracker`, which is always run, and the `RegionNormalizerWorker`, which is run most of the time, but not always. In a purely Guava + Juice application, this might be accomplished by conditionally injecting the `RegionNormalizerWorker`, based on the configuration.
   
   Another difference is that the `RegionNormalizerWorker` runs in a background thread, but the `RegionNormalizerTracker` does not. So using `Service` to manage the latter results in an extra thread we don't need. Likewise, most of the time, the `RegionNormalizerManager` will spawn the background thread for the worker, but if we're in safe-mode, that's not required. The most natural base class, `AbstractExecutionThreadService`, will always spawn a background thread, which is where it runs the instance of `RegionNormalizerWorker`. However, when in safe-mode, we don't need that thread and we cannot keep the `Service` instance in the `RUNNING` state without it.




----------------------------------------------------------------
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



[GitHub] [hbase] saintstack commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r501292046



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerManager.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * This class encapsulates the details of the {@link RegionNormalizer} subsystem.
+ */
+@InterfaceAudience.Private
+public class RegionNormalizerManager {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerManager.class);
+
+  private final RegionNormalizerTracker regionNormalizerTracker;
+  private final RegionNormalizerChore regionNormalizerChore;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RegionNormalizerWorker worker;
+  private final ExecutorService pool;
+
+  private final Object startStopLock = new Object();
+  private boolean started = false;
+  private boolean stopped = false;
+
+  public RegionNormalizerManager(
+    @NonNull  final RegionNormalizerTracker regionNormalizerTracker,
+    @Nullable final RegionNormalizerChore regionNormalizerChore,
+    @Nullable final RegionNormalizerWorkQueue<TableName> workQueue,
+    @Nullable final RegionNormalizerWorker worker
+  ) {
+    this.regionNormalizerTracker = regionNormalizerTracker;
+    this.regionNormalizerChore = regionNormalizerChore;
+    this.workQueue = workQueue;
+    this.worker = worker;
+    this.pool = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+      .setDaemon(true)
+      .setNameFormat("normalizer-worker-%d")
+      .setUncaughtExceptionHandler(
+        (thread, throwable) ->
+          LOG.error("Uncaught exception, worker thread likely terminated.", throwable))
+      .build());
+  }
+
+  public void start() {
+    if (started) {
+      return;
+    }
+    synchronized (startStopLock) {
+      if (started) {
+        return;
+      }
+      regionNormalizerTracker.start();
+      if (worker != null) {
+        // worker will be null when master is in maintenance mode.
+        pool.submit(worker);
+      }
+      started = true;
+    }
+  }
+
+  public void stop() {
+    if (!started) {
+      throw new IllegalStateException("calling `stop` without first calling `start`.");
+    }
+    if (stopped) {

Review comment:
       ok. Thanks for taking a look.




----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r496959173



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java
##########
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is
+ * an ordered collection class that has the following properties:
+ * <ul>
+ *   <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
+ *   <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
+ *   <li>Work is retrieved on a FIFO policy.</li>
+ *   <li>Work retrieval blocks the calling thread until new work is available, as a
+ *     {@link BlockingQueue}.</li>
+ *   <li>Allows a producer to insert an item at the head of the queue, if desired.</li>
+ * </ul>
+ * Assumes low-frequency and low-parallelism concurrent access, so protects state using a
+ * simplistic synchronization strategy.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorkQueue<E> {
+
+  /** Underlying storage structure that gives us the Set behavior and FIFO retrieval policy. */
+  private LinkedHashSet<E> delegate;
+
+  // the locking structure used here follows the example found in LinkedBlockingQueue. The
+  // difference is that our locks guard access to `delegate` rather than the head node.
+
+  /** Lock held by take, poll, etc */
+  private final ReentrantLock takeLock;
+
+  /** Wait queue for waiting takes */
+  private final Condition notEmpty;
+
+  /** Lock held by put, offer, etc */
+  private final ReentrantLock putLock;
+
+  public RegionNormalizerWorkQueue() {
+    delegate = new LinkedHashSet<>();
+    takeLock = new ReentrantLock();
+    notEmpty = takeLock.newCondition();
+    putLock = new ReentrantLock();
+  }
+
+  /**
+   * Signals a waiting take. Called only from put/offer (which do not
+   * otherwise ordinarily lock takeLock.)
+   */
+  private void signalNotEmpty() {
+    final ReentrantLock takeLock = this.takeLock;
+    takeLock.lock();
+    try {
+      notEmpty.signal();
+    } finally {
+      takeLock.unlock();
+    }
+  }
+
+  /**
+   * Locks to prevent both puts and takes.
+   */
+  private void fullyLock() {
+    putLock.lock();
+    takeLock.lock();
+  }
+
+  /**
+   * Unlocks to allow both puts and takes.
+   */
+  private void fullyUnlock() {
+    takeLock.unlock();
+    putLock.unlock();
+  }
+
+  /**
+   * Inserts the specified element at the tail of the queue, if it's not already present.
+   *
+   * @param e the element to add
+   */
+  public void put(E e) {
+    if (e == null) {
+      throw new NullPointerException();
+    }
+
+    putLock.lock();
+    try {
+      delegate.add(e);
+    } finally {
+      putLock.unlock();
+    }
+
+    if (!delegate.isEmpty()) {
+      signalNotEmpty();
+    }
+  }
+
+  /**
+   * Inserts the specified element at the head of the queue.
+   *
+   * @param e the element to add
+   */
+  public void putFirst(E e) {
+    if (e == null) {
+      throw new NullPointerException();
+    }
+    putAllFirst(Collections.singleton(e));
+  }
+
+  /**
+   * Inserts the specified elements at the tail of the queue. Any elements already present in
+   * the queue are ignored.
+   *
+   * @param c the elements to add
+   */
+  public void putAll(Collection<? extends E> c) {
+    if (c == null) {
+      throw new NullPointerException();
+    }
+
+    putLock.lock();
+    try {
+      delegate.addAll(c);
+    } finally {
+      putLock.unlock();
+    }
+
+    if (!delegate.isEmpty()) {
+      signalNotEmpty();
+    }
+  }
+
+  /**
+   * Inserts the specified elements at the head of the queue.
+   *
+   * @param c the elements to add
+   */
+  public void putAllFirst(Collection<? extends E> c) {
+    if (c == null) {
+      throw new NullPointerException();
+    }
+
+    fullyLock();
+    try {
+      final LinkedHashSet<E> copy = new LinkedHashSet<>(c.size() + delegate.size());
+      copy.addAll(c);
+      copy.addAll(delegate);
+      delegate = copy;
+    } finally {
+      fullyUnlock();
+    }
+
+    if (!delegate.isEmpty()) {
+      signalNotEmpty();
+    }
+  }
+
+  /**
+   * Retrieves and removes the head of this queue, waiting if necessary
+   * until an element becomes available.
+   *
+   * @return the head of this queue
+   * @throws InterruptedException if interrupted while waiting
+   */
+  public E take() throws InterruptedException {
+    E x;
+    takeLock.lockInterruptibly();
+    try {
+      while (delegate.isEmpty()) {
+        notEmpty.await();
+      }
+      final Iterator<E> iter = delegate.iterator();
+      x = iter.next();
+      iter.remove();

Review comment:
       Agree, only `putLock` should be enough for `clear`.




----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-703913381


   Rebased onto master and addressed PR feedback. Fixed `TestMasterChoreScheduled`. Filed [HBASE-25156](https://issues.apache.org/jira/browse/HBASE-25156) for `TestMasterFailover.testSimpleMasterFailover` and [HBASE-25157](https://issues.apache.org/jira/browse/HBASE-25157) for `TestMasterRegionCompaction`.


----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-699197970


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m  6s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 12s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m 14s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 38s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 11s |  hbase-server: The patch generated 0 new + 145 unchanged - 7 fixed = 145 total (was 152)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  11m 50s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   2m 25s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 15s |  The patch does not generate ASF License warnings.  |
   |  |   |  35m  4s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/3/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux c6a3819dd753 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 4e59014bed |
   | Max. process+thread count | 94 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/3/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-702674960


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  4s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m  9s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  0s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m  9s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 49s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  1s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  1s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m  8s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 193m 19s |  hbase-server in the patch failed.  |
   |  |   | 221m 31s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/6/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 62f3af595c1b 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 3b91a15183 |
   | Default Java | 1.8.0_232 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/6/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/6/testReport/ |
   | Max. process+thread count | 3549 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/6/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r496771793



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorker.java
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.RateLimiter;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+
+/**
+ * Consumes normalization request targets ({@link TableName}s) off the
+ * {@link RegionNormalizerWorkQueue}, dispatches them to the {@link RegionNormalizer},
+ * and executes the resulting {@link NormalizationPlan}s.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorker implements Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerWorker.class);
+  private static final String RATE_LIMIT_BYTES_PER_SEC_KEY =
+    "hbase.normalizer.throughput.max_bytes_per_sec";
+  private static final long RATE_UNLIMITED_BYTES = 1_000_000_000_000L; // 1TB/sec
+
+  private final MasterServices masterServices;
+  private final RegionNormalizer regionNormalizer;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RateLimiter rateLimiter;
+
+  private final long[] skippedCount;
+  private long splitPlanCount;
+  private long mergePlanCount;
+
+  public RegionNormalizerWorker(
+    final Configuration configuration,
+    final MasterServices masterServices,
+    final RegionNormalizer regionNormalizer,
+    final RegionNormalizerWorkQueue<TableName> workQueue
+  ) {
+    this.masterServices = masterServices;
+    this.regionNormalizer = regionNormalizer;
+    this.workQueue = workQueue;
+    this.skippedCount = new long[NormalizationPlan.PlanType.values().length];
+    this.splitPlanCount = 0;
+    this.mergePlanCount = 0;
+    this.rateLimiter = loadRateLimiter(configuration);
+  }
+
+  private static RateLimiter loadRateLimiter(final Configuration configuration) {
+    long rateLimitBytes =
+      configuration.getLongBytes(RATE_LIMIT_BYTES_PER_SEC_KEY, RATE_UNLIMITED_BYTES);
+    long rateLimitMbs = rateLimitBytes / 1_000_000L;
+    if (rateLimitMbs <= 0) {
+      LOG.warn("Configured value {}={} is <= 1MB. Falling back to default.",
+        RATE_LIMIT_BYTES_PER_SEC_KEY, rateLimitBytes);
+      rateLimitBytes = RATE_UNLIMITED_BYTES;
+      rateLimitMbs = RATE_UNLIMITED_BYTES / 1_000_000L;
+    }
+    LOG.info("Normalizer rate limit set to {}",
+      rateLimitBytes == RATE_UNLIMITED_BYTES ? "unlimited" : rateLimitMbs + " MB/sec");
+    return RateLimiter.create(rateLimitMbs);
+  }
+
+  /**
+   * @see RegionNormalizerManager#planSkipped(NormalizationPlan.PlanType)
+   */
+  void planSkipped(NormalizationPlan.PlanType type) {
+    synchronized (skippedCount) {
+      // updates come here via procedure threads, so synchronize access to this counter.
+      skippedCount[type.ordinal()]++;
+    }
+  }
+
+  /**
+   * @see RegionNormalizerManager#getSkippedCount(NormalizationPlan.PlanType)
+   */
+  long getSkippedCount(NormalizationPlan.PlanType type) {
+    return skippedCount[type.ordinal()];
+  }
+
+  /**
+   * @see HMaster#getSplitPlanCount()
+   */
+  long getSplitPlanCount() {
+    return splitPlanCount;
+  }
+
+  /**
+   * @see HMaster#getMergePlanCount()
+   */
+  long getMergePlanCount() {
+    return mergePlanCount;
+  }
+
+  @Override
+  public void run() {
+    while (true) {
+      if (Thread.interrupted()) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+      final TableName tableName;
+      try {
+        tableName = workQueue.take();
+      } catch (InterruptedException e) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+
+      final List<NormalizationPlan> plans = calculatePlans(tableName);
+      submitPlans(plans);
+    }
+  }
+
+  private List<NormalizationPlan> calculatePlans(final TableName tableName) {
+    try {
+      final TableDescriptor tblDesc = masterServices.getTableDescriptors().get(tableName);
+      if (tblDesc != null && !tblDesc.isNormalizationEnabled()) {
+        LOG.debug("Skipping table {} because normalization is disabled in its table properties.",
+          tableName);
+        return Collections.emptyList();
+      }
+    } catch (IOException e) {
+      LOG.debug("Skipping table {} because unable to access its table descriptor.", tableName, e);
+      return Collections.emptyList();
+    }
+
+    if (masterServices.skipRegionManagementAction("region normalizer")) {
+      return Collections.emptyList();
+    }
+
+    final List<NormalizationPlan> plans = regionNormalizer.computePlansForTable(tableName);
+    if (CollectionUtils.isEmpty(plans)) {
+      LOG.debug("No normalization required for table {}.", tableName);
+      return Collections.emptyList();
+    }
+    return plans;
+  }
+
+  private void submitPlans(final List<NormalizationPlan> plans) {
+    // as of this writing, `plan.submit()` is non-blocking and uses Async Admin APIs to submit
+    // task, so there's no artificial rate-limiting of merge/split requests due to this serial loop.
+    for (NormalizationPlan plan : plans) {
+      switch (plan.getType()) {
+        case MERGE: {
+          submitMergePlan((MergeNormalizationPlan) plan);
+          break;
+        }
+        case SPLIT: {
+          submitSplitPlan((SplitNormalizationPlan) plan);
+          break;
+        }
+        case NONE:
+          LOG.debug("Nothing to do for {} with PlanType=NONE. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+        default:
+          LOG.warn("Plan {} is of an unrecognized PlanType. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+      }
+    }
+  }
+
+  /**
+   * Interacts with {@link MasterServices} in order to execute a plan.
+   */
+  private void submitMergePlan(final MergeNormalizationPlan plan) {
+    final int totalSizeMb;
+    try {
+      final long totalSizeMbLong = plan.getNormalizationTargets()
+        .stream()
+        .mapToLong(NormalizationTarget::getRegionSizeMb)
+        .reduce(0, Math::addExact);

Review comment:
       Looks like `reduce(0, Math::addExact)` is preferred here over `sum()` due to long overflow. If it happens for two regions, perhaps we might get their merge request again? (and again in subsequent normalizer runs?)

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorker.java
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.RateLimiter;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+
+/**
+ * Consumes normalization request targets ({@link TableName}s) off the
+ * {@link RegionNormalizerWorkQueue}, dispatches them to the {@link RegionNormalizer},
+ * and executes the resulting {@link NormalizationPlan}s.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorker implements Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerWorker.class);
+  private static final String RATE_LIMIT_BYTES_PER_SEC_KEY =
+    "hbase.normalizer.throughput.max_bytes_per_sec";
+  private static final long RATE_UNLIMITED_BYTES = 1_000_000_000_000L; // 1TB/sec
+
+  private final MasterServices masterServices;
+  private final RegionNormalizer regionNormalizer;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RateLimiter rateLimiter;
+
+  private final long[] skippedCount;
+  private long splitPlanCount;
+  private long mergePlanCount;
+
+  public RegionNormalizerWorker(
+    final Configuration configuration,
+    final MasterServices masterServices,
+    final RegionNormalizer regionNormalizer,
+    final RegionNormalizerWorkQueue<TableName> workQueue
+  ) {
+    this.masterServices = masterServices;
+    this.regionNormalizer = regionNormalizer;
+    this.workQueue = workQueue;
+    this.skippedCount = new long[NormalizationPlan.PlanType.values().length];
+    this.splitPlanCount = 0;
+    this.mergePlanCount = 0;
+    this.rateLimiter = loadRateLimiter(configuration);
+  }
+
+  private static RateLimiter loadRateLimiter(final Configuration configuration) {
+    long rateLimitBytes =
+      configuration.getLongBytes(RATE_LIMIT_BYTES_PER_SEC_KEY, RATE_UNLIMITED_BYTES);
+    long rateLimitMbs = rateLimitBytes / 1_000_000L;
+    if (rateLimitMbs <= 0) {
+      LOG.warn("Configured value {}={} is <= 1MB. Falling back to default.",
+        RATE_LIMIT_BYTES_PER_SEC_KEY, rateLimitBytes);
+      rateLimitBytes = RATE_UNLIMITED_BYTES;
+      rateLimitMbs = RATE_UNLIMITED_BYTES / 1_000_000L;
+    }
+    LOG.info("Normalizer rate limit set to {}",
+      rateLimitBytes == RATE_UNLIMITED_BYTES ? "unlimited" : rateLimitMbs + " MB/sec");
+    return RateLimiter.create(rateLimitMbs);
+  }
+
+  /**
+   * @see RegionNormalizerManager#planSkipped(NormalizationPlan.PlanType)
+   */
+  void planSkipped(NormalizationPlan.PlanType type) {
+    synchronized (skippedCount) {
+      // updates come here via procedure threads, so synchronize access to this counter.
+      skippedCount[type.ordinal()]++;
+    }
+  }
+
+  /**
+   * @see RegionNormalizerManager#getSkippedCount(NormalizationPlan.PlanType)
+   */
+  long getSkippedCount(NormalizationPlan.PlanType type) {
+    return skippedCount[type.ordinal()];
+  }
+
+  /**
+   * @see HMaster#getSplitPlanCount()
+   */
+  long getSplitPlanCount() {
+    return splitPlanCount;
+  }
+
+  /**
+   * @see HMaster#getMergePlanCount()
+   */
+  long getMergePlanCount() {
+    return mergePlanCount;
+  }
+
+  @Override
+  public void run() {
+    while (true) {
+      if (Thread.interrupted()) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+      final TableName tableName;
+      try {
+        tableName = workQueue.take();
+      } catch (InterruptedException e) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+
+      final List<NormalizationPlan> plans = calculatePlans(tableName);
+      submitPlans(plans);
+    }
+  }
+
+  private List<NormalizationPlan> calculatePlans(final TableName tableName) {
+    try {
+      final TableDescriptor tblDesc = masterServices.getTableDescriptors().get(tableName);
+      if (tblDesc != null && !tblDesc.isNormalizationEnabled()) {
+        LOG.debug("Skipping table {} because normalization is disabled in its table properties.",
+          tableName);
+        return Collections.emptyList();
+      }
+    } catch (IOException e) {
+      LOG.debug("Skipping table {} because unable to access its table descriptor.", tableName, e);
+      return Collections.emptyList();
+    }
+
+    if (masterServices.skipRegionManagementAction("region normalizer")) {
+      return Collections.emptyList();
+    }
+
+    final List<NormalizationPlan> plans = regionNormalizer.computePlansForTable(tableName);
+    if (CollectionUtils.isEmpty(plans)) {
+      LOG.debug("No normalization required for table {}.", tableName);
+      return Collections.emptyList();
+    }
+    return plans;
+  }
+
+  private void submitPlans(final List<NormalizationPlan> plans) {
+    // as of this writing, `plan.submit()` is non-blocking and uses Async Admin APIs to submit
+    // task, so there's no artificial rate-limiting of merge/split requests due to this serial loop.
+    for (NormalizationPlan plan : plans) {
+      switch (plan.getType()) {
+        case MERGE: {
+          submitMergePlan((MergeNormalizationPlan) plan);
+          break;
+        }
+        case SPLIT: {
+          submitSplitPlan((SplitNormalizationPlan) plan);
+          break;
+        }
+        case NONE:
+          LOG.debug("Nothing to do for {} with PlanType=NONE. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+        default:
+          LOG.warn("Plan {} is of an unrecognized PlanType. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+      }
+    }
+  }
+
+  /**
+   * Interacts with {@link MasterServices} in order to execute a plan.
+   */
+  private void submitMergePlan(final MergeNormalizationPlan plan) {
+    final int totalSizeMb;
+    try {
+      final long totalSizeMbLong = plan.getNormalizationTargets()
+        .stream()
+        .mapToLong(NormalizationTarget::getRegionSizeMb)
+        .reduce(0, Math::addExact);
+      totalSizeMb = Math.toIntExact(totalSizeMbLong);
+    } catch (ArithmeticException e) {
+      LOG.debug("Sum of merge request size overflows rate limiter data type. {}", plan);
+      planSkipped(plan.getType());
+      return;
+    }
+
+    final RegionInfo[] infos = plan.getNormalizationTargets()
+      .stream()
+      .map(NormalizationTarget::getRegionInfo)
+      .toArray(RegionInfo[]::new);
+    final long rateLimitedSecs = Math.round(rateLimiter.acquire(Math.max(1, totalSizeMb)));
+    LOG.debug("Rate limiting delayed this operation by {}", Duration.ofSeconds(rateLimitedSecs));
+
+    final long pid;
+    try {
+      pid = masterServices.mergeRegions(
+        infos, false, HConstants.NO_NONCE, HConstants.NO_NONCE);
+    } catch (IOException e) {
+      LOG.debug("failed to submit plan {}.", plan, e);

Review comment:
       nit: `INFO` might be better for plan submit failures for both `split` and `merge`.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorker.java
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.RateLimiter;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+
+/**
+ * Consumes normalization request targets ({@link TableName}s) off the
+ * {@link RegionNormalizerWorkQueue}, dispatches them to the {@link RegionNormalizer},
+ * and executes the resulting {@link NormalizationPlan}s.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorker implements Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerWorker.class);
+  private static final String RATE_LIMIT_BYTES_PER_SEC_KEY =
+    "hbase.normalizer.throughput.max_bytes_per_sec";
+  private static final long RATE_UNLIMITED_BYTES = 1_000_000_000_000L; // 1TB/sec
+
+  private final MasterServices masterServices;
+  private final RegionNormalizer regionNormalizer;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RateLimiter rateLimiter;
+
+  private final long[] skippedCount;
+  private long splitPlanCount;
+  private long mergePlanCount;
+
+  public RegionNormalizerWorker(
+    final Configuration configuration,
+    final MasterServices masterServices,
+    final RegionNormalizer regionNormalizer,
+    final RegionNormalizerWorkQueue<TableName> workQueue
+  ) {
+    this.masterServices = masterServices;
+    this.regionNormalizer = regionNormalizer;
+    this.workQueue = workQueue;
+    this.skippedCount = new long[NormalizationPlan.PlanType.values().length];
+    this.splitPlanCount = 0;
+    this.mergePlanCount = 0;
+    this.rateLimiter = loadRateLimiter(configuration);
+  }
+
+  private static RateLimiter loadRateLimiter(final Configuration configuration) {
+    long rateLimitBytes =
+      configuration.getLongBytes(RATE_LIMIT_BYTES_PER_SEC_KEY, RATE_UNLIMITED_BYTES);
+    long rateLimitMbs = rateLimitBytes / 1_000_000L;
+    if (rateLimitMbs <= 0) {
+      LOG.warn("Configured value {}={} is <= 1MB. Falling back to default.",
+        RATE_LIMIT_BYTES_PER_SEC_KEY, rateLimitBytes);
+      rateLimitBytes = RATE_UNLIMITED_BYTES;
+      rateLimitMbs = RATE_UNLIMITED_BYTES / 1_000_000L;
+    }
+    LOG.info("Normalizer rate limit set to {}",
+      rateLimitBytes == RATE_UNLIMITED_BYTES ? "unlimited" : rateLimitMbs + " MB/sec");
+    return RateLimiter.create(rateLimitMbs);
+  }
+
+  /**
+   * @see RegionNormalizerManager#planSkipped(NormalizationPlan.PlanType)
+   */
+  void planSkipped(NormalizationPlan.PlanType type) {
+    synchronized (skippedCount) {
+      // updates come here via procedure threads, so synchronize access to this counter.
+      skippedCount[type.ordinal()]++;
+    }
+  }
+
+  /**
+   * @see RegionNormalizerManager#getSkippedCount(NormalizationPlan.PlanType)
+   */
+  long getSkippedCount(NormalizationPlan.PlanType type) {
+    return skippedCount[type.ordinal()];
+  }
+
+  /**
+   * @see HMaster#getSplitPlanCount()
+   */
+  long getSplitPlanCount() {
+    return splitPlanCount;
+  }
+
+  /**
+   * @see HMaster#getMergePlanCount()
+   */
+  long getMergePlanCount() {
+    return mergePlanCount;
+  }
+
+  @Override
+  public void run() {
+    while (true) {
+      if (Thread.interrupted()) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+      final TableName tableName;
+      try {
+        tableName = workQueue.take();
+      } catch (InterruptedException e) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+
+      final List<NormalizationPlan> plans = calculatePlans(tableName);
+      submitPlans(plans);
+    }
+  }
+
+  private List<NormalizationPlan> calculatePlans(final TableName tableName) {
+    try {
+      final TableDescriptor tblDesc = masterServices.getTableDescriptors().get(tableName);
+      if (tblDesc != null && !tblDesc.isNormalizationEnabled()) {
+        LOG.debug("Skipping table {} because normalization is disabled in its table properties.",
+          tableName);
+        return Collections.emptyList();
+      }
+    } catch (IOException e) {
+      LOG.debug("Skipping table {} because unable to access its table descriptor.", tableName, e);
+      return Collections.emptyList();
+    }
+
+    if (masterServices.skipRegionManagementAction("region normalizer")) {
+      return Collections.emptyList();
+    }
+
+    final List<NormalizationPlan> plans = regionNormalizer.computePlansForTable(tableName);
+    if (CollectionUtils.isEmpty(plans)) {
+      LOG.debug("No normalization required for table {}.", tableName);
+      return Collections.emptyList();
+    }
+    return plans;
+  }
+
+  private void submitPlans(final List<NormalizationPlan> plans) {
+    // as of this writing, `plan.submit()` is non-blocking and uses Async Admin APIs to submit
+    // task, so there's no artificial rate-limiting of merge/split requests due to this serial loop.
+    for (NormalizationPlan plan : plans) {
+      switch (plan.getType()) {
+        case MERGE: {
+          submitMergePlan((MergeNormalizationPlan) plan);
+          break;
+        }
+        case SPLIT: {
+          submitSplitPlan((SplitNormalizationPlan) plan);
+          break;
+        }
+        case NONE:
+          LOG.debug("Nothing to do for {} with PlanType=NONE. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+        default:
+          LOG.warn("Plan {} is of an unrecognized PlanType. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+      }
+    }
+  }
+
+  /**
+   * Interacts with {@link MasterServices} in order to execute a plan.
+   */
+  private void submitMergePlan(final MergeNormalizationPlan plan) {
+    final int totalSizeMb;
+    try {
+      final long totalSizeMbLong = plan.getNormalizationTargets()
+        .stream()
+        .mapToLong(NormalizationTarget::getRegionSizeMb)
+        .reduce(0, Math::addExact);
+      totalSizeMb = Math.toIntExact(totalSizeMbLong);
+    } catch (ArithmeticException e) {
+      LOG.debug("Sum of merge request size overflows rate limiter data type. {}", plan);
+      planSkipped(plan.getType());
+      return;
+    }
+
+    final RegionInfo[] infos = plan.getNormalizationTargets()
+      .stream()
+      .map(NormalizationTarget::getRegionInfo)
+      .toArray(RegionInfo[]::new);
+    final long rateLimitedSecs = Math.round(rateLimiter.acquire(Math.max(1, totalSizeMb)));
+    LOG.debug("Rate limiting delayed this operation by {}", Duration.ofSeconds(rateLimitedSecs));

Review comment:
       nit: logging `plan` also might be helpful

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestRegionNormalizerWorker.java
##########
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import static java.util.Collections.singletonList;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.comparesEqualTo;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.Mockito.when;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNameTestRule;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
+import org.hamcrest.StringDescription;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.mockito.Answers;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * A test over {@link RegionNormalizerWorker}. Being a background thread, the only points of
+ * interaction we have to this class are its input source ({@link RegionNormalizerWorkQueue} and
+ * its callbacks invoked against {@link RegionNormalizer} and {@link MasterServices}. The work
+ * queue is simple enough to use directly; for {@link MasterServices}, use a mock because, as of
+ * now, the worker only invokes 4 methods.
+ */
+@Category({ MasterTests.class, SmallTests.class})
+public class TestRegionNormalizerWorker {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestRegionNormalizerWorker.class);
+
+  @Rule
+  public TestName testName = new TestName();
+  @Rule
+  public TableNameTestRule tableName = new TableNameTestRule();
+
+  @Rule
+  public MockitoRule mockitoRule = MockitoJUnit.rule();
+
+  @Mock(answer = Answers.RETURNS_DEEP_STUBS)
+  private MasterServices masterServices;
+  @Mock
+  private RegionNormalizer regionNormalizer;
+
+  private HBaseCommonTestingUtility testingUtility;
+  private RegionNormalizerWorkQueue<TableName> queue;
+  private ExecutorService workerPool;
+
+  private final AtomicReference<Throwable> workerThreadThrowable = new AtomicReference<>();
+
+  @Before
+  public void before() throws Exception {
+    MockitoAnnotations.initMocks(this);
+    when(masterServices.skipRegionManagementAction(any())).thenReturn(false);
+    testingUtility = new HBaseCommonTestingUtility();
+    queue = new RegionNormalizerWorkQueue<>();
+    workerThreadThrowable.set(null);
+
+    final String threadNameFmt =
+      TestRegionNormalizerWorker.class.getSimpleName() + "-" + testName.getMethodName() + "-%d";
+    final ThreadFactory threadFactory = new ThreadFactoryBuilder()
+      .setNameFormat(threadNameFmt)
+      .setDaemon(true)
+      .setUncaughtExceptionHandler((t, e) -> workerThreadThrowable.set(e))
+      .build();
+    workerPool = Executors.newFixedThreadPool(1, threadFactory);

Review comment:
       nit: `newSingleThreadExecutor(factory)` to keep it similar with worker pool used in source code?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -371,7 +358,11 @@ private boolean skipForMerge(final RegionStates regionStates, final RegionInfo r
       final long nextSizeMb = getRegionSizeMB(next);
       // always merge away empty regions when they present themselves.
       if (currentSizeMb == 0 || nextSizeMb == 0 || currentSizeMb + nextSizeMb < avgRegionSizeMb) {
-        plans.add(new MergeNormalizationPlan(current, next));
+        final MergeNormalizationPlan plan = new MergeNormalizationPlan.Builder()
+          .addTarget(current, currentSizeMb)
+          .addTarget(next, nextSizeMb)

Review comment:
       Is there a foreseeable plan to start merging multiple regions in same plan? 
   Regardless, maybe we want to validate `list_size() % 2 == 0` in `MergeNormalizationPlan` constructor which initializes `normalizationTargets`?




----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-703979187


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  2s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 10s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 58s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m 12s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 50s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 57s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 57s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m 11s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 190m 13s |  hbase-server in the patch failed.  |
   |  |   | 218m 37s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/7/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 380cf9db1634 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 16251dbb53 |
   | Default Java | 1.8.0_232 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/7/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/7/testReport/ |
   | Max. process+thread count | 3334 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/7/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-705861790


   Thanks for the detailed reviews everyone!


----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r498413708



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -371,7 +358,11 @@ private boolean skipForMerge(final RegionStates regionStates, final RegionInfo r
       final long nextSizeMb = getRegionSizeMB(next);
       // always merge away empty regions when they present themselves.
       if (currentSizeMb == 0 || nextSizeMb == 0 || currentSizeMb + nextSizeMb < avgRegionSizeMb) {
-        plans.add(new MergeNormalizationPlan(current, next));
+        final MergeNormalizationPlan plan = new MergeNormalizationPlan.Builder()
+          .addTarget(current, currentSizeMb)
+          .addTarget(next, nextSizeMb)

Review comment:
       This PR is ~~not~~ now posted, #2490.




----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-704016910


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m 17s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  8s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 54s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 42s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m  4s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  6s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  6s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 43s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 39s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 123m 46s |  hbase-server in the patch failed.  |
   |  |   | 151m 57s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/8/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 7144f19c2df3 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 9f238bd79e |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/8/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/8/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/8/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/8/testReport/ |
   | Max. process+thread count | 4513 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/8/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] huaxiangsun commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
huaxiangsun commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r499093876



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorker.java
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.RateLimiter;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+
+/**
+ * Consumes normalization request targets ({@link TableName}s) off the
+ * {@link RegionNormalizerWorkQueue}, dispatches them to the {@link RegionNormalizer},
+ * and executes the resulting {@link NormalizationPlan}s.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorker implements Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerWorker.class);
+  private static final String RATE_LIMIT_BYTES_PER_SEC_KEY =
+    "hbase.normalizer.throughput.max_bytes_per_sec";
+  private static final long RATE_UNLIMITED_BYTES = 1_000_000_000_000L; // 1TB/sec
+
+  private final MasterServices masterServices;
+  private final RegionNormalizer regionNormalizer;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RateLimiter rateLimiter;
+
+  private final long[] skippedCount;
+  private long splitPlanCount;
+  private long mergePlanCount;
+
+  public RegionNormalizerWorker(
+    final Configuration configuration,
+    final MasterServices masterServices,
+    final RegionNormalizer regionNormalizer,
+    final RegionNormalizerWorkQueue<TableName> workQueue
+  ) {
+    this.masterServices = masterServices;
+    this.regionNormalizer = regionNormalizer;
+    this.workQueue = workQueue;
+    this.skippedCount = new long[NormalizationPlan.PlanType.values().length];
+    this.splitPlanCount = 0;
+    this.mergePlanCount = 0;
+    this.rateLimiter = loadRateLimiter(configuration);
+  }
+
+  private static RateLimiter loadRateLimiter(final Configuration configuration) {
+    long rateLimitBytes =
+      configuration.getLongBytes(RATE_LIMIT_BYTES_PER_SEC_KEY, RATE_UNLIMITED_BYTES);
+    long rateLimitMbs = rateLimitBytes / 1_000_000L;
+    if (rateLimitMbs <= 0) {
+      LOG.warn("Configured value {}={} is <= 1MB. Falling back to default.",
+        RATE_LIMIT_BYTES_PER_SEC_KEY, rateLimitBytes);
+      rateLimitBytes = RATE_UNLIMITED_BYTES;
+      rateLimitMbs = RATE_UNLIMITED_BYTES / 1_000_000L;
+    }
+    LOG.info("Normalizer rate limit set to {}",
+      rateLimitBytes == RATE_UNLIMITED_BYTES ? "unlimited" : rateLimitMbs + " MB/sec");
+    return RateLimiter.create(rateLimitMbs);
+  }
+
+  /**
+   * @see RegionNormalizerManager#planSkipped(NormalizationPlan.PlanType)
+   */
+  void planSkipped(NormalizationPlan.PlanType type) {
+    synchronized (skippedCount) {
+      // updates come here via procedure threads, so synchronize access to this counter.
+      skippedCount[type.ordinal()]++;
+    }
+  }
+
+  /**
+   * @see RegionNormalizerManager#getSkippedCount(NormalizationPlan.PlanType)
+   */
+  long getSkippedCount(NormalizationPlan.PlanType type) {
+    return skippedCount[type.ordinal()];
+  }
+
+  /**
+   * @see HMaster#getSplitPlanCount()
+   */
+  long getSplitPlanCount() {
+    return splitPlanCount;
+  }
+
+  /**
+   * @see HMaster#getMergePlanCount()
+   */
+  long getMergePlanCount() {
+    return mergePlanCount;
+  }
+
+  @Override
+  public void run() {
+    while (true) {
+      if (Thread.interrupted()) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+      final TableName tableName;
+      try {
+        tableName = workQueue.take();
+      } catch (InterruptedException e) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+
+      final List<NormalizationPlan> plans = calculatePlans(tableName);
+      submitPlans(plans);
+    }
+  }
+
+  private List<NormalizationPlan> calculatePlans(final TableName tableName) {
+    try {
+      final TableDescriptor tblDesc = masterServices.getTableDescriptors().get(tableName);
+      if (tblDesc != null && !tblDesc.isNormalizationEnabled()) {
+        LOG.debug("Skipping table {} because normalization is disabled in its table properties.",
+          tableName);
+        return Collections.emptyList();
+      }
+    } catch (IOException e) {
+      LOG.debug("Skipping table {} because unable to access its table descriptor.", tableName, e);
+      return Collections.emptyList();
+    }
+
+    if (masterServices.skipRegionManagementAction("region normalizer")) {
+      return Collections.emptyList();
+    }

Review comment:
       Nit: can it be moved to the first place? In case it needs to skip action, it will save some some time to look up the table  descriptor.
   




----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r496989734



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorker.java
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.RateLimiter;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+
+/**
+ * Consumes normalization request targets ({@link TableName}s) off the
+ * {@link RegionNormalizerWorkQueue}, dispatches them to the {@link RegionNormalizer},
+ * and executes the resulting {@link NormalizationPlan}s.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorker implements Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerWorker.class);
+  private static final String RATE_LIMIT_BYTES_PER_SEC_KEY =
+    "hbase.normalizer.throughput.max_bytes_per_sec";
+  private static final long RATE_UNLIMITED_BYTES = 1_000_000_000_000L; // 1TB/sec
+
+  private final MasterServices masterServices;
+  private final RegionNormalizer regionNormalizer;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RateLimiter rateLimiter;
+
+  private final long[] skippedCount;
+  private long splitPlanCount;
+  private long mergePlanCount;
+
+  public RegionNormalizerWorker(
+    final Configuration configuration,
+    final MasterServices masterServices,
+    final RegionNormalizer regionNormalizer,
+    final RegionNormalizerWorkQueue<TableName> workQueue
+  ) {
+    this.masterServices = masterServices;
+    this.regionNormalizer = regionNormalizer;
+    this.workQueue = workQueue;
+    this.skippedCount = new long[NormalizationPlan.PlanType.values().length];
+    this.splitPlanCount = 0;
+    this.mergePlanCount = 0;
+    this.rateLimiter = loadRateLimiter(configuration);
+  }
+
+  private static RateLimiter loadRateLimiter(final Configuration configuration) {
+    long rateLimitBytes =
+      configuration.getLongBytes(RATE_LIMIT_BYTES_PER_SEC_KEY, RATE_UNLIMITED_BYTES);
+    long rateLimitMbs = rateLimitBytes / 1_000_000L;
+    if (rateLimitMbs <= 0) {
+      LOG.warn("Configured value {}={} is <= 1MB. Falling back to default.",
+        RATE_LIMIT_BYTES_PER_SEC_KEY, rateLimitBytes);
+      rateLimitBytes = RATE_UNLIMITED_BYTES;
+      rateLimitMbs = RATE_UNLIMITED_BYTES / 1_000_000L;
+    }
+    LOG.info("Normalizer rate limit set to {}",
+      rateLimitBytes == RATE_UNLIMITED_BYTES ? "unlimited" : rateLimitMbs + " MB/sec");
+    return RateLimiter.create(rateLimitMbs);
+  }
+
+  /**
+   * @see RegionNormalizerManager#planSkipped(NormalizationPlan.PlanType)
+   */
+  void planSkipped(NormalizationPlan.PlanType type) {
+    synchronized (skippedCount) {
+      // updates come here via procedure threads, so synchronize access to this counter.
+      skippedCount[type.ordinal()]++;
+    }
+  }
+
+  /**
+   * @see RegionNormalizerManager#getSkippedCount(NormalizationPlan.PlanType)
+   */
+  long getSkippedCount(NormalizationPlan.PlanType type) {
+    return skippedCount[type.ordinal()];
+  }
+
+  /**
+   * @see HMaster#getSplitPlanCount()
+   */
+  long getSplitPlanCount() {
+    return splitPlanCount;
+  }
+
+  /**
+   * @see HMaster#getMergePlanCount()
+   */
+  long getMergePlanCount() {
+    return mergePlanCount;
+  }
+
+  @Override
+  public void run() {
+    while (true) {
+      if (Thread.interrupted()) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+      final TableName tableName;
+      try {
+        tableName = workQueue.take();
+      } catch (InterruptedException e) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+
+      final List<NormalizationPlan> plans = calculatePlans(tableName);
+      submitPlans(plans);
+    }
+  }
+
+  private List<NormalizationPlan> calculatePlans(final TableName tableName) {
+    try {
+      final TableDescriptor tblDesc = masterServices.getTableDescriptors().get(tableName);
+      if (tblDesc != null && !tblDesc.isNormalizationEnabled()) {
+        LOG.debug("Skipping table {} because normalization is disabled in its table properties.",
+          tableName);
+        return Collections.emptyList();
+      }
+    } catch (IOException e) {
+      LOG.debug("Skipping table {} because unable to access its table descriptor.", tableName, e);
+      return Collections.emptyList();
+    }
+
+    if (masterServices.skipRegionManagementAction("region normalizer")) {
+      return Collections.emptyList();
+    }
+
+    final List<NormalizationPlan> plans = regionNormalizer.computePlansForTable(tableName);
+    if (CollectionUtils.isEmpty(plans)) {
+      LOG.debug("No normalization required for table {}.", tableName);
+      return Collections.emptyList();
+    }
+    return plans;
+  }
+
+  private void submitPlans(final List<NormalizationPlan> plans) {
+    // as of this writing, `plan.submit()` is non-blocking and uses Async Admin APIs to submit
+    // task, so there's no artificial rate-limiting of merge/split requests due to this serial loop.
+    for (NormalizationPlan plan : plans) {
+      switch (plan.getType()) {
+        case MERGE: {
+          submitMergePlan((MergeNormalizationPlan) plan);
+          break;
+        }
+        case SPLIT: {
+          submitSplitPlan((SplitNormalizationPlan) plan);
+          break;
+        }
+        case NONE:
+          LOG.debug("Nothing to do for {} with PlanType=NONE. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+        default:
+          LOG.warn("Plan {} is of an unrecognized PlanType. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+      }
+    }
+  }
+
+  /**
+   * Interacts with {@link MasterServices} in order to execute a plan.
+   */
+  private void submitMergePlan(final MergeNormalizationPlan plan) {
+    final int totalSizeMb;
+    try {
+      final long totalSizeMbLong = plan.getNormalizationTargets()
+        .stream()
+        .mapToLong(NormalizationTarget::getRegionSizeMb)
+        .reduce(0, Math::addExact);

Review comment:
       Yes, if this happens, it's reasonable to assume that it would keep happening. However, given that these are region sizes in megabytes, I think it's highly unlikely that overflow would ever happen. I just chose to use `Math::addExact` so that we'd find out about it instead of silently operating on an overflow value.




----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r499859137



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -371,7 +358,11 @@ private boolean skipForMerge(final RegionStates regionStates, final RegionInfo r
       final long nextSizeMb = getRegionSizeMB(next);
       // always merge away empty regions when they present themselves.
       if (currentSizeMb == 0 || nextSizeMb == 0 || currentSizeMb + nextSizeMb < avgRegionSizeMb) {
-        plans.add(new MergeNormalizationPlan(current, next));
+        final MergeNormalizationPlan plan = new MergeNormalizationPlan.Builder()
+          .addTarget(current, currentSizeMb)
+          .addTarget(next, nextSizeMb)

Review comment:
       See the logic of `MergeTableRegionsProcedure#checkRegionsToMerge`, that's where validation of any merge request is applied. I think it's not the normalizer's job to verify these parameters. And anyway, the user isn't specifying the regions, they only select a table. Normalizer's algorithm generates the region merge list.




----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-705330820


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m 18s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m  5s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 58s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m  8s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 39s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 48s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 57s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 57s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m  8s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 36s |  hbase-server generated 11 new + 4 unchanged - 25 fixed = 15 total (was 29)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 199m 23s |  hbase-server in the patch failed.  |
   |  |   | 228m 47s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 1744c431b731 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / a8c49a6463 |
   | Default Java | 1.8.0_232 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/testReport/ |
   | Max. process+thread count | 3846 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r501326147



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerManager.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * This class encapsulates the details of the {@link RegionNormalizer} subsystem.
+ */
+@InterfaceAudience.Private
+public class RegionNormalizerManager {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerManager.class);
+
+  private final RegionNormalizerTracker regionNormalizerTracker;
+  private final RegionNormalizerChore regionNormalizerChore;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RegionNormalizerWorker worker;
+  private final ExecutorService pool;
+
+  private final Object startStopLock = new Object();
+  private boolean started = false;
+  private boolean stopped = false;
+
+  public RegionNormalizerManager(
+    @NonNull  final RegionNormalizerTracker regionNormalizerTracker,
+    @Nullable final RegionNormalizerChore regionNormalizerChore,
+    @Nullable final RegionNormalizerWorkQueue<TableName> workQueue,
+    @Nullable final RegionNormalizerWorker worker
+  ) {
+    this.regionNormalizerTracker = regionNormalizerTracker;
+    this.regionNormalizerChore = regionNormalizerChore;
+    this.workQueue = workQueue;
+    this.worker = worker;
+    this.pool = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+      .setDaemon(true)
+      .setNameFormat("normalizer-worker-%d")
+      .setUncaughtExceptionHandler(
+        (thread, throwable) ->
+          LOG.error("Uncaught exception, worker thread likely terminated.", throwable))
+      .build());
+  }
+
+  public void start() {
+    if (started) {
+      return;

Review comment:
       Done.




----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r499865060



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java
##########
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is
+ * an ordered collection class that has the following properties:
+ * <ul>
+ *   <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
+ *   <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
+ *   <li>Work is retrieved on a FIFO policy.</li>
+ *   <li>Work retrieval blocks the calling thread until new work is available, as a
+ *     {@link BlockingQueue}.</li>
+ *   <li>Allows a producer to insert an item at the head of the queue, if desired.</li>
+ * </ul>
+ * Assumes low-frequency and low-parallelism concurrent access, so protects state using a
+ * simplistic synchronization strategy.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorkQueue<E> {
+
+  /** Underlying storage structure that gives us the Set behavior and FIFO retrieval policy. */
+  private LinkedHashSet<E> delegate;
+
+  // the locking structure used here follows the example found in LinkedBlockingQueue. The
+  // difference is that our locks guard access to `delegate` rather than the head node.
+
+  /** Lock held by take, poll, etc */
+  private final ReentrantLock takeLock;
+
+  /** Wait queue for waiting takes */
+  private final Condition notEmpty;
+
+  /** Lock held by put, offer, etc */
+  private final ReentrantLock putLock;
+
+  public RegionNormalizerWorkQueue() {
+    delegate = new LinkedHashSet<>();
+    takeLock = new ReentrantLock();
+    notEmpty = takeLock.newCondition();
+    putLock = new ReentrantLock();
+  }
+
+  /**
+   * Signals a waiting take. Called only from put/offer (which do not
+   * otherwise ordinarily lock takeLock.)
+   */
+  private void signalNotEmpty() {
+    final ReentrantLock takeLock = this.takeLock;
+    takeLock.lock();
+    try {
+      notEmpty.signal();
+    } finally {
+      takeLock.unlock();
+    }
+  }
+
+  /**
+   * Locks to prevent both puts and takes.
+   */
+  private void fullyLock() {
+    putLock.lock();
+    takeLock.lock();
+  }
+
+  /**
+   * Unlocks to allow both puts and takes.
+   */
+  private void fullyUnlock() {

Review comment:
       I looked but did not find one. I'd be happy to replace this class with something off-the-shelf.




----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r500531573



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/NormalizationTarget.java
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A POJO that caries details about a region selected for normalization through the pipeline.
+ */
+@InterfaceAudience.Private
+public class NormalizationTarget {
+  private final RegionInfo regionInfo;
+  private final long regionSizeMb;
+
+  public NormalizationTarget(final RegionInfo regionInfo, final long regionSizeMb) {
+    this.regionInfo = regionInfo;
+    this.regionSizeMb = regionSizeMb;
+  }
+
+  public RegionInfo getRegionInfo() {
+    return regionInfo;
+  }
+
+  public long getRegionSizeMb() {
+    return regionSizeMb;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    NormalizationTarget that = (NormalizationTarget) o;
+
+    return new EqualsBuilder()
+      .append(regionSizeMb, that.regionSizeMb)
+      .append(regionInfo, that.regionInfo)
+      .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37)
+      .append(regionInfo)
+      .append(regionSizeMb)
+      .toHashCode();
+  }
+
+  @Override public String toString() {
+    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)

Review comment:
       _I_ use it everywhere ;) And HashBuilder, and EqualsBuilder. There's no sense in custom versions of this code in different places, unless we're explicitly maintaining backward compatibility is some way... which I find suspect. Would be super cool if we could enforce the use with static analysis.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerManager.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * This class encapsulates the details of the {@link RegionNormalizer} subsystem.
+ */
+@InterfaceAudience.Private
+public class RegionNormalizerManager {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerManager.class);
+
+  private final RegionNormalizerTracker regionNormalizerTracker;
+  private final RegionNormalizerChore regionNormalizerChore;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RegionNormalizerWorker worker;
+  private final ExecutorService pool;
+
+  private final Object startStopLock = new Object();
+  private boolean started = false;
+  private boolean stopped = false;
+
+  public RegionNormalizerManager(
+    @NonNull  final RegionNormalizerTracker regionNormalizerTracker,
+    @Nullable final RegionNormalizerChore regionNormalizerChore,
+    @Nullable final RegionNormalizerWorkQueue<TableName> workQueue,
+    @Nullable final RegionNormalizerWorker worker
+  ) {
+    this.regionNormalizerTracker = regionNormalizerTracker;
+    this.regionNormalizerChore = regionNormalizerChore;
+    this.workQueue = workQueue;
+    this.worker = worker;
+    this.pool = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+      .setDaemon(true)
+      .setNameFormat("normalizer-worker-%d")
+      .setUncaughtExceptionHandler(
+        (thread, throwable) ->
+          LOG.error("Uncaught exception, worker thread likely terminated.", throwable))
+      .build());
+  }
+
+  public void start() {
+    if (started) {
+      return;
+    }
+    synchronized (startStopLock) {
+      if (started) {
+        return;
+      }
+      regionNormalizerTracker.start();
+      if (worker != null) {
+        // worker will be null when master is in maintenance mode.
+        pool.submit(worker);
+      }
+      started = true;
+    }
+  }
+
+  public void stop() {
+    if (!started) {
+      throw new IllegalStateException("calling `stop` without first calling `start`.");
+    }
+    if (stopped) {

Review comment:
       > Or want to use a guava Service?
   
   I don't know it. Let me read.

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterChoreScheduled.java
##########
@@ -114,22 +113,27 @@ public void testDefaultScheduledChores() throws Exception {
     hbckChoreTestChoreField.testIfChoreScheduled(hbckChore);
   }
 
-
+  /**
+   * Reflect into the {@link HMaster} instance and find by field name a specified instance
+   * of {@link ScheduledChore}.
+   */
   private static class TestChoreField<E extends ScheduledChore> {
 
-    private E getChoreObj(String fieldName) throws NoSuchFieldException,
-        IllegalAccessException {
-      Field masterField = HMaster.class.getDeclaredField(fieldName);
-      masterField.setAccessible(true);
-      E choreFieldVal = (E) masterField.get(hMaster);
-      return choreFieldVal;
+    @SuppressWarnings("unchecked")
+    private E getChoreObj(String fieldName) {
+      try {
+        Field masterField = HMaster.class.getDeclaredField(fieldName);
+        masterField.setAccessible(true);
+        return (E) masterField.get(hMaster);
+      } catch (Exception e) {
+        throw new AssertionError(
+          "Unable to retrieve field '" + fieldName + "' from HMaster instance.", e);

Review comment:
       You didn't miss it, you let the exception bubble up, which is a common style in this code base. I just updated the test to have a bit of a friendlier error message, something that might help a confused dev to understand a failure, someone who doesn't know the system under test. I happen to like writing test asserts that include an error message, rather than decrypting a stack trace with generic exception messages and a line number. But @saintstack has given me grief about this habit for as long as he's been reviewing my patches ;)

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerManager.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * This class encapsulates the details of the {@link RegionNormalizer} subsystem.
+ */
+@InterfaceAudience.Private
+public class RegionNormalizerManager {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerManager.class);
+
+  private final RegionNormalizerTracker regionNormalizerTracker;
+  private final RegionNormalizerChore regionNormalizerChore;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RegionNormalizerWorker worker;
+  private final ExecutorService pool;
+
+  private final Object startStopLock = new Object();
+  private boolean started = false;
+  private boolean stopped = false;
+
+  public RegionNormalizerManager(
+    @NonNull  final RegionNormalizerTracker regionNormalizerTracker,
+    @Nullable final RegionNormalizerChore regionNormalizerChore,
+    @Nullable final RegionNormalizerWorkQueue<TableName> workQueue,
+    @Nullable final RegionNormalizerWorker worker
+  ) {
+    this.regionNormalizerTracker = regionNormalizerTracker;
+    this.regionNormalizerChore = regionNormalizerChore;
+    this.workQueue = workQueue;
+    this.worker = worker;
+    this.pool = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+      .setDaemon(true)
+      .setNameFormat("normalizer-worker-%d")
+      .setUncaughtExceptionHandler(
+        (thread, throwable) ->
+          LOG.error("Uncaught exception, worker thread likely terminated.", throwable))
+      .build());
+  }
+
+  public void start() {
+    if (started) {
+      return;

Review comment:
       I used a double-checked lock out of habit, to avoid the synchronized if we can. I don't have a strong opinion, and will make the change as you suggest.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java
##########
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is
+ * an ordered collection class that has the following properties:
+ * <ul>
+ *   <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
+ *   <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
+ *   <li>Work is retrieved on a FIFO policy.</li>

Review comment:
       @virajjasani has it right -- if the items is already present the addition is effectively ignored and no state is changed. You can intentionally have an existing entry jump to the front of the queue using `putFirst`/`putAllFirst`.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -371,7 +358,11 @@ private boolean skipForMerge(final RegionStates regionStates, final RegionInfo r
       final long nextSizeMb = getRegionSizeMB(next);
       // always merge away empty regions when they present themselves.
       if (currentSizeMb == 0 || nextSizeMb == 0 || currentSizeMb + nextSizeMb < avgRegionSizeMb) {
-        plans.add(new MergeNormalizationPlan(current, next));
+        final MergeNormalizationPlan plan = new MergeNormalizationPlan.Builder()
+          .addTarget(current, currentSizeMb)
+          .addTarget(next, nextSizeMb)

Review comment:
       I haven't rebased it yet, but have a look at 450c4d5 on #2490 . The limit to pairs is removed by that patch, so I'll just have to undo the pair-wise API you propose.
   
   I could remove the use of the list from this patch, go back to using the first and second member variables, if you think that aspect is so critical. My push-back is because, in my opinion, it's not the POJO's responsibility to ensure these invariants, it's the job of the merge system. And the merge system already makes these checks, so let's leave that logic written in one place, and leave it close to where it's consumed.




----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-703493303


   As for `TestMasterChoreScheduled` test, it tries to retrieve all default (unconditional) chores of HMaster by retrieving singleton objects of chores using reflection from `HMaster.class`. 
   With the current refactored PR, we should facilitate retrieval of `regionNormalizerChore` from `RegionNormalizerManager.class` using reflection in `TestMasterChoreScheduled` (`class` as well as `targetObj` as args) :
   
   ```
       private E getChoreObj(String fieldName, Class cls, Object targetObj)
           throws NoSuchFieldException, IllegalAccessException {
         Field masterField = cls.getDeclaredField(fieldName);
         masterField.setAccessible(true);
         E choreFieldVal = (E) masterField.get(targetObj);
         return choreFieldVal;
       }
   ```


----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r500199740



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -371,7 +358,11 @@ private boolean skipForMerge(final RegionStates regionStates, final RegionInfo r
       final long nextSizeMb = getRegionSizeMB(next);
       // always merge away empty regions when they present themselves.
       if (currentSizeMb == 0 || nextSizeMb == 0 || currentSizeMb + nextSizeMb < avgRegionSizeMb) {
-        plans.add(new MergeNormalizationPlan(current, next));
+        final MergeNormalizationPlan plan = new MergeNormalizationPlan.Builder()
+          .addTarget(current, currentSizeMb)
+          .addTarget(next, nextSizeMb)

Review comment:
       Sure this is not normalizer's job to validate but by keeping target regions as list, we tend to have a probability of keeping odd no of regions around a while just to get them abandoned later by `MergeTableRegionsProcedure#checkRegionsToMerge` if we keep it unbounded list. Not that `SimpleRegionNormalizer` will add odd no of regions, this is just to be treated as a probability.
   
   How about `List<Pair<NormalizerRegionInfo, NormalizerRegionInfo>>` as target regions in `MergeNormalizationPlan`? 
   New POJO `NormalizerRegionInfo` can just contain `regionInfo` and `regionSize`. This way, at least we indicate a clear way for clients to send multiple pair of regions to merge. Thought?




----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r499169951



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorker.java
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.RateLimiter;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+
+/**
+ * Consumes normalization request targets ({@link TableName}s) off the
+ * {@link RegionNormalizerWorkQueue}, dispatches them to the {@link RegionNormalizer},
+ * and executes the resulting {@link NormalizationPlan}s.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorker implements Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerWorker.class);
+  private static final String RATE_LIMIT_BYTES_PER_SEC_KEY =
+    "hbase.normalizer.throughput.max_bytes_per_sec";
+  private static final long RATE_UNLIMITED_BYTES = 1_000_000_000_000L; // 1TB/sec
+
+  private final MasterServices masterServices;
+  private final RegionNormalizer regionNormalizer;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RateLimiter rateLimiter;
+
+  private final long[] skippedCount;
+  private long splitPlanCount;
+  private long mergePlanCount;
+
+  public RegionNormalizerWorker(
+    final Configuration configuration,
+    final MasterServices masterServices,
+    final RegionNormalizer regionNormalizer,
+    final RegionNormalizerWorkQueue<TableName> workQueue
+  ) {
+    this.masterServices = masterServices;
+    this.regionNormalizer = regionNormalizer;
+    this.workQueue = workQueue;
+    this.skippedCount = new long[NormalizationPlan.PlanType.values().length];
+    this.splitPlanCount = 0;
+    this.mergePlanCount = 0;
+    this.rateLimiter = loadRateLimiter(configuration);
+  }
+
+  private static RateLimiter loadRateLimiter(final Configuration configuration) {
+    long rateLimitBytes =
+      configuration.getLongBytes(RATE_LIMIT_BYTES_PER_SEC_KEY, RATE_UNLIMITED_BYTES);
+    long rateLimitMbs = rateLimitBytes / 1_000_000L;
+    if (rateLimitMbs <= 0) {
+      LOG.warn("Configured value {}={} is <= 1MB. Falling back to default.",
+        RATE_LIMIT_BYTES_PER_SEC_KEY, rateLimitBytes);
+      rateLimitBytes = RATE_UNLIMITED_BYTES;
+      rateLimitMbs = RATE_UNLIMITED_BYTES / 1_000_000L;
+    }
+    LOG.info("Normalizer rate limit set to {}",
+      rateLimitBytes == RATE_UNLIMITED_BYTES ? "unlimited" : rateLimitMbs + " MB/sec");
+    return RateLimiter.create(rateLimitMbs);
+  }
+
+  /**
+   * @see RegionNormalizerManager#planSkipped(NormalizationPlan.PlanType)
+   */
+  void planSkipped(NormalizationPlan.PlanType type) {
+    synchronized (skippedCount) {
+      // updates come here via procedure threads, so synchronize access to this counter.
+      skippedCount[type.ordinal()]++;
+    }
+  }
+
+  /**
+   * @see RegionNormalizerManager#getSkippedCount(NormalizationPlan.PlanType)
+   */
+  long getSkippedCount(NormalizationPlan.PlanType type) {
+    return skippedCount[type.ordinal()];
+  }
+
+  /**
+   * @see HMaster#getSplitPlanCount()
+   */
+  long getSplitPlanCount() {
+    return splitPlanCount;
+  }
+
+  /**
+   * @see HMaster#getMergePlanCount()
+   */
+  long getMergePlanCount() {
+    return mergePlanCount;
+  }
+
+  @Override
+  public void run() {
+    while (true) {
+      if (Thread.interrupted()) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+      final TableName tableName;
+      try {
+        tableName = workQueue.take();
+      } catch (InterruptedException e) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+
+      final List<NormalizationPlan> plans = calculatePlans(tableName);
+      submitPlans(plans);
+    }
+  }
+
+  private List<NormalizationPlan> calculatePlans(final TableName tableName) {
+    try {
+      final TableDescriptor tblDesc = masterServices.getTableDescriptors().get(tableName);
+      if (tblDesc != null && !tblDesc.isNormalizationEnabled()) {
+        LOG.debug("Skipping table {} because normalization is disabled in its table properties.",
+          tableName);
+        return Collections.emptyList();
+      }
+    } catch (IOException e) {
+      LOG.debug("Skipping table {} because unable to access its table descriptor.", tableName, e);
+      return Collections.emptyList();
+    }
+
+    if (masterServices.skipRegionManagementAction("region normalizer")) {
+      return Collections.emptyList();
+    }
+
+    final List<NormalizationPlan> plans = regionNormalizer.computePlansForTable(tableName);
+    if (CollectionUtils.isEmpty(plans)) {
+      LOG.debug("No normalization required for table {}.", tableName);
+      return Collections.emptyList();
+    }
+    return plans;
+  }
+
+  private void submitPlans(final List<NormalizationPlan> plans) {
+    // as of this writing, `plan.submit()` is non-blocking and uses Async Admin APIs to submit
+    // task, so there's no artificial rate-limiting of merge/split requests due to this serial loop.
+    for (NormalizationPlan plan : plans) {
+      switch (plan.getType()) {
+        case MERGE: {
+          submitMergePlan((MergeNormalizationPlan) plan);
+          break;
+        }
+        case SPLIT: {
+          submitSplitPlan((SplitNormalizationPlan) plan);
+          break;
+        }
+        case NONE:
+          LOG.debug("Nothing to do for {} with PlanType=NONE. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+        default:
+          LOG.warn("Plan {} is of an unrecognized PlanType. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+      }
+    }
+  }
+
+  /**
+   * Interacts with {@link MasterServices} in order to execute a plan.
+   */
+  private void submitMergePlan(final MergeNormalizationPlan plan) {
+    final int totalSizeMb;
+    try {
+      final long totalSizeMbLong = plan.getNormalizationTargets()
+        .stream()
+        .mapToLong(NormalizationTarget::getRegionSizeMb)
+        .reduce(0, Math::addExact);
+      totalSizeMb = Math.toIntExact(totalSizeMbLong);
+    } catch (ArithmeticException e) {
+      LOG.debug("Sum of merge request size overflows rate limiter data type. {}", plan);
+      planSkipped(plan.getType());
+      return;
+    }
+
+    final RegionInfo[] infos = plan.getNormalizationTargets()
+      .stream()
+      .map(NormalizationTarget::getRegionInfo)
+      .toArray(RegionInfo[]::new);
+    final long rateLimitedSecs = Math.round(rateLimiter.acquire(Math.max(1, totalSizeMb)));
+    LOG.debug("Rate limiting delayed this operation by {}", Duration.ofSeconds(rateLimitedSecs));

Review comment:
       I think this is nice one from latency view point but this is all happening behind the scene by a single thread running continuously and hence, there is no actual API level latency coming into picture.
   However, it would be definitely better to get blocked on `RL.acquite()` while Proc is already submitted.
   I am +1 for this.
   
   And one more nice comment with this very reasoning behind keeping `RL.acquire()` after `masterServices.mergeRegions()` and `masterServices.splitRegion()` would be really great!




----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 WIP Normalizer needs a throttle

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-698648520






----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r500199740



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -371,7 +358,11 @@ private boolean skipForMerge(final RegionStates regionStates, final RegionInfo r
       final long nextSizeMb = getRegionSizeMB(next);
       // always merge away empty regions when they present themselves.
       if (currentSizeMb == 0 || nextSizeMb == 0 || currentSizeMb + nextSizeMb < avgRegionSizeMb) {
-        plans.add(new MergeNormalizationPlan(current, next));
+        final MergeNormalizationPlan plan = new MergeNormalizationPlan.Builder()
+          .addTarget(current, currentSizeMb)
+          .addTarget(next, nextSizeMb)

Review comment:
       Sure this is not normalizer's job to validate but by keeping target regions as list, we do have a probability of keeping odd no of regions around a while just to get them abandoned later by `MergeTableRegionsProcedure#checkRegionsToMerge` if we keep it unbounded list. Not that `SimpleRegionNormalizer` will add odd no of regions, this is just to be treated as a probability.
   
   How about `List<Pair<NormalizerRegionInfo, NormalizerRegionInfo>>` as target regions in `MergeNormalizationPlan`? 
   New POJO `NormalizerRegionInfo` can just contain `regionInfo` and `regionSize`. This way, at least we indicate a clear way for clients to send multiple pair of regions to merge. Thought?




----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r499870148



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorker.java
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.RateLimiter;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+
+/**
+ * Consumes normalization request targets ({@link TableName}s) off the
+ * {@link RegionNormalizerWorkQueue}, dispatches them to the {@link RegionNormalizer},
+ * and executes the resulting {@link NormalizationPlan}s.
+ */
+@InterfaceAudience.Private
+class RegionNormalizerWorker implements Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerWorker.class);
+  private static final String RATE_LIMIT_BYTES_PER_SEC_KEY =
+    "hbase.normalizer.throughput.max_bytes_per_sec";
+  private static final long RATE_UNLIMITED_BYTES = 1_000_000_000_000L; // 1TB/sec
+
+  private final MasterServices masterServices;
+  private final RegionNormalizer regionNormalizer;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RateLimiter rateLimiter;
+
+  private final long[] skippedCount;
+  private long splitPlanCount;
+  private long mergePlanCount;
+
+  public RegionNormalizerWorker(
+    final Configuration configuration,
+    final MasterServices masterServices,
+    final RegionNormalizer regionNormalizer,
+    final RegionNormalizerWorkQueue<TableName> workQueue
+  ) {
+    this.masterServices = masterServices;
+    this.regionNormalizer = regionNormalizer;
+    this.workQueue = workQueue;
+    this.skippedCount = new long[NormalizationPlan.PlanType.values().length];
+    this.splitPlanCount = 0;
+    this.mergePlanCount = 0;
+    this.rateLimiter = loadRateLimiter(configuration);
+  }
+
+  private static RateLimiter loadRateLimiter(final Configuration configuration) {
+    long rateLimitBytes =
+      configuration.getLongBytes(RATE_LIMIT_BYTES_PER_SEC_KEY, RATE_UNLIMITED_BYTES);
+    long rateLimitMbs = rateLimitBytes / 1_000_000L;
+    if (rateLimitMbs <= 0) {
+      LOG.warn("Configured value {}={} is <= 1MB. Falling back to default.",
+        RATE_LIMIT_BYTES_PER_SEC_KEY, rateLimitBytes);
+      rateLimitBytes = RATE_UNLIMITED_BYTES;
+      rateLimitMbs = RATE_UNLIMITED_BYTES / 1_000_000L;
+    }
+    LOG.info("Normalizer rate limit set to {}",
+      rateLimitBytes == RATE_UNLIMITED_BYTES ? "unlimited" : rateLimitMbs + " MB/sec");
+    return RateLimiter.create(rateLimitMbs);
+  }
+
+  /**
+   * @see RegionNormalizerManager#planSkipped(NormalizationPlan.PlanType)
+   */
+  void planSkipped(NormalizationPlan.PlanType type) {
+    synchronized (skippedCount) {
+      // updates come here via procedure threads, so synchronize access to this counter.
+      skippedCount[type.ordinal()]++;
+    }
+  }
+
+  /**
+   * @see RegionNormalizerManager#getSkippedCount(NormalizationPlan.PlanType)
+   */
+  long getSkippedCount(NormalizationPlan.PlanType type) {
+    return skippedCount[type.ordinal()];
+  }
+
+  /**
+   * @see HMaster#getSplitPlanCount()
+   */
+  long getSplitPlanCount() {
+    return splitPlanCount;
+  }
+
+  /**
+   * @see HMaster#getMergePlanCount()
+   */
+  long getMergePlanCount() {
+    return mergePlanCount;
+  }
+
+  @Override
+  public void run() {
+    while (true) {
+      if (Thread.interrupted()) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+      final TableName tableName;
+      try {
+        tableName = workQueue.take();
+      } catch (InterruptedException e) {
+        LOG.debug("interrupt detected. terminating.");
+        break;
+      }
+
+      final List<NormalizationPlan> plans = calculatePlans(tableName);
+      submitPlans(plans);
+    }
+  }
+
+  private List<NormalizationPlan> calculatePlans(final TableName tableName) {
+    try {
+      final TableDescriptor tblDesc = masterServices.getTableDescriptors().get(tableName);
+      if (tblDesc != null && !tblDesc.isNormalizationEnabled()) {
+        LOG.debug("Skipping table {} because normalization is disabled in its table properties.",
+          tableName);
+        return Collections.emptyList();
+      }
+    } catch (IOException e) {
+      LOG.debug("Skipping table {} because unable to access its table descriptor.", tableName, e);
+      return Collections.emptyList();
+    }
+
+    if (masterServices.skipRegionManagementAction("region normalizer")) {
+      return Collections.emptyList();
+    }
+
+    final List<NormalizationPlan> plans = regionNormalizer.computePlansForTable(tableName);
+    if (CollectionUtils.isEmpty(plans)) {
+      LOG.debug("No normalization required for table {}.", tableName);
+      return Collections.emptyList();
+    }
+    return plans;
+  }
+
+  private void submitPlans(final List<NormalizationPlan> plans) {
+    // as of this writing, `plan.submit()` is non-blocking and uses Async Admin APIs to submit
+    // task, so there's no artificial rate-limiting of merge/split requests due to this serial loop.
+    for (NormalizationPlan plan : plans) {
+      switch (plan.getType()) {
+        case MERGE: {
+          submitMergePlan((MergeNormalizationPlan) plan);
+          break;
+        }
+        case SPLIT: {
+          submitSplitPlan((SplitNormalizationPlan) plan);
+          break;
+        }
+        case NONE:
+          LOG.debug("Nothing to do for {} with PlanType=NONE. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+        default:
+          LOG.warn("Plan {} is of an unrecognized PlanType. Ignoring.", plan);
+          planSkipped(plan.getType());
+          break;
+      }
+    }
+  }
+
+  /**
+   * Interacts with {@link MasterServices} in order to execute a plan.
+   */
+  private void submitMergePlan(final MergeNormalizationPlan plan) {
+    final int totalSizeMb;
+    try {
+      final long totalSizeMbLong = plan.getNormalizationTargets()
+        .stream()
+        .mapToLong(NormalizationTarget::getRegionSizeMb)
+        .reduce(0, Math::addExact);
+      totalSizeMb = Math.toIntExact(totalSizeMbLong);
+    } catch (ArithmeticException e) {
+      LOG.debug("Sum of merge request size overflows rate limiter data type. {}", plan);
+      planSkipped(plan.getType());
+      return;
+    }
+
+    final RegionInfo[] infos = plan.getNormalizationTargets()
+      .stream()
+      .map(NormalizationTarget::getRegionInfo)
+      .toArray(RegionInfo[]::new);
+    final long rateLimitedSecs = Math.round(rateLimiter.acquire(Math.max(1, totalSizeMb)));
+    LOG.debug("Rate limiting delayed this operation by {}", Duration.ofSeconds(rateLimitedSecs));

Review comment:
       So you're suggesting that the thread's blocking could match more closely the way resources are consumed. By checking on the rate limit after submitting the procedure, the thread is held until after the submitted work as been conceptually processed. I think that makes sense.




----------------------------------------------------------------
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



[GitHub] [hbase] saintstack commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r501329639



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/package-info.java
##########
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * The Region Normalizer subsystem is responsible for coaxing all the regions in a table toward
+ * a "normal" size, according to their storefile size. It does this by splitting regions that
+ * are significantly larger than the norm, and merging regions that are significantly smaller than
+ * the norm.
+ * </p>
+ * The public interface to the Region Normalizer subsystem is limited to the following classes:
+ * <ul>
+ *   <li>
+ *     The {@link org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory} provides an
+ *     entry point for creating an instance of the
+ *     {@link org.apache.hadoop.hbase.master.normalizer.RegionNormalizerManager}.
+ *   </li>
+ *   <li>
+ *     The {@link org.apache.hadoop.hbase.master.normalizer.RegionNormalizerManager} encapsulates
+ *     the whole Region Normalizer subsystem. You'll find one of these hanging off of the
+ *     {@link org.apache.hadoop.hbase.master.HMaster}, which uses it to delegate API calls. There
+ *     is usually only a single instance of this class.
+ *   </li>
+ *   <li>
+ *     Various configuration points that share the common prefix of {@code hbase.normalizer}.
+ *     <ul>
+ *       <li>Whether to split a region as part of normalization. Configuration:
+ *         {@value org.apache.hadoop.hbase.master.normalizer.SimpleRegionNormalizer#SPLIT_ENABLED_KEY},
+ *         default: {@value org.apache.hadoop.hbase.master.normalizer.SimpleRegionNormalizer#DEFAULT_SPLIT_ENABLED}.
+ *       </li>
+ *       <li>Whether to merge a region as part of normalization. Configuration:
+ *         {@value org.apache.hadoop.hbase.master.normalizer.SimpleRegionNormalizer#MERGE_ENABLED_KEY},
+ *         default: {@value org.apache.hadoop.hbase.master.normalizer.SimpleRegionNormalizer#DEFAULT_MERGE_ENABLED}.
+ *       </li>
+ *       <li>The minimum number of regions in a table to consider it for merge normalization.
+ *         Configuration: {@value org.apache.hadoop.hbase.master.normalizer.SimpleRegionNormalizer#MIN_REGION_COUNT_KEY},
+ *         default: {@value org.apache.hadoop.hbase.master.normalizer.SimpleRegionNormalizer#DEFAULT_MIN_REGION_COUNT}.
+ *       </li>
+ *       <li>The minimum age for a region to be considered for a merge, in days. Configuration:
+ *         {@value org.apache.hadoop.hbase.master.normalizer.SimpleRegionNormalizer#MERGE_MIN_REGION_AGE_DAYS_KEY},
+ *         default: {@value org.apache.hadoop.hbase.master.normalizer.SimpleRegionNormalizer#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 org.apache.hadoop.hbase.master.normalizer.SimpleRegionNormalizer#MERGE_MIN_REGION_SIZE_MB_KEY},
+ *         default: {@value org.apache.hadoop.hbase.master.normalizer.SimpleRegionNormalizer#DEFAULT_MERGE_MIN_REGION_SIZE_MB}.
+ *       </li>
+ *       <li>The limit on total throughput of the Region Normalizer's actions, in whole MBs. Configuration:
+ *         {@value org.apache.hadoop.hbase.master.normalizer.RegionNormalizerWorker#RATE_LIMIT_BYTES_PER_SEC_KEY},
+ *         default: unlimited.
+ *       </li>
+ *     </ul>
+ *     <p>
+ *       To see detailed logging of the application of these configuration values, set the log
+ *       level for this package to `TRACE`.
+ *     </p>
+ *   </li>
+ * </ul>
+ * The Region Normalizer subsystem is composed of a handful of related classes:
+ * <ul>
+ *   <li>
+ *     The {@link org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker} provides a system by
+ *     which the Normalizer can be disabled at runtime. It currently does this by managing a znode,
+ *     but this is an implementation detail.
+ *   </li>
+ *   <li>
+ *     The {@link org.apache.hadoop.hbase.master.normalizer.RegionNormalizerWorkQueue} is a
+ *     {@link java.util.Set}-like {@link java.util.Queue} that permits a single copy of a given
+ *     work item to exist in the queue at one time. It also provides a facility for a producer to
+ *     add an item to the front of the line. Consumers are blocked waiting for new work.
+ *   </li>
+ *   <li>
+ *     The {@link org.apache.hadoop.hbase.master.normalizer.RegionNormalizerChore} wakes up
+ *     periodically and schedules new normalization work, adding targets to the queue.
+ *   </li>
+ *   <li>
+ *     The {@link org.apache.hadoop.hbase.master.normalizer.RegionNormalizerWorker} runs in a
+ *     daemon thread, grabbing work off the queue as is it becomes available.
+ *   </li>
+ *   <li>
+ *     The {@link org.apache.hadoop.hbase.master.normalizer.SimpleRegionNormalizer} implements the
+ *     logic for calculating target region sizes and emitting a list of corresponding
+ *     {@link org.apache.hadoop.hbase.master.normalizer.NormalizationPlan} objects.
+ *   </li>
+ * </ul>
+ */
+package org.apache.hadoop.hbase.master.normalizer;

Review comment:
       nice




----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r498407097



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
##########
@@ -1924,70 +1907,17 @@ public boolean normalizeRegions(final NormalizeTableFilterParams ntfp) throws IO
       return false;
     }
 
-    if (!normalizationInProgressLock.tryLock()) {
-      // Don't run the normalizer concurrently
-      LOG.info("Normalization already in progress. Skipping request.");
-      return true;
-    }
-
-    int affectedTables = 0;
-    try {
-      final Set<TableName> matchingTables = getTableDescriptors(new LinkedList<>(),
-        ntfp.getNamespace(), ntfp.getRegex(), ntfp.getTableNames(), false)
-        .stream()
-        .map(TableDescriptor::getTableName)
-        .collect(Collectors.toSet());
-      final Set<TableName> allEnabledTables =
-        tableStateManager.getTablesInStates(TableState.State.ENABLED);
-      final List<TableName> targetTables =
-        new ArrayList<>(Sets.intersection(matchingTables, allEnabledTables));
-      Collections.shuffle(targetTables);
-
-      final List<Long> submittedPlanProcIds = new ArrayList<>();
-      for (TableName table : targetTables) {
-        if (table.isSystemTable()) {

Review comment:
       All this moves to RegionNormalizerWorker.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
##########
@@ -354,6 +353,13 @@ long splitRegion(
    */
   boolean isInMaintenanceMode();
 
+  /**
+   * Checks master state before initiating action over region topology.
+   * @param action the name of the action under consideration, for logging.
+   * @return {@code true} when the caller should exit early, {@code false} otherwise.
+   */
+  boolean skipRegionManagementAction(final String action);

Review comment:
       I don't love exposing this up here on MasterServices, but imo, it's better to have the interface here than to expose its innards.
   
   Suggestions welcome.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
##########
@@ -1924,70 +1907,17 @@ public boolean normalizeRegions(final NormalizeTableFilterParams ntfp) throws IO
       return false;
     }
 
-    if (!normalizationInProgressLock.tryLock()) {
-      // Don't run the normalizer concurrently
-      LOG.info("Normalization already in progress. Skipping request.");
-      return true;
-    }
-
-    int affectedTables = 0;
-    try {
-      final Set<TableName> matchingTables = getTableDescriptors(new LinkedList<>(),

Review comment:
       Table selection code stays here in HMaster.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
##########
@@ -464,9 +451,6 @@ public void run() {
   // handle table states
   private TableStateManager tableStateManager;
 
-  private long splitPlanCount;

Review comment:
       Moved to RegionNormalizerWorker.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -203,16 +200,6 @@ public void setMasterServices(final MasterServices masterServices) {
     this.masterServices = masterServices;
   }
 
-  @Override
-  public void planSkipped(final RegionInfo hri, final PlanType type) {

Review comment:
       Moved to RegionNormalizerWorker.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
##########
@@ -1924,70 +1907,17 @@ public boolean normalizeRegions(final NormalizeTableFilterParams ntfp) throws IO
       return false;
     }
 
-    if (!normalizationInProgressLock.tryLock()) {
-      // Don't run the normalizer concurrently
-      LOG.info("Normalization already in progress. Skipping request.");
-      return true;
-    }
-
-    int affectedTables = 0;
-    try {
-      final Set<TableName> matchingTables = getTableDescriptors(new LinkedList<>(),
-        ntfp.getNamespace(), ntfp.getRegex(), ntfp.getTableNames(), false)
-        .stream()
-        .map(TableDescriptor::getTableName)
-        .collect(Collectors.toSet());
-      final Set<TableName> allEnabledTables =
-        tableStateManager.getTablesInStates(TableState.State.ENABLED);
-      final List<TableName> targetTables =
-        new ArrayList<>(Sets.intersection(matchingTables, allEnabledTables));
-      Collections.shuffle(targetTables);
-
-      final List<Long> submittedPlanProcIds = new ArrayList<>();
-      for (TableName table : targetTables) {
-        if (table.isSystemTable()) {
-          continue;
-        }
-        final TableDescriptor tblDesc = getTableDescriptors().get(table);
-        if (tblDesc != null && !tblDesc.isNormalizationEnabled()) {
-          LOG.debug(
-            "Skipping table {} because normalization is disabled in its table properties.", table);
-          continue;
-        }
-
-        // make one last check that the cluster isn't shutting down before proceeding.
-        if (skipRegionManagementAction("region normalizer")) {
-          return false;
-        }
-
-        final List<NormalizationPlan> plans = normalizer.computePlansForTable(table);
-        if (CollectionUtils.isEmpty(plans)) {
-          LOG.debug("No normalization required for table {}.", table);
-          continue;
-        }
-
-        affectedTables++;
-        // as of this writing, `plan.submit()` is non-blocking and uses Async Admin APIs to
-        // submit task , so there's no artificial rate-
-        // limiting of merge/split requests due to this serial loop.
-        for (NormalizationPlan plan : plans) {
-          long procId = plan.submit(this);

Review comment:
       Plan submission is not handled by the worker, the plans are converted into simple POJOs.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -371,7 +358,11 @@ private boolean skipForMerge(final RegionStates regionStates, final RegionInfo r
       final long nextSizeMb = getRegionSizeMB(next);
       // always merge away empty regions when they present themselves.
       if (currentSizeMb == 0 || nextSizeMb == 0 || currentSizeMb + nextSizeMb < avgRegionSizeMb) {
-        plans.add(new MergeNormalizationPlan(current, next));
+        final MergeNormalizationPlan plan = new MergeNormalizationPlan.Builder()
+          .addTarget(current, currentSizeMb)
+          .addTarget(next, nextSizeMb)

Review comment:
       This PR is not posted, #2490.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
##########
@@ -337,9 +330,6 @@ public void run() {
   // Tracker for split and merge state
   private SplitOrMergeTracker splitOrMergeTracker;
 
-  // Tracker for region normalizer state
-  private RegionNormalizerTracker regionNormalizerTracker;

Review comment:
       Moved to RegionNormalizerManager.




----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 WIP Normalizer needs a throttle

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-698648520


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 49s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 25s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 41s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   3m 44s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 21s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 51s |  hbase-server: The patch generated 2 new + 145 unchanged - 7 fixed = 147 total (was 152)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  19m 12s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   3m 45s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 21s |  The patch does not generate ASF License warnings.  |
   |  |   |  54m 33s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 67e3ed7af6f4 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 687e53b7e4 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 94 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-702578836


   We have two blocking calls in `NormalizerWorker` thread:
   
   1. While retrieving new table for normalizing from Queue
   2. RateLimiter blocking threads trying to submit plans


----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-705330820


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m 18s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   4m  5s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 58s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m  8s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 39s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 48s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 57s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 57s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m  8s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 36s |  hbase-server generated 11 new + 4 unchanged - 25 fixed = 15 total (was 29)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 199m 23s |  hbase-server in the patch failed.  |
   |  |   | 228m 47s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 1744c431b731 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / a8c49a6463 |
   | Default Java | 1.8.0_232 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/testReport/ |
   | Max. process+thread count | 3846 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/9/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r500388349



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerWorkQueue.java
##########
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A specialized collection that holds pending work for the {@link RegionNormalizerWorker}. It is
+ * an ordered collection class that has the following properties:
+ * <ul>
+ *   <li>Guarantees uniqueness of elements, as a {@link Set}.</li>
+ *   <li>Consumers retrieve objects from the head, as a {@link Queue}, via {@link #take()}.</li>
+ *   <li>Work is retrieved on a FIFO policy.</li>

Review comment:
       Since `LinkedHashSet` is used underneath, the order of element should not change if it is re-inserted.




----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r500199740



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -371,7 +358,11 @@ private boolean skipForMerge(final RegionStates regionStates, final RegionInfo r
       final long nextSizeMb = getRegionSizeMB(next);
       // always merge away empty regions when they present themselves.
       if (currentSizeMb == 0 || nextSizeMb == 0 || currentSizeMb + nextSizeMb < avgRegionSizeMb) {
-        plans.add(new MergeNormalizationPlan(current, next));
+        final MergeNormalizationPlan plan = new MergeNormalizationPlan.Builder()
+          .addTarget(current, currentSizeMb)
+          .addTarget(next, nextSizeMb)

Review comment:
       Sure this is not normalizer's job to validate but by keeping target regions as list, we tend to have a probability of keeping odd no of regions a while just to get them abandoned later by `MergeTableRegionsProcedure#checkRegionsToMerge` if we keep it unbounded list. Not that `SimpleRegionNormalizer` will do it, it's just a probability that someone can use in test also.
   
   How about `List<Pair<NormalizerRegionInfo, NormalizerRegionInfo>>` as target regions in `MergeNormalizationPlan`? 
   New POJO `NormalizerRegionInfo` can just contain `regionInfo` and `regionSize`. This way, it's a clear indication of sending list of pair of regions to merge.




----------------------------------------------------------------
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



[GitHub] [hbase] virajjasani commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r500199740



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -371,7 +358,11 @@ private boolean skipForMerge(final RegionStates regionStates, final RegionInfo r
       final long nextSizeMb = getRegionSizeMB(next);
       // always merge away empty regions when they present themselves.
       if (currentSizeMb == 0 || nextSizeMb == 0 || currentSizeMb + nextSizeMb < avgRegionSizeMb) {
-        plans.add(new MergeNormalizationPlan(current, next));
+        final MergeNormalizationPlan plan = new MergeNormalizationPlan.Builder()
+          .addTarget(current, currentSizeMb)
+          .addTarget(next, nextSizeMb)

Review comment:
       Sure this is not normalizer's job to validate but by keeping target regions as list, we do have a probability of keeping odd no of regions around a while just to get them abandoned later by `MergeTableRegionsProcedure#checkRegionsToMerge` if we keep it unbounded list. Not that `SimpleRegionNormalizer` will add odd no of regions, this is just to be treated as a probability.
   
   How about `List<Pair<NormalizerRegionInfo, NormalizerRegionInfo>>` as target regions in `MergeNormalizationPlan`? 
   New POJO `NormalizerRegionInfo` can just contain `regionInfo` and `regionSize`. 
   
   Or `List<MergeNormalizerRegionInfo>` where `MergeNormalizerRegionInfo` can keep `firstRegion` and `secondRegion` just like how it was before but this time we can have list of that object.
   
   This way, at least we indicate a clear way for clients to send list of pair of regions to merge. Thought?




----------------------------------------------------------------
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



[GitHub] [hbase] Apache-HBase commented on pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#issuecomment-703926016


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 43s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m  8s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m  0s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   3m 24s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m  7s |  hbase-server: The patch generated 0 new + 145 unchanged - 7 fixed = 145 total (was 152)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  11m 25s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   2m  9s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 15s |  The patch does not generate ASF License warnings.  |
   |  |   |  33m  0s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/7/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2454 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 5d782ef9305c 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 16251dbb53 |
   | Max. process+thread count | 94 (vs. ulimit of 30000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2454/7/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r501324001



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -54,29 +53,9 @@
  *   <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:

Review comment:
       All this doc moved to the package-level.




----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r501303527



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##########
@@ -371,7 +358,11 @@ private boolean skipForMerge(final RegionStates regionStates, final RegionInfo r
       final long nextSizeMb = getRegionSizeMB(next);
       // always merge away empty regions when they present themselves.
       if (currentSizeMb == 0 || nextSizeMb == 0 || currentSizeMb + nextSizeMb < avgRegionSizeMb) {
-        plans.add(new MergeNormalizationPlan(current, next));
+        final MergeNormalizationPlan plan = new MergeNormalizationPlan.Builder()
+          .addTarget(current, currentSizeMb)
+          .addTarget(next, nextSizeMb)

Review comment:
       Sure, let me buff up the javadoc on that class. FYI, technically an odd number of regions can be merged, so long a they're more than 2 and in a contiguous sequence.




----------------------------------------------------------------
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



[GitHub] [hbase] huaxiangsun commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
huaxiangsun commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r498979097



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
##########
@@ -1924,70 +1907,17 @@ public boolean normalizeRegions(final NormalizeTableFilterParams ntfp) throws IO
       return false;
     }
 
-    if (!normalizationInProgressLock.tryLock()) {
-      // Don't run the normalizer concurrently
-      LOG.info("Normalization already in progress. Skipping request.");
-      return true;
-    }
-
-    int affectedTables = 0;
-    try {
-      final Set<TableName> matchingTables = getTableDescriptors(new LinkedList<>(),

Review comment:
       Thanks, this is very helpful.




----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r501326034



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerManager.java
##########
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.normalizer;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * This class encapsulates the details of the {@link RegionNormalizer} subsystem.
+ */
+@InterfaceAudience.Private
+public class RegionNormalizerManager {
+  private static final Logger LOG = LoggerFactory.getLogger(RegionNormalizerManager.class);
+
+  private final RegionNormalizerTracker regionNormalizerTracker;
+  private final RegionNormalizerChore regionNormalizerChore;
+  private final RegionNormalizerWorkQueue<TableName> workQueue;
+  private final RegionNormalizerWorker worker;
+  private final ExecutorService pool;
+
+  private final Object startStopLock = new Object();
+  private boolean started = false;
+  private boolean stopped = false;
+
+  public RegionNormalizerManager(
+    @NonNull  final RegionNormalizerTracker regionNormalizerTracker,
+    @Nullable final RegionNormalizerChore regionNormalizerChore,
+    @Nullable final RegionNormalizerWorkQueue<TableName> workQueue,
+    @Nullable final RegionNormalizerWorker worker
+  ) {
+    this.regionNormalizerTracker = regionNormalizerTracker;
+    this.regionNormalizerChore = regionNormalizerChore;
+    this.workQueue = workQueue;
+    this.worker = worker;
+    this.pool = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+      .setDaemon(true)
+      .setNameFormat("normalizer-worker-%d")
+      .setUncaughtExceptionHandler(
+        (thread, throwable) ->
+          LOG.error("Uncaught exception, worker thread likely terminated.", throwable))
+      .build());
+  }
+
+  public void start() {
+    if (started) {
+      return;
+    }
+    synchronized (startStopLock) {
+      if (started) {
+        return;
+      }
+      regionNormalizerTracker.start();
+      if (worker != null) {
+        // worker will be null when master is in maintenance mode.
+        pool.submit(worker);
+      }
+      started = true;
+    }
+  }
+
+  public void stop() {
+    if (!started) {
+      throw new IllegalStateException("calling `stop` without first calling `start`.");
+    }
+    if (stopped) {

Review comment:
       I think it's possible to use Guava Service here, but it'll be a little messy because of encapsulation (or not) of the ZooKeeperTracker and because of the safe-mode master state. Let me leave this for a separate PR, perhaps and overhaul of everything in HMaster to be managed thusly.




----------------------------------------------------------------
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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2454: HBASE-24628 Region normalizer now respects a rate limit

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2454:
URL: https://github.com/apache/hbase/pull/2454#discussion_r499863287



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
##########
@@ -1953,20 +1952,27 @@ public SetNormalizerRunningResponse setNormalizerRunning(RpcController controlle
     rpcPreCheck("setNormalizerRunning");
 
     // Sets normalizer on/off flag in ZK.
-    boolean prevValue = master.getRegionNormalizerTracker().isNormalizerOn();
-    boolean newValue = request.getOn();
-    try {
-      master.getRegionNormalizerTracker().setNormalizerOn(newValue);
-    } catch (KeeperException ke) {
-      LOG.warn("Error flipping normalizer switch", ke);
-    }
+    // TODO: this method is totally broken in terms of atomicity of actions and values read.
+    //  1. The contract has this RPC returning the previous value. There isn't a ZKUtil method
+    //     that lets us retrieve the previous value as part of setting a new value, so we simply
+    //     perform a read before issuing the update. Thus we have a data race opportunity, between
+    //     when the `prevValue` is read and whatever is actually overwritten.
+    //  2. Down in `setNormalizerOn`, the call to `createAndWatch` inside of the catch clause can
+    //     itself fail in the event that the znode already exists. Thus, another data race, between
+    //     when the initial `setData` call is notified of the absence of the target znode and the
+    //     subsequent `createAndWatch`, with another client creating said node.
+    //  That said, there's supposed to be only one active master and thus there's supposed to be
+    //  only one process with the authority to modify the value.

Review comment:
       Thanks. Reading through this code surprised and disappointed me. Only after thinking through the macro-architecture did I realize it's probably safe anyway.




----------------------------------------------------------------
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