HBASE-13205 [branch-1] Backport HBASE-11598 Add simple rpc throttling (Ashish Singhi)

This commit is contained in:
tedyu 2015-04-07 18:29:03 -07:00
parent bbdd50b9c5
commit c031d8de23
70 changed files with 12982 additions and 242 deletions

View File

@ -41,6 +41,9 @@ import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
import org.apache.hadoop.hbase.quotas.QuotaFilter;
import org.apache.hadoop.hbase.quotas.QuotaRetriever;
import org.apache.hadoop.hbase.quotas.QuotaSettings;
import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
@ -1281,6 +1284,21 @@ public interface Admin extends Abortable, Closeable {
* @throws IOException if a remote or network exception occurs
*/
void deleteSnapshots(final Pattern pattern) throws IOException;
/**
* Apply the new quota settings.
* @param quota the quota settings
* @throws IOException if a remote or network exception occurs
*/
void setQuota(final QuotaSettings quota) throws IOException;
/**
* Return a QuotaRetriever to list the quotas based on the filter.
* @param filter the quota settings filter
* @return the quota retriever
* @throws IOException if a remote or network exception occurs
*/
QuotaRetriever getQuotaRetriever(final QuotaFilter filter) throws IOException;
/**
* Creates and returns a {@link com.google.protobuf.RpcChannel} instance connected to the active

View File

@ -18,9 +18,6 @@
*/
package org.apache.hadoop.hbase.client;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.io.Closeable;
import java.io.IOException;
import java.io.InterruptedIOException;
@ -45,11 +42,6 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.BlockingRpcChannel;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@ -164,6 +156,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanReq
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
@ -1996,6 +1990,12 @@ class ConnectionManager {
throws ServiceException {
return stub.getClusterStatus(controller, request);
}
@Override
public SetQuotaResponse setQuota(RpcController controller, SetQuotaRequest request)
throws ServiceException {
return stub.setQuota(controller, request);
}
@Override
public MajorCompactionTimestampResponse getLastMajorCompactionTimestamp(

View File

@ -131,6 +131,9 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
import org.apache.hadoop.hbase.quotas.QuotaFilter;
import org.apache.hadoop.hbase.quotas.QuotaRetriever;
import org.apache.hadoop.hbase.quotas.QuotaSettings;
import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
@ -3633,6 +3636,33 @@ public class HBaseAdmin implements Admin {
}
});
}
/**
* Apply the new quota settings.
* @param quota the quota settings
* @throws IOException if a remote or network exception occurs
*/
@Override
public void setQuota(final QuotaSettings quota) throws IOException {
executeCallable(new MasterCallable<Void>(getConnection()) {
@Override
public Void call(int callTimeout) throws ServiceException {
this.master.setQuota(null, QuotaSettings.buildSetQuotaRequestProto(quota));
return null;
}
});
}
/**
* Return a Quota Scanner to list the quotas based on the filter.
* @param filter the quota settings filter
* @return the quota scanner
* @throws IOException if a remote or network exception occurs
*/
@Override
public QuotaRetriever getQuotaRetriever(final QuotaFilter filter) throws IOException {
return QuotaRetriever.open(conf, filter);
}
private <V> V executeCallable(MasterCallable<V> callable) throws IOException {
RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller();

View File

@ -35,6 +35,7 @@ import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.NavigableSet;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
@ -117,6 +118,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
@ -127,6 +129,9 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescripto
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
import org.apache.hadoop.hbase.quotas.QuotaScope;
import org.apache.hadoop.hbase.quotas.QuotaType;
import org.apache.hadoop.hbase.quotas.ThrottleType;
import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
import org.apache.hadoop.hbase.security.access.Permission;
@ -2859,6 +2864,163 @@ public final class ProtobufUtil {
}
return result;
}
/**
* Convert a protocol buffer TimeUnit to a client TimeUnit
* @param proto
* @return the converted client TimeUnit
*/
public static TimeUnit toTimeUnit(final HBaseProtos.TimeUnit proto) {
switch (proto) {
case NANOSECONDS:
return TimeUnit.NANOSECONDS;
case MICROSECONDS:
return TimeUnit.MICROSECONDS;
case MILLISECONDS:
return TimeUnit.MILLISECONDS;
case SECONDS:
return TimeUnit.SECONDS;
case MINUTES:
return TimeUnit.MINUTES;
case HOURS:
return TimeUnit.HOURS;
case DAYS:
return TimeUnit.DAYS;
default:
throw new RuntimeException("Invalid TimeUnit " + proto);
}
}
/**
* Convert a client TimeUnit to a protocol buffer TimeUnit
* @param timeUnit
* @return the converted protocol buffer TimeUnit
*/
public static HBaseProtos.TimeUnit toProtoTimeUnit(final TimeUnit timeUnit) {
switch (timeUnit) {
case NANOSECONDS:
return HBaseProtos.TimeUnit.NANOSECONDS;
case MICROSECONDS:
return HBaseProtos.TimeUnit.MICROSECONDS;
case MILLISECONDS:
return HBaseProtos.TimeUnit.MILLISECONDS;
case SECONDS:
return HBaseProtos.TimeUnit.SECONDS;
case MINUTES:
return HBaseProtos.TimeUnit.MINUTES;
case HOURS:
return HBaseProtos.TimeUnit.HOURS;
case DAYS:
return HBaseProtos.TimeUnit.DAYS;
default:
throw new RuntimeException("Invalid TimeUnit " + timeUnit);
}
}
/**
* Convert a protocol buffer ThrottleType to a client ThrottleType
* @param proto
* @return the converted client ThrottleType
*/
public static ThrottleType toThrottleType(final QuotaProtos.ThrottleType proto) {
switch (proto) {
case REQUEST_NUMBER:
return ThrottleType.REQUEST_NUMBER;
case REQUEST_SIZE:
return ThrottleType.REQUEST_SIZE;
default:
throw new RuntimeException("Invalid ThrottleType " + proto);
}
}
/**
* Convert a client ThrottleType to a protocol buffer ThrottleType
* @param type
* @return the converted protocol buffer ThrottleType
*/
public static QuotaProtos.ThrottleType toProtoThrottleType(final ThrottleType type) {
switch (type) {
case REQUEST_NUMBER:
return QuotaProtos.ThrottleType.REQUEST_NUMBER;
case REQUEST_SIZE:
return QuotaProtos.ThrottleType.REQUEST_SIZE;
default:
throw new RuntimeException("Invalid ThrottleType " + type);
}
}
/**
* Convert a protocol buffer QuotaScope to a client QuotaScope
* @param proto
* @return the converted client QuotaScope
*/
public static QuotaScope toQuotaScope(final QuotaProtos.QuotaScope proto) {
switch (proto) {
case CLUSTER:
return QuotaScope.CLUSTER;
case MACHINE:
return QuotaScope.MACHINE;
default:
throw new RuntimeException("Invalid QuotaScope " + proto);
}
}
/**
* Convert a client QuotaScope to a protocol buffer QuotaScope
* @param scope
* @return the converted protocol buffer QuotaScope
*/
public static QuotaProtos.QuotaScope toProtoQuotaScope(final QuotaScope scope) {
switch (scope) {
case CLUSTER:
return QuotaProtos.QuotaScope.CLUSTER;
case MACHINE:
return QuotaProtos.QuotaScope.MACHINE;
default:
throw new RuntimeException("Invalid QuotaScope " + scope);
}
}
/**
* Convert a protocol buffer QuotaType to a client QuotaType
* @param proto
* @return the converted client QuotaType
*/
public static QuotaType toQuotaScope(final QuotaProtos.QuotaType proto) {
switch (proto) {
case THROTTLE:
return QuotaType.THROTTLE;
default:
throw new RuntimeException("Invalid QuotaType " + proto);
}
}
/**
* Convert a client QuotaType to a protocol buffer QuotaType
* @param type
* @return the converted protocol buffer QuotaType
*/
public static QuotaProtos.QuotaType toProtoQuotaScope(final QuotaType type) {
switch (type) {
case THROTTLE:
return QuotaProtos.QuotaType.THROTTLE;
default:
throw new RuntimeException("Invalid QuotaType " + type);
}
}
/**
* Build a protocol buffer TimedQuota
* @param limit the allowed number of request/data per timeUnit
* @param timeUnit the limit time unit
* @param scope the quota scope
* @return the protocol buffer TimedQuota
*/
public static QuotaProtos.TimedQuota toTimedQuota(final long limit, final TimeUnit timeUnit,
final QuotaScope scope) {
return QuotaProtos.TimedQuota.newBuilder().setSoftLimit(limit)
.setTimeUnit(toProtoTimeUnit(timeUnit)).setScope(toProtoQuotaScope(scope)).build();
}
/**
* Generates a marker for the WAL so that we propagate the notion of a bulk region load

View File

@ -0,0 +1,25 @@
/**
* 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 org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* Generic quota exceeded exception for invalid settings
*/
@InterfaceAudience.Private
public class InvalidQuotaSettingsException extends DoNotRetryIOException {
public InvalidQuotaSettingsException(String msg) {
super(msg);
}
}

View File

@ -0,0 +1,27 @@
/**
* 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 org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* Generic quota exceeded exception
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class QuotaExceededException extends DoNotRetryIOException {
public QuotaExceededException(String msg) {
super(msg);
}
}

View File

@ -0,0 +1,103 @@
/**
* 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.util.HashSet;
import java.util.Set;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.util.Strings;
/**
* Filter to use to filter the QuotaRetriever results.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class QuotaFilter {
private Set<QuotaType> types = new HashSet<QuotaType>();
private boolean hasFilters = false;
private String namespaceRegex;
private String tableRegex;
private String userRegex;
public QuotaFilter() {
}
/**
* Set the user filter regex
* @param regex the user filter
* @return the quota filter object
*/
public QuotaFilter setUserFilter(final String regex) {
this.userRegex = regex;
hasFilters |= !Strings.isEmpty(regex);
return this;
}
/**
* Set the table filter regex
* @param regex the table filter
* @return the quota filter object
*/
public QuotaFilter setTableFilter(final String regex) {
this.tableRegex = regex;
hasFilters |= !Strings.isEmpty(regex);
return this;
}
/**
* Set the namespace filter regex
* @param regex the namespace filter
* @return the quota filter object
*/
public QuotaFilter setNamespaceFilter(final String regex) {
this.namespaceRegex = regex;
hasFilters |= !Strings.isEmpty(regex);
return this;
}
/**
* Add a type to the filter list
* @param type the type to filter on
* @return the quota filter object
*/
public QuotaFilter addTypeFilter(final QuotaType type) {
this.types.add(type);
hasFilters |= true;
return this;
}
/** @return true if the filter is empty */
public boolean isNull() {
return !hasFilters;
}
/** @return the QuotaType types that we want to filter one */
public Set<QuotaType> getTypeFilters() {
return types;
}
/** @return the Namespace filter regex */
public String getNamespaceFilter() {
return namespaceRegex;
}
/** @return the Table filter regex */
public String getTableFilter() {
return tableRegex;
}
/** @return the User filter regex */
public String getUserFilter() {
return userRegex;
}
}

View File

@ -0,0 +1,178 @@
/**
* 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.Closeable;
import java.io.IOException;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.Queue;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
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.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.util.StringUtils;
/**
* Scanner to iterate over the quota settings.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public final class QuotaRetriever implements Closeable, Iterable<QuotaSettings> {
private static final Log LOG = LogFactory.getLog(QuotaRetriever.class);
private final Queue<QuotaSettings> cache = new LinkedList<QuotaSettings>();
private ResultScanner scanner;
/**
* Connection to use. Could pass one in and have this class use it but this class wants to be
* standalone.
*/
private Connection connection;
private Table table;
private QuotaRetriever() {
}
void init(final Configuration conf, final Scan scan) throws IOException {
this.connection = ConnectionFactory.createConnection(conf);
this.table = this.connection.getTable(QuotaTableUtil.QUOTA_TABLE_NAME);
try {
scanner = table.getScanner(scan);
} catch (IOException e) {
try {
close();
} catch (IOException ioe) {
LOG.warn("Failed getting scanner and then failed close on cleanup", e);
}
throw e;
}
}
public void close() throws IOException {
if (this.table != null) {
this.table.close();
this.table = null;
}
if (this.connection != null) {
this.connection.close();
this.connection = null;
}
}
public QuotaSettings next() throws IOException {
if (cache.isEmpty()) {
Result result = scanner.next();
if (result == null) return null;
QuotaTableUtil.parseResult(result, new QuotaTableUtil.QuotasVisitor() {
@Override
public void visitUserQuotas(String userName, Quotas quotas) {
cache.addAll(QuotaSettingsFactory.fromUserQuotas(userName, quotas));
}
@Override
public void visitUserQuotas(String userName, TableName table, Quotas quotas) {
cache.addAll(QuotaSettingsFactory.fromUserQuotas(userName, table, quotas));
}
@Override
public void visitUserQuotas(String userName, String namespace, Quotas quotas) {
cache.addAll(QuotaSettingsFactory.fromUserQuotas(userName, namespace, quotas));
}
@Override
public void visitTableQuotas(TableName tableName, Quotas quotas) {
cache.addAll(QuotaSettingsFactory.fromTableQuotas(tableName, quotas));
}
@Override
public void visitNamespaceQuotas(String namespace, Quotas quotas) {
cache.addAll(QuotaSettingsFactory.fromNamespaceQuotas(namespace, quotas));
}
});
}
return cache.poll();
}
@Override
public Iterator<QuotaSettings> iterator() {
return new Iter();
}
private class Iter implements Iterator<QuotaSettings> {
private QuotaSettings cache;
public Iter() {
try {
cache = QuotaRetriever.this.next();
} catch (IOException e) {
LOG.warn(StringUtils.stringifyException(e));
}
}
@Override
public boolean hasNext() {
return cache != null;
}
@Override
public QuotaSettings next() {
QuotaSettings result = cache;
try {
cache = QuotaRetriever.this.next();
} catch (IOException e) {
LOG.warn(StringUtils.stringifyException(e));
}
return result;
}
@Override
public void remove() {
throw new RuntimeException("remove() not supported");
}
}
/**
* Open a QuotaRetriever with no filter, all the quota settings will be returned.
* @param conf Configuration object to use.
* @return the QuotaRetriever
* @throws IOException if a remote or network exception occurs
*/
public static QuotaRetriever open(final Configuration conf) throws IOException {
return open(conf, null);
}
/**
* Open a QuotaRetriever with the specified filter.
* @param conf Configuration object to use.
* @param filter the QuotaFilter
* @return the QuotaRetriever
* @throws IOException if a remote or network exception occurs
*/
public static QuotaRetriever open(final Configuration conf, final QuotaFilter filter)
throws IOException {
Scan scan = QuotaTableUtil.makeScan(filter);
QuotaRetriever scanner = new QuotaRetriever();
scanner.init(conf, scan);
return scanner;
}
}

View File

@ -0,0 +1,35 @@
/**
* 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 org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* Describe the Scope of the quota rules. The quota can be enforced at the cluster level or at
* machine level.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public enum QuotaScope {
/**
* The specified throttling rules will be applied at the cluster level. A limit of 100req/min
* means 100req/min in total. If you execute 50req on a machine and then 50req on another machine
* then you have to wait your quota to fill up.
*/
CLUSTER,
/**
* The specified throttling rules will be applied on the machine level. A limit of 100req/min
* means that each machine can execute 100req/min.
*/
MACHINE,
}

View File

@ -0,0 +1,123 @@
/**
* 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.util.concurrent.TimeUnit;
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.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaRequest;
@InterfaceAudience.Public
@InterfaceStability.Evolving
public abstract class QuotaSettings {
private final String userName;
private final String namespace;
private final TableName tableName;
protected QuotaSettings(final String userName, final TableName tableName,
final String namespace) {
this.userName = userName;
this.namespace = namespace;
this.tableName = tableName;
}
public abstract QuotaType getQuotaType();
public String getUserName() {
return userName;
}
public TableName getTableName() {
return tableName;
}
public String getNamespace() {
return namespace;
}
/**
* Convert a QuotaSettings to a protocol buffer SetQuotaRequest. This is used internally by the
* Admin client to serialize the quota settings and send them to the master.
*/
public static SetQuotaRequest buildSetQuotaRequestProto(final QuotaSettings settings) {
SetQuotaRequest.Builder builder = SetQuotaRequest.newBuilder();
if (settings.getUserName() != null) {
builder.setUserName(settings.getUserName());
}
if (settings.getTableName() != null) {
builder.setTableName(ProtobufUtil.toProtoTableName(settings.getTableName()));
}
if (settings.getNamespace() != null) {
builder.setNamespace(settings.getNamespace());
}
settings.setupSetQuotaRequest(builder);
return builder.build();
}
/**
* Called by toSetQuotaRequestProto() the subclass should implement this method to set the
* specific SetQuotaRequest properties.
*/
protected abstract void setupSetQuotaRequest(SetQuotaRequest.Builder builder);
protected String ownerToString() {
StringBuilder builder = new StringBuilder();
if (userName != null) {
builder.append("USER => '");
builder.append(userName);
builder.append("', ");
}
if (tableName != null) {
builder.append("TABLE => '");
builder.append(tableName.toString());
builder.append("', ");
}
if (namespace != null) {
builder.append("NAMESPACE => '");
builder.append(namespace);
builder.append("', ");
}
return builder.toString();
}
protected static String sizeToString(final long size) {
if (size >= (1L << 50)) return String.format("%dP", size / (1L << 50));
if (size >= (1L << 40)) return String.format("%dT", size / (1L << 40));
if (size >= (1L << 30)) return String.format("%dG", size / (1L << 30));
if (size >= (1L << 20)) return String.format("%dM", size / (1L << 20));
if (size >= (1L << 10)) return String.format("%dK", size / (1L << 10));
return String.format("%dB", size);
}
protected static String timeToString(final TimeUnit timeUnit) {
switch (timeUnit) {
case NANOSECONDS:
return "nsec";
case MICROSECONDS:
return "usec";
case MILLISECONDS:
return "msec";
case SECONDS:
return "sec";
case MINUTES:
return "min";
case HOURS:
return "hour";
case DAYS:
return "day";
default:
throw new RuntimeException("Invalid TimeUnit " + timeUnit);
}
}
}

View File

@ -0,0 +1,253 @@
/**
* 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.util.ArrayList;
import java.util.List;
import java.util.concurrent.TimeUnit;
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.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaRequest;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class QuotaSettingsFactory {
private QuotaSettingsFactory() {
// Utility class
}
static class QuotaGlobalsSettingsBypass extends QuotaSettings {
private final boolean bypassGlobals;
QuotaGlobalsSettingsBypass(final String userName, final TableName tableName,
final String namespace, final boolean bypassGlobals) {
super(userName, tableName, namespace);
this.bypassGlobals = bypassGlobals;
}
@Override
public QuotaType getQuotaType() {
return QuotaType.GLOBAL_BYPASS;
}
@Override
protected void setupSetQuotaRequest(SetQuotaRequest.Builder builder) {
builder.setBypassGlobals(bypassGlobals);
}
@Override
public String toString() {
return "GLOBAL_BYPASS => " + bypassGlobals;
}
}
/*
* ========================================================================== QuotaSettings from
* the Quotas object
*/
static List<QuotaSettings> fromUserQuotas(final String userName, final Quotas quotas) {
return fromQuotas(userName, null, null, quotas);
}
static List<QuotaSettings> fromUserQuotas(final String userName, final TableName tableName,
final Quotas quotas) {
return fromQuotas(userName, tableName, null, quotas);
}
static List<QuotaSettings> fromUserQuotas(final String userName, final String namespace,
final Quotas quotas) {
return fromQuotas(userName, null, namespace, quotas);
}
static List<QuotaSettings> fromTableQuotas(final TableName tableName, final Quotas quotas) {
return fromQuotas(null, tableName, null, quotas);
}
static List<QuotaSettings> fromNamespaceQuotas(final String namespace, final Quotas quotas) {
return fromQuotas(null, null, namespace, quotas);
}
private static List<QuotaSettings> fromQuotas(final String userName, final TableName tableName,
final String namespace, final Quotas quotas) {
List<QuotaSettings> settings = new ArrayList<QuotaSettings>();
if (quotas.hasThrottle()) {
settings.addAll(fromThrottle(userName, tableName, namespace, quotas.getThrottle()));
}
if (quotas.getBypassGlobals() == true) {
settings.add(new QuotaGlobalsSettingsBypass(userName, tableName, namespace, true));
}
return settings;
}
private static List<QuotaSettings> fromThrottle(final String userName, final TableName tableName,
final String namespace, final QuotaProtos.Throttle throttle) {
List<QuotaSettings> settings = new ArrayList<QuotaSettings>();
if (throttle.hasReqNum()) {
settings.add(ThrottleSettings.fromTimedQuota(userName, tableName, namespace,
ThrottleType.REQUEST_NUMBER, throttle.getReqNum()));
}
if (throttle.hasReqSize()) {
settings.add(ThrottleSettings.fromTimedQuota(userName, tableName, namespace,
ThrottleType.REQUEST_SIZE, throttle.getReqSize()));
}
return settings;
}
/*
* ========================================================================== RPC Throttle
*/
/**
* Throttle the specified user.
* @param userName the user to throttle
* @param type the type of throttling
* @param limit the allowed number of request/data per timeUnit
* @param timeUnit the limit time unit
* @return the quota settings
*/
public static QuotaSettings throttleUser(final String userName, final ThrottleType type,
final long limit, final TimeUnit timeUnit) {
return throttle(userName, null, null, type, limit, timeUnit);
}
/**
* Throttle the specified user on the specified table.
* @param userName the user to throttle
* @param tableName the table to throttle
* @param type the type of throttling
* @param limit the allowed number of request/data per timeUnit
* @param timeUnit the limit time unit
* @return the quota settings
*/
public static QuotaSettings throttleUser(final String userName, final TableName tableName,
final ThrottleType type, final long limit, final TimeUnit timeUnit) {
return throttle(userName, tableName, null, type, limit, timeUnit);
}
/**
* Throttle the specified user on the specified namespace.
* @param userName the user to throttle
* @param namespace the namespace to throttle
* @param type the type of throttling
* @param limit the allowed number of request/data per timeUnit
* @param timeUnit the limit time unit
* @return the quota settings
*/
public static QuotaSettings throttleUser(final String userName, final String namespace,
final ThrottleType type, final long limit, final TimeUnit timeUnit) {
return throttle(userName, null, namespace, type, limit, timeUnit);
}
/**
* Remove the throttling for the specified user.
* @param userName the user
* @return the quota settings
*/
public static QuotaSettings unthrottleUser(final String userName) {
return throttle(userName, null, null, null, 0, null);
}
/**
* Remove the throttling for the specified user on the specified table.
* @param userName the user
* @param tableName the table
* @return the quota settings
*/
public static QuotaSettings unthrottleUser(final String userName, final TableName tableName) {
return throttle(userName, tableName, null, null, 0, null);
}
/**
* Remove the throttling for the specified user on the specified namespace.
* @param userName the user
* @param namespace the namespace
* @return the quota settings
*/
public static QuotaSettings unthrottleUser(final String userName, final String namespace) {
return throttle(userName, null, namespace, null, 0, null);
}
/**
* Throttle the specified table.
* @param tableName the table to throttle
* @param type the type of throttling
* @param limit the allowed number of request/data per timeUnit
* @param timeUnit the limit time unit
* @return the quota settings
*/
public static QuotaSettings throttleTable(final TableName tableName, final ThrottleType type,
final long limit, final TimeUnit timeUnit) {
return throttle(null, tableName, null, type, limit, timeUnit);
}
/**
* Remove the throttling for the specified table.
* @param tableName the table
* @return the quota settings
*/
public static QuotaSettings unthrottleTable(final TableName tableName) {
return throttle(null, tableName, null, null, 0, null);
}
/**
* Throttle the specified namespace.
* @param namespace the namespace to throttle
* @param type the type of throttling
* @param limit the allowed number of request/data per timeUnit
* @param timeUnit the limit time unit
* @return the quota settings
*/
public static QuotaSettings throttleNamespace(final String namespace, final ThrottleType type,
final long limit, final TimeUnit timeUnit) {
return throttle(null, null, namespace, type, limit, timeUnit);
}
/**
* Remove the throttling for the specified namespace.
* @param namespace the namespace
* @return the quota settings
*/
public static QuotaSettings unthrottleNamespace(final String namespace) {
return throttle(null, null, namespace, null, 0, null);
}
/* Throttle helper */
private static QuotaSettings throttle(final String userName, final TableName tableName,
final String namespace, final ThrottleType type, final long limit, final TimeUnit timeUnit) {
QuotaProtos.ThrottleRequest.Builder builder = QuotaProtos.ThrottleRequest.newBuilder();
if (type != null) {
builder.setType(ProtobufUtil.toProtoThrottleType(type));
}
if (timeUnit != null) {
builder.setTimedQuota(ProtobufUtil.toTimedQuota(limit, timeUnit, QuotaScope.MACHINE));
}
return new ThrottleSettings(userName, tableName, namespace, builder.build());
}
/*
* ========================================================================== Global Settings
*/
/**
* Set the "bypass global settings" for the specified user
* @param userName the user to throttle
* @param bypassGlobals true if the global settings should be bypassed
* @return the quota settings
*/
public static QuotaSettings bypassGlobals(final String userName, final boolean bypassGlobals) {
return new QuotaGlobalsSettingsBypass(userName, null, null, bypassGlobals);
}
}

