[GitHub] [hbase] ndimiduk commented on a change in pull request #2490: HBASE-24419 Normalizer merge plans should consider more than 2 regions when possible

2020-10-21 Thread GitBox


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



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##
@@ -315,35 +316,60 @@ private boolean skipForMerge(final RegionStates 
regionStates, final RegionInfo r
* towards target average or target region count.
*/
   private List computeMergeNormalizationPlans(final 
NormalizeContext ctx) {
-if (ctx.getTableRegions().size() < minRegionCount) {
+if (isEmpty(ctx.getTableRegions()) || ctx.getTableRegions().size() < 
minRegionCount) {
   LOG.debug("Table {} has {} regions, required min number of regions for 
normalizer to run"
 + " is {}, not computing merge plans.", ctx.getTableName(), 
ctx.getTableRegions().size(),
 minRegionCount);
   return Collections.emptyList();
 }
 
-final double avgRegionSizeMb = ctx.getAverageRegionSizeMb();
+final long avgRegionSizeMb = (long) ctx.getAverageRegionSizeMb();
+if (avgRegionSizeMb < mergeMinRegionSizeMb) {
+  return Collections.emptyList();
+}
 LOG.debug("Computing normalization plan for table {}. average region size: 
{}, number of"
   + " regions: {}.", ctx.getTableName(), avgRegionSizeMb, 
ctx.getTableRegions().size());
 
-final List plans = new ArrayList<>();
-for (int candidateIdx = 0; candidateIdx < ctx.getTableRegions().size() - 
1; candidateIdx++) {
-  final RegionInfo current = ctx.getTableRegions().get(candidateIdx);
-  final RegionInfo next = ctx.getTableRegions().get(candidateIdx + 1);
-  if (skipForMerge(ctx.getRegionStates(), current)
-|| skipForMerge(ctx.getRegionStates(), next)) {
-continue;
+// this nested loop walks the table's region chain once, looking for 
contiguous sequences of
+// regions that meet the criteria for merge. The outer loop tracks the 
starting point of the
+// next sequence, the inner loop looks for the end of that sequence. A 
single sequence becomes
+// an instance of MergeNormalizationPlan.
+
+final List plans = new LinkedList<>();
+final List rangeMembers = new LinkedList<>();
+long sumRangeMembersSizeMb;
+int current = 0;
+for (int rangeStart = 0;
+ rangeStart < ctx.getTableRegions().size() - 1 && current < 
ctx.getTableRegions().size();) {
+  // walk the region chain looking for contiguous sequences of regions 
that can be merged.
+  rangeMembers.clear();
+  sumRangeMembersSizeMb = 0;
+  for (current = rangeStart; current < ctx.getTableRegions().size(); 
current++) {
+final RegionInfo regionInfo = ctx.getTableRegions().get(current);
+final long regionSizeMb = getRegionSizeMB(regionInfo);
+if (skipForMerge(ctx.getRegionStates(), regionInfo)) {
+  // this region cannot participate in a range. resume the outer loop.
+  rangeStart = Math.max(current, rangeStart + 1);

Review comment:
   This line of questions convinces me you really read the code. Thank you 
for the thoughtful review!





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:
us...@infra.apache.org




[GitHub] [hbase] ndimiduk commented on a change in pull request #2490: HBASE-24419 Normalizer merge plans should consider more than 2 regions when possible

2020-10-09 Thread GitBox


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



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##
@@ -315,35 +317,61 @@ private boolean skipForMerge(final RegionStates 
regionStates, final RegionInfo r
* towards target average or target region count.
*/
   private List computeMergeNormalizationPlans(final 
NormalizeContext ctx) {
-if (ctx.getTableRegions().size() < minRegionCount) {
+if (isEmpty(ctx.getTableRegions()) || ctx.getTableRegions().size() < 
minRegionCount) {
   LOG.debug("Table {} has {} regions, required min number of regions for 
normalizer to run"
 + " is {}, not computing merge plans.", ctx.getTableName(), 
ctx.getTableRegions().size(),
 minRegionCount);
   return Collections.emptyList();
 }
 
-final double avgRegionSizeMb = ctx.getAverageRegionSizeMb();
+final long avgRegionSizeMb = (long) ctx.getAverageRegionSizeMb();
+if (avgRegionSizeMb < mergeMinRegionSizeMb) {
+  return Collections.emptyList();
+}
 LOG.debug("Computing normalization plan for table {}. average region size: 
{}, number of"
   + " regions: {}.", ctx.getTableName(), avgRegionSizeMb, 
ctx.getTableRegions().size());
 
-final List plans = new ArrayList<>();
-for (int candidateIdx = 0; candidateIdx < ctx.getTableRegions().size() - 
1; candidateIdx++) {
-  final RegionInfo current = ctx.getTableRegions().get(candidateIdx);
-  final RegionInfo next = ctx.getTableRegions().get(candidateIdx + 1);
-  if (skipForMerge(ctx.getRegionStates(), current)
-|| skipForMerge(ctx.getRegionStates(), next)) {
-continue;
+// this nested loop walks the table's region chain once, looking for 
contiguous sequences of
+// regions that meet the criteria for merge. The outer loop tracks the 
starting point of the
+// next sequence, the inner loop looks for the end of that sequence. A 
single sequence becomes
+// an instance of MergeNormalizationPlan.
+
+final List plans = new LinkedList<>();
+final List rangeMembers = new LinkedList<>();
+long sumRangeMembersSizeMb;
+int current = 0;
+for (int rangeStart = 0;

Review comment:
   The meat of this patch is this new nested loop.

##
File path: 
hbase-common/src/test/java/org/apache/hadoop/hbase/MatcherPredicate.java
##
@@ -0,0 +1,65 @@
+/*
+ * 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;
+
+import java.util.function.Supplier;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
+import org.hamcrest.StringDescription;
+
+/**
+ * An implementation of {@link Waiter.ExplainingPredicate} that uses Hamcrest 
{@link Matcher} for both

Review comment:
   I realized all my waiting in tests (1) use conditions that are shipped 
in the hamcrest library (2) have failure messages that look like failed 
`Matcher` explanations.

##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
##
@@ -124,10 +126,10 @@ private static Period parseMergeMinRegionAge(final 
Configuration conf) {
 return Period.ofDays(settledValue);
   }
 
-  private static int parseMergeMinRegionSizeMb(final Configuration conf) {
-final int parsedValue =
-  conf.getInt(MERGE_MIN_REGION_SIZE_MB_KEY, 
DEFAULT_MERGE_MIN_REGION_SIZE_MB);
-final int settledValue = Math.max(0, parsedValue);
+  private static long parseMergeMinRegionSizeMb(final Configuration conf) {

Review comment:
   Upgraded this datatype to a `long` because this is the type returned by 
the region size info we get from AM.





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:
us...@infra.apache.org




[GitHub] [hbase] ndimiduk commented on a change in pull request #2490: HBASE-24419 Normalizer merge plans should consider more than 2 regions when possible

2020-10-05 Thread GitBox


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



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerManager.java
##
@@ -0,0 +1,153 @@
+/*
+ * 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 workQueue;
+  private final RegionNormalizerWorker worker;
+  private final ExecutorService pool;
+
+  public RegionNormalizerManager(
+@NonNull  final RegionNormalizerTracker regionNormalizerTracker,
+@Nullable final RegionNormalizerChore regionNormalizerChore,
+@Nullable final RegionNormalizerWorkQueue 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() {
+regionNormalizerTracker.start();

Review comment:
   Now guarding this on a "`startStopLock`".

##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerManager.java
##
@@ -0,0 +1,153 @@
+/*
+ * 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.get

[GitHub] [hbase] ndimiduk commented on a change in pull request #2490: HBASE-24419 Normalizer merge plans should consider more than 2 regions when possible

2020-10-05 Thread GitBox


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



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerManager.java
##
@@ -0,0 +1,153 @@
+/*
+ * 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;

Review comment:
   These classes are on our classpath and come from the clean-room 
implementation of FindBugs, https://github.com/stephenc/findbugs-annotations, 
via the com.github.stephenc.findbugs:findbugs-annotations:1.3.9-1 dependency. 
We're okay here.





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:
us...@infra.apache.org




[GitHub] [hbase] ndimiduk commented on a change in pull request #2490: HBASE-24419 Normalizer merge plans should consider more than 2 regions when possible

2020-10-05 Thread GitBox


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



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/RegionNormalizerManager.java
##
@@ -0,0 +1,153 @@
+/*
+ * 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;

Review comment:
   It's on our class path... if it's LGPL, why is it there?





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:
us...@infra.apache.org




[GitHub] [hbase] ndimiduk commented on a change in pull request #2490: HBASE-24419 Normalizer merge plans should consider more than 2 regions when possible

2020-10-05 Thread GitBox


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



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
##
@@ -570,8 +569,10 @@ private void preSplitRegion(final MasterProcedureEnv env)
 try {
   
env.getMasterServices().getMasterQuotaManager().onRegionSplit(this.getParentRegion());
 } catch (QuotaExceededException e) {
-  
env.getMasterServices().getRegionNormalizer().planSkipped(this.getParentRegion(),
-  NormalizationPlan.PlanType.SPLIT);
+  // TODO: why is this here? merge requests can be submitted by actors 
other than the normalizer

Review comment:
   Sorry, this is a copy-paste error in the comment.
   
   I added it because these callbacks seem to me to be only partially 
implemented.





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:
us...@infra.apache.org




[GitHub] [hbase] ndimiduk commented on a change in pull request #2490: HBASE-24419 Normalizer merge plans should consider more than 2 regions when possible

2020-10-05 Thread GitBox


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



##
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.
+final boolean prevValue = 
master.getRegionNormalizerManager().isNormalizerOn();
+final boolean newValue = request.getOn();
+master.getRegionNormalizerManager().setNormalizerOn(newValue);
 LOG.info("{} set normalizerSwitch={}", master.getClientIdAuditPrefix(), 
newValue);
 return 
SetNormalizerRunningResponse.newBuilder().setPrevNormalizerValue(prevValue).build();
   }
 
   @Override
   public IsNormalizerEnabledResponse isNormalizerEnabled(RpcController 
controller,

Review comment:
   Yes there are a number of changed signatures on methods and classes, all 
`IA.Private`.





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:
us...@infra.apache.org




[GitHub] [hbase] ndimiduk commented on a change in pull request #2490: HBASE-24419 Normalizer merge plans should consider more than 2 regions when possible

2020-10-05 Thread GitBox


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



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
##
@@ -3003,18 +2933,20 @@ public double getAverageLoad() {
 return regionStates.getAverageLoad();
   }
 
-  /*
-   * @return the count of region split plans executed
+  /**
+   * Exposed here for metrics.
+   * @see RegionNormalizerManager#getSplitPlanCount()
*/
   public long getSplitPlanCount() {
-return splitPlanCount;
+return regionNormalizerManager.getSplitPlanCount();
   }
 
-  /*
-   * @return the count of region merge plans executed
+  /**
+   * Exposed here for metrics.
+   * @see RegionNormalizerManager#getMergePlanCount()
*/
   public long getMergePlanCount() {
-return mergePlanCount;
+return regionNormalizerManager.getMergePlanCount();

Review comment:
   Yeah these are exposed as metrics, all of which hang off of the HMaster 
instance. I think it's a larger refactor (and probably backward incompatible) 
to move these various metrics down a level in the metric name dotted-path.





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:
us...@infra.apache.org