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


Reply via email to