View File

@ -0,0 +1,408 @@
/**
* 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.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.NamespaceDescriptor;
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.Connection;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.CompareFilter;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.FilterList;
import org.apache.hadoop.hbase.filter.QualifierFilter;
import org.apache.hadoop.hbase.filter.RegexStringComparator;
import org.apache.hadoop.hbase.filter.RowFilter;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Strings;
/**
* Helper class to interact with the quota table.
*
* <pre>
* ROW-KEY FAM/QUAL DATA
* n.<namespace> q:s <global-quotas>
* t.<table> q:s <global-quotas>
* u.<user> q:s <global-quotas>
* u.<user> q:s.<table> <table-quotas>
* u.<user> q:s.<ns>: <namespace-quotas>
* </pre>
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class QuotaTableUtil {
private static final Log LOG = LogFactory.getLog(QuotaTableUtil.class);
/** System table for quotas */
public static final TableName QUOTA_TABLE_NAME = TableName.valueOf(
NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "quota");
protected static final byte[] QUOTA_FAMILY_INFO = Bytes.toBytes("q");
protected static final byte[] QUOTA_FAMILY_USAGE = Bytes.toBytes("u");
protected static final byte[] QUOTA_QUALIFIER_SETTINGS = Bytes.toBytes("s");
protected static final byte[] QUOTA_QUALIFIER_SETTINGS_PREFIX = Bytes.toBytes("s.");
protected static final byte[] QUOTA_USER_ROW_KEY_PREFIX = Bytes.toBytes("u.");
protected static final byte[] QUOTA_TABLE_ROW_KEY_PREFIX = Bytes.toBytes("t.");
protected static final byte[] QUOTA_NAMESPACE_ROW_KEY_PREFIX = Bytes.toBytes("n.");
/*
* ========================================================================= Quota "settings"
* helpers
*/
public static Quotas getTableQuota(final Connection connection, final TableName table)
throws IOException {
return getQuotas(connection, getTableRowKey(table));
}
public static Quotas getNamespaceQuota(final Connection connection, final String namespace)
throws IOException {
return getQuotas(connection, getNamespaceRowKey(namespace));
}
public static Quotas getUserQuota(final Connection connection, final String user)
throws IOException {
return getQuotas(connection, getUserRowKey(user));
}
public static Quotas getUserQuota(final Connection connection, final String user,
final TableName table) throws IOException {
return getQuotas(connection, getUserRowKey(user), getSettingsQualifierForUserTable(table));
}
public static Quotas getUserQuota(final Connection connection, final String user,
final String namespace) throws IOException {
return getQuotas(connection, getUserRowKey(user),
getSettingsQualifierForUserNamespace(namespace));
}
private static Quotas getQuotas(final Connection connection, final byte[] rowKey)
throws IOException {
return getQuotas(connection, rowKey, QUOTA_QUALIFIER_SETTINGS);
}
private static Quotas getQuotas(final Connection connection, final byte[] rowKey,
final byte[] qualifier) throws IOException {
Get get = new Get(rowKey);
get.addColumn(QUOTA_FAMILY_INFO, qualifier);
Result result = doGet(connection, get);
if (result.isEmpty()) {
return null;
}
return quotasFromData(result.getValue(QUOTA_FAMILY_INFO, qualifier));
}
public static Get makeGetForTableQuotas(final TableName table) {
Get get = new Get(getTableRowKey(table));
get.addFamily(QUOTA_FAMILY_INFO);
return get;
}
public static Get makeGetForNamespaceQuotas(final String namespace) {
Get get = new Get(getNamespaceRowKey(namespace));
get.addFamily(QUOTA_FAMILY_INFO);
return get;
}
public static Get makeGetForUserQuotas(final String user, final Iterable<TableName> tables,
final Iterable<String> namespaces) {
Get get = new Get(getUserRowKey(user));
get.addColumn(QUOTA_FAMILY_INFO, QUOTA_QUALIFIER_SETTINGS);
for (final TableName table : tables) {
get.addColumn(QUOTA_FAMILY_INFO, getSettingsQualifierForUserTable(table));
}
for (final String ns : namespaces) {
get.addColumn(QUOTA_FAMILY_INFO, getSettingsQualifierForUserNamespace(ns));
}
return get;
}
public static Scan makeScan(final QuotaFilter filter) {
Scan scan = new Scan();
scan.addFamily(QUOTA_FAMILY_INFO);
if (filter != null && !filter.isNull()) {
scan.setFilter(makeFilter(filter));
}
return scan;
}
/**
* converts quotafilter to serializeable filterlists.
*/
public static Filter makeFilter(final QuotaFilter filter) {
FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ALL);
if (!Strings.isEmpty(filter.getUserFilter())) {
FilterList userFilters = new FilterList(FilterList.Operator.MUST_PASS_ONE);
boolean hasFilter = false;
if (!Strings.isEmpty(filter.getNamespaceFilter())) {
FilterList nsFilters = new FilterList(FilterList.Operator.MUST_PASS_ALL);
nsFilters.addFilter(new RowFilter(CompareFilter.CompareOp.EQUAL, new RegexStringComparator(
getUserRowKeyRegex(filter.getUserFilter()), 0)));
nsFilters.addFilter(new QualifierFilter(CompareFilter.CompareOp.EQUAL,
new RegexStringComparator(getSettingsQualifierRegexForUserNamespace(filter
.getNamespaceFilter()), 0)));
userFilters.addFilter(nsFilters);
hasFilter = true;
}
if (!Strings.isEmpty(filter.getTableFilter())) {
FilterList tableFilters = new FilterList(FilterList.Operator.MUST_PASS_ALL);
tableFilters.addFilter(new RowFilter(CompareFilter.CompareOp.EQUAL,
new RegexStringComparator(getUserRowKeyRegex(filter.getUserFilter()), 0)));
tableFilters.addFilter(new QualifierFilter(CompareFilter.CompareOp.EQUAL,
new RegexStringComparator(
getSettingsQualifierRegexForUserTable(filter.getTableFilter()), 0)));
userFilters.addFilter(tableFilters);
hasFilter = true;
}
if (!hasFilter) {
userFilters.addFilter(new RowFilter(CompareFilter.CompareOp.EQUAL,
new RegexStringComparator(getUserRowKeyRegex(filter.getUserFilter()), 0)));
}
filterList.addFilter(userFilters);
} else if (!Strings.isEmpty(filter.getTableFilter())) {
filterList.addFilter(new RowFilter(CompareFilter.CompareOp.EQUAL, new RegexStringComparator(
getTableRowKeyRegex(filter.getTableFilter()), 0)));
} else if (!Strings.isEmpty(filter.getNamespaceFilter())) {
filterList.addFilter(new RowFilter(CompareFilter.CompareOp.EQUAL, new RegexStringComparator(
getNamespaceRowKeyRegex(filter.getNamespaceFilter()), 0)));
}
return filterList;
}
public static interface UserQuotasVisitor {
void visitUserQuotas(final String userName, final Quotas quotas) throws IOException;
void visitUserQuotas(final String userName, final TableName table, final Quotas quotas)
throws IOException;
void visitUserQuotas(final String userName, final String namespace, final Quotas quotas)
throws IOException;
}
public static interface TableQuotasVisitor {
void visitTableQuotas(final TableName tableName, final Quotas quotas) throws IOException;
}
public static interface NamespaceQuotasVisitor {
void visitNamespaceQuotas(final String namespace, final Quotas quotas) throws IOException;
}
public static interface QuotasVisitor extends UserQuotasVisitor, TableQuotasVisitor,
NamespaceQuotasVisitor {
}
public static void parseResult(final Result result, final QuotasVisitor visitor)
throws IOException {
byte[] row = result.getRow();
if (isNamespaceRowKey(row)) {
parseNamespaceResult(result, visitor);
} else if (isTableRowKey(row)) {
parseTableResult(result, visitor);
} else if (isUserRowKey(row)) {
parseUserResult(result, visitor);
} else {
LOG.warn("unexpected row-key: " + Bytes.toString(row));
}
}
public static void
parseNamespaceResult(final Result result, final NamespaceQuotasVisitor visitor)
throws IOException {
String namespace = getNamespaceFromRowKey(result.getRow());
parseNamespaceResult(namespace, result, visitor);
}
protected static void parseNamespaceResult(final String namespace, final Result result,
final NamespaceQuotasVisitor visitor) throws IOException {
byte[] data = result.getValue(QUOTA_FAMILY_INFO, QUOTA_QUALIFIER_SETTINGS);
if (data != null) {
Quotas quotas = quotasFromData(data);
visitor.visitNamespaceQuotas(namespace, quotas);
}
}
public static void parseTableResult(final Result result, final TableQuotasVisitor visitor)
throws IOException {
TableName table = getTableFromRowKey(result.getRow());
parseTableResult(table, result, visitor);
}
protected static void parseTableResult(final TableName table, final Result result,
final TableQuotasVisitor visitor) throws IOException {
byte[] data = result.getValue(QUOTA_FAMILY_INFO, QUOTA_QUALIFIER_SETTINGS);
if (data != null) {
Quotas quotas = quotasFromData(data);
visitor.visitTableQuotas(table, quotas);
}
}
public static void parseUserResult(final Result result, final UserQuotasVisitor visitor)
throws IOException {
String userName = getUserFromRowKey(result.getRow());
parseUserResult(userName, result, visitor);
}
protected static void parseUserResult(final String userName, final Result result,
final UserQuotasVisitor visitor) throws IOException {
Map<byte[], byte[]> familyMap = result.getFamilyMap(QUOTA_FAMILY_INFO);
if (familyMap == null || familyMap.isEmpty()) return;
for (Map.Entry<byte[], byte[]> entry : familyMap.entrySet()) {
Quotas quotas = quotasFromData(entry.getValue());
if (Bytes.startsWith(entry.getKey(), QUOTA_QUALIFIER_SETTINGS_PREFIX)) {
String name = Bytes.toString(entry.getKey(), QUOTA_QUALIFIER_SETTINGS_PREFIX.length);
if (name.charAt(name.length() - 1) == TableName.NAMESPACE_DELIM) {
String namespace = name.substring(0, name.length() - 1);
visitor.visitUserQuotas(userName, namespace, quotas);
} else {
TableName table = TableName.valueOf(name);
visitor.visitUserQuotas(userName, table, quotas);
}
} else if (Bytes.equals(entry.getKey(), QUOTA_QUALIFIER_SETTINGS)) {
visitor.visitUserQuotas(userName, quotas);
}
}
}
/*
* ========================================================================= Quotas protobuf
* helpers
*/
protected static Quotas quotasFromData(final byte[] data) throws IOException {
int magicLen = ProtobufUtil.lengthOfPBMagic();
if (!ProtobufUtil.isPBMagicPrefix(data, 0, magicLen)) {
throw new IOException("Missing pb magic prefix");
}
return Quotas.parseFrom(new ByteArrayInputStream(data, magicLen, data.length - magicLen));
}
protected static byte[] quotasToData(final Quotas data) throws IOException {
ByteArrayOutputStream stream = new ByteArrayOutputStream();
stream.write(ProtobufUtil.PB_MAGIC);
data.writeTo(stream);
return stream.toByteArray();
}
public static boolean isEmptyQuota(final Quotas quotas) {
boolean hasSettings = false;
hasSettings |= quotas.hasThrottle();
hasSettings |= quotas.hasBypassGlobals();
return !hasSettings;
}
/*
* ========================================================================= HTable helpers
*/
protected static Result doGet(final Connection connection, final Get get) throws IOException {
try (Table table = connection.getTable(QUOTA_TABLE_NAME)) {
return table.get(get);
}
}
protected static Result[] doGet(final Connection connection, final List<Get> gets)
throws IOException {
try (Table table = connection.getTable(QUOTA_TABLE_NAME)) {
return table.get(gets);
}
}
/*
* ========================================================================= Quota table row key
* helpers
*/
protected static byte[] getUserRowKey(final String user) {
return Bytes.add(QUOTA_USER_ROW_KEY_PREFIX, Bytes.toBytes(user));
}
protected static byte[] getTableRowKey(final TableName table) {
return Bytes.add(QUOTA_TABLE_ROW_KEY_PREFIX, table.getName());
}
protected static byte[] getNamespaceRowKey(final String namespace) {
return Bytes.add(QUOTA_NAMESPACE_ROW_KEY_PREFIX, Bytes.toBytes(namespace));
}
protected static byte[] getSettingsQualifierForUserTable(final TableName tableName) {
return Bytes.add(QUOTA_QUALIFIER_SETTINGS_PREFIX, tableName.getName());
}
protected static byte[] getSettingsQualifierForUserNamespace(final String namespace) {
return Bytes.add(QUOTA_QUALIFIER_SETTINGS_PREFIX,
Bytes.toBytes(namespace + TableName.NAMESPACE_DELIM));
}
protected static String getUserRowKeyRegex(final String user) {
return getRowKeyRegEx(QUOTA_USER_ROW_KEY_PREFIX, user);
}
protected static String getTableRowKeyRegex(final String table) {
return getRowKeyRegEx(QUOTA_TABLE_ROW_KEY_PREFIX, table);
}
protected static String getNamespaceRowKeyRegex(final String namespace) {
return getRowKeyRegEx(QUOTA_NAMESPACE_ROW_KEY_PREFIX, namespace);
}
private static String getRowKeyRegEx(final byte[] prefix, final String regex) {
return '^' + Pattern.quote(Bytes.toString(prefix)) + regex + '$';
}
protected static String getSettingsQualifierRegexForUserTable(final String table) {
return '^' + Pattern.quote(Bytes.toString(QUOTA_QUALIFIER_SETTINGS_PREFIX)) + table + "(?<!"
+ Pattern.quote(Character.toString(TableName.NAMESPACE_DELIM)) + ")$";
}
protected static String getSettingsQualifierRegexForUserNamespace(final String namespace) {
return '^' + Pattern.quote(Bytes.toString(QUOTA_QUALIFIER_SETTINGS_PREFIX)) + namespace
+ Pattern.quote(Character.toString(TableName.NAMESPACE_DELIM)) + '$';
}
protected static boolean isNamespaceRowKey(final byte[] key) {
return Bytes.startsWith(key, QUOTA_NAMESPACE_ROW_KEY_PREFIX);
}
protected static String getNamespaceFromRowKey(final byte[] key) {
return Bytes.toString(key, QUOTA_NAMESPACE_ROW_KEY_PREFIX.length);
}
protected static boolean isTableRowKey(final byte[] key) {
return Bytes.startsWith(key, QUOTA_TABLE_ROW_KEY_PREFIX);
}
protected static TableName getTableFromRowKey(final byte[] key) {
return TableName.valueOf(Bytes.toString(key, QUOTA_TABLE_ROW_KEY_PREFIX.length));
}
protected static boolean isUserRowKey(final byte[] key) {
return Bytes.startsWith(key, QUOTA_USER_ROW_KEY_PREFIX);
}
protected static String getUserFromRowKey(final byte[] key) {
return Bytes.toString(key, QUOTA_USER_ROW_KEY_PREFIX.length);
}
}

View File

@ -0,0 +1,23 @@
/**
* 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 org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* Describe the Quota Type.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public enum QuotaType {
THROTTLE, GLOBAL_BYPASS,
}

View File

@ -0,0 +1,100 @@
/**
* 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.util.concurrent.TimeUnit;
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.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaRequest;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos;
@InterfaceAudience.Private
@InterfaceStability.Evolving
class ThrottleSettings extends QuotaSettings {
private final QuotaProtos.ThrottleRequest proto;
ThrottleSettings(final String userName, final TableName tableName, final String namespace,
final QuotaProtos.ThrottleRequest proto) {
super(userName, tableName, namespace);
this.proto = proto;
}
public ThrottleType getThrottleType() {
return ProtobufUtil.toThrottleType(proto.getType());
}
public long getSoftLimit() {
return proto.hasTimedQuota() ? proto.getTimedQuota().getSoftLimit() : -1;
}
public TimeUnit getTimeUnit() {
return proto.hasTimedQuota() ? ProtobufUtil.toTimeUnit(proto.getTimedQuota().getTimeUnit())
: null;
}
@Override
public QuotaType getQuotaType() {
return QuotaType.THROTTLE;
}
@Override
protected void setupSetQuotaRequest(SetQuotaRequest.Builder builder) {
builder.setThrottle(proto);
}
@Override
public String toString() {
StringBuilder builder = new StringBuilder();
builder.append("TYPE => THROTTLE");
if (proto.hasType()) {
builder.append(", THROTTLE_TYPE => ");
builder.append(proto.getType().toString());
}
if (proto.hasTimedQuota()) {
QuotaProtos.TimedQuota timedQuota = proto.getTimedQuota();
builder.append(", LIMIT => ");
if (timedQuota.hasSoftLimit()) {
switch (getThrottleType()) {
case REQUEST_NUMBER:
builder.append(String.format("%dreq", timedQuota.getSoftLimit()));
break;
case REQUEST_SIZE:
builder.append(sizeToString(timedQuota.getSoftLimit()));
break;
default:
throw new RuntimeException("Invalid throttle type: " + getThrottleType());
}
} else if (timedQuota.hasShare()) {
builder.append(String.format("%.2f%%", timedQuota.getShare()));
}
builder.append('/');
builder.append(timeToString(ProtobufUtil.toTimeUnit(timedQuota.getTimeUnit())));
if (timedQuota.hasScope()) {
builder.append(", SCOPE => ");
builder.append(timedQuota.getScope().toString());
}
} else {
builder.append(", LIMIT => NONE");
}
return builder.toString();
}
static ThrottleSettings fromTimedQuota(final String userName, final TableName tableName,
final String namespace, ThrottleType type, QuotaProtos.TimedQuota timedQuota) {
QuotaProtos.ThrottleRequest.Builder builder = QuotaProtos.ThrottleRequest.newBuilder();
builder.setType(ProtobufUtil.toProtoThrottleType(type));
builder.setTimedQuota(timedQuota);
return new ThrottleSettings(userName, tableName, namespace, builder.build());
}
}

View File

@ -0,0 +1,34 @@
/**
* 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 org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* Describe the Throttle Type.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public enum ThrottleType {
/** Throttling based on the number of request per time-unit */
REQUEST_NUMBER,
/** Throttling based on the read+write data size */
REQUEST_SIZE,
}

View File

@ -0,0 +1,146 @@
/**
* 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.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* Describe the throttling result. TODO: At some point this will be handled on the client side to
* prevent operation to go on the server if the waitInterval is grater than the one got as result of
* this exception.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class ThrottlingException extends QuotaExceededException {
private static final long serialVersionUID = 1406576492085155743L;
@InterfaceAudience.Public
@InterfaceStability.Evolving
public enum Type {
NumRequestsExceeded, NumReadRequestsExceeded, NumWriteRequestsExceeded, WriteSizeExceeded,
ReadSizeExceeded,
}
private static final String[] MSG_TYPE = new String[] { "number of requests exceeded",
"number of read requests exceeded", "number of write requests exceeded",
"write size limit exceeded", "read size limit exceeded", };
private static final String MSG_WAIT = " - wait ";
private long waitInterval;
private Type type;
public ThrottlingException(String msg) {
super(msg);
// Dirty workaround to get the information after
// ((RemoteException)e.getCause()).unwrapRemoteException()
for (int i = 0; i < MSG_TYPE.length; ++i) {
int index = msg.indexOf(MSG_TYPE[i]);
if (index >= 0) {
String waitTimeStr = msg.substring(index + MSG_TYPE[i].length() + MSG_WAIT.length());
type = Type.values()[i];
waitInterval = timeFromString(waitTimeStr);
break;
}
}
}
public ThrottlingException(final Type type, final long waitInterval, final String msg) {
super(msg);
this.waitInterval = waitInterval;
this.type = type;
}
public Type getType() {
return this.type;
}
public long getWaitInterval() {
return this.waitInterval;
}
public static void throwNumRequestsExceeded(final long waitInterval) throws ThrottlingException {
throwThrottlingException(Type.NumRequestsExceeded, waitInterval);
}
public static void throwNumReadRequestsExceeded(final long waitInterval)
throws ThrottlingException {
throwThrottlingException(Type.NumReadRequestsExceeded, waitInterval);
}
public static void throwNumWriteRequestsExceeded(final long waitInterval)
throws ThrottlingException {
throwThrottlingException(Type.NumWriteRequestsExceeded, waitInterval);
}
public static void throwWriteSizeExceeded(final long waitInterval) throws ThrottlingException {
throwThrottlingException(Type.WriteSizeExceeded, waitInterval);
}
public static void throwReadSizeExceeded(final long waitInterval) throws ThrottlingException {
throwThrottlingException(Type.ReadSizeExceeded, waitInterval);
}
private static void throwThrottlingException(final Type type, final long waitInterval)
throws ThrottlingException {
String msg = MSG_TYPE[type.ordinal()] + MSG_WAIT + formatTime(waitInterval);
throw new ThrottlingException(type, waitInterval, msg);
}
public static String formatTime(long timeDiff) {
StringBuilder buf = new StringBuilder();
long hours = timeDiff / (60 * 60 * 1000);
long rem = (timeDiff % (60 * 60 * 1000));
long minutes = rem / (60 * 1000);
rem = rem % (60 * 1000);
float seconds = rem / 1000.0f;
if (hours != 0) {
buf.append(hours);
buf.append("hrs, ");
}
if (minutes != 0) {
buf.append(minutes);
buf.append("mins, ");
}
buf.append(String.format("%.2fsec", seconds));
return buf.toString();
}
private static long timeFromString(String timeDiff) {
Pattern[] patterns =
new Pattern[] { Pattern.compile("^(\\d+\\.\\d\\d)sec"),
Pattern.compile("^(\\d+)mins, (\\d+\\.\\d\\d)sec"),
Pattern.compile("^(\\d+)hrs, (\\d+)mins, (\\d+\\.\\d\\d)sec") };
for (int i = 0; i < patterns.length; ++i) {
Matcher m = patterns[i].matcher(timeDiff);
if (m.find()) {
long time = Math.round(Float.parseFloat(m.group(1 + i)) * 1000);
if (i > 0) {
time += Long.parseLong(m.group(i)) * (60 * 1000);
}
if (i > 1) {
time += Long.parseLong(m.group(i - 1)) * (60 * 60 * 1000);
}
return time;
}
}
return -1;
}
}

View File

@ -362,6 +362,24 @@ public class Bytes {
final byte [] b2) {
return toString(b1, 0, b1.length) + sep + toString(b2, 0, b2.length);
}
/**
* This method will convert utf8 encoded bytes into a string. If the given byte array is null,
* this method will return null.
* @param b Presumed UTF-8 encoded byte array.
* @param off offset into array
* @return String made from <code>b</code> or null
*/
public static String toString(final byte[] b, int off) {
if (b == null) {
return null;
}
int len = b.length - off;
if (len <= 0) {
return "";
}
return new String(b, off, len, UTF8_CHARSET);
}
/**
* This method will convert utf8 encoded bytes into a string. If

View File

@ -114,4 +114,11 @@ public class Sleeper {
triggerWake = false;
}
}
/**
* @return the sleep period in milliseconds
*/
public final int getPeriod() {
return period;
}
}

