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/04/26 13:34:43 UTC

[GitHub] [hbase] nyl3532016 opened a new pull request #3202: HBASE-25803 Add compaction offload switch

nyl3532016 opened a new pull request #3202:
URL: https://github.com/apache/hbase/pull/3202


   


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



[GitHub] [hbase] Apache-HBase commented on pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#issuecomment-829955853


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  8s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-25714 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 30s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 12s |  HBASE-25714 passed  |
   | +1 :green_heart: |  compile  |   3m 34s |  HBASE-25714 passed  |
   | +1 :green_heart: |  shadedjars  |   8m 26s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 17s |  HBASE-25714 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  3s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 30s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 30s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 28s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 15s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 48s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 49s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 19s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 216m 17s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |   5m 17s |  hbase-thrift in the patch passed.  |
   | +1 :green_heart: |  unit  |   8m 33s |  hbase-shell in the patch passed.  |
   |  |   | 275m 24s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/2/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3202 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 43d5c1b8dfff 4.15.0-128-generic #131-Ubuntu SMP Wed Dec 9 06:57:35 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-25714 / 4d9a360a78 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/2/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/2/testReport/ |
   | Max. process+thread count | 3087 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-client hbase-server hbase-thrift hbase-shell U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/2/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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



[GitHub] [hbase] Apache9 commented on a change in pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#discussion_r631623181



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchCompactionOffloadProcedure.java
##########
@@ -0,0 +1,171 @@
+/**
+ * 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.procedure;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.regionserver.CompactionOffloadSwitchStorage;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SwitchCompactionOffloadState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SwitchCompactionOffloadStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+/**
+ * The procedure to switch compaction offload
+ */
+@InterfaceAudience.Private
+public class SwitchCompactionOffloadProcedure

Review comment:
       The code is almost a clone of SwitchRpcThrottleProcedure, is it possible make a common parent class for these two classes?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/compaction/CompactionOffloadManager.java
