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.quotas; 019 020import java.io.IOException; 021 022import org.apache.hadoop.conf.Configuration; 023import org.apache.hadoop.hbase.util.Bytes; 024import org.apache.hadoop.hbase.zookeeper.ZKUtil; 025import org.apache.hadoop.hbase.zookeeper.ZKWatcher; 026import org.apache.hadoop.hbase.zookeeper.ZNodePaths; 027import org.apache.yetus.audience.InterfaceAudience; 028import org.apache.zookeeper.KeeperException; 029 030/** 031 * ZK based rpc throttle storage. 032 */ 033@InterfaceAudience.Private 034public class RpcThrottleStorage { 035 public static final String RPC_THROTTLE_ZNODE = "zookeeper.znode.quota.rpc.throttle"; 036 public static final String RPC_THROTTLE_ZNODE_DEFAULT = "rpc-throttle"; 037 038 private final ZKWatcher zookeeper; 039 private final String rpcThrottleZNode; 040 041 public RpcThrottleStorage(ZKWatcher zookeeper, Configuration conf) { 042 this.zookeeper = zookeeper; 043 this.rpcThrottleZNode = ZNodePaths.joinZNode(zookeeper.getZNodePaths().baseZNode, 044 conf.get(RPC_THROTTLE_ZNODE, RPC_THROTTLE_ZNODE_DEFAULT)); 045 } 046 047 public boolean isRpcThrottleEnabled() throws IOException { 048 try { 049 byte[] upData = ZKUtil.getData(zookeeper, rpcThrottleZNode); 050 return upData == null || Bytes.toBoolean(upData); 051 } catch (KeeperException | InterruptedException e) { 052 throw new IOException("Failed to get rpc throttle", e); 053 } 054 } 055 056 /** 057 * Store the rpc throttle value. 058 * @param enable Set to <code>true</code> to enable, <code>false</code> to disable. 059 * @throws IOException if an unexpected io exception occurs 060 */ 061 public void switchRpcThrottle(boolean enable) throws IOException { 062 try { 063 byte[] upData = Bytes.toBytes(enable); 064 ZKUtil.createSetData(zookeeper, rpcThrottleZNode, upData); 065 } catch (KeeperException e) { 066 throw new IOException("Failed to store rpc throttle", e); 067 } 068 } 069}