org.apache.hadoop.hbase.quotas.QuotaCache Maven / Gradle / Ivy
Go to download
Show more of this group Show more artifacts with this name
Show all versions of hbase-server Show documentation
Show all versions of hbase-server Show documentation
Server functionality for HBase
/**
* 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.quotas;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ScheduledChore;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.security.UserGroupInformation;
import com.google.common.annotations.VisibleForTesting;
/**
* Cache that keeps track of the quota settings for the users and tables that are interacting with
* it. To avoid blocking the operations if the requested quota is not in cache an "empty quota" will
* be returned and the request to fetch the quota information will be enqueued for the next refresh.
* TODO: At the moment the Cache has a Chore that will be triggered every 5min or on cache-miss
* events. Later the Quotas will be pushed using the notification system.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class QuotaCache implements Stoppable {
private static final Log LOG = LogFactory.getLog(QuotaCache.class);
public static final String REFRESH_CONF_KEY = "hbase.quota.refresh.period";
private static final int REFRESH_DEFAULT_PERIOD = 5 * 60000; // 5min
private static final int EVICT_PERIOD_FACTOR = 5; // N * REFRESH_DEFAULT_PERIOD
// for testing purpose only, enforce the cache to be always refreshed
private static boolean TEST_FORCE_REFRESH = false;
private final ConcurrentHashMap namespaceQuotaCache =
new ConcurrentHashMap();
private final ConcurrentHashMap tableQuotaCache =
new ConcurrentHashMap();
private final ConcurrentHashMap userQuotaCache =
new ConcurrentHashMap();
private final RegionServerServices rsServices;
private QuotaRefresherChore refreshChore;
private boolean stopped = true;
public QuotaCache(final RegionServerServices rsServices) {
this.rsServices = rsServices;
}
public void start() throws IOException {
stopped = false;
// TODO: This will be replaced once we have the notification bus ready.
Configuration conf = rsServices.getConfiguration();
int period = conf.getInt(REFRESH_CONF_KEY, REFRESH_DEFAULT_PERIOD);
refreshChore = new QuotaRefresherChore(period, this);
rsServices.getChoreService().scheduleChore(refreshChore);
}
@Override
public void stop(final String why) {
stopped = true;
}
@Override
public boolean isStopped() {
return stopped;
}
/**
* Returns the limiter associated to the specified user/table.
* @param ugi the user to limit
* @param table the table to limit
* @return the limiter associated to the specified user/table
*/
public QuotaLimiter getUserLimiter(final UserGroupInformation ugi, final TableName table) {
if (table.isSystemTable()) {
return NoopQuotaLimiter.get();
}
return getUserQuotaState(ugi).getTableLimiter(table);
}
/**
* Returns the QuotaState associated to the specified user.
* @param ugi the user
* @return the quota info associated to specified user
*/
public UserQuotaState getUserQuotaState(final UserGroupInformation ugi) {
String key = ugi.getShortUserName();
UserQuotaState quotaInfo = userQuotaCache.get(key);
if (quotaInfo == null) {
quotaInfo = new UserQuotaState();
if (userQuotaCache.putIfAbsent(key, quotaInfo) == null) {
triggerCacheRefresh();
}
}
return quotaInfo;
}
/**
* Returns the limiter associated to the specified table.
* @param table the table to limit
* @return the limiter associated to the specified table
*/
public QuotaLimiter getTableLimiter(final TableName table) {
return getQuotaState(this.tableQuotaCache, table).getGlobalLimiter();
}
/**
* Returns the limiter associated to the specified namespace.
* @param namespace the namespace to limit
* @return the limiter associated to the specified namespace
*/
public QuotaLimiter getNamespaceLimiter(final String namespace) {
return getQuotaState(this.namespaceQuotaCache, namespace).getGlobalLimiter();
}
/**
* Returns the QuotaState requested. If the quota info is not in cache an empty one will be
* returned and the quota request will be enqueued for the next cache refresh.
*/
private QuotaState
getQuotaState(final ConcurrentHashMap quotasMap, final K key) {
QuotaState quotaInfo = quotasMap.get(key);
if (quotaInfo == null) {
quotaInfo = new QuotaState();
if (quotasMap.putIfAbsent(key, quotaInfo) == null) {
triggerCacheRefresh();
}
}
return quotaInfo;
}
@VisibleForTesting
void triggerCacheRefresh() {
refreshChore.triggerNow();
}
@VisibleForTesting
long getLastUpdate() {
return refreshChore.lastUpdate;
}
@VisibleForTesting
Map getNamespaceQuotaCache() {
return namespaceQuotaCache;
}
@VisibleForTesting
Map getTableQuotaCache() {
return tableQuotaCache;
}
@VisibleForTesting
Map getUserQuotaCache() {
return userQuotaCache;
}
public static boolean isTEST_FORCE_REFRESH() {
return TEST_FORCE_REFRESH;
}
public static void setTEST_FORCE_REFRESH(boolean tEST_FORCE_REFRESH) {
TEST_FORCE_REFRESH = tEST_FORCE_REFRESH;
}
// TODO: Remove this once we have the notification bus
private class QuotaRefresherChore extends ScheduledChore {
private long lastUpdate = 0;
public QuotaRefresherChore(final int period, final Stoppable stoppable) {
super("QuotaRefresherChore", stoppable, period);
}
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "GC_UNRELATED_TYPES",
justification = "I do not understand why the complaints, it looks good to me -- FIX")
protected void chore() {
// Prefetch online tables/namespaces
for (TableName table : QuotaCache.this.rsServices.getOnlineTables()) {
if (table.isSystemTable()) continue;
if (!QuotaCache.this.tableQuotaCache.containsKey(table)) {
QuotaCache.this.tableQuotaCache.putIfAbsent(table, new QuotaState());
}
String ns = table.getNamespaceAsString();
if (!QuotaCache.this.namespaceQuotaCache.containsKey(ns)) {
QuotaCache.this.namespaceQuotaCache.putIfAbsent(ns, new QuotaState());
}
}
fetchNamespaceQuotaState();
fetchTableQuotaState();
fetchUserQuotaState();
lastUpdate = EnvironmentEdgeManager.currentTime();
}
private void fetchNamespaceQuotaState() {
fetch("namespace", QuotaCache.this.namespaceQuotaCache, new Fetcher() {
@Override
public Get makeGet(final Map.Entry entry) {
return QuotaUtil.makeGetForNamespaceQuotas(entry.getKey());
}
@Override
public Map fetchEntries(final List gets) throws IOException {
return QuotaUtil.fetchNamespaceQuotas(rsServices.getConnection(), gets);
}
});
}
private void fetchTableQuotaState() {
fetch("table", QuotaCache.this.tableQuotaCache, new Fetcher() {
@Override
public Get makeGet(final Map.Entry entry) {
return QuotaUtil.makeGetForTableQuotas(entry.getKey());
}
@Override
public Map fetchEntries(final List gets) throws IOException {
return QuotaUtil.fetchTableQuotas(rsServices.getConnection(), gets);
}
});
}
private void fetchUserQuotaState() {
final Set namespaces = QuotaCache.this.namespaceQuotaCache.keySet();
final Set tables = QuotaCache.this.tableQuotaCache.keySet();
fetch("user", QuotaCache.this.userQuotaCache, new Fetcher() {
@Override
public Get makeGet(final Map.Entry entry) {
return QuotaUtil.makeGetForUserQuotas(entry.getKey(), tables, namespaces);
}
@Override
public Map fetchEntries(final List gets) throws IOException {
return QuotaUtil.fetchUserQuotas(rsServices.getConnection(), gets);
}
});
}
private void fetch(final String type,
final ConcurrentHashMap quotasMap, final Fetcher fetcher) {
long now = EnvironmentEdgeManager.currentTime();
long refreshPeriod = getPeriod();
long evictPeriod = refreshPeriod * EVICT_PERIOD_FACTOR;
// Find the quota entries to update
List gets = new ArrayList();
List toRemove = new ArrayList();
for (Map.Entry entry : quotasMap.entrySet()) {
long lastUpdate = entry.getValue().getLastUpdate();
long lastQuery = entry.getValue().getLastQuery();
if (lastQuery > 0 && (now - lastQuery) >= evictPeriod) {
toRemove.add(entry.getKey());
} else if (isTEST_FORCE_REFRESH() || (now - lastUpdate) >= refreshPeriod) {
gets.add(fetcher.makeGet(entry));
}
}
for (final K key : toRemove) {
if (LOG.isTraceEnabled()) {
LOG.trace("evict " + type + " key=" + key);
}
quotasMap.remove(key);
}
// fetch and update the quota entries
if (!gets.isEmpty()) {
try {
for (Map.Entry entry : fetcher.fetchEntries(gets).entrySet()) {
V quotaInfo = quotasMap.putIfAbsent(entry.getKey(), entry.getValue());
if (quotaInfo != null) {
quotaInfo.update(entry.getValue());
}
if (LOG.isTraceEnabled()) {
LOG.trace("refresh " + type + " key=" + entry.getKey() + " quotas=" + quotaInfo);
}
}
} catch (IOException e) {
LOG.warn("Unable to read " + type + " from quota table", e);
}
}
}
}
static interface Fetcher {
Get makeGet(Map.Entry entry);
Map fetchEntries(List gets) throws IOException;
}
}