##########
@@ -0,0 +1,134 @@
+/**
+ * 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.compaction;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerMetrics;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
+import org.apache.hadoop.hbase.master.procedure.SwitchCompactionOffloadProcedure;
+import org.apache.hadoop.hbase.regionserver.CompactionOffloadSwitchStorage;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.cache.Cache;
+import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCompactionOffloadEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCompactionOffloadEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchCompactionOffloadRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchCompactionOffloadResponse;
+
+@InterfaceAudience.Private
+public class CompactionOffloadManager {
+  private final MasterServices masterServices;
+  /** Map of registered servers to their current load */
+  private final Cache<ServerName, ServerMetrics> onlineServers;
+  private CompactionOffloadSwitchStorage compactionOffloadSwitchStorage;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CompactionOffloadManager.class.getName());
+
+  public CompactionOffloadManager(final MasterServices master) {
+    this.masterServices = master;
+    int compactionServerMsgInterval =
+        master.getConfiguration().getInt(HConstants.COMPACTION_SERVER_MSG_INTERVAL, 3 * 1000);
+    int compactionServerExpiredFactor =
+        master.getConfiguration().getInt("hbase.compaction.server.expired.factor", 2);
+    this.onlineServers = CacheBuilder.newBuilder().expireAfterWrite(
+      compactionServerMsgInterval * compactionServerExpiredFactor, TimeUnit.MILLISECONDS).build();
+    this.compactionOffloadSwitchStorage = new CompactionOffloadSwitchStorage(
+        masterServices.getZooKeeper(), masterServices.getConfiguration());
+  }
+
+  public void compactionServerReport(ServerName sn, ServerMetrics sl) {
+    this.onlineServers.put(sn, sl);
+  }
+
+  /**
+   * @return A copy of the internal list of online servers.
+   */
+  public List<ServerName> getOnlineServersList() {
+    return new ArrayList<>(this.onlineServers.asMap().keySet());
+  }
+
+  /**
+   * @return Read-only map of servers to serverinfo
+   */
+  public Map<ServerName, ServerMetrics> getOnlineServers() {
+    return Collections.unmodifiableMap(this.onlineServers.asMap());
+  }
+
+  /**
+   * May return "0.0.0" when server is not online
+   */
+  public String getVersion(ServerName serverName) {
+    ServerMetrics serverMetrics = onlineServers.asMap().get(serverName);
+    return serverMetrics != null ? serverMetrics.getVersion() : "0.0.0";
+  }
+
+  public int getInfoPort(ServerName serverName) {
+    ServerMetrics serverMetrics = onlineServers.asMap().get(serverName);
+    return serverMetrics != null ? serverMetrics.getInfoServerPort() : 0;
+  }
+
+  public IsCompactionOffloadEnabledResponse
+      isCompactionOffloadEnabled(IsCompactionOffloadEnabledRequest request) throws IOException {
+    masterServices.getMasterCoprocessorHost().preIsCompactionOffloadEnabled();
+    boolean enabled = compactionOffloadSwitchStorage.isCompactionOffloadEnabled();
+    IsCompactionOffloadEnabledResponse response = IsCompactionOffloadEnabledResponse.newBuilder()
+        .setCompactionOffloadEnabled(enabled).build();
+    masterServices.getMasterCoprocessorHost().postIsCompactionOffloadEnabled(enabled);
+    return response;
+  }
+
+  public SwitchCompactionOffloadResponse
+      switchCompactionOffload(SwitchCompactionOffloadRequest request) throws IOException {
+    boolean compactionOffloadEnabled = request.getCompactionOffloadEnabled();
+    masterServices.getMasterCoprocessorHost().preSwitchCompactionOffload(compactionOffloadEnabled);
+    boolean oldCompactionOffloadEnable =
+        compactionOffloadSwitchStorage.isCompactionOffloadEnabled();
+    if (compactionOffloadEnabled != oldCompactionOffloadEnable) {

Review comment:
       Should we add this check in the procedure implementation?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchCompactionOffloadProcedure.java
##########
@@ -0,0 +1,171 @@
+/**
+ * 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.procedure;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.regionserver.CompactionOffloadSwitchStorage;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SwitchCompactionOffloadState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SwitchCompactionOffloadStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+/**
+ * The procedure to switch compaction offload
+ */
+@InterfaceAudience.Private
+public class SwitchCompactionOffloadProcedure
+    extends StateMachineProcedure<MasterProcedureEnv, SwitchCompactionOffloadState>
+    implements ServerProcedureInterface {
+
+  private static Logger LOG = LoggerFactory.getLogger(SwitchCompactionOffloadProcedure.class);
+
+  private CompactionOffloadSwitchStorage compactionOffloadSwitchStorage;
+  private boolean CompactionOffloadEnabled;
+  private ProcedurePrepareLatch syncLatch;
+  private ServerName serverName;
+  private RetryCounter retryCounter;
+
+  public SwitchCompactionOffloadProcedure() {
+  }
+
+  public SwitchCompactionOffloadProcedure(
+      CompactionOffloadSwitchStorage compactionOffloadSwitchStorage,
+      boolean CompactionOffloadEnabled, ServerName serverName,
+      final ProcedurePrepareLatch syncLatch) {
+    this.compactionOffloadSwitchStorage = compactionOffloadSwitchStorage;
+    this.syncLatch = syncLatch;
+    this.CompactionOffloadEnabled = CompactionOffloadEnabled;
+    this.serverName = serverName;
+  }
+
+  @Override
+  protected Flow executeFromState(MasterProcedureEnv env, SwitchCompactionOffloadState state)
+      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    switch (state) {
+      case UPDATE_SWITCH_COMPACTION_OFFLOAD_STORAGE:
+        try {
+          switchThrottleState(env, CompactionOffloadEnabled);
+        } catch (IOException e) {
+          if (retryCounter == null) {
+            retryCounter = ProcedureUtil.createRetryCounter(env.getMasterConfiguration());
+          }
+          long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
+          LOG.warn("Failed to store compaction offload value {}, sleep {} secs and retry",
+            CompactionOffloadEnabled, backoff / 1000, e);
+          setTimeout(Math.toIntExact(backoff));
+          setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT);
+          skipPersistence();
+          throw new ProcedureSuspendedException();
+        }
+        setNextState(SwitchCompactionOffloadState.SWITCH_COMPACTION_OFFLOAD_ON_RS);
+        return Flow.HAS_MORE_STATE;
+      case SWITCH_COMPACTION_OFFLOAD_ON_RS:
+        SwitchCompactionOffloadRemoteProcedure[] subProcedures =
+            env.getMasterServices().getServerManager().getOnlineServersList().stream()
+                .map(sn -> new SwitchCompactionOffloadRemoteProcedure(sn, CompactionOffloadEnabled))
+                .toArray(SwitchCompactionOffloadRemoteProcedure[]::new);
+        addChildProcedure(subProcedures);
+        setNextState(SwitchCompactionOffloadState.POST_SWITCH_COMPACTION_OFFLOAD);
+        return Flow.HAS_MORE_STATE;
+      case POST_SWITCH_COMPACTION_OFFLOAD:
+        ProcedurePrepareLatch.releaseLatch(syncLatch, this);
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException("unhandled state=" + state);
+    }
+  }
+
+  @Override
+  protected void rollbackState(MasterProcedureEnv env, SwitchCompactionOffloadState state)

Review comment:
       Not implemented?




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



[GitHub] [hbase] Apache9 commented on a change in pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#discussion_r636074862



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchCompactionOffloadProcedure.java
##########
@@ -0,0 +1,171 @@
+/**
+ * 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.procedure;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.regionserver.CompactionOffloadSwitchStorage;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SwitchCompactionOffloadState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SwitchCompactionOffloadStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+/**
+ * The procedure to switch compaction offload
+ */
+@InterfaceAudience.Private
+public class SwitchCompactionOffloadProcedure

Review comment:
       But you need to keep the old SwitchRpcThrottleProcedure here for compatibility? And I'm fine with any approach that could reduce duplication.




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



[GitHub] [hbase] nyl3532016 commented on pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
nyl3532016 commented on pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#issuecomment-835211978


   @Apache9 @wchevreuil Design doc has been updated, I'll appreciate it if you could take a look. Thanks!


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



[GitHub] [hbase] Apache-HBase commented on pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#issuecomment-837872302


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 55s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ HBASE-25714 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 32s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 10s |  HBASE-25714 passed  |
   | +1 :green_heart: |  compile  |   7m 15s |  HBASE-25714 passed  |
   | +1 :green_heart: |  checkstyle  |   2m 49s |  HBASE-25714 passed  |
   | +1 :green_heart: |  spotbugs  |   8m 49s |  HBASE-25714 passed  |
   | -0 :warning: |  patch  |   2m  1s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  8s |  the patch passed  |
   | +1 :green_heart: |  compile  |   7m  5s |  the patch passed  |
   | +1 :green_heart: |  cc  |   7m  5s |  the patch passed  |
   | +1 :green_heart: |  javac  |   7m  5s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m  9s |  The patch passed checkstyle in hbase-protocol-shaded  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  The patch passed checkstyle in hbase-client  |
   | +1 :green_heart: |  checkstyle  |   1m 18s |  hbase-server: The patch generated 0 new + 316 unchanged - 1 fixed = 316 total (was 317)  |
   | +1 :green_heart: |  checkstyle  |   0m 42s |  The patch passed checkstyle in hbase-thrift  |
   | +1 :green_heart: |  checkstyle  |   0m 10s |  The patch passed checkstyle in hbase-shell  |
   | -0 :warning: |  rubocop  |   0m 20s |  The patch generated 3 new + 652 unchanged - 0 fixed = 655 total (was 652)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  20m  2s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m 55s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   9m 42s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 52s |  The patch does not generate ASF License warnings.  |
   |  |   |  82m 41s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/5/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3202 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool rubocop |
   | uname | Linux 590ad1ba89eb 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-25714 / 7f32d6cfc5 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/5/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | Max. process+thread count | 86 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server hbase-thrift hbase-shell U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/5/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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



[GitHub] [hbase] wchevreuil commented on a change in pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
wchevreuil commented on a change in pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#discussion_r621026222



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
##########
@@ -705,6 +705,17 @@ void majorCompact(TableName tableName, byte[] columnFamily, CompactType compactT
   Map<ServerName, Boolean> compactionSwitch(boolean switchState, List<String> serverNamesList)
       throws IOException;
 
+  /**
+   * Turn the compaction offload on or off. This state is ephemeral. The setting will be lost on
+   * restart.
+   * @param switchState Set to <code>true</code> to enable, <code>false</code> to disable.
+   * @param serverNamesList list of region servers.
+   * @return Previous compaction offload states for region servers
+   * @throws IOException if a remote or network exception occurs
+   */
+  Map<ServerName, Boolean> compactionOffloadSwitch(boolean switchState, List<String> serverNamesList)

Review comment:
       Why adding a separate API? We'll either have embedded or offload compaction configured to run, so the existing `compactionSwitch` command should transparently switch on/off whatever compaction execution strategy is in place.




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



[GitHub] [hbase] nyl3532016 commented on a change in pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
nyl3532016 commented on a change in pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#discussion_r621124076



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
##########
@@ -705,6 +705,17 @@ void majorCompact(TableName tableName, byte[] columnFamily, CompactType compactT
   Map<ServerName, Boolean> compactionSwitch(boolean switchState, List<String> serverNamesList)
       throws IOException;
 
+  /**
+   * Turn the compaction offload on or off. This state is ephemeral. The setting will be lost on
+   * restart.
+   * @param switchState Set to <code>true</code> to enable, <code>false</code> to disable.
+   * @param serverNamesList list of region servers.
+   * @return Previous compaction offload states for region servers
+   * @throws IOException if a remote or network exception occurs
+   */
+  Map<ServerName, Boolean> compactionOffloadSwitch(boolean switchState, List<String> serverNamesList)

Review comment:
       `compactionOffloadSwitch` can't change the enablement of compaction, It can change compaction execution strategy for specific regionservers conveniently. It's useful, isn't 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:
users@infra.apache.org



[GitHub] [hbase] nyl3532016 commented on a change in pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
nyl3532016 commented on a change in pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#discussion_r636017799



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchCompactionOffloadProcedure.java
##########
@@ -0,0 +1,171 @@
+/**
+ * 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.procedure;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.regionserver.CompactionOffloadSwitchStorage;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SwitchCompactionOffloadState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SwitchCompactionOffloadStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+/**
+ * The procedure to switch compaction offload
+ */
+@InterfaceAudience.Private
+public class SwitchCompactionOffloadProcedure

Review comment:
       To eliminate duplicate code. will introduce a procedure `doSwitchProcedure`, It can switch rpcThrottle态compactionOffload and so on, No class inherit this class ?




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



[GitHub] [hbase] nyl3532016 commented on pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
nyl3532016 commented on pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#issuecomment-838127385


   @Apache9 @wchevreuil Sir, now implement this switch according to design doc , mind taking a look?


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



[GitHub] [hbase] Apache-HBase commented on pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#issuecomment-837766619


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  8s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-25714 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  5s |  HBASE-25714 passed  |
   | +1 :green_heart: |  compile  |   3m  8s |  HBASE-25714 passed  |
   | +1 :green_heart: |  shadedjars  |   8m 24s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 57s |  HBASE-25714 passed  |
   | -0 :warning: |  patch  |  11m  6s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   2m  4s |  root in the patch failed.  |
   | -1 :x: |  compile  |   0m 42s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javac  |   0m 42s |  hbase-server in the patch failed.  |
   | -1 :x: |  shadedjars  |   6m 14s |  patch has 20 errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 55s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 48s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 20s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  |   0m 41s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |   5m  1s |  hbase-thrift in the patch passed.  |
   | -1 :x: |  unit  |   8m 20s |  hbase-shell in the patch failed.  |
   |  |   |  50m  9s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3202 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux f85014656f9d 4.15.0-128-generic #131-Ubuntu SMP Wed Dec 9 06:57:35 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-25714 / 7f32d6cfc5 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-jdk8-hadoop3-check/output/patch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-jdk8-hadoop3-check/output/patch-compile-hbase-server.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-jdk8-hadoop3-check/output/patch-compile-hbase-server.txt |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-jdk8-hadoop3-check/output/patch-shadedjars.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-shell.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/testReport/ |
   | Max. process+thread count | 1485 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server hbase-thrift hbase-shell U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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



[GitHub] [hbase] Apache-HBase commented on pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#issuecomment-837775180


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   6m 36s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ HBASE-25714 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 33s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 21s |  HBASE-25714 passed  |
   | +1 :green_heart: |  compile  |   7m  3s |  HBASE-25714 passed  |
   | +1 :green_heart: |  checkstyle  |   2m 48s |  HBASE-25714 passed  |
   | +1 :green_heart: |  spotbugs  |   8m 29s |  HBASE-25714 passed  |
   | -0 :warning: |  patch  |   2m  1s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   1m 58s |  root in the patch failed.  |
   | -1 :x: |  compile  |   0m 44s |  hbase-server in the patch failed.  |
   | -0 :warning: |  cc  |   0m 44s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javac  |   0m 44s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  checkstyle  |   0m  8s |  The patch passed checkstyle in hbase-protocol-shaded  |
   | +1 :green_heart: |  checkstyle  |   0m 25s |  The patch passed checkstyle in hbase-client  |
   | +1 :green_heart: |  checkstyle  |   1m 15s |  hbase-server: The patch generated 0 new + 316 unchanged - 1 fixed = 316 total (was 317)  |
   | +1 :green_heart: |  checkstyle  |   0m 40s |  The patch passed checkstyle in hbase-thrift  |
   | +1 :green_heart: |  checkstyle  |   0m  9s |  The patch passed checkstyle in hbase-shell  |
   | -0 :warning: |  rubocop  |   0m 20s |  The patch generated 3 new + 652 unchanged - 0 fixed = 655 total (was 652)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | -1 :x: |  hadoopcheck  |   2m  8s |  The patch causes 20 errors with Hadoop v3.1.2.  |
   | -1 :x: |  hadoopcheck  |   4m 19s |  The patch causes 20 errors with Hadoop v3.2.1.  |
   | -1 :x: |  hadoopcheck  |   6m 32s |  The patch causes 20 errors with Hadoop v3.3.0.  |
   | -1 :x: |  hbaseprotoc  |   0m 34s |  hbase-server in the patch failed.  |
   | -1 :x: |  spotbugs  |   0m 27s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 46s |  The patch does not generate ASF License warnings.  |
   |  |   |  59m 17s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3202 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool rubocop |
   | uname | Linux b10c4594217e 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-25714 / 7f32d6cfc5 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-general-check/output/patch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | cc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-general-check/output/patch-compile-hbase-server.txt |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | hadoopcheck | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-general-check/output/patch-javac-3.1.2.txt |
   | hadoopcheck | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-general-check/output/patch-javac-3.2.1.txt |
   | hadoopcheck | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-general-check/output/patch-javac-3.3.0.txt |
   | hbaseprotoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-general-check/output/patch-hbaseprotoc-hbase-server.txt |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-general-check/output/patch-spotbugs-hbase-server.txt |
   | Max. process+thread count | 86 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server hbase-thrift hbase-shell U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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



[GitHub] [hbase] Apache-HBase commented on pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#issuecomment-826913963


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   6m 52s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ HBASE-25714 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 20s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 19s |  HBASE-25714 passed  |
   | +1 :green_heart: |  compile  |   8m  0s |  HBASE-25714 passed  |
   | +1 :green_heart: |  checkstyle  |   3m  9s |  HBASE-25714 passed  |
   | +1 :green_heart: |  spotbugs  |   9m 16s |  HBASE-25714 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  4s |  the patch passed  |
   | +1 :green_heart: |  compile  |   7m 51s |  the patch passed  |
   | +1 :green_heart: |  cc  |   7m 51s |  the patch passed  |
   | +1 :green_heart: |  javac  |   7m 51s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 10s |  The patch passed checkstyle in hbase-protocol-shaded  |
   | +1 :green_heart: |  checkstyle  |   0m 24s |  The patch passed checkstyle in hbase-common  |
   | -0 :warning: |  checkstyle  |   0m 27s |  hbase-client: The patch generated 4 new + 38 unchanged - 0 fixed = 42 total (was 38)  |
   | +1 :green_heart: |  checkstyle  |   1m 15s |  hbase-server: The patch generated 0 new + 102 unchanged - 3 fixed = 102 total (was 105)  |
   | +1 :green_heart: |  checkstyle  |   0m 44s |  The patch passed checkstyle in hbase-thrift  |
   | +1 :green_heart: |  checkstyle  |   0m 11s |  The patch passed checkstyle in hbase-shell  |
   | -0 :warning: |  rubocop  |   0m 13s |  The patch generated 3 new + 377 unchanged - 0 fixed = 380 total (was 377)  |
   | -0 :warning: |  whitespace  |   0m  0s |  The patch has 1 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  hadoopcheck  |  20m  7s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   3m 24s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |  11m 35s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 12s |  The patch does not generate ASF License warnings.  |
   |  |   |  93m 41s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3202 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool rubocop |
   | uname | Linux fd057f4c561c 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-25714 / 51e17e1446 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-client.txt |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/1/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | whitespace | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/1/artifact/yetus-general-check/output/whitespace-eol.txt |
   | Max. process+thread count | 86 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-client hbase-server hbase-thrift hbase-shell U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/1/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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



[GitHub] [hbase] Apache9 commented on a change in pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#discussion_r625094672



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
##########
@@ -1684,6 +1684,9 @@
    */
   public static final int BATCH_ROWS_THRESHOLD_DEFAULT = 5000;
 
+  public final static String COMPACTION_OFFLOAD_ENABLED = "hbase.compaction.offload.enabled";

Review comment:
       Does this need to be in HConstants?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
##########
@@ -3152,11 +3155,31 @@ private void getProcedureResult(long procId, CompletableFuture<Void> future, int
     return future;
   }
 
-  @Override
-  public CompletableFuture<Map<ServerName, Boolean>> compactionSwitch(boolean switchState,
-      List<String> serverNamesList) {
+  private CompletableFuture<List<ServerName>> getMasterAndRegionServerList() {
+    CompletableFuture<List<ServerName>> future = new CompletableFuture<>();
+    addListener(getRegionServerList(new ArrayList<>()), (serverNames, err) -> {
+      if (err != null) {
+        future.completeExceptionally(err);
+        return;
+      }
+      List<ServerName> serverNameList = new ArrayList<>(serverNames);
+      addListener(getMaster(), (masterName, err1) -> {
+        if (err1 != null) {
+          future.completeExceptionally(err1);
+        } else {
+          serverNameList.add(masterName);
+          future.complete(serverNameList);
+        }
+      });
+    });
+    return future;
+  }
+
+  public CompletableFuture<Map<ServerName, Boolean>> doSwitchForServers(boolean switchState,

Review comment:
       Why public here?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
##########
@@ -542,6 +544,28 @@ boolean synchronousBalanceSwitch(final boolean b) throws IOException {
     return switchBalancer(b, BalanceSwitchMode.SYNC);
   }
 
+  @Override
+  public CompactionOffloadSwitchResponse compactionOffloadSwitch(RpcController controller,
+    CompactionOffloadSwitchRequest request) throws ServiceException {
+    // write compaction offload switch to zk
+    CompactionOffloadSwitchResponse.Builder response =
+      CompactionOffloadSwitchResponse.newBuilder();
+    try {
+      master.checkInitialized();
+      boolean newValue = request.getEnabled();
+      boolean oldValue = master.isSplitOrMergeEnabled(MasterSwitchType.COMPACTION_OFFLOAD);
+      response.setPrevState(oldValue);
+      LOG.info("Set compaction offload enabled to {}", newValue);
+      master.getSplitOrMergeTracker().setSplitOrMergeEnabled(newValue,

Review comment:
       Still called SplitOrMergeTracker?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
##########
@@ -282,6 +284,9 @@
 
   private HeapMemoryManager hMemManager;
 
+  private volatile boolean compactionOffloadEnabled = false;
+  private SplitOrMergeTracker switchTracker;

Review comment:
       We do not use this field at region server side in the past? Then I think we need to consider whether this is the correct approach.
   
   I checked the design doc, not too much information about this. Could you please explain the design more?
   
   Thanks.




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



[GitHub] [hbase] Apache-HBase commented on pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#issuecomment-836829376


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 36s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-25714 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 20s |  HBASE-25714 passed  |
   | +1 :green_heart: |  compile  |   3m 39s |  HBASE-25714 passed  |
   | +1 :green_heart: |  shadedjars  |   7m 43s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 25s |  HBASE-25714 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 13s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 42s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 42s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m 44s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 23s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 59s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 20s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 142m 16s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |   4m 35s |  hbase-thrift in the patch passed.  |
   | -1 :x: |  unit  |   7m 20s |  hbase-shell in the patch failed.  |
   |  |   | 197m  5s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/3/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3202 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 102f0b54e826 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-25714 / 7f32d6cfc5 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/3/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/3/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-shell.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/3/testReport/ |
   | Max. process+thread count | 4683 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server hbase-thrift hbase-shell U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/3/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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



[GitHub] [hbase] Apache-HBase commented on pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#issuecomment-827042935


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  9s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-25714 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 20s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 11s |  HBASE-25714 passed  |
   | +1 :green_heart: |  compile  |   3m 34s |  HBASE-25714 passed  |
   | +1 :green_heart: |  shadedjars  |   8m 25s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 17s |  HBASE-25714 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  2s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 32s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 32s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 34s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 15s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 47s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 49s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 18s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  | 213m 59s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |   5m 17s |  hbase-thrift in the patch passed.  |
   | -1 :x: |  unit  |   9m  3s |  hbase-shell in the patch failed.  |
   |  |   | 273m 33s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/1/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3202 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux bdd42bce9363 4.15.0-128-generic #131-Ubuntu SMP Wed Dec 9 06:57:35 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-25714 / 51e17e1446 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/1/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-shell.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/1/testReport/ |
   | Max. process+thread count | 3817 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-client hbase-server hbase-thrift hbase-shell U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/1/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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



[GitHub] [hbase] nyl3532016 commented on a change in pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
nyl3532016 commented on a change in pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#discussion_r630008352



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/compaction/CompactionOffloadManager.java
##########
@@ -0,0 +1,134 @@
+/**
+ * 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.compaction;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerMetrics;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
+import org.apache.hadoop.hbase.master.procedure.SwitchCompactionOffloadProcedure;
+import org.apache.hadoop.hbase.regionserver.CompactionOffloadSwitchStorage;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.cache.Cache;
+import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCompactionOffloadEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCompactionOffloadEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchCompactionOffloadRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchCompactionOffloadResponse;
+
+@InterfaceAudience.Private
+public class CompactionOffloadManager {

Review comment:
       Change class name from `CompactionServerManager` to `CompactionOffloadManager`. For now this class handle compactionServer and compaction offload switch logic




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



[GitHub] [hbase] Apache-HBase commented on pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#issuecomment-838112976


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  9s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-25714 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 58s |  HBASE-25714 passed  |
   | +1 :green_heart: |  compile  |   3m  8s |  HBASE-25714 passed  |
   | +1 :green_heart: |  shadedjars  |   8m 28s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 56s |  HBASE-25714 passed  |
   | -0 :warning: |  patch  |  11m  8s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  0s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m  8s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m  8s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 27s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 56s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 49s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 19s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  | 217m 41s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |   5m 18s |  hbase-thrift in the patch passed.  |
   | +1 :green_heart: |  unit  |   8m 30s |  hbase-shell in the patch passed.  |
   |  |   | 272m 48s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/5/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3202 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux a3c745aaf65c 4.15.0-128-generic #131-Ubuntu SMP Wed Dec 9 06:57:35 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-25714 / 7f32d6cfc5 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/5/testReport/ |
   | Max. process+thread count | 3443 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server hbase-thrift hbase-shell U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/5/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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



[GitHub] [hbase] Apache-HBase commented on pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#issuecomment-836696015


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 20s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ HBASE-25714 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  9s |  HBASE-25714 passed  |
   | +1 :green_heart: |  compile  |   7m  8s |  HBASE-25714 passed  |
   | +1 :green_heart: |  checkstyle  |   2m 45s |  HBASE-25714 passed  |
   | +1 :green_heart: |  spotbugs  |   8m 43s |  HBASE-25714 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  4s |  the patch passed  |
   | +1 :green_heart: |  compile  |   7m 15s |  the patch passed  |
   | +1 :green_heart: |  cc  |   7m 15s |  the patch passed  |
   | +1 :green_heart: |  javac  |   7m 15s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 16s |  hbase-server: The patch generated 7 new + 187 unchanged - 0 fixed = 194 total (was 187)  |
   | -0 :warning: |  checkstyle  |   0m 44s |  hbase-thrift: The patch generated 1 new + 1 unchanged - 0 fixed = 2 total (was 1)  |
   | -0 :warning: |  rubocop  |   0m 21s |  The patch generated 7 new + 652 unchanged - 0 fixed = 659 total (was 652)  |
   | +1 :green_heart: |  whitespace  |   0m  1s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  20m 15s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   3m  0s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   9m 51s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 52s |  The patch does not generate ASF License warnings.  |
   |  |   |  82m 33s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/3/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3202 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool rubocop |
   | uname | Linux 98dd05a5f449 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-25714 / 7f32d6cfc5 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/3/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/3/artifact/yetus-general-check/output/diff-checkstyle-hbase-thrift.txt |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/3/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | Max. process+thread count | 86 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server hbase-thrift hbase-shell U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/3/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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



[GitHub] [hbase] nyl3532016 commented on a change in pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
nyl3532016 commented on a change in pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#discussion_r639614126



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchCompactionOffloadProcedure.java
##########
@@ -0,0 +1,171 @@
+/**
+ * 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.procedure;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.regionserver.CompactionOffloadSwitchStorage;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SwitchCompactionOffloadState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SwitchCompactionOffloadStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+/**
+ * The procedure to switch compaction offload
+ */
+@InterfaceAudience.Private
+public class SwitchCompactionOffloadProcedure

Review comment:
       I think it is no need, It only effect procedure replay at rare situation. And we can skip the procedure via HBCK.
   If we want to keep compatibility, We must retain `SwitchRpcThrottleProcedure/SwitchRpcThrottleRemoteProcedure/SwitchRpcThrottleRemoteCallable/SwitchRpcThrottleStateData/SwitchRpcThrottleRemoteStateData` , we need to add too many class for switchCompacionOffload




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



[GitHub] [hbase] Apache9 commented on a change in pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#discussion_r636008706



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchCompactionOffloadProcedure.java
##########
@@ -0,0 +1,171 @@
+/**
+ * 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.procedure;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.regionserver.CompactionOffloadSwitchStorage;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SwitchCompactionOffloadState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SwitchCompactionOffloadStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+/**
+ * The procedure to switch compaction offload
+ */
+@InterfaceAudience.Private
+public class SwitchCompactionOffloadProcedure

Review comment:
       What do you mean by Combination? What I can see is that there are duplicate code...




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



[GitHub] [hbase] Apache-HBase commented on pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#issuecomment-829953793


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  2s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-25714 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 56s |  HBASE-25714 passed  |
   | +1 :green_heart: |  compile  |   4m 23s |  HBASE-25714 passed  |
   | +1 :green_heart: |  shadedjars  |   8m 34s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 54s |  HBASE-25714 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 44s |  the patch passed  |
   | +1 :green_heart: |  compile  |   4m 15s |  the patch passed  |
   | +1 :green_heart: |  javac  |   4m 15s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 26s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 54s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  1s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 11s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 33s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 209m 24s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |   5m  3s |  hbase-thrift in the patch passed.  |
   | +1 :green_heart: |  unit  |   7m  5s |  hbase-shell in the patch passed.  |
   |  |   | 271m 50s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/2/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3202 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux ea141dac78b7 4.15.0-136-generic #140-Ubuntu SMP Thu Jan 28 05:20:47 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-25714 / 4d9a360a78 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/2/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/2/testReport/ |
   | Max. process+thread count | 3526 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-client hbase-server hbase-thrift hbase-shell U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/2/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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



[GitHub] [hbase] nyl3532016 commented on a change in pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
nyl3532016 commented on a change in pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#discussion_r636013340



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/compaction/CompactionOffloadManager.java
##########
@@ -0,0 +1,134 @@
+/**
+ * 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.compaction;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerMetrics;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
+import org.apache.hadoop.hbase.master.procedure.SwitchCompactionOffloadProcedure;
+import org.apache.hadoop.hbase.regionserver.CompactionOffloadSwitchStorage;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.cache.Cache;
+import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCompactionOffloadEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCompactionOffloadEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchCompactionOffloadRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SwitchCompactionOffloadResponse;
+
+@InterfaceAudience.Private
+public class CompactionOffloadManager {
+  private final MasterServices masterServices;
+  /** Map of registered servers to their current load */
+  private final Cache<ServerName, ServerMetrics> onlineServers;
+  private CompactionOffloadSwitchStorage compactionOffloadSwitchStorage;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CompactionOffloadManager.class.getName());
+
+  public CompactionOffloadManager(final MasterServices master) {
+    this.masterServices = master;
+    int compactionServerMsgInterval =
+        master.getConfiguration().getInt(HConstants.COMPACTION_SERVER_MSG_INTERVAL, 3 * 1000);
+    int compactionServerExpiredFactor =
+        master.getConfiguration().getInt("hbase.compaction.server.expired.factor", 2);
+    this.onlineServers = CacheBuilder.newBuilder().expireAfterWrite(
+      compactionServerMsgInterval * compactionServerExpiredFactor, TimeUnit.MILLISECONDS).build();
+    this.compactionOffloadSwitchStorage = new CompactionOffloadSwitchStorage(
+        masterServices.getZooKeeper(), masterServices.getConfiguration());
+  }
+
+  public void compactionServerReport(ServerName sn, ServerMetrics sl) {
+    this.onlineServers.put(sn, sl);
+  }
+
+  /**
+   * @return A copy of the internal list of online servers.
+   */
+  public List<ServerName> getOnlineServersList() {
+    return new ArrayList<>(this.onlineServers.asMap().keySet());
+  }
+
+  /**
+   * @return Read-only map of servers to serverinfo
+   */
+  public Map<ServerName, ServerMetrics> getOnlineServers() {
+    return Collections.unmodifiableMap(this.onlineServers.asMap());
+  }
+
+  /**
+   * May return "0.0.0" when server is not online
+   */
+  public String getVersion(ServerName serverName) {
+    ServerMetrics serverMetrics = onlineServers.asMap().get(serverName);
+    return serverMetrics != null ? serverMetrics.getVersion() : "0.0.0";
+  }
+
+  public int getInfoPort(ServerName serverName) {
+    ServerMetrics serverMetrics = onlineServers.asMap().get(serverName);
+    return serverMetrics != null ? serverMetrics.getInfoServerPort() : 0;
+  }
+
+  public IsCompactionOffloadEnabledResponse
+      isCompactionOffloadEnabled(IsCompactionOffloadEnabledRequest request) throws IOException {
+    masterServices.getMasterCoprocessorHost().preIsCompactionOffloadEnabled();
+    boolean enabled = compactionOffloadSwitchStorage.isCompactionOffloadEnabled();
+    IsCompactionOffloadEnabledResponse response = IsCompactionOffloadEnabledResponse.newBuilder()
+        .setCompactionOffloadEnabled(enabled).build();
+    masterServices.getMasterCoprocessorHost().postIsCompactionOffloadEnabled(enabled);
+    return response;
+  }
+
+  public SwitchCompactionOffloadResponse
+      switchCompactionOffload(SwitchCompactionOffloadRequest request) throws IOException {
+    boolean compactionOffloadEnabled = request.getCompactionOffloadEnabled();
+    masterServices.getMasterCoprocessorHost().preSwitchCompactionOffload(compactionOffloadEnabled);
+    boolean oldCompactionOffloadEnable =
+        compactionOffloadSwitchStorage.isCompactionOffloadEnabled();
+    if (compactionOffloadEnabled != oldCompactionOffloadEnable) {

Review comment:
       Yes, double check to avoid duplicated procedure




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



[GitHub] [hbase] Apache-HBase commented on pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#issuecomment-829858981


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  8s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ HBASE-25714 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 13s |  HBASE-25714 passed  |
   | +1 :green_heart: |  compile  |   7m 52s |  HBASE-25714 passed  |
   | +1 :green_heart: |  checkstyle  |   3m  5s |  HBASE-25714 passed  |
   | +1 :green_heart: |  spotbugs  |   9m 30s |  HBASE-25714 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  1s |  the patch passed  |
   | +1 :green_heart: |  compile  |   7m 47s |  the patch passed  |
   | +1 :green_heart: |  cc  |   7m 47s |  the patch passed  |
   | +1 :green_heart: |  javac  |   7m 47s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m  9s |  The patch passed checkstyle in hbase-protocol-shaded  |
   | +1 :green_heart: |  checkstyle  |   0m 24s |  The patch passed checkstyle in hbase-common  |
   | +1 :green_heart: |  checkstyle  |   0m 29s |  The patch passed checkstyle in hbase-client  |
   | +1 :green_heart: |  checkstyle  |   1m 11s |  hbase-server: The patch generated 0 new + 102 unchanged - 3 fixed = 102 total (was 105)  |
   | +1 :green_heart: |  checkstyle  |   0m 45s |  The patch passed checkstyle in hbase-thrift  |
   | +1 :green_heart: |  checkstyle  |   0m 10s |  The patch passed checkstyle in hbase-shell  |
   | -0 :warning: |  rubocop  |   0m 13s |  The patch generated 2 new + 377 unchanged - 0 fixed = 379 total (was 377)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  20m  2s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   3m 20s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |  10m 14s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m  3s |  The patch does not generate ASF License warnings.  |
   |  |   |  85m 46s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3202 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile cc hbaseprotoc prototool rubocop |
   | uname | Linux 8906569b4674 4.15.0-136-generic #140-Ubuntu SMP Thu Jan 28 05:20:47 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-25714 / 4d9a360a78 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | rubocop | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/2/artifact/yetus-general-check/output/diff-patch-rubocop.txt |
   | Max. process+thread count | 86 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-client hbase-server hbase-thrift hbase-shell U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/2/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 rubocop=0.80.0 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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



[GitHub] [hbase] Apache-HBase commented on pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#issuecomment-837765986


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-25714 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 13s |  HBASE-25714 passed  |
   | +1 :green_heart: |  compile  |   3m 39s |  HBASE-25714 passed  |
   | +1 :green_heart: |  shadedjars  |   7m 54s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 22s |  HBASE-25714 passed  |
   | -0 :warning: |  patch  |  11m 11s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   2m 12s |  root in the patch failed.  |
   | -1 :x: |  compile  |   0m 45s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javac  |   0m 45s |  hbase-server in the patch failed.  |
   | -1 :x: |  shadedjars  |   5m 53s |  patch has 20 errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 26s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 59s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 21s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  |   0m 46s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |   4m 23s |  hbase-thrift in the patch passed.  |
   | -1 :x: |  unit  |   7m  2s |  hbase-shell in the patch failed.  |
   |  |   |  49m 29s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3202 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 69c12fd02cfa 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-25714 / 7f32d6cfc5 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-jdk11-hadoop3-check/output/patch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-server.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-server.txt |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-jdk11-hadoop3-check/output/patch-shadedjars.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-shell.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/testReport/ |
   | Max. process+thread count | 2387 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server hbase-thrift hbase-shell U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/4/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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



[GitHub] [hbase] nyl3532016 commented on pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
nyl3532016 commented on pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#issuecomment-831100801


   Test failed not related, fix `TestRollingRestart` in #3220 
   @wchevreuil @Apache9, mind have a look ?


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



[GitHub] [hbase] Apache-HBase commented on pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#issuecomment-838009173


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-25714 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 27s |  HBASE-25714 passed  |
   | +1 :green_heart: |  compile  |   3m 39s |  HBASE-25714 passed  |
   | +1 :green_heart: |  shadedjars  |   7m 44s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 28s |  HBASE-25714 passed  |
   | -0 :warning: |  patch  |  11m  7s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 14s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 40s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 40s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m 43s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 23s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  0s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 21s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  | 142m 20s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  unit  |   5m  3s |  hbase-thrift in the patch passed.  |
   | +1 :green_heart: |  unit  |   7m 16s |  hbase-shell in the patch passed.  |
   |  |   | 197m 12s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/5/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3202 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux e26cbc6d7f5f 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-25714 / 7f32d6cfc5 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/5/testReport/ |
   | Max. process+thread count | 4724 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server hbase-thrift hbase-shell U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/5/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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



[GitHub] [hbase] nyl3532016 commented on a change in pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
nyl3532016 commented on a change in pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#discussion_r621124076



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
##########
@@ -705,6 +705,17 @@ void majorCompact(TableName tableName, byte[] columnFamily, CompactType compactT
   Map<ServerName, Boolean> compactionSwitch(boolean switchState, List<String> serverNamesList)
       throws IOException;
 
+  /**
+   * Turn the compaction offload on or off. This state is ephemeral. The setting will be lost on
+   * restart.
+   * @param switchState Set to <code>true</code> to enable, <code>false</code> to disable.
+   * @param serverNamesList list of region servers.
+   * @return Previous compaction offload states for region servers
+   * @throws IOException if a remote or network exception occurs
+   */
+  Map<ServerName, Boolean> compactionOffloadSwitch(boolean switchState, List<String> serverNamesList)

Review comment:
       This api can change compaction execution strategy for specific regionservers conveniently. 
   It's useful, isn't 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:
users@infra.apache.org



[GitHub] [hbase] nyl3532016 merged pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
nyl3532016 merged pull request #3202:
URL: https://github.com/apache/hbase/pull/3202


   


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



[GitHub] [hbase] Apache9 commented on a change in pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#discussion_r640254840



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchCompactionOffloadProcedure.java
##########
@@ -0,0 +1,171 @@
+/**
+ * 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.procedure;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.regionserver.CompactionOffloadSwitchStorage;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SwitchCompactionOffloadState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SwitchCompactionOffloadStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+/**
+ * The procedure to switch compaction offload
+ */
+@InterfaceAudience.Private
+public class SwitchCompactionOffloadProcedure

Review comment:
       Requiring HBCK operation for updating is not a good idea usually...
   
   But let me see your approach first.




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



[GitHub] [hbase] Apache-HBase commented on pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#issuecomment-836932706


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  8s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-25714 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 31s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  3s |  HBASE-25714 passed  |
   | +1 :green_heart: |  compile  |   3m  9s |  HBASE-25714 passed  |
   | +1 :green_heart: |  shadedjars  |   8m 28s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 54s |  HBASE-25714 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 59s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m  8s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m  8s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 25s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 57s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 48s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 19s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 216m 36s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |   5m 20s |  hbase-thrift in the patch passed.  |
   | -1 :x: |  unit  |   8m 43s |  hbase-shell in the patch failed.  |
   |  |   | 272m 17s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/3/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3202 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux c511be2b8a77 4.15.0-128-generic #131-Ubuntu SMP Wed Dec 9 06:57:35 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-25714 / 7f32d6cfc5 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/3/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/3/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-shell.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/3/testReport/ |
   | Max. process+thread count | 3276 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server hbase-thrift hbase-shell U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/3/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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



[GitHub] [hbase] nyl3532016 commented on a change in pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
nyl3532016 commented on a change in pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#discussion_r636006494



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchCompactionOffloadProcedure.java
##########
@@ -0,0 +1,171 @@
+/**
+ * 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.procedure;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.regionserver.CompactionOffloadSwitchStorage;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SwitchCompactionOffloadState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SwitchCompactionOffloadStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+/**
+ * The procedure to switch compaction offload
+ */
+@InterfaceAudience.Private
+public class SwitchCompactionOffloadProcedure

Review comment:
       the switch of rpcThrottle and compactionOffload has same behavior. Therefore Combination is more proper than Inheritance?




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



[GitHub] [hbase] Apache-HBase commented on pull request #3202: HBASE-25803 Add compaction offload switch

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3202:
URL: https://github.com/apache/hbase/pull/3202#issuecomment-826998031


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 31s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ HBASE-25714 Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 26s |  HBASE-25714 passed  |
   | +1 :green_heart: |  compile  |   4m  9s |  HBASE-25714 passed  |
   | +1 :green_heart: |  shadedjars  |   7m 41s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 53s |  HBASE-25714 passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 15s |  the patch passed  |
   | +1 :green_heart: |  compile  |   4m  9s |  the patch passed  |
   | +1 :green_heart: |  javac  |   4m  9s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m 39s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 49s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  1s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  1s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 16s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 144m 31s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  unit  |   4m 39s |  hbase-thrift in the patch passed.  |
   | -1 :x: |  unit  |   7m  8s |  hbase-shell in the patch failed.  |
   |  |   | 203m  7s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/1/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3202 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux fc2beeb9e0e0 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | HBASE-25714 / 51e17e1446 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/1/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/1/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-shell.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/1/testReport/ |
   | Max. process+thread count | 3905 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-client hbase-server hbase-thrift hbase-shell U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3202/1/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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