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


Reply via email to