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 2021/06/28 17:21:27 UTC

[GitHub] [hbase] saintstack commented on a change in pull request #3425: HBASE-25991 Do compaction on compaction server

saintstack commented on a change in pull request #3425:
URL: https://github.com/apache/hbase/pull/3425#discussion_r659933917



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CSRpcServices.java
##########
@@ -88,8 +93,18 @@ void start() {
   public CompactResponse requestCompaction(RpcController controller,
       CompactionProtos.CompactRequest request) {
     requestCount.increment();
+    ServerName rsServerName = ProtobufUtil.toServerName(request.getServer());
+    RegionInfo regionInfo = ProtobufUtil.toRegionInfo(request.getRegionInfo());
+    ColumnFamilyDescriptor cfd = ProtobufUtil.toColumnFamilyDescriptor(request.getFamily());
+    boolean major = request.getMajor();
+    int priority = request.getPriority();
+    List<HBaseProtos.ServerName> favoredNodes = Collections.singletonList(request.getServer());
     LOG.info("Receive compaction request from {}", ProtobufUtil.toString(request));
-    compactionServer.compactionThreadManager.requestCompaction();
+    CompactionTask compactionTask =
+        CompactionTask.newBuilder().setRsServerName(rsServerName).setRegionInfo(regionInfo)
+            .setColumnFamilyDescriptor(cfd).setRequestMajor(major).setPriority(priority)
+            .setFavoredNodes(favoredNodes).setSubmitTime(System.currentTimeMillis()).build();
+    compactionServer.compactionThreadManager.requestCompaction(compactionTask);

Review comment:
       So 'throttling' is done via the current compaction throttling mechanism it seems. Good. Maybe later Master would have an overall view on compactions and do its own macro-level throttling...
   
   

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactThreadControl.java
##########
@@ -0,0 +1,151 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControllerService;
+import org.apache.hadoop.hbase.util.StealJobQueue;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.RejectedExecutionHandler;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+
+@InterfaceAudience.Private
+public class CompactThreadControl {

Review comment:
       What does a CompactThreadControl do?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CompactionServerStorage.java
##########
@@ -0,0 +1,139 @@
+/**
+ * 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.compactionserver;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+/**
+ * since we do not maintain StoreFileManager in compaction server(can't refresh when flush). we use
+ * external storage(this class) to record compacting files, and initialize a new HStore in
+ * {@link CompactionThreadManager#selectCompaction} every time when request compaction

Review comment:
       Can you say more in here?
   
   This is how we interface w/ storage? Does the compaction classes use this or this is used by them?
   
   The CS server is doing file movement and not the Master or the RS? Do we want processes other than RS doing this sort of file movement?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CompactionTask.java
##########
@@ -0,0 +1,173 @@
+/**
+ * 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.compactionserver;
+
+import java.util.List;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.monitoring.MonitoredTask;
+import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+
+@InterfaceAudience.Private
+public final class CompactionTask {

Review comment:
       How does CompactionTask relate to CompactionRequest? Class comment would help. Thanks.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CompactionServerStorage.java
##########
@@ -0,0 +1,139 @@
+/**
+ * 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.compactionserver;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+/**
+ * since we do not maintain StoreFileManager in compaction server(can't refresh when flush). we use
+ * external storage(this class) to record compacting files, and initialize a new HStore in
+ * {@link CompactionThreadManager#selectCompaction} every time when request compaction
+ */
+class CompactionServerStorage {
+  private static Logger LOG = LoggerFactory.getLogger(CompactionServerStorage.class);
+  private final ConcurrentMap<String, ConcurrentMap<String, Set<String>>> selectedFiles =
+      new ConcurrentHashMap<>();
+  private final ConcurrentMap<String, ConcurrentMap<String, Set<String>>> compactedFiles =
+      new ConcurrentHashMap<>();
+  /**
+   * Mark files as completed, called after CS finished compaction and RS accepted the results of
+   * this compaction, these compacted files will be deleted by RS if no reader referenced to them.
+   */
+  boolean addCompactedFiles(RegionInfo regionInfo, ColumnFamilyDescriptor cfd,
+      List<String> compactedFiles) {
+    Set<String> compactedFileSet = getCompactedStoreFiles(regionInfo, cfd);
+    synchronized (compactedFileSet) {
+      compactedFileSet.addAll(compactedFiles);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Mark files as compacted, region: {}, cf, files: {}", regionInfo,
+          cfd.getNameAsString(), compactedFileSet);
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Mark files as selected, called after the files are selected and before the compaction is
+   * started. Avoid a file is selected twice in two compaction.
+   * @return True if these files don't be selected, false if these files are already selected.
+   */
+  boolean addSelectedFiles(RegionInfo regionInfo, ColumnFamilyDescriptor cfd,
+      List<String> selectedFiles) {
+    Set<String> selectedFileSet = getSelectedStoreFiles(regionInfo, cfd);
+    synchronized (selectedFileSet) {

Review comment:
       Is this synchronize any good? It is a local variable. Two concurrent requests could sync on two different local variables. The data member selectedFiles meantime could be manipulated in background?

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

Review comment:
       We have a o.a.h.h.compaction package. Should the o.a.h.h.cs be under it? Or in it? Would that make access to internals easier on you?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CSRpcServices.java
##########
@@ -19,11 +19,15 @@
 

Review comment:
       The CS could not be in its own module? It can only be in the hbase-server module? Or perhaps there is a plan to move it out of hbase-server at a later time?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/throttle/ThroughputControllerService.java
##########
@@ -0,0 +1,31 @@
+/**
+ * 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.regionserver.throttle;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ChoreService;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private

Review comment:
       Good that it is private.....

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/throttle/ThroughputControllerService.java
##########
@@ -0,0 +1,31 @@
+/**
+ * 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.regionserver.throttle;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ChoreService;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public interface ThroughputControllerService {
+  ChoreService getChoreService();
+  double getCompactionPressure();
+  @Deprecated
+  double getFlushPressure();

Review comment:
       Move the javadoc here?
   
   If this method is deprecated, do you need to add it here at all?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/throttle/ThroughputControllerService.java
##########
@@ -0,0 +1,31 @@
+/**
+ * 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.regionserver.throttle;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ChoreService;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public interface ThroughputControllerService {

Review comment:
       No harm in class comment -- this Interface is provider of methods the TC needs to run.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/throttle/ThroughputControllerService.java
##########
@@ -0,0 +1,31 @@
+/**
+ * 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.regionserver.throttle;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ChoreService;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public interface ThroughputControllerService {
+  ChoreService getChoreService();

Review comment:
       This does not belong in this interface? It is a general call not particular to TC.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/throttle/ThroughputControllerService.java
##########
@@ -0,0 +1,31 @@
+/**
+ * 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.regionserver.throttle;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ChoreService;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public interface ThroughputControllerService {
+  ChoreService getChoreService();
+  double getCompactionPressure();
+  @Deprecated
+  double getFlushPressure();
+  Configuration getConfiguration();

Review comment:
       Ditto. This does not belong in here. TC is too specialized to host this general call.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
##########
@@ -100,7 +100,7 @@
    * @param tableDir {@link Path} to where the table is being stored
    * @param regionInfo {@link RegionInfo} for region
    */
-  HRegionFileSystem(final Configuration conf, final FileSystem fs, final Path tableDir,
+  public HRegionFileSystem(final Configuration conf, final FileSystem fs, final Path tableDir,

Review comment:
       Ouch on making this public and HStore public.  The compaction server is a subset of the RegionServer? Should it be in the RegionServer package?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CompactionTask.java
##########
@@ -0,0 +1,173 @@
+/**
+ * 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.compactionserver;
+
+import java.util.List;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.monitoring.MonitoredTask;
+import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+
+@InterfaceAudience.Private
+public final class CompactionTask {
+  private ServerName rsServerName;
+  private RegionInfo regionInfo;

Review comment:
       This is the name of the server we're to return the complete compaction too? Even if the Region has moved location?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactThreadControl.java
##########
@@ -0,0 +1,151 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControllerService;
+import org.apache.hadoop.hbase.util.StealJobQueue;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.RejectedExecutionHandler;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+
+@InterfaceAudience.Private
+public class CompactThreadControl {

Review comment:
       It looks like you are collecting the throttling up into a dedicated class. Thats good. A bit of a class comment would help devs figure what this is about. THanks.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactThreadControl.java
##########
@@ -0,0 +1,151 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControllerService;
+import org.apache.hadoop.hbase.util.StealJobQueue;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.RejectedExecutionHandler;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+
+@InterfaceAudience.Private
+public class CompactThreadControl {

Review comment:
       Is it used by the CS too?




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

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

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