Apache9 commented on a change in pull request #3378:
URL: https://github.com/apache/hbase/pull/3378#discussion_r654932817



##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
##########
@@ -1966,6 +2021,34 @@ public boolean shouldPerformMajorCompaction() throws 
IOException {
     return Optional.of(compaction);
   }
 
+  private void preCompactionSelection(CompactionContext compaction,

Review comment:
       Just refactoring?

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncCompactionServerService.java
##########
@@ -0,0 +1,85 @@
+/**
+ * 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.client;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.CompactionProtos.CompactRequest;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.CompactionProtos.CompactResponse;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.CompactionProtos.CompactionService;
+
+
+/**
+ * A simple wrapper of the {@link CompactionService} for a compaction server, 
which returns a
+ * {@link CompletableFuture}. This is easier to use, as if you use the raw 
protobuf interface, you
+ * need to get the result from the {@link RpcCallback}, and if there is an 
exception, you need to
+ * get it from the {@link RpcController} passed in.
+ * <p/>
+ * Notice that there is no retry, and this is intentional. We have different 
retry for different
+ * usage for now, if later we want to unify them, we can move the retry logic 
into this class.
+ */
[email protected]
+public class AsyncCompactionServerService {
+
+  private final ServerName server;
+
+  private final AsyncClusterConnectionImpl conn;
+
+  AsyncCompactionServerService(ServerName server, AsyncClusterConnectionImpl 
conn) {
+    this.server = server;
+    this.conn = conn;
+  }
+
+  @FunctionalInterface
+  private interface RpcCall<RESP> {
+    void call(CompactionService.Interface stub, HBaseRpcController controller,
+        RpcCallback<RESP> done);
+  }
+
+  private <RESP> CompletableFuture<RESP> call(RpcCall<RESP> rpcCall) {

Review comment:
       And this one is a call to master? If we want to follow the pattern here, 
maybe we could also change the way on how to do regionServerReport...

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
##########
@@ -3762,6 +3766,40 @@ public ClearRegionBlockCacheResponse 
clearRegionBlockCache(RpcController control
     return 
builder.setStats(ProtobufUtil.toCacheEvictionStats(stats.build())).build();
   }
 
+
+  @Override
+  public CompleteCompactionResponse completeCompaction(RpcController 
controller,
+      CompleteCompactionRequest request) throws ServiceException {
+    RegionInfo regionInfo = ProtobufUtil.toRegionInfo(request.getRegionInfo());
+    ColumnFamilyDescriptor family = 
ProtobufUtil.toColumnFamilyDescriptor(request.getFamily());
+    LOG.debug("Region server receive complete compaction for region: {}, cf: 
{}",
+      regionInfo.getRegionNameAsString(), family.getNameAsString());
+    boolean success = false;
+    HRegion onlineRegion = 
regionServer.getOnlineRegion(regionInfo.getRegionName());
+    if (onlineRegion != null) {
+      HStore store = onlineRegion.getStore(family.getName());
+      if (store != null) {
+        if (store.getForceMajor()) {
+          store.setForceMajor(request.getNewForceMajor());
+        }
+        List<String> selectedFiles =
+            
request.getSelectedFilesList().stream().collect(Collectors.toList());
+        List<String> newFiles = 
request.getNewFilesList().stream().collect(Collectors.toList());
+        try {
+          success = store.completeCompaction(null, selectedFiles, null, 
newFiles);

Review comment:
       If we could write HFile directly into the data directory, here the 
completion will be easier. Just a note.

##########
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:
       So the logic is already implemented? I saw in the last catch block, we 
will return false, which is the same return value with compaction offload 
disabled.
   
   This will be a challenge of the CP design, as for the same table, the 
compaction could be run in a compaction server or the region server randomly... 
Anyway, not the problem for this patch, but we need to consider again.

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncCompactionServerService.java
##########
@@ -0,0 +1,85 @@
+/**
+ * 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.client;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.CompactionProtos.CompactRequest;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.CompactionProtos.CompactResponse;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.CompactionProtos.CompactionService;
+
+
+/**
+ * A simple wrapper of the {@link CompactionService} for a compaction server, 
which returns a
+ * {@link CompletableFuture}. This is easier to use, as if you use the raw 
protobuf interface, you
+ * need to get the result from the {@link RpcCallback}, and if there is an 
exception, you need to
+ * get it from the {@link RpcController} passed in.
+ * <p/>
+ * Notice that there is no retry, and this is intentional. We have different 
retry for different
+ * usage for now, if later we want to unify them, we can move the retry logic 
into this class.
+ */
[email protected]
+public class AsyncCompactionServerService {
+
+  private final ServerName server;
+
+  private final AsyncClusterConnectionImpl conn;
+
+  AsyncCompactionServerService(ServerName server, AsyncClusterConnectionImpl 
conn) {
+    this.server = server;
+    this.conn = conn;
+  }
+
+  @FunctionalInterface
+  private interface RpcCall<RESP> {
+    void call(CompactionService.Interface stub, HBaseRpcController controller,
+        RpcCallback<RESP> done);
+  }
+
+  private <RESP> CompletableFuture<RESP> call(RpcCall<RESP> rpcCall) {

Review comment:
       Oh, the one in ConnectionUtils is not suitable here as it requires 
ClientService.Interface. But we do have another version in 
AsyncRegionServerAdmin which has almost the same code. Should find a way to 
share the code in the future.




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


Reply via email to