View File

@ -176,6 +176,7 @@
<include>MapReduce.proto</include>
<include>Master.proto</include>
<include>MultiRowMutation.proto</include>
<include>Quota.proto</include>
<include>RegionServerStatus.proto</include>
<include>RowProcessor.proto</include>
<include>RPC.proto</include>

View File

@ -139,6 +139,133 @@ public final class HBaseProtos {
// @@protoc_insertion_point(enum_scope:CompareType)
}
/**
* Protobuf enum {@code TimeUnit}
*/
public enum TimeUnit
implements com.google.protobuf.ProtocolMessageEnum {
/**
* <code>NANOSECONDS = 1;</code>
*/
NANOSECONDS(0, 1),
/**
* <code>MICROSECONDS = 2;</code>
*/
MICROSECONDS(1, 2),
/**
* <code>MILLISECONDS = 3;</code>
*/
MILLISECONDS(2, 3),
/**
* <code>SECONDS = 4;</code>
*/
SECONDS(3, 4),
/**
* <code>MINUTES = 5;</code>
*/
MINUTES(4, 5),
/**
* <code>HOURS = 6;</code>
*/
HOURS(5, 6),
/**
* <code>DAYS = 7;</code>
*/
DAYS(6, 7),
;
/**
* <code>NANOSECONDS = 1;</code>
*/
public static final int NANOSECONDS_VALUE = 1;
/**
* <code>MICROSECONDS = 2;</code>
*/
public static final int MICROSECONDS_VALUE = 2;
/**
* <code>MILLISECONDS = 3;</code>
*/
public static final int MILLISECONDS_VALUE = 3;
/**
* <code>SECONDS = 4;</code>
*/
public static final int SECONDS_VALUE = 4;
/**
* <code>MINUTES = 5;</code>
*/
public static final int MINUTES_VALUE = 5;
/**
* <code>HOURS = 6;</code>
*/
public static final int HOURS_VALUE = 6;
/**
* <code>DAYS = 7;</code>
*/
public static final int DAYS_VALUE = 7;
public final int getNumber() { return value; }
public static TimeUnit valueOf(int value) {
switch (value) {
case 1: return NANOSECONDS;
case 2: return MICROSECONDS;
case 3: return MILLISECONDS;
case 4: return SECONDS;
case 5: return MINUTES;
case 6: return HOURS;
case 7: return DAYS;
default: return null;
}
}
public static com.google.protobuf.Internal.EnumLiteMap<TimeUnit>
internalGetValueMap() {
return internalValueMap;
}
private static com.google.protobuf.Internal.EnumLiteMap<TimeUnit>
internalValueMap =
new com.google.protobuf.Internal.EnumLiteMap<TimeUnit>() {
public TimeUnit findValueByNumber(int number) {
return TimeUnit.valueOf(number);
}
};
public final com.google.protobuf.Descriptors.EnumValueDescriptor
getValueDescriptor() {
return getDescriptor().getValues().get(index);
}
public final com.google.protobuf.Descriptors.EnumDescriptor
getDescriptorForType() {
return getDescriptor();
}
public static final com.google.protobuf.Descriptors.EnumDescriptor
getDescriptor() {
return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor().getEnumTypes().get(1);
}
private static final TimeUnit[] VALUES = values();
public static TimeUnit valueOf(
com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
if (desc.getType() != getDescriptor()) {
throw new java.lang.IllegalArgumentException(
"EnumValueDescriptor is not for this type.");
}
return VALUES[desc.getIndex()];
}
private final int index;
private final int value;
private TimeUnit(int index, int value) {
this.index = index;
this.value = value;
}
// @@protoc_insertion_point(enum_scope:TimeUnit)
}
public interface TableNameOrBuilder
extends com.google.protobuf.MessageOrBuilder {
@ -16524,9 +16651,11 @@ public final class HBaseProtos {
"Info\022\020\n\010infoPort\030\001 \001(\005*r\n\013CompareType\022\010\n" +
"\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n" +
"\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR_EQUAL\020\004\022\013\n\007GR" +
"EATER\020\005\022\t\n\005NO_OP\020\006B>\n*org.apache.hadoop." +
"hbase.protobuf.generatedB\013HBaseProtosH\001\240" +
"\001\001"
"EATER\020\005\022\t\n\005NO_OP\020\006*n\n\010TimeUnit\022\017\n\013NANOSE" +
"CONDS\020\001\022\020\n\014MICROSECONDS\020\002\022\020\n\014MILLISECOND" +
"S\020\003\022\013\n\007SECONDS\020\004\022\013\n\007MINUTES\020\005\022\t\n\005HOURS\020\006" +
"\022\010\n\004DAYS\020\007B>\n*org.apache.hadoop.hbase.pr" +
"otobuf.generatedB\013HBaseProtosH\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {

View File

@ -180,6 +180,16 @@ message ProcedureDescription {
message EmptyMsg {
}
enum TimeUnit {
NANOSECONDS = 1;
MICROSECONDS = 2;
MILLISECONDS = 3;
SECONDS = 4;
MINUTES = 5;
HOURS = 6;
DAYS = 7;
}
message LongMsg {
required int64 long_msg = 1;
}

View File

@ -28,6 +28,7 @@ option optimize_for = SPEED;
import "HBase.proto";
import "Client.proto";
import "ClusterStatus.proto";
import "Quota.proto";
/* Column-level protobufs */
@ -371,6 +372,20 @@ message IsProcedureDoneResponse {
optional ProcedureDescription snapshot = 2;
}
message SetQuotaRequest {
optional string user_name = 1;
optional string user_group = 2;
optional string namespace = 3;
optional TableName table_name = 4;
optional bool remove_all = 5;
optional bool bypass_globals = 6;
optional ThrottleRequest throttle = 7;
}
message SetQuotaResponse {
}
message MajorCompactionTimestampRequest {
required TableName table_name = 1;
}
@ -597,6 +612,9 @@ service MasterService {
rpc ListTableNamesByNamespace(ListTableNamesByNamespaceRequest)
returns(ListTableNamesByNamespaceResponse);
/** Apply the new quota settings */
rpc SetQuota(SetQuotaRequest) returns(SetQuotaResponse);
/** Returns the timestamp of the last major compaction */
rpc getLastMajorCompactionTimestamp(MajorCompactionTimestampRequest)
returns(MajorCompactionTimestampResponse);

View File

@ -0,0 +1,73 @@
/**
* 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.
*/
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
option java_outer_classname = "QuotaProtos";
option java_generic_services = true;
option java_generate_equals_and_hash = true;
option optimize_for = SPEED;
import "HBase.proto";
enum QuotaScope {
CLUSTER = 1;
MACHINE = 2;
}
message TimedQuota {
required TimeUnit time_unit = 1;
optional uint64 soft_limit = 2;
optional float share = 3;
optional QuotaScope scope = 4 [default = MACHINE];
}
enum ThrottleType {
REQUEST_NUMBER = 1;
REQUEST_SIZE = 2;
WRITE_NUMBER = 3;
WRITE_SIZE = 4;
READ_NUMBER = 5;
READ_SIZE = 6;
}
message Throttle {
optional TimedQuota req_num = 1;
optional TimedQuota req_size = 2;
optional TimedQuota write_num = 3;
optional TimedQuota write_size = 4;
optional TimedQuota read_num = 5;
optional TimedQuota read_size = 6;
}
message ThrottleRequest {
optional ThrottleType type = 1;
optional TimedQuota timed_quota = 2;
}
enum QuotaType {
THROTTLE = 1;
}
message Quotas {
optional bool bypass_globals = 1 [default = false];
optional Throttle throttle = 2;
}
message QuotaUsage {
}

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
import java.io.IOException;
import java.util.List;
@ -467,4 +468,54 @@ public abstract class BaseMasterAndRegionObserver extends BaseRegionObserver
public void postTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName) throws IOException {
}
@Override
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final Quotas quotas) throws IOException {
}
@Override
public void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final Quotas quotas) throws IOException {
}
@Override
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final TableName tableName, final Quotas quotas) throws IOException {
}
@Override
public void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final TableName tableName, final Quotas quotas) throws IOException {
}
@Override
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final String namespace, final Quotas quotas) throws IOException {
}
@Override
public void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final String namespace, final Quotas quotas) throws IOException {
}
@Override
public void preSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName, final Quotas quotas) throws IOException {
}
@Override
public void postSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName, final Quotas quotas) throws IOException {
}
@Override
public void preSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String namespace, final Quotas quotas) throws IOException {
}
@Override
public void postSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String namespace, final Quotas quotas) throws IOException {
}
}

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
import java.io.IOException;
import java.util.List;
@ -462,4 +463,53 @@ public class BaseMasterObserver implements MasterObserver {
TableName tableName) throws IOException {
}
@Override
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final Quotas quotas) throws IOException {
}
@Override
public void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final Quotas quotas) throws IOException {
}
@Override
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final TableName tableName, final Quotas quotas) throws IOException {
}
@Override
public void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final TableName tableName, final Quotas quotas) throws IOException {
}
@Override
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final String namespace, final Quotas quotas) throws IOException {
}
@Override
public void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final String namespace, final Quotas quotas) throws IOException {
}
@Override
public void preSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName, final Quotas quotas) throws IOException {
}
@Override
public void postSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName, final Quotas quotas) throws IOException {
}
@Override
public void preSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String namespace, final Quotas quotas) throws IOException {
}
@Override
public void postSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String namespace, final Quotas quotas) throws IOException {
}
}

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
/**
* Defines coprocessor hooks for interacting with operations on the
@ -842,4 +843,108 @@ public interface MasterObserver extends Coprocessor {
*/
void postTableFlush(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName) throws IOException;
/**
* Called before the quota for the user is stored.
* @param ctx the environment to interact with the framework and master
* @param userName the name of user
* @param quotas the quota settings
* @throws IOException
*/
void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final Quotas quotas) throws IOException;
/**
* Called after the quota for the user is stored.
* @param ctx the environment to interact with the framework and master
* @param userName the name of user
* @param quotas the quota settings
* @throws IOException
*/
void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final Quotas quotas) throws IOException;
/**
* Called before the quota for the user on the specified table is stored.
* @param ctx the environment to interact with the framework and master
* @param userName the name of user
* @param tableName the name of the table
* @param quotas the quota settings
* @throws IOException
*/
void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final TableName tableName, final Quotas quotas) throws IOException;
/**
* Called after the quota for the user on the specified table is stored.
* @param ctx the environment to interact with the framework and master
* @param userName the name of user
* @param tableName the name of the table
* @param quotas the quota settings
* @throws IOException
*/
void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final TableName tableName, final Quotas quotas) throws IOException;
/**
* Called before the quota for the user on the specified namespace is stored.
* @param ctx the environment to interact with the framework and master
* @param userName the name of user
* @param namespace the name of the namespace
* @param quotas the quota settings
* @throws IOException
*/
void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final String namespace, final Quotas quotas) throws IOException;
/**
* Called after the quota for the user on the specified namespace is stored.
* @param ctx the environment to interact with the framework and master
* @param userName the name of user
* @param namespace the name of the namespace
* @param quotas the quota settings
* @throws IOException
*/
void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final String namespace, final Quotas quotas) throws IOException;
/**
* Called before the quota for the table is stored.
* @param ctx the environment to interact with the framework and master
* @param tableName the name of the table
* @param quotas the quota settings
* @throws IOException
*/
void preSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName, final Quotas quotas) throws IOException;
/**
* Called after the quota for the table is stored.
* @param ctx the environment to interact with the framework and master
* @param tableName the name of the table
* @param quotas the quota settings
* @throws IOException
*/
void postSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName, final Quotas quotas) throws IOException;
/**
* Called before the quota for the namespace is stored.
* @param ctx the environment to interact with the framework and master
* @param namespace the name of the namespace
* @param quotas the quota settings
* @throws IOException
*/
void preSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String namespace, final Quotas quotas) throws IOException;
/**
* Called after the quota for the namespace is stored.
* @param ctx the environment to interact with the framework and master
* @param namespace the name of the namespace
* @param quotas the quota settings
* @throws IOException
*/
void postSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String namespace, final Quotas quotas) throws IOException;
}

View File

@ -2477,6 +2477,7 @@ public class RpcServer implements RpcServerInterface {
}
}
@Override
public RpcScheduler getScheduler() {
return scheduler;
}

View File

@ -73,4 +73,6 @@ public interface RpcServerInterface {
*/
@VisibleForTesting
void refreshAuthManager(PolicyProvider pp);
RpcScheduler getScheduler();
}

View File

@ -109,6 +109,7 @@ import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
@ -290,6 +291,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
SnapshotManager snapshotManager;
// monitor for distributed procedures
MasterProcedureManagerHost mpmHost;
private MasterQuotaManager quotaManager;
/** flag used in test cases in order to simulate RS failures during master initialization */
private volatile boolean initializationBeforeMetaAssignment = false;
@ -721,6 +724,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
status.setStatus("Starting namespace manager");
initNamespace();
status.setStatus("Starting quota manager");
initQuotaManager();
if (this.cpHost != null) {
try {
@ -761,6 +767,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
zombieDetector.interrupt();
}
private void initQuotaManager() throws IOException {
quotaManager = new MasterQuotaManager(this);
quotaManager.start();
}
/**
* Create a {@link ServerManager} instance.
* @param master
@ -1063,6 +1074,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
// Clean up and close up shop
if (this.logCleaner != null) this.logCleaner.cancel(true);
if (this.hfileCleaner != null) this.hfileCleaner.cancel(true);
if (this.quotaManager != null) this.quotaManager.stop();
if (this.activeMasterManager != null) this.activeMasterManager.stop();
if (this.serverManager != null) this.serverManager.stop();
if (this.assignmentManager != null) this.assignmentManager.stop();
@ -1861,6 +1873,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
public MasterCoprocessorHost getMasterCoprocessorHost() {
return cpHost;
}
@Override
public MasterQuotaManager getMasterQuotaManager() {
return quotaManager;
}
@Override
public ServerName getServerName() {

View File

@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.coprocessor.*;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
import java.io.IOException;
import java.util.List;
@ -930,6 +931,111 @@ public class MasterCoprocessorHost
}
});
}
public void preSetUserQuota(final String user, final Quotas quotas) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.preSetUserQuota(ctx, user, quotas);
}
});
}
public void postSetUserQuota(final String user, final Quotas quotas) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.postSetUserQuota(ctx, user, quotas);
}
});
}
public void preSetUserQuota(final String user, final TableName table, final Quotas quotas)
throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.preSetUserQuota(ctx, user, table, quotas);
}
});
}
public void postSetUserQuota(final String user, final TableName table, final Quotas quotas)
throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.postSetUserQuota(ctx, user, table, quotas);
}
});
}
public void preSetUserQuota(final String user, final String namespace, final Quotas quotas)
throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.preSetUserQuota(ctx, user, namespace, quotas);
}
});
}
public void postSetUserQuota(final String user, final String namespace, final Quotas quotas)
throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.postSetUserQuota(ctx, user, namespace, quotas);
}
});
}
public void preSetTableQuota(final TableName table, final Quotas quotas) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.preSetTableQuota(ctx, table, quotas);
}
});
}
public void postSetTableQuota(final TableName table, final Quotas quotas) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.postSetTableQuota(ctx, table, quotas);
}
});
}
public void preSetNamespaceQuota(final String namespace, final Quotas quotas) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.preSetNamespaceQuota(ctx, namespace, quotas);
}
});
}
public void postSetNamespaceQuota(final String namespace, final Quotas quotas)
throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
oserver.postSetNamespaceQuota(ctx, namespace, quotas);
}
});
}
private static abstract class CoprocessorOperation
extends ObserverContext<MasterCoprocessorEnvironment> {

View File

@ -129,6 +129,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanReq
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
@ -1290,4 +1292,14 @@ public class MasterRpcServices extends RSRpcServices
response.setEnabled(master.isBalancerOn());
return response.build();
}
@Override
public SetQuotaResponse setQuota(RpcController c, SetQuotaRequest req) throws ServiceException {
try {
master.checkInitialized();
return master.getMasterQuotaManager().setQuota(req);
} catch (Exception e) {
throw new ServiceException(e);
}
}
}

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
import com.google.protobuf.Service;
@ -266,4 +267,10 @@ public interface MasterServices extends Server {
* @throws IOException
*/
public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException;
/**
* @return Master's instance of {@link MasterQuotaManager}
*/
MasterQuotaManager getMasterQuotaManager();
}

View File

@ -0,0 +1,135 @@
/**
* 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.util.Arrays;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Result;
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class DefaultOperationQuota implements OperationQuota {
private static final Log LOG = LogFactory.getLog(DefaultOperationQuota.class);
private final List<QuotaLimiter> limiters;
private long writeAvailable = 0;
private long readAvailable = 0;
private long writeConsumed = 0;
private long readConsumed = 0;
private AvgOperationSize avgOpSize = new AvgOperationSize();
public DefaultOperationQuota(final QuotaLimiter... limiters) {
this(Arrays.asList(limiters));
}
/**
* NOTE: The order matters. It should be something like [user, table, namespace, global]
*/
public DefaultOperationQuota(final List<QuotaLimiter> limiters) {
this.limiters = limiters;
}
@Override
public void checkQuota(int numWrites, int numReads, int numScans) throws ThrottlingException {
writeConsumed = estimateConsume(OperationType.MUTATE, numWrites, 100);
readConsumed = estimateConsume(OperationType.GET, numReads, 100);
readConsumed += estimateConsume(OperationType.SCAN, numScans, 1000);
writeAvailable = Long.MAX_VALUE;
readAvailable = Long.MAX_VALUE;
for (final QuotaLimiter limiter : limiters) {
if (limiter.isBypass()) continue;
limiter.checkQuota(writeConsumed, readConsumed);
readAvailable = Math.min(readAvailable, limiter.getReadAvailable());
writeAvailable = Math.min(writeAvailable, limiter.getWriteAvailable());
}
for (final QuotaLimiter limiter : limiters) {
limiter.grabQuota(writeConsumed, readConsumed);
}
}
@Override
public void close() {
// Calculate and set the average size of get, scan and mutate for the current operation
long getSize = avgOpSize.getAvgOperationSize(OperationType.GET);
long scanSize = avgOpSize.getAvgOperationSize(OperationType.SCAN);
long mutationSize = avgOpSize.getAvgOperationSize(OperationType.MUTATE);
for (final QuotaLimiter limiter : limiters) {
limiter.addOperationSize(OperationType.GET, getSize);
limiter.addOperationSize(OperationType.SCAN, scanSize);
limiter.addOperationSize(OperationType.MUTATE, mutationSize);
}
// Adjust the quota consumed for the specified operation
long writeDiff = avgOpSize.getOperationSize(OperationType.MUTATE) - writeConsumed;
long readDiff =
(avgOpSize.getOperationSize(OperationType.GET) + avgOpSize
.getOperationSize(OperationType.SCAN)) - readConsumed;
for (final QuotaLimiter limiter : limiters) {
if (writeDiff != 0) limiter.consumeWrite(writeDiff);
if (readDiff != 0) limiter.consumeRead(readDiff);
}
}
@Override
public long getReadAvailable() {
return readAvailable;
}
@Override
public long getWriteAvailable() {
return writeAvailable;
}
@Override
public void addGetResult(final Result result) {
avgOpSize.addGetResult(result);
}
@Override
public void addScanResult(final List<Result> results) {
avgOpSize.addScanResult(results);
}
@Override
public void addMutation(final Mutation mutation) {
avgOpSize.addMutation(mutation);
}
@Override
public long getAvgOperationSize(OperationType type) {
return avgOpSize.getAvgOperationSize(type);
}
private long estimateConsume(final OperationType type, int numReqs, long avgSize) {
if (numReqs > 0) {
for (final QuotaLimiter limiter : limiters) {
long size = limiter.getAvgOperationSize(type);
if (size > 0) {
avgSize = size;
break;
}
}
return avgSize * numReqs;
}
return 0;
}
}

View File

