001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.master.procedure; 019 020import java.io.IOException; 021import java.util.Optional; 022import org.apache.hadoop.hbase.ServerName; 023import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; 024import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher; 025import org.apache.hadoop.hbase.replication.regionserver.SwitchRpcThrottleRemoteCallable; 026import org.apache.yetus.audience.InterfaceAudience; 027import org.slf4j.Logger; 028import org.slf4j.LoggerFactory; 029 030import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 031import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SwitchRpcThrottleRemoteStateData; 032 033/** 034 * The procedure to switch rpc throttle on region server 035 */ 036@InterfaceAudience.Private 037public class SwitchRpcThrottleRemoteProcedure extends ServerRemoteProcedure 038 implements ServerProcedureInterface { 039 040 private static final Logger LOG = LoggerFactory.getLogger(SwitchRpcThrottleRemoteProcedure.class); 041 private boolean rpcThrottleEnabled; 042 043 public SwitchRpcThrottleRemoteProcedure() { 044 } 045 046 public SwitchRpcThrottleRemoteProcedure(ServerName serverName, boolean rpcThrottleEnabled) { 047 this.targetServer = serverName; 048 this.rpcThrottleEnabled = rpcThrottleEnabled; 049 } 050 051 @Override 052 protected void rollback(MasterProcedureEnv env) throws IOException, InterruptedException { 053 } 054 055 @Override 056 protected boolean abort(MasterProcedureEnv env) { 057 return false; 058 } 059 060 @Override 061 protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { 062 SwitchRpcThrottleRemoteStateData.newBuilder() 063 .setTargetServer(ProtobufUtil.toServerName(targetServer)) 064 .setRpcThrottleEnabled(rpcThrottleEnabled).build(); 065 } 066 067 @Override 068 protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { 069 SwitchRpcThrottleRemoteStateData data = 070 serializer.deserialize(SwitchRpcThrottleRemoteStateData.class); 071 targetServer = ProtobufUtil.toServerName(data.getTargetServer()); 072 rpcThrottleEnabled = data.getRpcThrottleEnabled(); 073 } 074 075 @Override 076 public Optional<RemoteProcedureDispatcher.RemoteOperation> 077 remoteCallBuild(MasterProcedureEnv masterProcedureEnv, ServerName remote) { 078 assert targetServer.equals(remote); 079 return Optional.of(new RSProcedureDispatcher.ServerOperation(this, getProcId(), 080 SwitchRpcThrottleRemoteCallable.class, 081 SwitchRpcThrottleRemoteStateData.newBuilder() 082 .setTargetServer(ProtobufUtil.toServerName(remote)) 083 .setRpcThrottleEnabled(rpcThrottleEnabled).build().toByteArray())); 084 } 085 086 @Override 087 public ServerName getServerName() { 088 return targetServer; 089 } 090 091 @Override 092 public boolean hasMetaTableRegion() { 093 return false; 094 } 095 096 @Override 097 public ServerOperationType getServerOperationType() { 098 return ServerOperationType.SWITCH_RPC_THROTTLE; 099 } 100 101 @Override 102 protected void complete(MasterProcedureEnv env, Throwable error) { 103 if (error != null) { 104 LOG.warn("Failed to switch rpc throttle to {} on server {}", rpcThrottleEnabled, targetServer, 105 error); 106 this.succ = false; 107 } else { 108 this.succ = true; 109 } 110 } 111 112 @Override 113 public void toStringClassDetails(StringBuilder sb) { 114 sb.append(getClass().getSimpleName()); 115 sb.append(" server="); 116 sb.append(targetServer); 117 sb.append(", rpcThrottleEnabled="); 118 sb.append(rpcThrottleEnabled); 119 } 120}