Apache9 commented on a change in pull request #3378:
URL: https://github.com/apache/hbase/pull/3378#discussion_r650031294
##########
File path:
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
##########
@@ -101,6 +102,7 @@
private final ConcurrentMap<String, ClientService.Interface> rsStubs = new
ConcurrentHashMap<>();
private final ConcurrentMap<String, AdminService.Interface> adminSubs = new
ConcurrentHashMap<>();
+ private final ConcurrentMap<String, CompactionService.Interface>
CompactionSubs = new ConcurrentHashMap<>();
Review comment:
Is this the suitable place? I do not think we want to expose this to
client?
nit: the first letter should be lowercase, and it is Stubs, not Subs. The
adminSubs should be a typo...
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CompactionThreadManager.java
##########
@@ -0,0 +1,61 @@
+/**
+ *
+ * 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.io.IOException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
+import org.apache.hadoop.hbase.trace.TraceUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
[email protected]
+public class CompactionThreadManager {
+ private static Logger LOG =
LoggerFactory.getLogger(CompactionThreadManager.class);
+
+ // process name
+ private final Configuration conf;
+ private final ConcurrentMap<ServerName, AsyncRegionServerAdmin> rsAdmins =
new ConcurrentHashMap<>();
Review comment:
OK, now we will call back to region server to finish the compaction...
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/compactionserver/CompactionThreadManager.java
##########
@@ -0,0 +1,61 @@
+/**
+ *
+ * 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.io.IOException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin;
+import org.apache.hadoop.hbase.trace.TraceUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
[email protected]
+public class CompactionThreadManager {
+ private static Logger LOG =
LoggerFactory.getLogger(CompactionThreadManager.class);
+
+ // process name
+ private final Configuration conf;
+ private final ConcurrentMap<ServerName, AsyncRegionServerAdmin> rsAdmins =
new ConcurrentHashMap<>();
+ private final HCompactionServer server;
+
+ public CompactionThreadManager(final Configuration conf, HCompactionServer
server) {
+ TraceUtil.initTracer(conf);
+ this.conf = conf;
+ this.server = server;
+ }
+
+ private AsyncRegionServerAdmin getRsAdmin(final ServerName sn) throws
IOException {
+ AsyncRegionServerAdmin admin = this.rsAdmins.get(sn);
+ if (admin == null) {
+ LOG.debug("New RS admin connection to {}", sn);
+ admin = this.server.getAsyncClusterConnection().getRegionServerAdmin(sn);
+ this.rsAdmins.put(sn, admin);
+ }
+ return admin;
+ }
+
+ public void requestCompaction() {
Review comment:
Not implemented?
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnection.java
##########
@@ -41,6 +41,11 @@
*/
AsyncRegionServerAdmin getRegionServerAdmin(ServerName serverName);
+ /**
+ * Get the compaction service for the given compaction server.
+ */
+ AsyncCompactionServerCaller getCompactionServerCaller(ServerName serverName);
Review comment:
Why call it caller?
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
##########
@@ -3736,4 +3741,67 @@ public CompactedHFilesDischarger
getCompactedHFilesDischarger() {
public long getRetryPauseTime() {
return this.retryPauseTime;
}
+
+ @Override
+ public boolean isCompactionOffloadEnabled(){
+ return regionServerCompactionOffloadManager.isCompactionOffloadEnabled();
+ }
+
+ private synchronized void createCompactionManagerStub(boolean refresh) {
+ // Create Master Compaction service stub without refreshing the master
node from ZK,
+ // use cached data
+ if (cmsStub == null) {
+ cmsStub =
+ (CompactionService.BlockingInterface)
createMasterStub(CompactionService.class, refresh);
+ }
+ }
+
+ /**
+ * Send compaction request to compaction manager
+ * @return True if send request successfully, otherwise false
+ * @throws IOException If an error occurs
+ */
+ @Override
+ public boolean requestCompactRegion(RegionInfo regionInfo,
ColumnFamilyDescriptor cfd,
+ boolean major, int priority) {
+ if (!isCompactionOffloadEnabled()) {
+ return false;
+ }
+ if (cmsStub == null) {
+ createCompactionManagerStub(false);
Review comment:
OK, so we will call to master first. What is the purpose here? Better
throttle?
--
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:
[email protected]