@ -0,0 +1,441 @@
/**
* 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.HashSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
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.master.MasterServices;
import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaResponse;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Throttle;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.ThrottleRequest;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota;
/**
* Master Quota Manager. It is responsible for initialize the quota table on the first-run and
* provide the admin operations to interact with the quota table. TODO: FUTURE: The master will be
* responsible to notify each RS of quota changes and it will do the "quota aggregation" when the
* QuotaScope is CLUSTER.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class MasterQuotaManager {
private static final Log LOG = LogFactory.getLog(MasterQuotaManager.class);
private final MasterServices masterServices;
private NamedLock<String> namespaceLocks;
private NamedLock<TableName> tableLocks;
private NamedLock<String> userLocks;
private boolean enabled = false;
public MasterQuotaManager(final MasterServices masterServices) {
this.masterServices = masterServices;
}
public void start() throws IOException {
// If the user doesn't want the quota support skip all the initializations.
if (!QuotaUtil.isQuotaEnabled(masterServices.getConfiguration())) {
LOG.info("Quota support disabled");
return;
}
// Create the quota table if missing
if (!MetaTableAccessor.tableExists(masterServices.getConnection(),
QuotaUtil.QUOTA_TABLE_NAME)) {
LOG.info("Quota table not found. Creating...");
createQuotaTable();
}
LOG.info("Initializing quota support");
namespaceLocks = new NamedLock<String>();
tableLocks = new NamedLock<TableName>();
userLocks = new NamedLock<String>();
enabled = true;
}
public void stop() {
}
public boolean isQuotaEnabled() {
return enabled;
}
/*
* ========================================================================== Admin operations to
* manage the quota table
*/
public SetQuotaResponse setQuota(final SetQuotaRequest req) throws IOException,
InterruptedException {
checkQuotaSupport();
if (req.hasUserName()) {
userLocks.lock(req.getUserName());
try {
if (req.hasTableName()) {
setUserQuota(req.getUserName(), ProtobufUtil.toTableName(req.getTableName()), req);
} else if (req.hasNamespace()) {
setUserQuota(req.getUserName(), req.getNamespace(), req);
} else {
setUserQuota(req.getUserName(), req);
}
} finally {
userLocks.unlock(req.getUserName());
}
} else if (req.hasTableName()) {
TableName table = ProtobufUtil.toTableName(req.getTableName());
tableLocks.lock(table);
try {
setTableQuota(table, req);
} finally {
tableLocks.unlock(table);
}
} else if (req.hasNamespace()) {
namespaceLocks.lock(req.getNamespace());
try {
setNamespaceQuota(req.getNamespace(), req);
} finally {
namespaceLocks.unlock(req.getNamespace());
}
} else {
throw new DoNotRetryIOException(new UnsupportedOperationException(
"a user, a table or a namespace must be specified"));
}
return SetQuotaResponse.newBuilder().build();
}
public void setUserQuota(final String userName, final SetQuotaRequest req) throws IOException,
InterruptedException {
setQuota(req, new SetQuotaOperations() {
@Override
public Quotas fetch() throws IOException {
return QuotaUtil.getUserQuota(masterServices.getConnection(), userName);
}
@Override
public void update(final Quotas quotas) throws IOException {
QuotaUtil.addUserQuota(masterServices.getConnection(), userName, quotas);
}
@Override
public void delete() throws IOException {
QuotaUtil.deleteUserQuota(masterServices.getConnection(), userName);
}
@Override
public void preApply(final Quotas quotas) throws IOException {
masterServices.getMasterCoprocessorHost().preSetUserQuota(userName, quotas);
}
@Override
public void postApply(final Quotas quotas) throws IOException {
masterServices.getMasterCoprocessorHost().postSetUserQuota(userName, quotas);
}
});
}
public void setUserQuota(final String userName, final TableName table, final SetQuotaRequest req)
throws IOException, InterruptedException {
setQuota(req, new SetQuotaOperations() {
@Override
public Quotas fetch() throws IOException {
return QuotaUtil.getUserQuota(masterServices.getConnection(), userName, table);
}
@Override
public void update(final Quotas quotas) throws IOException {
QuotaUtil.addUserQuota(masterServices.getConnection(), userName, table, quotas);
}
@Override
public void delete() throws IOException {
QuotaUtil.deleteUserQuota(masterServices.getConnection(), userName, table);
}
@Override
public void preApply(final Quotas quotas) throws IOException {
masterServices.getMasterCoprocessorHost().preSetUserQuota(userName, table, quotas);
}
@Override
public void postApply(final Quotas quotas) throws IOException {
masterServices.getMasterCoprocessorHost().postSetUserQuota(userName, table, quotas);
}
});
}
public void
setUserQuota(final String userName, final String namespace, final SetQuotaRequest req)
throws IOException, InterruptedException {
setQuota(req, new SetQuotaOperations() {
@Override
public Quotas fetch() throws IOException {
return QuotaUtil.getUserQuota(masterServices.getConnection(), userName, namespace);
}
@Override
public void update(final Quotas quotas) throws IOException {
QuotaUtil.addUserQuota(masterServices.getConnection(), userName, namespace, quotas);
}
@Override
public void delete() throws IOException {
QuotaUtil.deleteUserQuota(masterServices.getConnection(), userName, namespace);
}
@Override
public void preApply(final Quotas quotas) throws IOException {
masterServices.getMasterCoprocessorHost().preSetUserQuota(userName, namespace, quotas);
}
@Override
public void postApply(final Quotas quotas) throws IOException {
masterServices.getMasterCoprocessorHost().postSetUserQuota(userName, namespace, quotas);
}
});
}
public void setTableQuota(final TableName table, final SetQuotaRequest req) throws IOException,
InterruptedException {
setQuota(req, new SetQuotaOperations() {
@Override
public Quotas fetch() throws IOException {
return QuotaUtil.getTableQuota(masterServices.getConnection(), table);
}
@Override
public void update(final Quotas quotas) throws IOException {
QuotaUtil.addTableQuota(masterServices.getConnection(), table, quotas);
}
@Override
public void delete() throws IOException {
QuotaUtil.deleteTableQuota(masterServices.getConnection(), table);
}
@Override
public void preApply(final Quotas quotas) throws IOException {
masterServices.getMasterCoprocessorHost().preSetTableQuota(table, quotas);
}
@Override
public void postApply(final Quotas quotas) throws IOException {
masterServices.getMasterCoprocessorHost().postSetTableQuota(table, quotas);
}
});
}
public void setNamespaceQuota(final String namespace, final SetQuotaRequest req)
throws IOException, InterruptedException {
setQuota(req, new SetQuotaOperations() {
@Override
public Quotas fetch() throws IOException {
return QuotaUtil.getNamespaceQuota(masterServices.getConnection(), namespace);
}
@Override
public void update(final Quotas quotas) throws IOException {
QuotaUtil.addNamespaceQuota(masterServices.getConnection(), namespace, quotas);
}
@Override
public void delete() throws IOException {
QuotaUtil.deleteNamespaceQuota(masterServices.getConnection(), namespace);
}
@Override
public void preApply(final Quotas quotas) throws IOException {
masterServices.getMasterCoprocessorHost().preSetNamespaceQuota(namespace, quotas);
}
@Override
public void postApply(final Quotas quotas) throws IOException {
masterServices.getMasterCoprocessorHost().postSetNamespaceQuota(namespace, quotas);
}
});
}
private void setQuota(final SetQuotaRequest req, final SetQuotaOperations quotaOps)
throws IOException, InterruptedException {
if (req.hasRemoveAll() && req.getRemoveAll() == true) {
quotaOps.preApply(null);
quotaOps.delete();
quotaOps.postApply(null);
return;
}
// Apply quota changes
Quotas quotas = quotaOps.fetch();
quotaOps.preApply(quotas);
Quotas.Builder builder = (quotas != null) ? quotas.toBuilder() : Quotas.newBuilder();
if (req.hasThrottle()) applyThrottle(builder, req.getThrottle());
if (req.hasBypassGlobals()) applyBypassGlobals(builder, req.getBypassGlobals());
// Submit new changes
quotas = builder.build();
if (QuotaUtil.isEmptyQuota(quotas)) {
quotaOps.delete();
} else {
quotaOps.update(quotas);
}
quotaOps.postApply(quotas);
}
private static interface SetQuotaOperations {
Quotas fetch() throws IOException;
void delete() throws IOException;
void update(final Quotas quotas) throws IOException;
void preApply(final Quotas quotas) throws IOException;
void postApply(final Quotas quotas) throws IOException;
}
/*
* ========================================================================== Helpers to apply
* changes to the quotas
*/
private void applyThrottle(final Quotas.Builder quotas, final ThrottleRequest req)
throws IOException {
Throttle.Builder throttle;
if (req.hasType() && (req.hasTimedQuota() || quotas.hasThrottle())) {
// Validate timed quota if present
if (req.hasTimedQuota()) {
validateTimedQuota(req.getTimedQuota());
}
// apply the new settings
throttle = quotas.hasThrottle() ? quotas.getThrottle().toBuilder() : Throttle.newBuilder();
switch (req.getType()) {
case REQUEST_NUMBER:
if (req.hasTimedQuota()) {
throttle.setReqNum(req.getTimedQuota());
} else {
throttle.clearReqNum();
}
break;
case REQUEST_SIZE:
if (req.hasTimedQuota()) {
throttle.setReqSize(req.getTimedQuota());
} else {
throttle.clearReqSize();
}
break;
case WRITE_NUMBER:
if (req.hasTimedQuota()) {
throttle.setWriteNum(req.getTimedQuota());
} else {
throttle.clearWriteNum();
}
break;
case WRITE_SIZE:
if (req.hasTimedQuota()) {
throttle.setWriteSize(req.getTimedQuota());
} else {
throttle.clearWriteSize();
}
break;
case READ_NUMBER:
if (req.hasTimedQuota()) {
throttle.setReadNum(req.getTimedQuota());
} else {
throttle.clearReqNum();
}
break;
case READ_SIZE:
if (req.hasTimedQuota()) {
throttle.setReadSize(req.getTimedQuota());
} else {
throttle.clearReadSize();
}
break;
default:
throw new RuntimeException("Invalid throttle type: " + req.getType());
}
quotas.setThrottle(throttle.build());
} else {
quotas.clearThrottle();
}
}
private void applyBypassGlobals(final Quotas.Builder quotas, boolean bypassGlobals) {
if (bypassGlobals) {
quotas.setBypassGlobals(bypassGlobals);
} else {
quotas.clearBypassGlobals();
}
}
private void validateTimedQuota(final TimedQuota timedQuota) throws IOException {
if (timedQuota.getSoftLimit() < 1) {
throw new DoNotRetryIOException(new UnsupportedOperationException(
"The throttle limit must be greater then 0, got " + timedQuota.getSoftLimit()));
}
}
/*
* ========================================================================== Helpers
*/
private void checkQuotaSupport() throws IOException {
if (!enabled) {
throw new DoNotRetryIOException(new UnsupportedOperationException("quota support disabled"));
}
}
private void createQuotaTable() throws IOException {
HRegionInfo[] newRegions = new HRegionInfo[] { new HRegionInfo(QuotaUtil.QUOTA_TABLE_NAME) };
masterServices.getExecutorService()
.submit(
new CreateTableHandler(masterServices, masterServices.getMasterFileSystem(),
QuotaUtil.QUOTA_TABLE_DESC, masterServices.getConfiguration(), newRegions,
masterServices).prepare());
}
private static class NamedLock<T> {
private HashSet<T> locks = new HashSet<T>();
public void lock(final T name) throws InterruptedException {
synchronized (locks) {
while (locks.contains(name)) {
locks.wait();
}
locks.add(name);
}
}
public void unlock(final T name) {
synchronized (locks) {
locks.remove(name);
locks.notifyAll();
}
}
}
}

View File

@ -0,0 +1,76 @@
/**
* 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.util.List;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Result;
/**
* Noop operation quota returned when no quota is associated to the user/table
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
final class NoopOperationQuota implements OperationQuota {
private static OperationQuota instance = new NoopOperationQuota();
private NoopOperationQuota() {
// no-op
}
public static OperationQuota get() {
return instance;
}
@Override
public void checkQuota(int numWrites, int numReads, int numScans) throws ThrottlingException {
// no-op
}
@Override
public void close() {
// no-op
}
@Override
public void addGetResult(final Result result) {
// no-op
}
@Override
public void addScanResult(final List<Result> results) {
// no-op
}
@Override
public void addMutation(final Mutation mutation) {
// no-op
}
@Override
public long getReadAvailable() {
return Long.MAX_VALUE;
}
@Override
public long getWriteAvailable() {
return Long.MAX_VALUE;
}
@Override
public long getAvgOperationSize(OperationType type) {
return -1;
}
}

View File

@ -0,0 +1,82 @@
/**
* 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 org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.quotas.OperationQuota.OperationType;
/**
* Noop quota limiter returned when no limiter is associated to the user/table
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
final class NoopQuotaLimiter implements QuotaLimiter {
private static QuotaLimiter instance = new NoopQuotaLimiter();
private NoopQuotaLimiter() {
// no-op
}
@Override
public void checkQuota(long estimateWriteSize, long estimateReadSize) throws ThrottlingException {
// no-op
}
@Override
public void grabQuota(long writeSize, long readSize) {
// no-op
}
@Override
public void consumeWrite(final long size) {
// no-op
}
@Override
public void consumeRead(final long size) {
// no-op
}
@Override
public boolean isBypass() {
return true;
}
@Override
public long getWriteAvailable() {
throw new UnsupportedOperationException();
}
@Override
public long getReadAvailable() {
throw new UnsupportedOperationException();
}
@Override
public void addOperationSize(OperationType type, long size) {
}
@Override
public long getAvgOperationSize(OperationType type) {
return -1;
}
@Override
public String toString() {
return "NoopQuotaLimiter";
}
public static QuotaLimiter get() {
return instance;
}
}

View File

@ -0,0 +1,120 @@
/**
* 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.util.List;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Result;
/**
* Interface that allows to check the quota available for an operation.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public interface OperationQuota {
public enum OperationType {
MUTATE, GET, SCAN
}
/**
* Keeps track of the average data size of operations like get, scan, mutate
*/
public class AvgOperationSize {
private final long[] sizeSum;
private final long[] count;
public AvgOperationSize() {
int size = OperationType.values().length;
sizeSum = new long[size];
count = new long[size];
for (int i = 0; i < size; ++i) {
sizeSum[i] = 0;
count[i] = 0;
}
}
public void addOperationSize(OperationType type, long size) {
if (size > 0) {
int index = type.ordinal();
sizeSum[index] += size;
count[index]++;
}
}
public long getAvgOperationSize(OperationType type) {
int index = type.ordinal();
return count[index] > 0 ? sizeSum[index] / count[index] : 0;
}
public long getOperationSize(OperationType type) {
return sizeSum[type.ordinal()];
}
public void addGetResult(final Result result) {
long size = QuotaUtil.calculateResultSize(result);
addOperationSize(OperationType.GET, size);
}
public void addScanResult(final List<Result> results) {
long size = QuotaUtil.calculateResultSize(results);
addOperationSize(OperationType.SCAN, size);
}
public void addMutation(final Mutation mutation) {
long size = QuotaUtil.calculateMutationSize(mutation);
addOperationSize(OperationType.MUTATE, size);
}
}
/**
* Checks if it is possible to execute the specified operation. The quota will be estimated based
* on the number of operations to perform and the average size accumulated during time.
* @param numWrites number of write operation that will be performed
* @param numReads number of small-read operation that will be performed
* @param numScans number of long-read operation that will be performed
* @throws ThrottlingException if the operation cannot be performed
*/
void checkQuota(int numWrites, int numReads, int numScans) throws ThrottlingException;
/** Cleanup method on operation completion */
void close();
/**
* Add a get result. This will be used to calculate the exact quota and have a better short-read
* average size for the next time.
*/
void addGetResult(Result result);
/**
* Add a scan result. This will be used to calculate the exact quota and have a better long-read
* average size for the next time.
*/
void addScanResult(List<Result> results);
/**
* Add a mutation result. This will be used to calculate the exact quota and have a better
* mutation average size for the next time.
*/
void addMutation(Mutation mutation);
/** @return the number of bytes available to read to avoid exceeding the quota */
long getReadAvailable();
/** @return the number of bytes available to write to avoid exceeding the quota */
long getWriteAvailable();
/** @return the average data size of the specified operation */
long getAvgOperationSize(OperationType type);
}

View File

@ -0,0 +1,312 @@
/**
* 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<String, QuotaState> namespaceQuotaCache =
new ConcurrentHashMap<String, QuotaState>();
private final ConcurrentHashMap<TableName, QuotaState> tableQuotaCache =
new ConcurrentHashMap<TableName, QuotaState>();
private final ConcurrentHashMap<String, UserQuotaState> userQuotaCache =
new ConcurrentHashMap<String, UserQuotaState>();
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 <K> QuotaState
getQuotaState(final ConcurrentHashMap<K, QuotaState> 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<String, QuotaState> getNamespaceQuotaCache() {
return namespaceQuotaCache;
}
@VisibleForTesting
Map<TableName, QuotaState> getTableQuotaCache() {
return tableQuotaCache;
}
@VisibleForTesting
Map<String, UserQuotaState> 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.contains(table)) {
QuotaCache.this.tableQuotaCache.putIfAbsent(table, new QuotaState());
}
String ns = table.getNamespaceAsString();
if (!QuotaCache.this.namespaceQuotaCache.contains(ns)) {
QuotaCache.this.namespaceQuotaCache.putIfAbsent(ns, new QuotaState());
}
}
fetchNamespaceQuotaState();
fetchTableQuotaState();
fetchUserQuotaState();
lastUpdate = EnvironmentEdgeManager.currentTime();
}
private void fetchNamespaceQuotaState() {
fetch("namespace", QuotaCache.this.namespaceQuotaCache, new Fetcher<String, QuotaState>() {
@Override
public Get makeGet(final Map.Entry<String, QuotaState> entry) {
return QuotaUtil.makeGetForNamespaceQuotas(entry.getKey());
}
@Override
public Map<String, QuotaState> fetchEntries(final List<Get> gets) throws IOException {
return QuotaUtil.fetchNamespaceQuotas(rsServices.getConnection(), gets);
}
});
}
private void fetchTableQuotaState() {
fetch("table", QuotaCache.this.tableQuotaCache, new Fetcher<TableName, QuotaState>() {
@Override
public Get makeGet(final Map.Entry<TableName, QuotaState> entry) {
return QuotaUtil.makeGetForTableQuotas(entry.getKey());
}
@Override
public Map<TableName, QuotaState> fetchEntries(final List<Get> gets) throws IOException {
return QuotaUtil.fetchTableQuotas(rsServices.getConnection(), gets);
}
});
}
private void fetchUserQuotaState() {
final Set<String> namespaces = QuotaCache.this.namespaceQuotaCache.keySet();
final Set<TableName> tables = QuotaCache.this.tableQuotaCache.keySet();
fetch("user", QuotaCache.this.userQuotaCache, new Fetcher<String, UserQuotaState>() {
@Override
public Get makeGet(final Map.Entry<String, UserQuotaState> entry) {
return QuotaUtil.makeGetForUserQuotas(entry.getKey(), tables, namespaces);
}
@Override
public Map<String, UserQuotaState> fetchEntries(final List<Get> gets) throws IOException {
return QuotaUtil.fetchUserQuotas(rsServices.getConnection(), gets);
}
});
}
private <K, V extends QuotaState> void fetch(final String type,
final ConcurrentHashMap<K, V> quotasMap, final Fetcher<K, V> fetcher) {
long now = EnvironmentEdgeManager.currentTime();
long refreshPeriod = getPeriod();
long evictPeriod = refreshPeriod * EVICT_PERIOD_FACTOR;
// Find the quota entries to update
List<Get> gets = new ArrayList<Get>();
List<K> toRemove = new ArrayList<K>();
for (Map.Entry<K, V> 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<K, V> 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<Key, Value> {
Get makeGet(Map.Entry<Key, Value> entry);
Map<Key, Value> fetchEntries(List<Get> gets) throws IOException;
}
}

View File

@ -0,0 +1,80 @@
/**
* 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 org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.quotas.OperationQuota.OperationType;
/**
* Internal interface used to interact with the user/table quota.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public interface QuotaLimiter {
/**
* Checks if it is possible to execute the specified operation.
*
* @param estimateWriteSize the write size that will be checked against the available quota
* @param estimateReadSize the read size that will be checked against the available quota
* @throws ThrottlingException thrown if not enough avialable resources to perform operation.
*/
void checkQuota(long estimateWriteSize, long estimateReadSize)
throws ThrottlingException;
/**
* Removes the specified write and read amount from the quota.
* At this point the write and read amount will be an estimate,
* that will be later adjusted with a consumeWrite()/consumeRead() call.
*
* @param writeSize the write size that will be removed from the current quota
* @param readSize the read size that will be removed from the current quota
*/
void grabQuota(long writeSize, long readSize);
/**
* Removes or add back some write amount to the quota.
* (called at the end of an operation in case the estimate quota was off)
*/
void consumeWrite(long size);
/**
* Removes or add back some read amount to the quota.
* (called at the end of an operation in case the estimate quota was off)
*/
void consumeRead(long size);
/** @return true if the limiter is a noop */
boolean isBypass();
/** @return the number of bytes available to read to avoid exceeding the quota */
long getReadAvailable();
/** @return the number of bytes available to write to avoid exceeding the quota */
long getWriteAvailable();
/**
* Add the average size of the specified operation type.
* The average will be used as estimate for the next operations.
*/
void addOperationSize(OperationType type, long size);
/** @return the average data size of the specified operation */
long getAvgOperationSize(OperationType type);
}

View File

@ -0,0 +1,37 @@
/**
* 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 org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Throttle;
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class QuotaLimiterFactory {
private QuotaLimiterFactory() {
// utility class
}
public static QuotaLimiter fromThrottle(final Throttle throttle) {
return TimeBasedLimiter.fromThrottle(throttle);
}
public static QuotaLimiter update(final QuotaLimiter a, final QuotaLimiter b) {
if (a.getClass().equals(b.getClass()) && a instanceof TimeBasedLimiter) {
((TimeBasedLimiter) a).update(((TimeBasedLimiter) b));
return a;
}
throw new UnsupportedOperationException("TODO not implemented yet");
}
}

View File

@ -0,0 +1,114 @@
/**
* 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 org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* In-Memory state of table or namespace quotas
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class QuotaState {
private long lastUpdate = 0;
private long lastQuery = 0;
private QuotaLimiter globalLimiter = NoopQuotaLimiter.get();
public QuotaState() {
this(0);
}
public QuotaState(final long updateTs) {
lastUpdate = updateTs;
}
public synchronized long getLastUpdate() {
return lastUpdate;
}
public synchronized long getLastQuery() {
return lastQuery;
}
@Override
public synchronized String toString() {
StringBuilder builder = new StringBuilder();
builder.append("QuotaState(ts=" + getLastUpdate());
if (isBypass()) {
builder.append(" bypass");
} else {
if (globalLimiter != NoopQuotaLimiter.get()) {
// builder.append(" global-limiter");
builder.append(" " + globalLimiter);
}
}
builder.append(')');
return builder.toString();
}
/**
* @return true if there is no quota information associated to this object
*/
public synchronized boolean isBypass() {
return globalLimiter == NoopQuotaLimiter.get();
}
/**
* Setup the global quota information. (This operation is part of the QuotaState setup)
*/
public synchronized void setQuotas(final Quotas quotas) {
if (quotas.hasThrottle()) {
globalLimiter = QuotaLimiterFactory.fromThrottle(quotas.getThrottle());
} else {
globalLimiter = NoopQuotaLimiter.get();
}
}
/**
* Perform an update of the quota info based on the other quota info object. (This operation is
* executed by the QuotaCache)
*/
public synchronized void update(final QuotaState other) {
if (globalLimiter == NoopQuotaLimiter.get()) {
globalLimiter = other.globalLimiter;
} else if (other.globalLimiter == NoopQuotaLimiter.get()) {
globalLimiter = NoopQuotaLimiter.get();
} else {
globalLimiter = QuotaLimiterFactory.update(globalLimiter, other.globalLimiter);
}
lastUpdate = other.lastUpdate;
}
/**
* Return the limiter associated with this quota.
* @return the quota limiter
*/
public synchronized QuotaLimiter getGlobalLimiter() {
setLastQuery(EnvironmentEdgeManager.currentTime());
return globalLimiter;
}
/**
* Return the limiter associated with this quota without updating internal last query stats
* @return the quota limiter
*/
synchronized QuotaLimiter getGlobalLimiterWithoutUpdatingLastQuery() {
return globalLimiter;
}
public synchronized void setLastQuery(long lastQuery) {
this.lastQuery = lastQuery;
}
}

View File

@ -0,0 +1,297 @@
/**
* 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.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValueUtil;
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.Connection;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* Helper class to interact with the quota table
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class QuotaUtil extends QuotaTableUtil {
private static final Log LOG = LogFactory.getLog(QuotaUtil.class);
public static final String QUOTA_CONF_KEY = "hbase.quota.enabled";
private static final boolean QUOTA_ENABLED_DEFAULT = false;
/** Table descriptor for Quota internal table */
public static final HTableDescriptor QUOTA_TABLE_DESC = new HTableDescriptor(QUOTA_TABLE_NAME);
static {
QUOTA_TABLE_DESC.addFamily(new HColumnDescriptor(QUOTA_FAMILY_INFO)
.setScope(HConstants.REPLICATION_SCOPE_LOCAL).setBloomFilterType(BloomType.ROW)
.setMaxVersions(1));
QUOTA_TABLE_DESC.addFamily(new HColumnDescriptor(QUOTA_FAMILY_USAGE)
.setScope(HConstants.REPLICATION_SCOPE_LOCAL).setBloomFilterType(BloomType.ROW)
.setMaxVersions(1));
}
/** Returns true if the support for quota is enabled */
public static boolean isQuotaEnabled(final Configuration conf) {
return conf.getBoolean(QUOTA_CONF_KEY, QUOTA_ENABLED_DEFAULT);
}
/*
* ========================================================================= Quota "settings"
* helpers
*/
public static void addTableQuota(final Connection connection, final TableName table,
final Quotas data) throws IOException {
addQuotas(connection, getTableRowKey(table), data);
}
public static void deleteTableQuota(final Connection connection, final TableName table)
throws IOException {
deleteQuotas(connection, getTableRowKey(table));
}
public static void addNamespaceQuota(final Connection connection, final String namespace,
final Quotas data) throws IOException {
addQuotas(connection, getNamespaceRowKey(namespace), data);
}
public static void deleteNamespaceQuota(final Connection connection, final String namespace)
throws IOException {
deleteQuotas(connection, getNamespaceRowKey(namespace));
}
public static void
addUserQuota(final Connection connection, final String user, final Quotas data)
throws IOException {
addQuotas(connection, getUserRowKey(user), data);
}
public static void addUserQuota(final Connection connection, final String user,
final TableName table, final Quotas data) throws IOException {
addQuotas(connection, getUserRowKey(user), getSettingsQualifierForUserTable(table), data);
}
public static void addUserQuota(final Connection connection, final String user,
final String namespace, final Quotas data) throws IOException {
addQuotas(connection, getUserRowKey(user), getSettingsQualifierForUserNamespace(namespace),
data);
}
public static void deleteUserQuota(final Connection connection, final String user)
throws IOException {
deleteQuotas(connection, getUserRowKey(user));
}
public static void deleteUserQuota(final Connection connection, final String user,
final TableName table) throws IOException {
deleteQuotas(connection, getUserRowKey(user), getSettingsQualifierForUserTable(table));
}
public static void deleteUserQuota(final Connection connection, final String user,
final String namespace) throws IOException {
deleteQuotas(connection, getUserRowKey(user), getSettingsQualifierForUserNamespace(namespace));
}
private static void
addQuotas(final Connection connection, final byte[] rowKey, final Quotas data)
throws IOException {
addQuotas(connection, rowKey, QUOTA_QUALIFIER_SETTINGS, data);
}
private static void addQuotas(final Connection connection, final byte[] rowKey,
final byte[] qualifier, final Quotas data) throws IOException {
Put put = new Put(rowKey);
put.addColumn(QUOTA_FAMILY_INFO, qualifier, quotasToData(data));
doPut(connection, put);
}
private static void deleteQuotas(final Connection connection, final byte[] rowKey)
throws IOException {
deleteQuotas(connection, rowKey, null);
}
private static void deleteQuotas(final Connection connection, final byte[] rowKey,
final byte[] qualifier) throws IOException {
Delete delete = new Delete(rowKey);
if (qualifier != null) {
delete.addColumns(QUOTA_FAMILY_INFO, qualifier);
}
doDelete(connection, delete);
}
public static Map<String, UserQuotaState> fetchUserQuotas(final Connection connection,
final List<Get> gets) throws IOException {
long nowTs = EnvironmentEdgeManager.currentTime();
Result[] results = doGet(connection, gets);
Map<String, UserQuotaState> userQuotas = new HashMap<String, UserQuotaState>(results.length);
for (int i = 0; i < results.length; ++i) {
byte[] key = gets.get(i).getRow();
assert isUserRowKey(key);
String user = getUserFromRowKey(key);
final UserQuotaState quotaInfo = new UserQuotaState(nowTs);
userQuotas.put(user, quotaInfo);
if (results[i].isEmpty()) continue;
assert Bytes.equals(key, results[i].getRow());
try {
parseUserResult(user, results[i], new UserQuotasVisitor() {
@Override
public void visitUserQuotas(String userName, String namespace, Quotas quotas) {
quotaInfo.setQuotas(namespace, quotas);
}
@Override
public void visitUserQuotas(String userName, TableName table, Quotas quotas) {
quotaInfo.setQuotas(table, quotas);
}
@Override
public void visitUserQuotas(String userName, Quotas quotas) {
quotaInfo.setQuotas(quotas);
}
});
} catch (IOException e) {
LOG.error("Unable to parse user '" + user + "' quotas", e);
userQuotas.remove(user);
}
}
return userQuotas;
}
public static Map<TableName, QuotaState> fetchTableQuotas(final Connection connection,
final List<Get> gets) throws IOException {
return fetchGlobalQuotas("table", connection, gets, new KeyFromRow<TableName>() {
@Override
public TableName getKeyFromRow(final byte[] row) {
assert isTableRowKey(row);
return getTableFromRowKey(row);
}
});
}
public static Map<String, QuotaState> fetchNamespaceQuotas(final Connection connection,
final List<Get> gets) throws IOException {
return fetchGlobalQuotas("namespace", connection, gets, new KeyFromRow<String>() {
@Override
public String getKeyFromRow(final byte[] row) {
assert isNamespaceRowKey(row);
return getNamespaceFromRowKey(row);
}
});
}
public static <K> Map<K, QuotaState> fetchGlobalQuotas(final String type,
final Connection connection, final List<Get> gets, final KeyFromRow<K> kfr)
throws IOException {
long nowTs = EnvironmentEdgeManager.currentTime();
Result[] results = doGet(connection, gets);
Map<K, QuotaState> globalQuotas = new HashMap<K, QuotaState>(results.length);
for (int i = 0; i < results.length; ++i) {
byte[] row = gets.get(i).getRow();
K key = kfr.getKeyFromRow(row);
QuotaState quotaInfo = new QuotaState(nowTs);
globalQuotas.put(key, quotaInfo);
if (results[i].isEmpty()) continue;
assert Bytes.equals(row, results[i].getRow());
byte[] data = results[i].getValue(QUOTA_FAMILY_INFO, QUOTA_QUALIFIER_SETTINGS);
if (data == null) continue;
try {
Quotas quotas = quotasFromData(data);
quotaInfo.setQuotas(quotas);
} catch (IOException e) {
LOG.error("Unable to parse " + type + " '" + key + "' quotas", e);
globalQuotas.remove(key);
}
}
return globalQuotas;
}
private static interface KeyFromRow<T> {
T getKeyFromRow(final byte[] row);
}
/*
* ========================================================================= HTable helpers
*/
private static void doPut(final Connection connection, final Put put) throws IOException {
try (Table table = connection.getTable(QuotaUtil.QUOTA_TABLE_NAME)) {
table.put(put);
}
}
private static void doDelete(final Connection connection, final Delete delete)
throws IOException {
try (Table table = connection.getTable(QuotaUtil.QUOTA_TABLE_NAME)) {
table.delete(delete);
}
}
/*
* ========================================================================= Data Size Helpers
*/
public static long calculateMutationSize(final Mutation mutation) {
long size = 0;
for (Map.Entry<byte[], List<Cell>> entry : mutation.getFamilyCellMap().entrySet()) {
for (Cell cell : entry.getValue()) {
size += KeyValueUtil.length(cell);
}
}
return size;
}
public static long calculateResultSize(final Result result) {
long size = 0;
for (Cell cell : result.rawCells()) {
size += KeyValueUtil.length(cell);
}
return size;
}
public static long calculateResultSize(final List<Result> results) {
long size = 0;
for (Result result : results) {
for (Cell cell : result.rawCells()) {
size += KeyValueUtil.length(cell);
}
}
return size;
}
}

View File

@ -0,0 +1,165 @@
/**
* 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.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* Simple rate limiter. Usage Example: RateLimiter limiter = new RateLimiter(); // At this point you
* have a unlimited resource limiter limiter.set(10, TimeUnit.SECONDS); // set 10 resources/sec long
* lastTs = 0; // You need to keep track of the last update timestamp while (true) { long now =
* System.currentTimeMillis(); // call canExecute before performing resource consuming operation
* bool canExecute = limiter.canExecute(now, lastTs); // If there are no available resources, wait
* until one is available if (!canExecute) Thread.sleep(limiter.waitInterval()); // ...execute the
* work and consume the resource... limiter.consume(); }
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class RateLimiter {
private long tunit = 1000; // Timeunit factor for translating to ms.
private long limit = Long.MAX_VALUE; // The max value available resource units can be refilled to.
private long avail = Long.MAX_VALUE; // Currently available resource units
public RateLimiter() {
}
/**
* Set the RateLimiter max available resources and refill period.
* @param limit The max value available resource units can be refilled to.
* @param timeUnit Timeunit factor for translating to ms.
*/
public synchronized void set(final long limit, final TimeUnit timeUnit) {
switch (timeUnit) {
case NANOSECONDS:
throw new RuntimeException("Unsupported NANOSECONDS TimeUnit");
case MICROSECONDS:
throw new RuntimeException("Unsupported MICROSECONDS TimeUnit");
case MILLISECONDS:
tunit = 1;
break;
case SECONDS:
tunit = 1000;
break;
case MINUTES:
tunit = 60 * 1000;
break;
case HOURS:
tunit = 60 * 60 * 1000;
break;
case DAYS:
tunit = 24 * 60 * 60 * 1000;
break;
default:
throw new RuntimeException("Invalid TimeUnit " + timeUnit);
}
this.limit = limit;
this.avail = limit;
}
public String toString() {
if (limit == Long.MAX_VALUE) {
return "RateLimiter(Bypass)";
}
return "RateLimiter(avail=" + avail + " limit=" + limit + " tunit=" + tunit + ")";
}
/**
* Sets the current instance of RateLimiter to a new values. if current limit is smaller than the
* new limit, bump up the available resources. Otherwise allow clients to use up the previously
* available resources.
*/
public synchronized void update(final RateLimiter other) {
this.tunit = other.tunit;
if (this.limit < other.limit) {
this.avail += (other.limit - this.limit);
}
this.limit = other.limit;
}
public synchronized boolean isBypass() {
return limit == Long.MAX_VALUE;
}
public synchronized long getLimit() {
return limit;
}
public synchronized long getAvailable() {
return avail;
}
/**
* given the time interval, is there at least one resource available to allow execution?
* @param now the current timestamp
* @param lastTs the timestamp of the last update
* @return true if there is at least one resource available, otherwise false
*/
public boolean canExecute(final long now, final long lastTs) {
return canExecute(now, lastTs, 1);
}
/**
* given the time interval, are there enough available resources to allow execution?
* @param now the current timestamp
* @param lastTs the timestamp of the last update
* @param amount the number of required resources
* @return true if there are enough available resources, otherwise false
*/
public synchronized boolean canExecute(final long now, final long lastTs, final long amount) {
return avail >= amount ? true : refill(now, lastTs) >= amount;
}
/**
* consume one available unit.
*/
public void consume() {
consume(1);
}
/**
* consume amount available units.
* @param amount the number of units to consume
*/
public synchronized void consume(final long amount) {
this.avail -= amount;
}
/**
* @return estimate of the ms required to wait before being able to provide 1 resource.
*/
public long waitInterval() {
return waitInterval(1);
}
/**
* @return estimate of the ms required to wait before being able to provide "amount" resources.
*/
public synchronized long waitInterval(final long amount) {
// TODO Handle over quota?
return (amount <= avail) ? 0 : ((amount * tunit) / limit) - ((avail * tunit) / limit);
}
/**
* given the specified time interval, refill the avilable units to the proportionate to elapsed
* time or to the prespecified limit.
*/
private long refill(final long now, final long lastTs) {
long delta = (limit * (now - lastTs)) / tunit;
if (delta > 0) {
avail = Math.min(limit, avail + delta);
}
return avail;
}
}

View File

@ -0,0 +1,189 @@
/**
* 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.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
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.ipc.RpcScheduler;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.security.UserGroupInformation;
import com.google.common.annotations.VisibleForTesting;
/**
* Region Server Quota Manager. It is responsible to provide access to the quota information of each
* user/table. The direct user of this class is the RegionServer that will get and check the
* user/table quota for each operation (put, get, scan). For system tables and user/table with a
* quota specified, the quota check will be a noop.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class RegionServerQuotaManager {
private static final Log LOG = LogFactory.getLog(RegionServerQuotaManager.class);
private final RegionServerServices rsServices;
private QuotaCache quotaCache = null;
public RegionServerQuotaManager(final RegionServerServices rsServices) {
this.rsServices = rsServices;
}
public void start(final RpcScheduler rpcScheduler) throws IOException {
if (!QuotaUtil.isQuotaEnabled(rsServices.getConfiguration())) {
LOG.info("Quota support disabled");
return;
}
LOG.info("Initializing quota support");
// Initialize quota cache
quotaCache = new QuotaCache(rsServices);
quotaCache.start();
}
public void stop() {
if (isQuotaEnabled()) {
quotaCache.stop("shutdown");
}
}
public boolean isQuotaEnabled() {
return quotaCache != null;
}
@VisibleForTesting
QuotaCache getQuotaCache() {
return quotaCache;
}
/**
* Returns the quota for an operation.
* @param ugi the user that is executing the operation
* @param table the table where the operation will be executed
* @return the OperationQuota
*/
public OperationQuota getQuota(final UserGroupInformation ugi, final TableName table) {
if (isQuotaEnabled() && !table.isSystemTable()) {
UserQuotaState userQuotaState = quotaCache.getUserQuotaState(ugi);
QuotaLimiter userLimiter = userQuotaState.getTableLimiter(table);
boolean useNoop = userLimiter.isBypass();
if (userQuotaState.hasBypassGlobals()) {
if (LOG.isTraceEnabled()) {
LOG.trace("get quota for ugi=" + ugi + " table=" + table + " userLimiter=" + userLimiter);
}
if (!useNoop) {
return new DefaultOperationQuota(userLimiter);
}
} else {
QuotaLimiter nsLimiter = quotaCache.getNamespaceLimiter(table.getNamespaceAsString());
QuotaLimiter tableLimiter = quotaCache.getTableLimiter(table);
useNoop &= tableLimiter.isBypass() && nsLimiter.isBypass();
if (LOG.isTraceEnabled()) {
LOG.trace("get quota for ugi=" + ugi + " table=" + table + " userLimiter=" + userLimiter
+ " tableLimiter=" + tableLimiter + " nsLimiter=" + nsLimiter);
}
if (!useNoop) {
return new DefaultOperationQuota(userLimiter, tableLimiter, nsLimiter);
}
}
}
return NoopOperationQuota.get();
}
/**
* Check the quota for the current (rpc-context) user. Returns the OperationQuota used to get the
* available quota and to report the data/usage of the operation.
* @param region the region where the operation will be performed
* @param type the operation type
* @return the OperationQuota
* @throws ThrottlingException if the operation cannot be executed due to quota exceeded.
*/
public OperationQuota checkQuota(final Region region, final OperationQuota.OperationType type)
throws IOException, ThrottlingException {
switch (type) {
case SCAN:
return checkQuota(region, 0, 0, 1);
case GET:
return checkQuota(region, 0, 1, 0);
case MUTATE:
return checkQuota(region, 1, 0, 0);
default:
throw new RuntimeException("Invalid operation type: " + type);
}
}
/**
* Check the quota for the current (rpc-context) user. Returns the OperationQuota used to get the
* available quota and to report the data/usage of the operation.
* @param region the region where the operation will be performed
* @param actions the "multi" actions to perform
* @return the OperationQuota
* @throws ThrottlingException if the operation cannot be executed due to quota exceeded.
*/
public OperationQuota checkQuota(final Region region, final List<ClientProtos.Action> actions)
throws IOException, ThrottlingException {
int numWrites = 0;
int numReads = 0;
for (final ClientProtos.Action action : actions) {
if (action.hasMutation()) {
numWrites++;
} else if (action.hasGet()) {
numReads++;
}
}
return checkQuota(region, numWrites, numReads, 0);
}
/**
* Check the quota for the current (rpc-context) user. Returns the OperationQuota used to get the
* available quota and to report the data/usage of the operation.
* @param region the region where the operation will be performed
* @param numWrites number of writes to perform
* @param numReads number of short-reads to perform
* @param numScans number of scan to perform
* @return the OperationQuota
* @throws ThrottlingException if the operation cannot be executed due to quota exceeded.
*/
private OperationQuota checkQuota(final Region region, final int numWrites, final int numReads,
final int numScans) throws IOException, ThrottlingException {
User user = RpcServer.getRequestUser();
UserGroupInformation ugi;
if (user != null) {
ugi = user.getUGI();
} else {
ugi = User.getCurrent().getUGI();
}
TableName table = region.getTableDesc().getTableName();
OperationQuota quota = getQuota(ugi, table);
try {
quota.checkQuota(numWrites, numReads, numScans);
} catch (ThrottlingException e) {
LOG.debug("Throttling exception for user=" + ugi.getUserName() + " table=" + table
+ " numWrites=" + numWrites + " numReads=" + numReads + " numScans=" + numScans + ": "
+ e.getMessage());
throw e;
}
return quota;
}
}

View File

@ -0,0 +1,194 @@
/**
* 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 org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Throttle;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.TimedQuota;
import org.apache.hadoop.hbase.quotas.OperationQuota.AvgOperationSize;
import org.apache.hadoop.hbase.quotas.OperationQuota.OperationType;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* Simple time based limiter that checks the quota Throttle
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public final class TimeBasedLimiter implements QuotaLimiter {
private long writeLastTs = 0;
private long readLastTs = 0;
private RateLimiter reqsLimiter = new RateLimiter();
private RateLimiter reqSizeLimiter = new RateLimiter();
private RateLimiter writeReqsLimiter = new RateLimiter();
private RateLimiter writeSizeLimiter = new RateLimiter();
private RateLimiter readReqsLimiter = new RateLimiter();
private RateLimiter readSizeLimiter = new RateLimiter();
private AvgOperationSize avgOpSize = new AvgOperationSize();
private TimeBasedLimiter() {
}
static QuotaLimiter fromThrottle(final Throttle throttle) {
TimeBasedLimiter limiter = new TimeBasedLimiter();
boolean isBypass = true;
if (throttle.hasReqNum()) {
setFromTimedQuota(limiter.reqsLimiter, throttle.getReqNum());
isBypass = false;
}
if (throttle.hasReqSize()) {
setFromTimedQuota(limiter.reqSizeLimiter, throttle.getReqSize());
isBypass = false;
}
if (throttle.hasWriteNum()) {
setFromTimedQuota(limiter.writeReqsLimiter, throttle.getWriteNum());
isBypass = false;
}
if (throttle.hasWriteSize()) {
setFromTimedQuota(limiter.writeSizeLimiter, throttle.getWriteSize());
isBypass = false;
}
if (throttle.hasReadNum()) {
setFromTimedQuota(limiter.readReqsLimiter, throttle.getReadNum());
isBypass = false;
}
if (throttle.hasReadSize()) {
setFromTimedQuota(limiter.readSizeLimiter, throttle.getReadSize());
isBypass = false;
}
return isBypass ? NoopQuotaLimiter.get() : limiter;
}
public void update(final TimeBasedLimiter other) {
reqsLimiter.update(other.reqsLimiter);
reqSizeLimiter.update(other.reqSizeLimiter);
writeReqsLimiter.update(other.writeReqsLimiter);
writeSizeLimiter.update(other.writeSizeLimiter);
readReqsLimiter.update(other.readReqsLimiter);
readSizeLimiter.update(other.readSizeLimiter);
}
private static void setFromTimedQuota(final RateLimiter limiter, final TimedQuota timedQuota) {
limiter.set(timedQuota.getSoftLimit(), ProtobufUtil.toTimeUnit(timedQuota.getTimeUnit()));
}
@Override
public void checkQuota(long writeSize, long readSize) throws ThrottlingException {
long now = EnvironmentEdgeManager.currentTime();
long lastTs = Math.max(readLastTs, writeLastTs);
if (!reqsLimiter.canExecute(now, lastTs)) {
ThrottlingException.throwNumRequestsExceeded(reqsLimiter.waitInterval());
}
if (!reqSizeLimiter.canExecute(now, lastTs, writeSize + readSize)) {
ThrottlingException.throwNumRequestsExceeded(reqSizeLimiter
.waitInterval(writeSize + readSize));
}
if (writeSize > 0) {
if (!writeReqsLimiter.canExecute(now, writeLastTs)) {
ThrottlingException.throwNumWriteRequestsExceeded(writeReqsLimiter.waitInterval());
}
if (!writeSizeLimiter.canExecute(now, writeLastTs, writeSize)) {
ThrottlingException.throwWriteSizeExceeded(writeSizeLimiter.waitInterval(writeSize));
}
}
if (readSize > 0) {
if (!readReqsLimiter.canExecute(now, readLastTs)) {
ThrottlingException.throwNumReadRequestsExceeded(readReqsLimiter.waitInterval());
}
if (!readSizeLimiter.canExecute(now, readLastTs, readSize)) {
ThrottlingException.throwReadSizeExceeded(readSizeLimiter.waitInterval(readSize));
}
}
}
@Override
public void grabQuota(long writeSize, long readSize) {
assert writeSize != 0 || readSize != 0;
long now = EnvironmentEdgeManager.currentTime();
reqsLimiter.consume(1);
reqSizeLimiter.consume(writeSize + readSize);
if (writeSize > 0) {
writeReqsLimiter.consume(1);
writeSizeLimiter.consume(writeSize);
writeLastTs = now;
}
if (readSize > 0) {
readReqsLimiter.consume(1);
readSizeLimiter.consume(readSize);
readLastTs = now;
}
}
@Override
public void consumeWrite(final long size) {
reqSizeLimiter.consume(size);
writeSizeLimiter.consume(size);
}
@Override
public void consumeRead(final long size) {
reqSizeLimiter.consume(size);
readSizeLimiter.consume(size);
}
@Override
public boolean isBypass() {
return false;
}
@Override
public long getWriteAvailable() {
return writeSizeLimiter.getAvailable();
}
@Override
public long getReadAvailable() {
return readSizeLimiter.getAvailable();
}
@Override
public void addOperationSize(OperationType type, long size) {
avgOpSize.addOperationSize(type, size);
}
@Override
public long getAvgOperationSize(OperationType type) {
return avgOpSize.getAvgOperationSize(type);
}
@Override
public String toString() {
StringBuilder builder = new StringBuilder();
builder.append("TimeBasedLimiter(");
if (!reqsLimiter.isBypass()) builder.append("reqs=" + reqsLimiter);
if (!reqSizeLimiter.isBypass()) builder.append(" resSize=" + reqSizeLimiter);
if (!writeReqsLimiter.isBypass()) builder.append(" writeReqs=" + writeReqsLimiter);
if (!writeSizeLimiter.isBypass()) builder.append(" writeSize=" + writeSizeLimiter);
if (!readReqsLimiter.isBypass()) builder.append(" readReqs=" + readReqsLimiter);
if (!readSizeLimiter.isBypass()) builder.append(" readSize=" + readSizeLimiter);
builder.append(')');
return builder.toString();
}
}

View File

@ -0,0 +1,193 @@
/**
* 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.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
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.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* In-Memory state of the user quotas
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class UserQuotaState extends QuotaState {
private Map<String, QuotaLimiter> namespaceLimiters = null;
private Map<TableName, QuotaLimiter> tableLimiters = null;
private boolean bypassGlobals = false;
public UserQuotaState() {
super();
}
public UserQuotaState(final long updateTs) {
super(updateTs);
}
@Override
public synchronized String toString() {
StringBuilder builder = new StringBuilder();
builder.append("UserQuotaState(ts=" + getLastUpdate());
if (bypassGlobals) builder.append(" bypass-globals");
if (isBypass()) {
builder.append(" bypass");
} else {
if (getGlobalLimiterWithoutUpdatingLastQuery() != NoopQuotaLimiter.get()) {
builder.append(" global-limiter");
}
if (tableLimiters != null && !tableLimiters.isEmpty()) {
builder.append(" [");
for (TableName table : tableLimiters.keySet()) {
builder.append(" " + table);
}
builder.append(" ]");
}
if (namespaceLimiters != null && !namespaceLimiters.isEmpty()) {
builder.append(" [");
for (String ns : namespaceLimiters.keySet()) {
builder.append(" " + ns);
}
builder.append(" ]");
}
}
builder.append(')');
return builder.toString();
}
/**
* @return true if there is no quota information associated to this object
*/
@Override
public synchronized boolean isBypass() {
return !bypassGlobals && getGlobalLimiterWithoutUpdatingLastQuery() == NoopQuotaLimiter.get()
&& (tableLimiters == null || tableLimiters.isEmpty())
&& (namespaceLimiters == null || namespaceLimiters.isEmpty());
}
public synchronized boolean hasBypassGlobals() {
return bypassGlobals;
}
@Override
public synchronized void setQuotas(final Quotas quotas) {
super.setQuotas(quotas);
bypassGlobals = quotas.getBypassGlobals();
}
/**
* Add the quota information of the specified table. (This operation is part of the QuotaState
* setup)
*/
public synchronized void setQuotas(final TableName table, Quotas quotas) {
tableLimiters = setLimiter(tableLimiters, table, quotas);
}
/**
* Add the quota information of the specified namespace. (This operation is part of the QuotaState
* setup)
*/
public synchronized void setQuotas(final String namespace, Quotas quotas) {
namespaceLimiters = setLimiter(namespaceLimiters, namespace, quotas);
}
private <K> Map<K, QuotaLimiter> setLimiter(Map<K, QuotaLimiter> limiters, final K key,
final Quotas quotas) {
if (limiters == null) {
limiters = new HashMap<K, QuotaLimiter>();
}
QuotaLimiter limiter =
quotas.hasThrottle() ? QuotaLimiterFactory.fromThrottle(quotas.getThrottle()) : null;
if (limiter != null && !limiter.isBypass()) {
limiters.put(key, limiter);
} else {
limiters.remove(key);
}
return limiters;
}
/**
* Perform an update of the quota state based on the other quota state object. (This operation is
* executed by the QuotaCache)
*/
@Override
public synchronized void update(final QuotaState other) {
super.update(other);
if (other instanceof UserQuotaState) {
UserQuotaState uOther = (UserQuotaState) other;
tableLimiters = updateLimiters(tableLimiters, uOther.tableLimiters);
namespaceLimiters = updateLimiters(namespaceLimiters, uOther.namespaceLimiters);
bypassGlobals = uOther.bypassGlobals;
} else {
tableLimiters = null;
namespaceLimiters = null;
bypassGlobals = false;
}
}
private static <K> Map<K, QuotaLimiter> updateLimiters(final Map<K, QuotaLimiter> map,
final Map<K, QuotaLimiter> otherMap) {
if (map == null) {
return otherMap;
}
if (otherMap != null) {
// To Remove
Set<K> toRemove = new HashSet<K>(map.keySet());
toRemove.removeAll(otherMap.keySet());
map.keySet().removeAll(toRemove);
// To Update/Add
for (final Map.Entry<K, QuotaLimiter> entry : otherMap.entrySet()) {
QuotaLimiter limiter = map.get(entry.getKey());
if (limiter == null) {
limiter = entry.getValue();
} else {
limiter = QuotaLimiterFactory.update(limiter, entry.getValue());
}
map.put(entry.getKey(), limiter);
}
return map;
}
return null;
}
/**
* Return the limiter for the specified table associated with this quota. If the table does not
* have its own quota limiter the global one will be returned. In case there is no quota limiter
* associated with this object a noop limiter will be returned.
* @return the quota limiter for the specified table
*/
public synchronized QuotaLimiter getTableLimiter(final TableName table) {
setLastQuery(EnvironmentEdgeManager.currentTime());
if (tableLimiters != null) {
QuotaLimiter limiter = tableLimiters.get(table);
if (limiter != null) return limiter;
}
if (namespaceLimiters != null) {
QuotaLimiter limiter = namespaceLimiters.get(table.getNamespaceAsString());
if (limiter != null) return limiter;
}
return getGlobalLimiterWithoutUpdatingLastQuery();
}
}

View File

@ -130,6 +130,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Regio
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
@ -431,6 +432,8 @@ public class HRegionServer extends HasThread implements
private RegionServerCoprocessorHost rsHost;
private RegionServerProcedureManagerHost rspmHost;
private RegionServerQuotaManager rsQuotaManager;
// Table level lock manager for locking for region operations
protected TableLockManager tableLockManager;
@ -825,6 +828,9 @@ public class HRegionServer extends HasThread implements
nonceManagerChore = this.nonceManager.createCleanupScheduledChore(this);
}
// Setup the Quota Manager
rsQuotaManager = new RegionServerQuotaManager(this);
// Setup RPC client for master communication
rpcClient = RpcClientFactory.createClient(conf, clusterId, new InetSocketAddress(
rpcServices.isa.getAddress(), 0));
@ -891,6 +897,9 @@ public class HRegionServer extends HasThread implements
// since the server is ready to run
rspmHost.start();
}
// Start the Quota Manager
rsQuotaManager.start(getRpcServer().getScheduler());
// We registered with the Master. Go into run mode.
long lastMsg = System.currentTimeMillis();
@ -976,6 +985,11 @@ public class HRegionServer extends HasThread implements
if (this.nonceManagerChore != null) this.nonceManagerChore.cancel(true);
if (this.storefileRefresher != null) this.storefileRefresher.cancel(true);
// Stop the quota manager
if (rsQuotaManager != null) {
rsQuotaManager.stop();
}
// Stop the snapshot and other procedure handlers, forcefully killing all running tasks
if (rspmHost != null) {
rspmHost.stop(this.abortRequested || this.killed);
@ -2486,6 +2500,11 @@ public class HRegionServer extends HasThread implements
public ChoreService getChoreService() {
return choreService;
}
@Override
public RegionServerQuotaManager getRegionServerQuotaManager() {
return rsQuotaManager;
}
//
// Main program and support routines
@ -2604,6 +2623,22 @@ public class HRegionServer extends HasThread implements
}
return tableRegions;
}
/**
* Gets the online tables in this RS.
* This method looks at the in-memory onlineRegions.
* @return all the online tables in this RS
*/
@Override
public Set<TableName> getOnlineTables() {
Set<TableName> tables = new HashSet<TableName>();
synchronized (this.onlineRegions) {
for (Region region: this.onlineRegions.values()) {
tables.add(region.getTableDesc().getTableName());
}
}
return tables;
}
// used by org/apache/hbase/tmpl/regionserver/RSStatusTmpl.jamon (HBASE-4070).
public String[] getRegionServerCoprocessors() {

View File

@ -152,6 +152,8 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
import org.apache.hadoop.hbase.quotas.OperationQuota;
import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
import org.apache.hadoop.hbase.regionserver.Region.FlushResult;
@ -448,10 +450,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
* bypassed as indicated by RegionObserver, null otherwise
* @throws IOException
*/
private Result append(final Region region, final MutationProto m,
private Result append(final Region region, final OperationQuota quota, final MutationProto m,
final CellScanner cellScanner, long nonceGroup) throws IOException {
long before = EnvironmentEdgeManager.currentTime();
Append append = ProtobufUtil.toAppend(m, cellScanner);
quota.addMutation(append);
Result r = null;
if (region.getCoprocessorHost() != null) {
r = region.getCoprocessorHost().preAppend(append);
@ -484,10 +487,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
* @return the Result
* @throws IOException
*/
private Result increment(final Region region, final MutationProto mutation,
final CellScanner cells, long nonceGroup) throws IOException {
private Result increment(final Region region, final OperationQuota quota,
final MutationProto mutation, final CellScanner cells, long nonceGroup) throws IOException {
long before = EnvironmentEdgeManager.currentTime();
Increment increment = ProtobufUtil.toIncrement(mutation, cells);
quota.addMutation(increment);
Result r = null;
if (region.getCoprocessorHost() != null) {
r = region.getCoprocessorHost().preIncrement(increment);
@ -524,7 +528,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
* @return Return the <code>cellScanner</code> passed
*/
private List<CellScannable> doNonAtomicRegionMutation(final Region region,
final RegionAction actions, final CellScanner cellScanner,
final OperationQuota quota, final RegionAction actions, final CellScanner cellScanner,
final RegionActionResult.Builder builder, List<CellScannable> cellsToReturn, long nonceGroup) {
// Gather up CONTIGUOUS Puts and Deletes in this mutations List. Idea is that rather than do
// one at a time, we instead pass them in batch. Be aware that the corresponding
@ -557,15 +561,15 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
if (type != MutationType.PUT && type != MutationType.DELETE && mutations != null &&
!mutations.isEmpty()) {
// Flush out any Puts or Deletes already collected.
doBatchOp(builder, region, mutations, cellScanner);
doBatchOp(builder, region, quota, mutations, cellScanner);
mutations.clear();
}
switch (type) {
case APPEND:
r = append(region, action.getMutation(), cellScanner, nonceGroup);
r = append(region, quota, action.getMutation(), cellScanner, nonceGroup);
break;
case INCREMENT:
r = increment(region, action.getMutation(), cellScanner, nonceGroup);
r = increment(region, quota, action.getMutation(), cellScanner, nonceGroup);
break;
case PUT:
case DELETE:
@ -610,7 +614,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
// Finish up any outstanding mutations
if (mutations != null && !mutations.isEmpty()) {
doBatchOp(builder, region, mutations, cellScanner);
doBatchOp(builder, region, quota, mutations, cellScanner);
}
return cellsToReturn;
}
@ -623,6 +627,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
* @param mutations
*/
private void doBatchOp(final RegionActionResult.Builder builder, final Region region,
final OperationQuota quota,
final List<ClientProtos.Action> mutations, final CellScanner cells) {
Mutation[] mArray = new Mutation[mutations.size()];
long before = EnvironmentEdgeManager.currentTime();
@ -640,6 +645,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
batchContainsDelete = true;
}
mArray[i++] = mutation;
quota.addMutation(mutation);
}
if (!region.getRegionInfo().isMetaTable()) {
@ -893,6 +899,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
Configuration getConfiguration() {
return regionServer.getConfiguration();
}
private RegionServerQuotaManager getQuotaManager() {
return regionServer.getRegionServerQuotaManager();
}
void start() {
rpcServer.start();
@ -1813,6 +1823,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
public GetResponse get(final RpcController controller,
final GetRequest request) throws ServiceException {
long before = EnvironmentEdgeManager.currentTime();
OperationQuota quota = null;
try {
checkOpen();
requestCount.increment();
@ -1822,6 +1833,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
ClientProtos.Get get = request.getGet();
Boolean existence = null;
Result r = null;
quota = getQuotaManager().checkQuota(region, OperationQuota.OperationType.GET);
if (get.hasClosestRowBefore() && get.getClosestRowBefore()) {
if (get.getColumnCount() != 1) {
@ -1856,6 +1868,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
ClientProtos.Result pbr = ProtobufUtil.toResult(r);
builder.setResult(pbr);
}
if (r != null) {
quota.addGetResult(r);
}
return builder.build();
} catch (IOException ie) {
throw new ServiceException(ie);
@ -1864,6 +1879,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
regionServer.metricsRegionServer.updateGet(
EnvironmentEdgeManager.currentTime() - before);
}
if (quota != null) {
quota.close();
}
}
}
@ -1899,10 +1917,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
for (RegionAction regionAction : request.getRegionActionList()) {
this.requestCount.add(regionAction.getActionCount());
OperationQuota quota;
Region region;
regionActionResultBuilder.clear();
try {
region = getRegion(regionAction.getRegion());
quota = getQuotaManager().checkQuota(region, regionAction.getActionList());
} catch (IOException e) {
regionActionResultBuilder.setException(ResponseConverter.buildException(e));
responseBuilder.addRegionActionResult(regionActionResultBuilder.build());
@ -1939,10 +1959,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
} else {
// doNonAtomicRegionMutation manages the exception internally
cellsToReturn = doNonAtomicRegionMutation(region, regionAction, cellScanner,
cellsToReturn = doNonAtomicRegionMutation(region, quota, regionAction, cellScanner,
regionActionResultBuilder, cellsToReturn, nonceGroup);
}
responseBuilder.addRegionActionResult(regionActionResultBuilder.build());
quota.close();
}
// Load the controller with the Cells to return.
if (cellsToReturn != null && !cellsToReturn.isEmpty() && controller != null) {
@ -1966,6 +1987,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
// It is also the conduit via which we pass back data.
PayloadCarryingRpcController controller = (PayloadCarryingRpcController)rpcc;
CellScanner cellScanner = controller != null? controller.cellScanner(): null;
OperationQuota quota = null;
// Clear scanner so we are not holding on to reference across call.
if (controller != null) controller.setCellScanner(null);
try {
@ -1981,17 +2003,20 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
Result r = null;
Boolean processed = null;
MutationType type = mutation.getMutateType();
long mutationSize = 0;
quota = getQuotaManager().checkQuota(region, OperationQuota.OperationType.MUTATE);
switch (type) {
case APPEND:
// TODO: this doesn't actually check anything.
r = append(region, mutation, cellScanner, nonceGroup);
r = append(region, quota, mutation, cellScanner, nonceGroup);
break;
case INCREMENT:
// TODO: this doesn't actually check anything.
r = increment(region, mutation, cellScanner, nonceGroup);
r = increment(region, quota, mutation, cellScanner, nonceGroup);
break;
case PUT:
Put put = ProtobufUtil.toPut(mutation, cellScanner);
quota.addMutation(put);
if (request.hasCondition()) {
Condition condition = request.getCondition();
byte[] row = condition.getRow().toByteArray();
@ -2020,6 +2045,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
break;
case DELETE:
Delete delete = ProtobufUtil.toDelete(mutation, cellScanner);
quota.addMutation(delete);
if (request.hasCondition()) {
Condition condition = request.getCondition();
byte[] row = condition.getRow().toByteArray();
@ -2056,6 +2082,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
} catch (IOException ie) {
regionServer.checkFileSystem();
throw new ServiceException(ie);
} finally {
if (quota != null) {
quota.close();
}
}
}
@ -2069,6 +2099,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
@Override
public ScanResponse scan(final RpcController controller, final ScanRequest request)
throws ServiceException {
OperationQuota quota = null;
Leases.Lease lease = null;
String scannerName = null;
try {
@ -2162,6 +2193,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
ttl = this.scannerLeaseTimeoutPeriod;
}
quota = getQuotaManager().checkQuota(region, OperationQuota.OperationType.SCAN);
long maxQuotaResultSize = Math.min(maxScannerResultSize, quota.getReadAvailable());
if (rows > 0) {
// if nextCallSeq does not match throw Exception straight away. This needs to be
// performed even before checking of Lease.
@ -2207,9 +2240,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
if (!done) {
long maxResultSize = scanner.getMaxResultSize();
long maxResultSize = Math.min(scanner.getMaxResultSize(), maxQuotaResultSize);
if (maxResultSize <= 0) {
maxResultSize = maxScannerResultSize;
maxResultSize = maxQuotaResultSize;
}
List<Cell> values = new ArrayList<Cell>();
region.startRegionOperation(Operation.SCAN);
@ -2302,6 +2335,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
region.getCoprocessorHost().postScannerNext(scanner, results, rows, true);
}
}
quota.addScanResult(results);
// If the scanner's filter - if any - is done with the scan
// and wants to tell the client to stop the scan. This is done by passing
@ -2362,6 +2397,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
}
throw new ServiceException(ie);
} finally {
if (quota != null) {
quota.close();
}
}
}

View File

@ -20,11 +20,13 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
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.executor.ExecutorService;
@ -32,6 +34,7 @@ import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.master.TableLockManager;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
import org.apache.zookeeper.KeeperException;
import com.google.protobuf.Service;
@ -70,6 +73,11 @@ public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegi
* @return RegionServer's instance of {@link TableLockManager}
*/
TableLockManager getTableLockManager();
/**
* @return RegionServer's instance of {@link RegionServerQuotaManager}
*/
RegionServerQuotaManager getRegionServerQuotaManager();
/**
* Tasks to perform after region open to complete deploy of region on
@ -148,4 +156,9 @@ public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegi
* @see org.apache.hadoop.hbase.regionserver.Store#getCompactionPressure()
*/
double getCompactionPressure();
/**
* @return all the online tables in this RS
*/
Set<TableName> getOnlineTables();
}

View File

@ -83,6 +83,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadRequest;
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.PrepareBulkLoadRequest;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
@ -2464,4 +2465,34 @@ public class AccessController extends BaseMasterAndRegionObserver
public void postReplicateLogEntries(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
List<WALEntry> entries, CellScanner cells) throws IOException {
}
@Override
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final Quotas quotas) throws IOException {
requirePermission("setUserQuota", Action.ADMIN);
}
@Override
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final TableName tableName, final Quotas quotas) throws IOException {
requirePermission("setUserTableQuota", tableName, null, null, Action.ADMIN);
}
@Override
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final String namespace, final Quotas quotas) throws IOException {
requirePermission("setUserNamespaceQuota", Action.ADMIN);
}
@Override
public void preSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName, final Quotas quotas) throws IOException {
requirePermission("setTableQuota", tableName, null, null, Action.ADMIN);
}
@Override
public void preSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String namespace, final Quotas quotas) throws IOException {
requirePermission("setNamespaceQuota", Action.ADMIN);
}
}

View File

@ -90,6 +90,7 @@ public class BoundedPriorityBlockingQueue<E> extends AbstractQueue<E> implements
public E poll() {
E elem = objects[head];
objects[head] = null;
head = (head + 1) % objects.length;
if (head == 0) tail = 0;
return elem;

View File

@ -22,6 +22,7 @@ import java.net.InetSocketAddress;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.atomic.AtomicBoolean;
@ -36,6 +37,7 @@ import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.master.TableLockManager;
import org.apache.hadoop.hbase.master.TableLockManager.NullTableLockManager;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
import org.apache.hadoop.hbase.regionserver.FlushRequester;
import org.apache.hadoop.hbase.regionserver.HeapMemoryManager;
@ -100,6 +102,11 @@ public class MockRegionServerServices implements RegionServerServices {
public List<Region> getOnlineRegions(TableName tableName) throws IOException {
return null;
}
@Override
public Set<TableName> getOnlineTables() {
return null;
}
@Override
public void addToOnlineRegions(Region r) {
@ -168,6 +175,11 @@ public class MockRegionServerServices implements RegionServerServices {
public TableLockManager getTableLockManager() {
return new NullTableLockManager();
}
@Override
public RegionServerQuotaManager getRegionServerQuotaManager() {
return null;
}
@Override
public ServerName getServerName() {

View File

@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
@ -1092,6 +1093,56 @@ public class TestMasterObserver {
public void postTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName) throws IOException {
}
@Override
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final Quotas quotas) throws IOException {
}
@Override
public void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final Quotas quotas) throws IOException {
}
@Override
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final TableName tableName, final Quotas quotas) throws IOException {
}
@Override
public void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final TableName tableName, final Quotas quotas) throws IOException {
}
@Override
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final String namespace, final Quotas quotas) throws IOException {
}
@Override
public void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String userName, final String namespace, final Quotas quotas) throws IOException {
}
@Override
public void preSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName, final Quotas quotas) throws IOException {
}
@Override
public void postSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName, final Quotas quotas) throws IOException {
}
@Override
public void preSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String namespace, final Quotas quotas) throws IOException {
}
@Override
public void postSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final String namespace, final Quotas quotas) throws IOException {
}
}
private static HBaseTestingUtility UTIL = new HBaseTestingUtility();

View File

@ -24,6 +24,7 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.ConcurrentSkipListMap;
@ -92,6 +93,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
import org.apache.hadoop.hbase.regionserver.FlushRequester;
import org.apache.hadoop.hbase.regionserver.HRegion;
@ -326,6 +328,11 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
public TableLockManager getTableLockManager() {
return new NullTableLockManager();
}
@Override
public RegionServerQuotaManager getRegionServerQuotaManager() {
return null;
}
@Override
public void postOpenDeployTasks(Region r) throws KeeperException, IOException {
@ -528,6 +535,11 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
// TODO Auto-generated method stub
return null;
}
@Override
public Set<TableName> getOnlineTables() {
return null;
}
@Override
public Leases getLeases() {

View File

@ -72,6 +72,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException;
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
@ -250,6 +251,11 @@ public class TestCatalogJanitor {
public MasterCoprocessorHost getMasterCoprocessorHost() {
return null;
}
@Override
public MasterQuotaManager getMasterQuotaManager() {
return null;
}
@Override
public ServerManager getServerManager() {

View File

@ -0,0 +1,199 @@
/**
* 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 static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* minicluster tests that validate that quota entries are properly set in the quota table
*/
@Category({ MediumTests.class })
public class TestQuotaAdmin {
final Log LOG = LogFactory.getLog(getClass());
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.getConfiguration().setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
TEST_UTIL.getConfiguration().setInt(QuotaCache.REFRESH_CONF_KEY, 2000);
TEST_UTIL.getConfiguration().setInt("hbase.hstore.compactionThreshold", 10);
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
TEST_UTIL.startMiniCluster(1);
TEST_UTIL.waitTableAvailable(QuotaTableUtil.QUOTA_TABLE_NAME);
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
@Test
public void testSimpleScan() throws Exception {
Admin admin = TEST_UTIL.getHBaseAdmin();
String userName = User.getCurrent().getShortName();
admin.setQuota(QuotaSettingsFactory.throttleUser(userName, ThrottleType.REQUEST_NUMBER, 6,
TimeUnit.MINUTES));
admin.setQuota(QuotaSettingsFactory.bypassGlobals(userName, true));
QuotaRetriever scanner = QuotaRetriever.open(TEST_UTIL.getConfiguration());
try {
int countThrottle = 0;
int countGlobalBypass = 0;
for (QuotaSettings settings : scanner) {
LOG.debug(settings);
switch (settings.getQuotaType()) {
case THROTTLE:
ThrottleSettings throttle = (ThrottleSettings) settings;
assertEquals(userName, throttle.getUserName());
assertEquals(null, throttle.getTableName());
assertEquals(null, throttle.getNamespace());
assertEquals(6, throttle.getSoftLimit());
assertEquals(TimeUnit.MINUTES, throttle.getTimeUnit());
countThrottle++;
break;
case GLOBAL_BYPASS:
countGlobalBypass++;
break;
default:
fail("unexpected settings type: " + settings.getQuotaType());
}
}
assertEquals(1, countThrottle);
assertEquals(1, countGlobalBypass);
} finally {
scanner.close();
}
admin.setQuota(QuotaSettingsFactory.unthrottleUser(userName));
assertNumResults(1, null);
admin.setQuota(QuotaSettingsFactory.bypassGlobals(userName, false));
assertNumResults(0, null);
}
@Test
public void testQuotaRetrieverFilter() throws Exception {
Admin admin = TEST_UTIL.getHBaseAdmin();
TableName[] tables =
new TableName[] { TableName.valueOf("T0"), TableName.valueOf("T01"),
TableName.valueOf("NS0:T2"), };
String[] namespaces = new String[] { "NS0", "NS01", "NS2" };
String[] users = new String[] { "User0", "User01", "User2" };
for (String user : users) {
admin.setQuota(QuotaSettingsFactory.throttleUser(user, ThrottleType.REQUEST_NUMBER, 1,
TimeUnit.MINUTES));
for (TableName table : tables) {
admin.setQuota(QuotaSettingsFactory.throttleUser(user, table, ThrottleType.REQUEST_NUMBER,
2, TimeUnit.MINUTES));
}
for (String ns : namespaces) {
admin.setQuota(QuotaSettingsFactory.throttleUser(user, ns, ThrottleType.REQUEST_NUMBER, 3,
TimeUnit.MINUTES));
}
}
assertNumResults(21, null);
for (TableName table : tables) {
admin.setQuota(QuotaSettingsFactory.throttleTable(table, ThrottleType.REQUEST_NUMBER, 4,
TimeUnit.MINUTES));
}
assertNumResults(24, null);
for (String ns : namespaces) {
admin.setQuota(QuotaSettingsFactory.throttleNamespace(ns, ThrottleType.REQUEST_NUMBER, 5,
TimeUnit.MINUTES));
}
assertNumResults(27, null);
assertNumResults(7, new QuotaFilter().setUserFilter("User0"));
assertNumResults(0, new QuotaFilter().setUserFilter("User"));
assertNumResults(21, new QuotaFilter().setUserFilter("User.*"));
assertNumResults(3, new QuotaFilter().setUserFilter("User.*").setTableFilter("T0"));
assertNumResults(3, new QuotaFilter().setUserFilter("User.*").setTableFilter("NS.*"));
assertNumResults(0, new QuotaFilter().setUserFilter("User.*").setTableFilter("T"));
assertNumResults(6, new QuotaFilter().setUserFilter("User.*").setTableFilter("T.*"));
assertNumResults(3, new QuotaFilter().setUserFilter("User.*").setNamespaceFilter("NS0"));
assertNumResults(0, new QuotaFilter().setUserFilter("User.*").setNamespaceFilter("NS"));
assertNumResults(9, new QuotaFilter().setUserFilter("User.*").setNamespaceFilter("NS.*"));
assertNumResults(6, new QuotaFilter().setUserFilter("User.*").setTableFilter("T0")
.setNamespaceFilter("NS0"));
assertNumResults(1, new QuotaFilter().setTableFilter("T0"));
assertNumResults(0, new QuotaFilter().setTableFilter("T"));
assertNumResults(2, new QuotaFilter().setTableFilter("T.*"));
assertNumResults(3, new QuotaFilter().setTableFilter(".*T.*"));
assertNumResults(1, new QuotaFilter().setNamespaceFilter("NS0"));
assertNumResults(0, new QuotaFilter().setNamespaceFilter("NS"));
assertNumResults(3, new QuotaFilter().setNamespaceFilter("NS.*"));
for (String user : users) {
admin.setQuota(QuotaSettingsFactory.unthrottleUser(user));
for (TableName table : tables) {
admin.setQuota(QuotaSettingsFactory.unthrottleUser(user, table));
}
for (String ns : namespaces) {
admin.setQuota(QuotaSettingsFactory.unthrottleUser(user, ns));
}
}
assertNumResults(6, null);
for (TableName table : tables) {
admin.setQuota(QuotaSettingsFactory.unthrottleTable(table));
}
assertNumResults(3, null);
for (String ns : namespaces) {
admin.setQuota(QuotaSettingsFactory.unthrottleNamespace(ns));
}
assertNumResults(0, null);
}
private void assertNumResults(int expected, final QuotaFilter filter) throws Exception {
assertEquals(expected, countResults(filter));
}
private int countResults(final QuotaFilter filter) throws Exception {
QuotaRetriever scanner = QuotaRetriever.open(TEST_UTIL.getConfiguration(), filter);
try {
int count = 0;
for (QuotaSettings settings : scanner) {
LOG.debug(settings);
count++;
}
return count;
} finally {
scanner.close();
}
}
}

View File

@ -0,0 +1,221 @@
/**
* 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 static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Throttle;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({ SmallTests.class })
public class TestQuotaState {
private static final TableName UNKNOWN_TABLE_NAME = TableName.valueOf("unknownTable");
@Test(timeout = 60000)
public void testQuotaStateBypass() {
QuotaState quotaInfo = new QuotaState();
assertTrue(quotaInfo.isBypass());
assertNoopLimiter(quotaInfo.getGlobalLimiter());
UserQuotaState userQuotaState = new UserQuotaState();
assertTrue(userQuotaState.isBypass());
assertNoopLimiter(userQuotaState.getTableLimiter(UNKNOWN_TABLE_NAME));
}
@Test(timeout = 60000)
public void testSimpleQuotaStateOperation() {
final TableName table = TableName.valueOf("testSimpleQuotaStateOperationTable");
final int NUM_GLOBAL_THROTTLE = 3;
final int NUM_TABLE_THROTTLE = 2;
UserQuotaState quotaInfo = new UserQuotaState();
assertTrue(quotaInfo.isBypass());
// Set global quota
quotaInfo.setQuotas(buildReqNumThrottle(NUM_GLOBAL_THROTTLE));
assertFalse(quotaInfo.isBypass());
// Set table quota
quotaInfo.setQuotas(table, buildReqNumThrottle(NUM_TABLE_THROTTLE));
assertFalse(quotaInfo.isBypass());
assertTrue(quotaInfo.getGlobalLimiter() == quotaInfo.getTableLimiter(UNKNOWN_TABLE_NAME));
assertThrottleException(quotaInfo.getTableLimiter(UNKNOWN_TABLE_NAME), NUM_GLOBAL_THROTTLE);
assertThrottleException(quotaInfo.getTableLimiter(table), NUM_TABLE_THROTTLE);
}
@Test(timeout = 60000)
public void testQuotaStateUpdateBypassThrottle() {
final long LAST_UPDATE = 10;
UserQuotaState quotaInfo = new UserQuotaState();
assertEquals(0, quotaInfo.getLastUpdate());
assertTrue(quotaInfo.isBypass());
UserQuotaState otherQuotaState = new UserQuotaState(LAST_UPDATE);
assertEquals(LAST_UPDATE, otherQuotaState.getLastUpdate());
assertTrue(otherQuotaState.isBypass());
quotaInfo.update(otherQuotaState);
assertEquals(LAST_UPDATE, quotaInfo.getLastUpdate());
assertTrue(quotaInfo.isBypass());
assertTrue(quotaInfo.getGlobalLimiter() == quotaInfo.getTableLimiter(UNKNOWN_TABLE_NAME));
assertNoopLimiter(quotaInfo.getTableLimiter(UNKNOWN_TABLE_NAME));
}
@Test(timeout = 60000)
public void testQuotaStateUpdateGlobalThrottle() {
final int NUM_GLOBAL_THROTTLE_1 = 3;
final int NUM_GLOBAL_THROTTLE_2 = 11;
final long LAST_UPDATE_1 = 10;
final long LAST_UPDATE_2 = 20;
final long LAST_UPDATE_3 = 30;
QuotaState quotaInfo = new QuotaState();
assertEquals(0, quotaInfo.getLastUpdate());
assertTrue(quotaInfo.isBypass());
// Add global throttle
QuotaState otherQuotaState = new QuotaState(LAST_UPDATE_1);
otherQuotaState.setQuotas(buildReqNumThrottle(NUM_GLOBAL_THROTTLE_1));
assertEquals(LAST_UPDATE_1, otherQuotaState.getLastUpdate());
assertFalse(otherQuotaState.isBypass());
quotaInfo.update(otherQuotaState);
assertEquals(LAST_UPDATE_1, quotaInfo.getLastUpdate());
assertFalse(quotaInfo.isBypass());
assertThrottleException(quotaInfo.getGlobalLimiter(), NUM_GLOBAL_THROTTLE_1);
// Update global Throttle
otherQuotaState = new QuotaState(LAST_UPDATE_2);
otherQuotaState.setQuotas(buildReqNumThrottle(NUM_GLOBAL_THROTTLE_2));
assertEquals(LAST_UPDATE_2, otherQuotaState.getLastUpdate());
assertFalse(otherQuotaState.isBypass());
quotaInfo.update(otherQuotaState);
assertEquals(LAST_UPDATE_2, quotaInfo.getLastUpdate());
assertFalse(quotaInfo.isBypass());
assertThrottleException(quotaInfo.getGlobalLimiter(), NUM_GLOBAL_THROTTLE_2
- NUM_GLOBAL_THROTTLE_1);
// Remove global throttle
otherQuotaState = new QuotaState(LAST_UPDATE_3);
assertEquals(LAST_UPDATE_3, otherQuotaState.getLastUpdate());
assertTrue(otherQuotaState.isBypass());
quotaInfo.update(otherQuotaState);
assertEquals(LAST_UPDATE_3, quotaInfo.getLastUpdate());
assertTrue(quotaInfo.isBypass());
assertNoopLimiter(quotaInfo.getGlobalLimiter());
}
@Test(timeout = 60000)
public void testQuotaStateUpdateTableThrottle() {
final TableName TABLE_A = TableName.valueOf("TableA");
final TableName TABLE_B = TableName.valueOf("TableB");
final TableName TABLE_C = TableName.valueOf("TableC");
final int TABLE_A_THROTTLE_1 = 3;
final int TABLE_A_THROTTLE_2 = 11;
final int TABLE_B_THROTTLE = 4;
final int TABLE_C_THROTTLE = 5;
final long LAST_UPDATE_1 = 10;
final long LAST_UPDATE_2 = 20;
final long LAST_UPDATE_3 = 30;
UserQuotaState quotaInfo = new UserQuotaState();
assertEquals(0, quotaInfo.getLastUpdate());
assertTrue(quotaInfo.isBypass());
// Add A B table limiters
UserQuotaState otherQuotaState = new UserQuotaState(LAST_UPDATE_1);
otherQuotaState.setQuotas(TABLE_A, buildReqNumThrottle(TABLE_A_THROTTLE_1));
otherQuotaState.setQuotas(TABLE_B, buildReqNumThrottle(TABLE_B_THROTTLE));
assertEquals(LAST_UPDATE_1, otherQuotaState.getLastUpdate());
assertFalse(otherQuotaState.isBypass());
quotaInfo.update(otherQuotaState);
assertEquals(LAST_UPDATE_1, quotaInfo.getLastUpdate());
assertFalse(quotaInfo.isBypass());
assertThrottleException(quotaInfo.getTableLimiter(TABLE_A), TABLE_A_THROTTLE_1);
assertThrottleException(quotaInfo.getTableLimiter(TABLE_B), TABLE_B_THROTTLE);
assertNoopLimiter(quotaInfo.getTableLimiter(TABLE_C));
// Add C, Remove B, Update A table limiters
otherQuotaState = new UserQuotaState(LAST_UPDATE_2);
otherQuotaState.setQuotas(TABLE_A, buildReqNumThrottle(TABLE_A_THROTTLE_2));
otherQuotaState.setQuotas(TABLE_C, buildReqNumThrottle(TABLE_C_THROTTLE));
assertEquals(LAST_UPDATE_2, otherQuotaState.getLastUpdate());
assertFalse(otherQuotaState.isBypass());
quotaInfo.update(otherQuotaState);
assertEquals(LAST_UPDATE_2, quotaInfo.getLastUpdate());
assertFalse(quotaInfo.isBypass());
assertThrottleException(quotaInfo.getTableLimiter(TABLE_A), TABLE_A_THROTTLE_2
- TABLE_A_THROTTLE_1);
assertThrottleException(quotaInfo.getTableLimiter(TABLE_C), TABLE_C_THROTTLE);
assertNoopLimiter(quotaInfo.getTableLimiter(TABLE_B));
// Remove table limiters
otherQuotaState = new UserQuotaState(LAST_UPDATE_3);
assertEquals(LAST_UPDATE_3, otherQuotaState.getLastUpdate());
assertTrue(otherQuotaState.isBypass());
quotaInfo.update(otherQuotaState);
assertEquals(LAST_UPDATE_3, quotaInfo.getLastUpdate());
assertTrue(quotaInfo.isBypass());
assertNoopLimiter(quotaInfo.getTableLimiter(UNKNOWN_TABLE_NAME));
}
private Quotas buildReqNumThrottle(final long limit) {
return Quotas
.newBuilder()
.setThrottle(
Throttle.newBuilder()
.setReqNum(ProtobufUtil.toTimedQuota(limit, TimeUnit.MINUTES, QuotaScope.MACHINE))
.build()).build();
}
private void assertThrottleException(final QuotaLimiter limiter, final int availReqs) {
assertNoThrottleException(limiter, availReqs);
try {
limiter.checkQuota(1, 1);
fail("Should have thrown ThrottlingException");
} catch (ThrottlingException e) {
// expected
}
}
private void assertNoThrottleException(final QuotaLimiter limiter, final int availReqs) {
for (int i = 0; i < availReqs; ++i) {
try {
limiter.checkQuota(1, 1);
} catch (ThrottlingException e) {
fail("Unexpected ThrottlingException after " + i + " requests. limit=" + availReqs);
}
limiter.grabQuota(1, 1);
}
}
private void assertNoopLimiter(final QuotaLimiter limiter) {
assertTrue(limiter == NoopQuotaLimiter.get());
assertNoThrottleException(limiter, 100);
}
}

View File

@ -0,0 +1,196 @@
/**
* 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 static org.junit.Assert.assertEquals;
import java.io.IOException;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Throttle;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test the quota table helpers (e.g. CRUD operations)
*/
@Category({ MediumTests.class })
public class TestQuotaTableUtil {
final Log LOG = LogFactory.getLog(getClass());
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private Connection connection;
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.getConfiguration().setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
TEST_UTIL.getConfiguration().setInt(QuotaCache.REFRESH_CONF_KEY, 2000);
TEST_UTIL.getConfiguration().setInt("hbase.hstore.compactionThreshold", 10);
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
TEST_UTIL.startMiniCluster(1);
TEST_UTIL.waitTableAvailable(QuotaTableUtil.QUOTA_TABLE_NAME);
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
@Before
public void before() throws IOException {
this.connection = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
}
@After
public void after() throws IOException {
this.connection.close();
}
@Test
public void testTableQuotaUtil() throws Exception {
final TableName table = TableName.valueOf("testTableQuotaUtilTable");
Quotas quota =
Quotas
.newBuilder()
.setThrottle(
Throttle
.newBuilder()
.setReqNum(ProtobufUtil.toTimedQuota(1000, TimeUnit.SECONDS, QuotaScope.MACHINE))
.setWriteNum(ProtobufUtil.toTimedQuota(600, TimeUnit.SECONDS, QuotaScope.MACHINE))
.setReadSize(
ProtobufUtil.toTimedQuota(8192, TimeUnit.SECONDS, QuotaScope.MACHINE)).build())
.build();
// Add user quota and verify it
QuotaUtil.addTableQuota(this.connection, table, quota);
Quotas resQuota = QuotaUtil.getTableQuota(this.connection, table);
assertEquals(quota, resQuota);
// Remove user quota and verify it
QuotaUtil.deleteTableQuota(this.connection, table);
resQuota = QuotaUtil.getTableQuota(this.connection, table);
assertEquals(null, resQuota);
}
@Test
public void testNamespaceQuotaUtil() throws Exception {
final String namespace = "testNamespaceQuotaUtilNS";
Quotas quota =
Quotas
.newBuilder()
.setThrottle(
Throttle
.newBuilder()
.setReqNum(ProtobufUtil.toTimedQuota(1000, TimeUnit.SECONDS, QuotaScope.MACHINE))
.setWriteNum(ProtobufUtil.toTimedQuota(600, TimeUnit.SECONDS, QuotaScope.MACHINE))
.setReadSize(
ProtobufUtil.toTimedQuota(8192, TimeUnit.SECONDS, QuotaScope.MACHINE)).build())
.build();
// Add user quota and verify it
QuotaUtil.addNamespaceQuota(this.connection, namespace, quota);
Quotas resQuota = QuotaUtil.getNamespaceQuota(this.connection, namespace);
assertEquals(quota, resQuota);
// Remove user quota and verify it
QuotaUtil.deleteNamespaceQuota(this.connection, namespace);
resQuota = QuotaUtil.getNamespaceQuota(this.connection, namespace);
assertEquals(null, resQuota);
}
@Test
public void testUserQuotaUtil() throws Exception {
final TableName table = TableName.valueOf("testUserQuotaUtilTable");
final String namespace = "testNS";
final String user = "testUser";
Quotas quotaNamespace =
Quotas
.newBuilder()
.setThrottle(
Throttle
.newBuilder()
.setReqNum(ProtobufUtil.toTimedQuota(50000, TimeUnit.SECONDS, QuotaScope.MACHINE))
.build()).build();
Quotas quotaTable =
Quotas
.newBuilder()
.setThrottle(
Throttle
.newBuilder()
.setReqNum(ProtobufUtil.toTimedQuota(1000, TimeUnit.SECONDS, QuotaScope.MACHINE))
.setWriteNum(ProtobufUtil.toTimedQuota(600, TimeUnit.SECONDS, QuotaScope.MACHINE))
.setReadSize(
ProtobufUtil.toTimedQuota(10000, TimeUnit.SECONDS, QuotaScope.MACHINE)).build())
.build();
Quotas quota =
Quotas
.newBuilder()
.setThrottle(
Throttle
.newBuilder()
.setReqSize(ProtobufUtil.toTimedQuota(8192, TimeUnit.SECONDS, QuotaScope.MACHINE))
.setWriteSize(
ProtobufUtil.toTimedQuota(4096, TimeUnit.SECONDS, QuotaScope.MACHINE))
.setReadNum(ProtobufUtil.toTimedQuota(1000, TimeUnit.SECONDS, QuotaScope.MACHINE))
.build()).build();
// Add user global quota
QuotaUtil.addUserQuota(this.connection, user, quota);
Quotas resQuota = QuotaUtil.getUserQuota(this.connection, user);
assertEquals(quota, resQuota);
// Add user quota for table
QuotaUtil.addUserQuota(this.connection, user, table, quotaTable);
Quotas resQuotaTable = QuotaUtil.getUserQuota(this.connection, user, table);
assertEquals(quotaTable, resQuotaTable);
// Add user quota for namespace
QuotaUtil.addUserQuota(this.connection, user, namespace, quotaNamespace);
Quotas resQuotaNS = QuotaUtil.getUserQuota(this.connection, user, namespace);
assertEquals(quotaNamespace, resQuotaNS);
// Delete user global quota
QuotaUtil.deleteUserQuota(this.connection, user);
resQuota = QuotaUtil.getUserQuota(this.connection, user);
assertEquals(null, resQuota);
// Delete user quota for table
QuotaUtil.deleteUserQuota(this.connection, user, table);
resQuotaTable = QuotaUtil.getUserQuota(this.connection, user, table);
assertEquals(null, resQuotaTable);
// Delete user quota for namespace
QuotaUtil.deleteUserQuota(this.connection, user, namespace);
resQuotaNS = QuotaUtil.getUserQuota(this.connection, user, namespace);
assertEquals(null, resQuotaNS);
}
}

View File

@ -0,0 +1,409 @@
/**
* 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 static org.junit.Assert.assertEquals;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({ MediumTests.class })
public class TestQuotaThrottle {
final Log LOG = LogFactory.getLog(getClass());
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private final static byte[] FAMILY = Bytes.toBytes("cf");
private final static byte[] QUALIFIER = Bytes.toBytes("q");
private final static TableName[] TABLE_NAMES = new TableName[] {
TableName.valueOf("TestQuotaAdmin0"), TableName.valueOf("TestQuotaAdmin1"),
TableName.valueOf("TestQuotaAdmin2") };
private static HTable[] tables;
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.getConfiguration().setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
TEST_UTIL.getConfiguration().setInt("hbase.hstore.compactionThreshold", 10);
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
TEST_UTIL.startMiniCluster(1);
TEST_UTIL.waitTableAvailable(QuotaTableUtil.QUOTA_TABLE_NAME);
QuotaCache.setTEST_FORCE_REFRESH(true);
tables = new HTable[TABLE_NAMES.length];
for (int i = 0; i < TABLE_NAMES.length; ++i) {
tables[i] = TEST_UTIL.createTable(TABLE_NAMES[i], FAMILY);
}
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
for (int i = 0; i < tables.length; ++i) {
if (tables[i] != null) {
tables[i].close();
TEST_UTIL.deleteTable(TABLE_NAMES[i]);
}
}
TEST_UTIL.shutdownMiniCluster();
}
@After
public void tearDown() throws Exception {
for (RegionServerThread rst : TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads()) {
RegionServerQuotaManager quotaManager = rst.getRegionServer().getRegionServerQuotaManager();
QuotaCache quotaCache = quotaManager.getQuotaCache();
quotaCache.getNamespaceQuotaCache().clear();
quotaCache.getTableQuotaCache().clear();
quotaCache.getUserQuotaCache().clear();
}
}
@Test(timeout = 60000)
public void testUserGlobalThrottle() throws Exception {
final Admin admin = TEST_UTIL.getHBaseAdmin();
final String userName = User.getCurrent().getShortName();
// Add 6req/min limit
admin.setQuota(QuotaSettingsFactory.throttleUser(userName, ThrottleType.REQUEST_NUMBER, 6,
TimeUnit.MINUTES));
triggerUserCacheRefresh(false, TABLE_NAMES);
// should execute at max 6 requests
assertEquals(6, doPuts(100, tables));
// wait a minute and you should get other 6 requests executed
waitMinuteQuota();
assertEquals(6, doPuts(100, tables));
// Remove all the limits
admin.setQuota(QuotaSettingsFactory.unthrottleUser(userName));
triggerUserCacheRefresh(true, TABLE_NAMES);
assertEquals(60, doPuts(60, tables));
assertEquals(60, doGets(60, tables));
}
@Test(timeout = 60000)
public void testUserTableThrottle() throws Exception {
final Admin admin = TEST_UTIL.getHBaseAdmin();
final String userName = User.getCurrent().getShortName();
// Add 6req/min limit
admin.setQuota(QuotaSettingsFactory.throttleUser(userName, TABLE_NAMES[0],
ThrottleType.REQUEST_NUMBER, 6, TimeUnit.MINUTES));
triggerUserCacheRefresh(false, TABLE_NAMES[0]);
// should execute at max 6 requests on tables[0] and have no limit on tables[1]
assertEquals(6, doPuts(100, tables[0]));
assertEquals(30, doPuts(30, tables[1]));
// wait a minute and you should get other 6 requests executed
waitMinuteQuota();
assertEquals(6, doPuts(100, tables[0]));
// Remove all the limits
admin.setQuota(QuotaSettingsFactory.unthrottleUser(userName, TABLE_NAMES[0]));
triggerUserCacheRefresh(true, TABLE_NAMES);
assertEquals(60, doPuts(60, tables));
assertEquals(60, doGets(60, tables));
}
@Test(timeout = 60000)
public void testUserNamespaceThrottle() throws Exception {
final Admin admin = TEST_UTIL.getHBaseAdmin();
final String userName = User.getCurrent().getShortName();
final String NAMESPACE = "default";
// Add 6req/min limit
admin.setQuota(QuotaSettingsFactory.throttleUser(userName, NAMESPACE,
ThrottleType.REQUEST_NUMBER, 6, TimeUnit.MINUTES));
triggerUserCacheRefresh(false, TABLE_NAMES[0]);
// should execute at max 6 requests on tables[0] and have no limit on tables[1]
assertEquals(6, doPuts(100, tables[0]));
// wait a minute and you should get other 6 requests executed
waitMinuteQuota();
assertEquals(6, doPuts(100, tables[1]));
// Remove all the limits
admin.setQuota(QuotaSettingsFactory.unthrottleUser(userName, NAMESPACE));
triggerUserCacheRefresh(true, TABLE_NAMES);
assertEquals(60, doPuts(60, tables));
assertEquals(60, doGets(60, tables));
}
@Test(timeout = 60000)
public void testTableGlobalThrottle() throws Exception {
final Admin admin = TEST_UTIL.getHBaseAdmin();
// Add 6req/min limit
admin.setQuota(QuotaSettingsFactory.throttleTable(TABLE_NAMES[0], ThrottleType.REQUEST_NUMBER,
6, TimeUnit.MINUTES));
triggerTableCacheRefresh(false, TABLE_NAMES[0]);
// should execute at max 6 requests
assertEquals(6, doPuts(100, tables[0]));
// should have no limits
assertEquals(30, doPuts(30, tables[1]));
// wait a minute and you should get other 6 requests executed
waitMinuteQuota();
assertEquals(6, doPuts(100, tables[0]));
// Remove all the limits
admin.setQuota(QuotaSettingsFactory.unthrottleTable(TABLE_NAMES[0]));
triggerTableCacheRefresh(true, TABLE_NAMES[0]);
assertEquals(80, doGets(80, tables[0], tables[1]));
}
@Test(timeout = 60000)
public void testNamespaceGlobalThrottle() throws Exception {
final Admin admin = TEST_UTIL.getHBaseAdmin();
final String NAMESPACE = "default";
// Add 6req/min limit
admin.setQuota(QuotaSettingsFactory.throttleNamespace(NAMESPACE, ThrottleType.REQUEST_NUMBER,
6, TimeUnit.MINUTES));
triggerNamespaceCacheRefresh(false, TABLE_NAMES[0]);
// should execute at max 6 requests
assertEquals(6, doPuts(100, tables[0]));
// wait a minute and you should get other 6 requests executed
waitMinuteQuota();
assertEquals(6, doPuts(100, tables[1]));
admin.setQuota(QuotaSettingsFactory.unthrottleNamespace(NAMESPACE));
triggerNamespaceCacheRefresh(true, TABLE_NAMES[0]);
assertEquals(40, doPuts(40, tables[0]));
}
@Test(timeout = 60000)
public void testUserAndTableThrottle() throws Exception {
final Admin admin = TEST_UTIL.getHBaseAdmin();
final String userName = User.getCurrent().getShortName();
// Add 6req/min limit for the user on tables[0]
admin.setQuota(QuotaSettingsFactory.throttleUser(userName, TABLE_NAMES[0],
ThrottleType.REQUEST_NUMBER, 6, TimeUnit.MINUTES));
triggerUserCacheRefresh(false, TABLE_NAMES[0]);
// Add 12req/min limit for the user
admin.setQuota(QuotaSettingsFactory.throttleUser(userName, ThrottleType.REQUEST_NUMBER, 12,
TimeUnit.MINUTES));
triggerUserCacheRefresh(false, TABLE_NAMES[1], TABLE_NAMES[2]);
// Add 8req/min limit for the tables[1]
admin.setQuota(QuotaSettingsFactory.throttleTable(TABLE_NAMES[1], ThrottleType.REQUEST_NUMBER,
8, TimeUnit.MINUTES));
triggerTableCacheRefresh(false, TABLE_NAMES[1]);
// Add a lower table level throttle on tables[0]
admin.setQuota(QuotaSettingsFactory.throttleTable(TABLE_NAMES[0], ThrottleType.REQUEST_NUMBER,
3, TimeUnit.MINUTES));
triggerTableCacheRefresh(false, TABLE_NAMES[0]);
// should execute at max 12 requests
assertEquals(12, doGets(100, tables[2]));
// should execute at max 8 requests
waitMinuteQuota();
assertEquals(8, doGets(100, tables[1]));
// should execute at max 3 requests
waitMinuteQuota();
assertEquals(3, doPuts(100, tables[0]));
// Remove all the throttling rules
admin.setQuota(QuotaSettingsFactory.unthrottleUser(userName, TABLE_NAMES[0]));
admin.setQuota(QuotaSettingsFactory.unthrottleUser(userName));
triggerUserCacheRefresh(true, TABLE_NAMES[0], TABLE_NAMES[1]);
admin.setQuota(QuotaSettingsFactory.unthrottleTable(TABLE_NAMES[1]));
triggerTableCacheRefresh(true, TABLE_NAMES[1]);
waitMinuteQuota();
assertEquals(40, doGets(40, tables[1]));
admin.setQuota(QuotaSettingsFactory.unthrottleTable(TABLE_NAMES[0]));
triggerTableCacheRefresh(true, TABLE_NAMES[0]);
waitMinuteQuota();
assertEquals(40, doGets(40, tables[0]));
}
@Test(timeout = 60000)
public void testUserGlobalBypassThrottle() throws Exception {
final Admin admin = TEST_UTIL.getHBaseAdmin();
final String userName = User.getCurrent().getShortName();
final String NAMESPACE = "default";
// Add 6req/min limit for tables[0]
admin.setQuota(QuotaSettingsFactory.throttleTable(TABLE_NAMES[0], ThrottleType.REQUEST_NUMBER,
6, TimeUnit.MINUTES));
triggerTableCacheRefresh(false, TABLE_NAMES[0]);
// Add 13req/min limit for the user
admin.setQuota(QuotaSettingsFactory.throttleNamespace(NAMESPACE, ThrottleType.REQUEST_NUMBER,
13, TimeUnit.MINUTES));
triggerNamespaceCacheRefresh(false, TABLE_NAMES[1]);
// should execute at max 6 requests on table[0] and (13 - 6) on table[1]
assertEquals(6, doPuts(100, tables[0]));
assertEquals(7, doGets(100, tables[1]));
waitMinuteQuota();
// Set the global bypass for the user
admin.setQuota(QuotaSettingsFactory.bypassGlobals(userName, true));
admin.setQuota(QuotaSettingsFactory.throttleUser(userName, TABLE_NAMES[2],
ThrottleType.REQUEST_NUMBER, 6, TimeUnit.MINUTES));
triggerUserCacheRefresh(false, TABLE_NAMES[2]);
assertEquals(30, doGets(30, tables[0]));
assertEquals(30, doGets(30, tables[1]));
waitMinuteQuota();
// Remove the global bypass
// should execute at max 6 requests on table[0] and (13 - 6) on table[1]
admin.setQuota(QuotaSettingsFactory.bypassGlobals(userName, false));
admin.setQuota(QuotaSettingsFactory.unthrottleUser(userName, TABLE_NAMES[2]));
triggerUserCacheRefresh(true, TABLE_NAMES[2]);
assertEquals(6, doPuts(100, tables[0]));
assertEquals(7, doGets(100, tables[1]));
// unset throttle
admin.setQuota(QuotaSettingsFactory.unthrottleTable(TABLE_NAMES[0]));
admin.setQuota(QuotaSettingsFactory.unthrottleNamespace(NAMESPACE));
waitMinuteQuota();
triggerTableCacheRefresh(true, TABLE_NAMES[0]);
triggerNamespaceCacheRefresh(true, TABLE_NAMES[1]);
assertEquals(30, doGets(30, tables[0]));
assertEquals(30, doGets(30, tables[1]));
}
private int doPuts(int maxOps, final HTable... tables) throws Exception {
int count = 0;
try {
while (count < maxOps) {
Put put = new Put(Bytes.toBytes("row-" + count));
put.addColumn(FAMILY, QUALIFIER, Bytes.toBytes("data-" + count));
for (final HTable table : tables) {
table.put(put);
}
count += tables.length;
}
} catch (RetriesExhaustedWithDetailsException e) {
for (Throwable t : e.getCauses()) {
if (!(t instanceof ThrottlingException)) {
throw e;
}
}
LOG.error("put failed after nRetries=" + count, e);
}
return count;
}
private long doGets(int maxOps, final HTable... tables) throws Exception {
int count = 0;
try {
while (count < maxOps) {
Get get = new Get(Bytes.toBytes("row-" + count));
for (final HTable table : tables) {
table.get(get);
}
count += tables.length;
}
} catch (ThrottlingException e) {
LOG.error("get failed after nRetries=" + count, e);
}
return count;
}
private void triggerUserCacheRefresh(boolean bypass, TableName... tables) throws Exception {
triggerCacheRefresh(bypass, true, false, false, tables);
}
private void triggerTableCacheRefresh(boolean bypass, TableName... tables) throws Exception {
triggerCacheRefresh(bypass, false, true, false, tables);
}
private void triggerNamespaceCacheRefresh(boolean bypass, TableName... tables) throws Exception {
triggerCacheRefresh(bypass, false, false, true, tables);
}
private void triggerCacheRefresh(boolean bypass, boolean userLimiter, boolean tableLimiter,
boolean nsLimiter, final TableName... tables) throws Exception {
for (RegionServerThread rst : TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads()) {
RegionServerQuotaManager quotaManager = rst.getRegionServer().getRegionServerQuotaManager();
QuotaCache quotaCache = quotaManager.getQuotaCache();
quotaCache.triggerCacheRefresh();
Thread.sleep(250);
for (TableName table : tables) {
quotaCache.getTableLimiter(table);
}
boolean isUpdated = false;
while (!isUpdated) {
isUpdated = true;
for (TableName table : tables) {
boolean isBypass = true;
if (userLimiter) {
isBypass &= quotaCache.getUserLimiter(User.getCurrent().getUGI(), table).isBypass();
}
if (tableLimiter) {
isBypass &= quotaCache.getTableLimiter(table).isBypass();
}
if (nsLimiter) {
isBypass &= quotaCache.getNamespaceLimiter(table.getNamespaceAsString()).isBypass();
}
if (isBypass != bypass) {
isUpdated = false;
Thread.sleep(250);
break;
}
}
}
LOG.debug("QuotaCache");
LOG.debug(quotaCache.getNamespaceQuotaCache());
LOG.debug(quotaCache.getTableQuotaCache());
LOG.debug(quotaCache.getUserQuotaCache());
}
}
private void waitMinuteQuota() {
EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge(
EnvironmentEdgeManager.currentTime() + 70000));
}
}

View File

@ -0,0 +1,105 @@
/**
* 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 static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Verify the behaviour of the Rate Limiter.
*/
@Category({ SmallTests.class })
public class TestRateLimiter {
@Test
public void testWaitIntervalTimeUnitSeconds() {
testWaitInterval(TimeUnit.SECONDS, 10, 100);
}
@Test
public void testWaitIntervalTimeUnitMinutes() {
testWaitInterval(TimeUnit.MINUTES, 10, 6000);
}
@Test
public void testWaitIntervalTimeUnitHours() {
testWaitInterval(TimeUnit.HOURS, 10, 360000);
}
@Test
public void testWaitIntervalTimeUnitDays() {
testWaitInterval(TimeUnit.DAYS, 10, 8640000);
}
private void testWaitInterval(final TimeUnit timeUnit, final long limit,
final long expectedWaitInterval) {
RateLimiter limiter = new RateLimiter();
limiter.set(limit, timeUnit);
long nowTs = 0;
long lastTs = 0;
// consume all the available resources, one request at the time.
// the wait interval should be 0
for (int i = 0; i < (limit - 1); ++i) {
assertTrue(limiter.canExecute(nowTs, lastTs));
limiter.consume();
long waitInterval = limiter.waitInterval();
assertEquals(0, waitInterval);
}
for (int i = 0; i < (limit * 4); ++i) {
// There is one resource available, so we should be able to
// consume it without waiting.
assertTrue(limiter.canExecute(nowTs, lastTs));
assertEquals(0, limiter.waitInterval());
limiter.consume();
lastTs = nowTs;
// No more resources are available, we should wait for at least an interval.
long waitInterval = limiter.waitInterval();
assertEquals(expectedWaitInterval, waitInterval);
// set the nowTs to be the exact time when resources should be available again.
nowTs += waitInterval;
// artificially go into the past to prove that when too early we should fail.
assertFalse(limiter.canExecute(nowTs - 500, lastTs));
}
}
@Test
public void testOverconsumption() {
RateLimiter limiter = new RateLimiter();
limiter.set(10, TimeUnit.SECONDS);
// 10 resources are available, but we need to consume 20 resources
// Verify that we have to wait at least 1.1sec to have 1 resource available
assertTrue(limiter.canExecute(0, 0));
limiter.consume(20);
assertEquals(1100, limiter.waitInterval());
// Verify that after 1sec we need to wait for another 0.1sec to get a resource available
assertFalse(limiter.canExecute(1000, 0));
assertEquals(100, limiter.waitInterval());
// Verify that after 1.1sec the resource is available
assertTrue(limiter.canExecute(1100, 0));
assertEquals(0, limiter.waitInterval());
}
}

View File

@ -2484,4 +2484,67 @@ public class TestAccessController extends SecureTestUtil {
verifyAllowed(replicateLogEntriesAction, SUPERUSER, USER_ADMIN);
verifyDenied(replicateLogEntriesAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
}
@Test
public void testSetQuota() throws Exception {
AccessTestAction setUserQuotaAction = new AccessTestAction() {
@Override
public Object run() throws Exception {
ACCESS_CONTROLLER.preSetUserQuota(ObserverContext.createAndPrepare(CP_ENV, null),
null, null);
return null;
}
};
AccessTestAction setUserTableQuotaAction = new AccessTestAction() {
@Override
public Object run() throws Exception {
ACCESS_CONTROLLER.preSetUserQuota(ObserverContext.createAndPrepare(CP_ENV, null),
null, TEST_TABLE.getTableName(), null);
return null;
}
};
AccessTestAction setUserNamespaceQuotaAction = new AccessTestAction() {
@Override
public Object run() throws Exception {
ACCESS_CONTROLLER.preSetUserQuota(ObserverContext.createAndPrepare(CP_ENV, null),
null, (String)null, null);
return null;
}
};
AccessTestAction setTableQuotaAction = new AccessTestAction() {
@Override
public Object run() throws Exception {
ACCESS_CONTROLLER.preSetTableQuota(ObserverContext.createAndPrepare(CP_ENV, null),
TEST_TABLE.getTableName(), null);
return null;
}
};
AccessTestAction setNamespaceQuotaAction = new AccessTestAction() {
@Override
public Object run() throws Exception {
ACCESS_CONTROLLER.preSetNamespaceQuota(ObserverContext.createAndPrepare(CP_ENV, null),
null, null);
return null;
}
};
verifyAllowed(setUserQuotaAction, SUPERUSER, USER_ADMIN);
verifyDenied(setUserQuotaAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
verifyAllowed(setUserTableQuotaAction, SUPERUSER, USER_ADMIN, USER_OWNER);
verifyDenied(setUserTableQuotaAction, USER_CREATE, USER_RW, USER_RO, USER_NONE);
verifyAllowed(setUserNamespaceQuotaAction, SUPERUSER, USER_ADMIN);
verifyDenied(setUserNamespaceQuotaAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
verifyAllowed(setTableQuotaAction, SUPERUSER, USER_ADMIN, USER_OWNER);
verifyDenied(setTableQuotaAction, USER_CREATE, USER_RW, USER_RO, USER_NONE);
verifyAllowed(setNamespaceQuotaAction, SUPERUSER, USER_ADMIN);
verifyDenied(setNamespaceQuotaAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
}
}

View File

@ -71,6 +71,12 @@ module HBaseConstants
TABLE_CFS = 'TABLE_CFS'
CONFIG = 'CONFIG'
DATA = 'DATA'
USER = 'USER'
TABLE = 'TABLE'
NAMESPACE = 'NAMESPACE'
TYPE = 'TYPE'
NONE = 'NONE'
VALUE = 'VALUE'
# Load constants from hbase java API
def self.promote_constants(constants)
@ -90,6 +96,10 @@ end
require 'hbase/hbase'
require 'hbase/admin'
require 'hbase/table'
require 'hbase/quotas'
require 'hbase/replication_admin'
require 'hbase/security'
require 'hbase/visibility_labels'
include HBaseQuotasConstants

View File

@ -21,6 +21,7 @@ include Java
require 'hbase/admin'
require 'hbase/table'
require 'hbase/quotas'
require 'hbase/security'
require 'hbase/visibility_labels'
@ -62,6 +63,10 @@ module Hbase
def visibility_labels_admin(formatter)
::Hbase::VisibilityLabelsAdmin.new(@connection.getAdmin, formatter)
end
def quotas_admin(formatter)
::Hbase::QuotasAdmin.new(@connection.getAdmin, formatter)
end
def shutdown
@connection.close

View File

@ -0,0 +1,216 @@
#
#
# 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.
#
include Java
java_import java.util.concurrent.TimeUnit
java_import org.apache.hadoop.hbase.TableName
java_import org.apache.hadoop.hbase.quotas.ThrottleType
java_import org.apache.hadoop.hbase.quotas.QuotaFilter
java_import org.apache.hadoop.hbase.quotas.QuotaRetriever
java_import org.apache.hadoop.hbase.quotas.QuotaSettingsFactory
module HBaseQuotasConstants
GLOBAL_BYPASS = 'GLOBAL_BYPASS'
THROTTLE_TYPE = 'THROTTLE_TYPE'
THROTTLE = 'THROTTLE'
REQUEST = 'REQUEST'
end
module Hbase
class QuotasAdmin
def initialize(admin, formatter)
@admin = admin
@formatter = formatter
end
def close
@admin.close
end
def throttle(args)
raise(ArgumentError, "Arguments should be a Hash") unless args.kind_of?(Hash)
type = args.fetch(THROTTLE_TYPE, REQUEST)
type, limit, time_unit = _parse_limit(args.delete(LIMIT), ThrottleType, type)
if args.has_key?(USER)
user = args.delete(USER)
if args.has_key?(TABLE)
table = TableName.valueOf(args.delete(TABLE))
raise(ArgumentError, "Unexpected arguments: " + args.inspect) unless args.empty?
settings = QuotaSettingsFactory.throttleUser(user, table, type, limit, time_unit)
elsif args.has_key?(NAMESPACE)
namespace = args.delete(NAMESPACE)
raise(ArgumentError, "Unexpected arguments: " + args.inspect) unless args.empty?
settings = QuotaSettingsFactory.throttleUser(user, namespace, type, limit, time_unit)
else
raise(ArgumentError, "Unexpected arguments: " + args.inspect) unless args.empty?
settings = QuotaSettingsFactory.throttleUser(user, type, limit, time_unit)
end
elsif args.has_key?(TABLE)
table = TableName.valueOf(args.delete(TABLE))
raise(ArgumentError, "Unexpected arguments: " + args.inspect) unless args.empty?
settings = QuotaSettingsFactory.throttleTable(table, type, limit, time_unit)
elsif args.has_key?(NAMESPACE)
namespace = args.delete(NAMESPACE)
raise(ArgumentError, "Unexpected arguments: " + args.inspect) unless args.empty?
settings = QuotaSettingsFactory.throttleNamespace(namespace, type, limit, time_unit)
else
raise "One of USER, TABLE or NAMESPACE must be specified"
end
@admin.setQuota(settings)
end
def unthrottle(args)
raise(ArgumentError, "Arguments should be a Hash") unless args.kind_of?(Hash)
if args.has_key?(USER)
user = args.delete(USER)
if args.has_key?(TABLE)
table = TableName.valueOf(args.delete(TABLE))
raise(ArgumentError, "Unexpected arguments: " + args.inspect) unless args.empty?
settings = QuotaSettingsFactory.unthrottleUser(user, table)
elsif args.has_key?(NAMESPACE)
namespace = args.delete(NAMESPACE)
raise(ArgumentError, "Unexpected arguments: " + args.inspect) unless args.empty?
settings = QuotaSettingsFactory.unthrottleUser(user, namespace)
else
raise(ArgumentError, "Unexpected arguments: " + args.inspect) unless args.empty?
settings = QuotaSettingsFactory.unthrottleUser(user)
end
elsif args.has_key?(TABLE)
table = TableName.valueOf(args.delete(TABLE))
raise(ArgumentError, "Unexpected arguments: " + args.inspect) unless args.empty?
settings = QuotaSettingsFactory.unthrottleTable(table)
elsif args.has_key?(NAMESPACE)
namespace = args.delete(NAMESPACE)
raise(ArgumentError, "Unexpected arguments: " + args.inspect) unless args.empty?
settings = QuotaSettingsFactory.unthrottleNamespace(namespace)
else
raise "One of USER, TABLE or NAMESPACE must be specified"
end
@admin.setQuota(settings)
end
def set_global_bypass(bypass, args)
raise(ArgumentError, "Arguments should be a Hash") unless args.kind_of?(Hash)
if args.has_key?(USER)
user = args.delete(USER)
raise(ArgumentError, "Unexpected arguments: " + args.inspect) unless args.empty?
settings = QuotaSettingsFactory.bypassGlobals(user, bypass)
else
raise "Expected USER"
end
@admin.setQuota(settings)
end
def list_quotas(args = {})
raise(ArgumentError, "Arguments should be a Hash") unless args.kind_of?(Hash)
limit = args.delete("LIMIT") || -1
count = 0
filter = QuotaFilter.new()
filter.setUserFilter(args.delete(USER)) if args.has_key?(USER)
filter.setTableFilter(args.delete(TABLE)) if args.has_key?(TABLE)
filter.setNamespaceFilter(args.delete(NAMESPACE)) if args.has_key?(NAMESPACE)
raise(ArgumentError, "Unexpected arguments: " + args.inspect) unless args.empty?
# Start the scanner
scanner = @admin.getQuotaRetriever(filter)
begin
iter = scanner.iterator
# Iterate results
while iter.hasNext
if limit > 0 && count >= limit
break
end
settings = iter.next
owner = {
USER => settings.getUserName(),
TABLE => settings.getTableName(),
NAMESPACE => settings.getNamespace(),
}.delete_if { |k, v| v.nil? }.map {|k, v| k.to_s + " => " + v.to_s} * ', '
yield owner, settings.to_s
count += 1
end
ensure
scanner.close()
end
return count
end
def _parse_size(str_limit)
str_limit = str_limit.downcase
match = /(\d+)([bkmgtp%]*)/.match(str_limit)
if match
if match[2] == '%'
return match[1].to_i
else
return _size_from_str(match[1].to_i, match[2])
end
else
raise "Invalid size limit syntax"
end
end
def _parse_limit(str_limit, type_cls, type)
str_limit = str_limit.downcase
match = /(\d+)(req|[bkmgtp])\/(sec|min|hour|day)/.match(str_limit)
if match
if match[2] == 'req'
limit = match[1].to_i
type = type_cls.valueOf(type + "_NUMBER")
else
limit = _size_from_str(match[1].to_i, match[2])
type = type_cls.valueOf(type + "_SIZE")
end
if limit <= 0
raise "Invalid throttle limit, must be greater then 0"
end
case match[3]
when 'sec' then time_unit = TimeUnit::SECONDS
when 'min' then time_unit = TimeUnit::MINUTES
when 'hour' then time_unit = TimeUnit::HOURS
when 'day' then time_unit = TimeUnit::DAYS
end
return type, limit, time_unit
else
raise "Invalid throttle limit syntax"
end
end
def _size_from_str(value, suffix)
case suffix
when 'k' then value <<= 10
when 'm' then value <<= 20
when 'g' then value <<= 30
when 't' then value <<= 40
when 'p' then value <<= 50
end
return value
end
end
end

View File

@ -99,6 +99,10 @@ module Shell
def hbase_visibility_labels_admin
@hbase_visibility_labels_admin ||= hbase.visibility_labels_admin(formatter)
end
def hbase_quotas_admin
@hbase_quotas_admin ||= hbase.quotas_admin(formatter)
end
def export_commands(where)
::Shell.commands.keys.each do |cmd|
@ -371,6 +375,15 @@ Shell.load_command_group(
]
)
Shell.load_command_group(
'quotas',
:full_name => 'CLUSTER QUOTAS TOOLS',
:commands => %w[
set_quota
list_quotas
]
)
Shell.load_command_group(
'security',
:full_name => 'SECURITY TOOLS',

View File

@ -65,6 +65,10 @@ module Shell
def visibility_labels_admin
@shell.hbase_visibility_labels_admin
end
def quotas_admin
@shell.hbase_quotas_admin
end
#----------------------------------------------------------------------

View File

@ -0,0 +1,52 @@
#
#
# 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.
#
module Shell
module Commands
class ListQuotas < Command
def help
return <<-EOF
List the quota settings added to the system.
You can filter the result based on USER, TABLE, or NAMESPACE.
For example:
hbase> list_quotas
hbase> list_quotas USER => 'bob.*'
hbase> list_quotas USER => 'bob.*', TABLE => 't1'
hbase> list_quotas USER => 'bob.*', NAMESPACE => 'ns.*'
hbase> list_quotas TABLE => 'myTable'
hbase> list_quotas NAMESPACE => 'ns.*'
EOF
end
def command(args = {})
now = Time.now
formatter.header(["OWNER", "QUOTAS"])
#actually do the scanning
count = quotas_admin.list_quotas(args) do |row, cells|
formatter.row([ row, cells ])
end
formatter.footer(now, count)
end
end
end
end

View File

@ -0,0 +1,70 @@
#
#
# 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.
#
module Shell
module Commands
class SetQuota < Command
def help
return <<-EOF
Set a quota for a user, table, or namespace.
Syntax : set_quota TYPE => <type>, <args>
TYPE => THROTTLE
The request limit can be expressed using the form 100req/sec, 100req/min
and the size limit can be expressed using the form 100k/sec, 100M/min
with (B, K, M, G, T, P) as valid size unit and (sec, min, hour, day) as valid time unit.
Currently the throttle limit is per machine - a limit of 100req/min
means that each machine can execute 100req/min.
For example:
hbase> set_quota TYPE => THROTTLE, USER => 'u1', LIMIT => '10req/sec'
hbase> set_quota TYPE => THROTTLE, USER => 'u1', LIMIT => '10M/sec'
hbase> set_quota TYPE => THROTTLE, USER => 'u1', TABLE => 't2', LIMIT => '5K/min'
hbase> set_quota TYPE => THROTTLE, USER => 'u1', NAMESPACE => 'ns2', LIMIT => NONE
hbase> set_quota TYPE => THROTTLE, NAMESPACE => 'ns1', LIMIT => '10req/sec'
hbase> set_quota TYPE => THROTTLE, TABLE => 't1', LIMIT => '10M/sec'
hbase> set_quota TYPE => THROTTLE, USER => 'u1', LIMIT => NONE
hbase> set_quota USER => 'u1', GLOBAL_BYPASS => true
EOF
end
def command(args = {})
if args.has_key?(TYPE)
qtype = args.delete(TYPE)
case qtype
when THROTTLE
if args[LIMIT].eql? NONE
args.delete(LIMIT)
quotas_admin.unthrottle(args)
else
quotas_admin.throttle(args)
end
else
raise "Invalid TYPE argument. got " + qtype
end
elsif args.has_key?(GLOBAL_BYPASS)
quotas_admin.set_global_bypass(args.delete(GLOBAL_BYPASS), args)
else
raise "Expected TYPE argument"
end
end
end
end
end