HBASE-18873 Move protobufs to private implementation on GlobalQuotaSettings
A hack to "hide" the protobufs, but it's not going to be a trivial
change to remove use of protobufs entirely as they're serialized
into the hbase:quota table.
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/81133f89
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/81133f89
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/81133f89
Branch: refs/heads/HBASE-18410
Commit: 81133f89fc9a80fbd03aff5a3b51184eeb90f130
Parents: b7db62c
Author: Josh Elser <elserj@apache.org>
Authored: Wed Oct 11 18:37:42 2017 -0400
Committer: Josh Elser <elserj@apache.org>
Committed: Mon Oct 23 22:37:10 2017 -0400
----------------------------------------------------------------------
.../hbase/quotas/QuotaSettingsFactory.java | 2 +-
.../hbase/quotas/GlobalQuotaSettings.java | 290 +---------------
.../hbase/quotas/GlobalQuotaSettingsImpl.java | 332 +++++++++++++++++++
.../hadoop/hbase/quotas/MasterQuotaManager.java | 72 ++--
.../hbase/quotas/TestGlobalQuotaSettings.java | 122 -------
.../quotas/TestGlobalQuotaSettingsImpl.java | 122 +++++++
6 files changed, 501 insertions(+), 439 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/81133f89/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
index 185365b..2a20c51 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
@@ -116,7 +116,7 @@ public class QuotaSettingsFactory {
return settings;
}
- private static List<QuotaSettings> fromThrottle(final String userName, final TableName tableName,
+ protected static List<QuotaSettings> fromThrottle(final String userName, final TableName tableName,
final String namespace, final QuotaProtos.Throttle throttle) {
List<QuotaSettings> settings = new ArrayList<>();
if (throttle.hasReqNum()) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/81133f89/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettings.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettings.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettings.java
index 079edf0..107523b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettings.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettings.java
@@ -16,23 +16,12 @@
*/
package org.apache.hadoop.hbase.quotas;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Objects;
+import java.util.List;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.quotas.QuotaSettingsFactory.QuotaGlobalsSettingsBypass;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest.Builder;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
@@ -43,28 +32,19 @@ import org.apache.yetus.audience.InterfaceStability;
*/
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC})
@InterfaceStability.Evolving
-public class GlobalQuotaSettings extends QuotaSettings {
- private final QuotaProtos.Throttle throttleProto;
- private final Boolean bypassGlobals;
- private final QuotaProtos.SpaceQuota spaceProto;
+public abstract class GlobalQuotaSettings extends QuotaSettings {
- protected GlobalQuotaSettings(
- String username, TableName tableName, String namespace, QuotaProtos.Quotas quotas) {
- this(username, tableName, namespace,
- (quotas != null && quotas.hasThrottle() ? quotas.getThrottle() : null),
- (quotas != null && quotas.hasBypassGlobals() ? quotas.getBypassGlobals() : null),
- (quotas != null && quotas.hasSpace() ? quotas.getSpace() : null));
- }
-
- protected GlobalQuotaSettings(
- String userName, TableName tableName, String namespace, QuotaProtos.Throttle throttleProto,
- Boolean bypassGlobals, QuotaProtos.SpaceQuota spaceProto) {
+ protected GlobalQuotaSettings(String userName, TableName tableName, String namespace) {
super(userName, tableName, namespace);
- this.throttleProto = throttleProto;
- this.bypassGlobals = bypassGlobals;
- this.spaceProto = spaceProto;
}
+ /**
+ * Computes a list of QuotaSettings that present the complete quota state of the combination of
+ * this user, table, and/or namespace. Beware in calling this method repeatedly as the
+ * implementation of it may be costly.
+ */
+ public abstract List<QuotaSettings> getQuotaSettings();
+
@Override
public QuotaType getQuotaType() {
throw new UnsupportedOperationException();
@@ -76,254 +56,4 @@ public class GlobalQuotaSettings extends QuotaSettings {
throw new UnsupportedOperationException(
"This class should not be used to generate a SetQuotaRequest.");
}
-
- protected QuotaProtos.Throttle getThrottleProto() {
- return this.throttleProto;
- }
-
- protected Boolean getGlobalBypass() {
- return this.bypassGlobals;
- }
-
- protected QuotaProtos.SpaceQuota getSpaceProto() {
- return this.spaceProto;
- }
-
- /**
- * Constructs a new {@link Quotas} message from {@code this}.
- */
- protected Quotas toQuotas() {
- QuotaProtos.Quotas.Builder builder = QuotaProtos.Quotas.newBuilder();
- if (getThrottleProto() != null) {
- builder.setThrottle(getThrottleProto());
- }
- if (getGlobalBypass() != null) {
- builder.setBypassGlobals(getGlobalBypass());
- }
- if (getSpaceProto() != null) {
- builder.setSpace(getSpaceProto());
- }
- return builder.build();
- }
-
- @Override
- protected GlobalQuotaSettings merge(QuotaSettings other) throws IOException {
- // Validate the quota subject
- validateQuotaTarget(other);
-
- // Propagate the Throttle
- QuotaProtos.Throttle.Builder throttleBuilder = (throttleProto == null
- ? null : throttleProto.toBuilder());
- if (other instanceof ThrottleSettings) {
- if (throttleBuilder == null) {
- throttleBuilder = QuotaProtos.Throttle.newBuilder();
- }
- ThrottleSettings otherThrottle = (ThrottleSettings) other;
-
- if (otherThrottle.proto.hasType()) {
- QuotaProtos.ThrottleRequest otherProto = otherThrottle.proto;
- if (otherProto.hasTimedQuota()) {
- if (otherProto.hasTimedQuota()) {
- validateTimedQuota(otherProto.getTimedQuota());
- }
-
- switch (otherProto.getType()) {
- case REQUEST_NUMBER:
- if (otherProto.hasTimedQuota()) {
- throttleBuilder.setReqNum(otherProto.getTimedQuota());
- } else {
- throttleBuilder.clearReqNum();
- }
- break;
- case REQUEST_SIZE:
- if (otherProto.hasTimedQuota()) {
- throttleBuilder.setReqSize(otherProto.getTimedQuota());
- } else {
- throttleBuilder.clearReqSize();
- }
- break;
- case WRITE_NUMBER:
- if (otherProto.hasTimedQuota()) {
- throttleBuilder.setWriteNum(otherProto.getTimedQuota());
- } else {
- throttleBuilder.clearWriteNum();
- }
- break;
- case WRITE_SIZE:
- if (otherProto.hasTimedQuota()) {
- throttleBuilder.setWriteSize(otherProto.getTimedQuota());
- } else {
- throttleBuilder.clearWriteSize();
- }
- break;
- case READ_NUMBER:
- if (otherProto.hasTimedQuota()) {
- throttleBuilder.setReadNum(otherProto.getTimedQuota());
- } else {
- throttleBuilder.clearReqNum();
- }
- break;
- case READ_SIZE:
- if (otherProto.hasTimedQuota()) {
- throttleBuilder.setReadSize(otherProto.getTimedQuota());
- } else {
- throttleBuilder.clearReadSize();
- }
- break;
- }
- } else {
- clearThrottleBuilder(throttleBuilder);
- }
- } else {
- clearThrottleBuilder(throttleBuilder);
- }
- }
-
- // Propagate the space quota portion
- QuotaProtos.SpaceQuota.Builder spaceBuilder = (spaceProto == null
- ? null : spaceProto.toBuilder());
- if (other instanceof SpaceLimitSettings) {
- if (spaceBuilder == null) {
- spaceBuilder = QuotaProtos.SpaceQuota.newBuilder();
- }
- SpaceLimitSettings settingsToMerge = (SpaceLimitSettings) other;
-
- QuotaProtos.SpaceLimitRequest spaceRequest = settingsToMerge.getProto();
-
- // The message contained the expect SpaceQuota object
- if (spaceRequest.hasQuota()) {
- SpaceQuota quotaToMerge = spaceRequest.getQuota();
- // Validate that the two settings are for the same target.
- // SpaceQuotas either apply to a table or a namespace (no user spacequota).
- if (!Objects.equals(getTableName(), settingsToMerge.getTableName())
- && !Objects.equals(getNamespace(), settingsToMerge.getNamespace())) {
- throw new IllegalArgumentException(
- "Cannot merge " + settingsToMerge + " into " + this);
- }
-
- if (quotaToMerge.getRemove()) {
- // Update the builder to propagate the removal
- spaceBuilder.setRemove(true).clearSoftLimit().clearViolationPolicy();
- } else {
- // Add the new settings to the existing settings
- spaceBuilder.mergeFrom(quotaToMerge);
- }
- }
- }
-
- Boolean bypassGlobals = this.bypassGlobals;
- if (other instanceof QuotaGlobalsSettingsBypass) {
- bypassGlobals = ((QuotaGlobalsSettingsBypass) other).getBypass();
- }
-
- if (throttleBuilder == null &&
- (spaceBuilder == null || (spaceBuilder.hasRemove() && spaceBuilder.getRemove()))
- && bypassGlobals == null) {
- return null;
- }
-
- return new GlobalQuotaSettings(
- getUserName(), getTableName(), getNamespace(),
- (throttleBuilder == null ? null : throttleBuilder.build()), bypassGlobals,
- (spaceBuilder == null ? null : spaceBuilder.build()));
- }
-
- 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()));
- }
- }
-
- @Override
- public String toString() {
- StringBuilder builder = new StringBuilder();
- builder.append("GlobalQuota: ");
- if (throttleProto != null) {
- Map<ThrottleType,TimedQuota> throttleQuotas = buildThrottleQuotas(throttleProto);
- builder.append(" { TYPE => THROTTLE ");
- for (Entry<ThrottleType,TimedQuota> entry : throttleQuotas.entrySet()) {
- final ThrottleType type = entry.getKey();
- final TimedQuota timedQuota = entry.getValue();
- builder.append("{THROTTLE_TYPE => ").append(type.name()).append(", LIMIT => ");
- if (timedQuota.hasSoftLimit()) {
- switch (type) {
- case REQUEST_NUMBER:
- case WRITE_NUMBER:
- case READ_NUMBER:
- builder.append(String.format("%dreq", timedQuota.getSoftLimit()));
- break;
- case REQUEST_SIZE:
- case WRITE_SIZE:
- case READ_SIZE:
- builder.append(sizeToString(timedQuota.getSoftLimit()));
- break;
- }
- } 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());
- }
- }
- builder.append( "} } ");
- } else {
- builder.append(" {} ");
- }
- if (bypassGlobals != null) {
- builder.append(" { GLOBAL_BYPASS => " + bypassGlobals + " } ");
- }
- if (spaceProto != null) {
- builder.append(" { TYPE => SPACE");
- if (getTableName() != null) {
- builder.append(", TABLE => ").append(getTableName());
- }
- if (getNamespace() != null) {
- builder.append(", NAMESPACE => ").append(getNamespace());
- }
- if (spaceProto.getRemove()) {
- builder.append(", REMOVE => ").append(spaceProto.getRemove());
- } else {
- builder.append(", LIMIT => ").append(spaceProto.getSoftLimit());
- builder.append(", VIOLATION_POLICY => ").append(spaceProto.getViolationPolicy());
- }
- builder.append(" } ");
- }
- return builder.toString();
- }
-
- private Map<ThrottleType,TimedQuota> buildThrottleQuotas(Throttle proto) {
- HashMap<ThrottleType,TimedQuota> quotas = new HashMap<>();
- if (proto.hasReadNum()) {
- quotas.put(ThrottleType.READ_NUMBER, proto.getReadNum());
- }
- if (proto.hasReadSize()) {
- quotas.put(ThrottleType.READ_SIZE, proto.getReadSize());
- }
- if (proto.hasReqNum()) {
- quotas.put(ThrottleType.REQUEST_NUMBER, proto.getReqNum());
- }
- if (proto.hasReqSize()) {
- quotas.put(ThrottleType.REQUEST_SIZE, proto.getReqSize());
- }
- if (proto.hasWriteNum()) {
- quotas.put(ThrottleType.WRITE_NUMBER, proto.getWriteNum());
- }
- if (proto.hasWriteSize()) {
- quotas.put(ThrottleType.WRITE_SIZE, proto.getWriteSize());
- }
- return quotas;
- }
-
- private void clearThrottleBuilder(QuotaProtos.Throttle.Builder builder) {
- builder.clearReadNum();
- builder.clearReadSize();
- builder.clearReqNum();
- builder.clearReqSize();
- builder.clearWriteNum();
- builder.clearWriteSize();
- }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/81133f89/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java
new file mode 100644
index 0000000..3893d00
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java
@@ -0,0 +1,332 @@
+/*
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.quotas.QuotaSettingsFactory.QuotaGlobalsSettingsBypass;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Implementation of {@link GlobalQuotaSettings} to hide the Protobuf messages we use internally.
+ */
+@InterfaceAudience.Private
+public class GlobalQuotaSettingsImpl extends GlobalQuotaSettings {
+
+ private final QuotaProtos.Throttle throttleProto;
+ private final Boolean bypassGlobals;
+ private final QuotaProtos.SpaceQuota spaceProto;
+
+ protected GlobalQuotaSettingsImpl(
+ String username, TableName tableName, String namespace, QuotaProtos.Quotas quotas) {
+ this(username, tableName, namespace,
+ (quotas != null && quotas.hasThrottle() ? quotas.getThrottle() : null),
+ (quotas != null && quotas.hasBypassGlobals() ? quotas.getBypassGlobals() : null),
+ (quotas != null && quotas.hasSpace() ? quotas.getSpace() : null));
+ }
+
+ protected GlobalQuotaSettingsImpl(
+ String userName, TableName tableName, String namespace, QuotaProtos.Throttle throttleProto,
+ Boolean bypassGlobals, QuotaProtos.SpaceQuota spaceProto) {
+ super(userName, tableName, namespace);
+ this.throttleProto = throttleProto;
+ this.bypassGlobals = bypassGlobals;
+ this.spaceProto = spaceProto;
+ }
+
+ @Override
+ public List<QuotaSettings> getQuotaSettings() {
+ // Very similar to QuotaSettingsFactory
+ List<QuotaSettings> settings = new ArrayList<>();
+ if (throttleProto != null) {
+ settings.addAll(QuotaSettingsFactory.fromThrottle(
+ getUserName(), getTableName(), getNamespace(), throttleProto));
+ }
+ if (bypassGlobals != null && bypassGlobals.booleanValue()) {
+ settings.add(new QuotaGlobalsSettingsBypass(
+ getUserName(), getTableName(), getNamespace(), true));
+ }
+ if (spaceProto != null) {
+ settings.add(QuotaSettingsFactory.fromSpace(getTableName(), getNamespace(), spaceProto));
+ }
+ return settings;
+ }
+
+ protected QuotaProtos.Throttle getThrottleProto() {
+ return this.throttleProto;
+ }
+
+ protected Boolean getBypassGlobals() {
+ return this.bypassGlobals;
+ }
+
+ protected QuotaProtos.SpaceQuota getSpaceProto() {
+ return this.spaceProto;
+ }
+
+ /**
+ * Constructs a new {@link Quotas} message from {@code this}.
+ */
+ protected Quotas toQuotas() {
+ QuotaProtos.Quotas.Builder builder = QuotaProtos.Quotas.newBuilder();
+ if (getThrottleProto() != null) {
+ builder.setThrottle(getThrottleProto());
+ }
+ if (getBypassGlobals() != null) {
+ builder.setBypassGlobals(getBypassGlobals());
+ }
+ if (getSpaceProto() != null) {
+ builder.setSpace(getSpaceProto());
+ }
+ return builder.build();
+ }
+
+ @Override
+ protected GlobalQuotaSettingsImpl merge(QuotaSettings other) throws IOException {
+ // Validate the quota subject
+ validateQuotaTarget(other);
+
+ // Propagate the Throttle
+ QuotaProtos.Throttle.Builder throttleBuilder = (throttleProto == null
+ ? null : throttleProto.toBuilder());
+ if (other instanceof ThrottleSettings) {
+ if (throttleBuilder == null) {
+ throttleBuilder = QuotaProtos.Throttle.newBuilder();
+ }
+ ThrottleSettings otherThrottle = (ThrottleSettings) other;
+
+ if (otherThrottle.proto.hasType()) {
+ QuotaProtos.ThrottleRequest otherProto = otherThrottle.proto;
+ if (otherProto.hasTimedQuota()) {
+ if (otherProto.hasTimedQuota()) {
+ validateTimedQuota(otherProto.getTimedQuota());
+ }
+
+ switch (otherProto.getType()) {
+ case REQUEST_NUMBER:
+ if (otherProto.hasTimedQuota()) {
+ throttleBuilder.setReqNum(otherProto.getTimedQuota());
+ } else {
+ throttleBuilder.clearReqNum();
+ }
+ break;
+ case REQUEST_SIZE:
+ if (otherProto.hasTimedQuota()) {
+ throttleBuilder.setReqSize(otherProto.getTimedQuota());
+ } else {
+ throttleBuilder.clearReqSize();
+ }
+ break;
+ case WRITE_NUMBER:
+ if (otherProto.hasTimedQuota()) {
+ throttleBuilder.setWriteNum(otherProto.getTimedQuota());
+ } else {
+ throttleBuilder.clearWriteNum();
+ }
+ break;
+ case WRITE_SIZE:
+ if (otherProto.hasTimedQuota()) {
+ throttleBuilder.setWriteSize(otherProto.getTimedQuota());
+ } else {
+ throttleBuilder.clearWriteSize();
+ }
+ break;
+ case READ_NUMBER:
+ if (otherProto.hasTimedQuota()) {
+ throttleBuilder.setReadNum(otherProto.getTimedQuota());
+ } else {
+ throttleBuilder.clearReqNum();
+ }
+ break;
+ case READ_SIZE:
+ if (otherProto.hasTimedQuota()) {
+ throttleBuilder.setReadSize(otherProto.getTimedQuota());
+ } else {
+ throttleBuilder.clearReadSize();
+ }
+ break;
+ }
+ } else {
+ clearThrottleBuilder(throttleBuilder);
+ }
+ } else {
+ clearThrottleBuilder(throttleBuilder);
+ }
+ }
+
+ // Propagate the space quota portion
+ QuotaProtos.SpaceQuota.Builder spaceBuilder = (spaceProto == null
+ ? null : spaceProto.toBuilder());
+ if (other instanceof SpaceLimitSettings) {
+ if (spaceBuilder == null) {
+ spaceBuilder = QuotaProtos.SpaceQuota.newBuilder();
+ }
+ SpaceLimitSettings settingsToMerge = (SpaceLimitSettings) other;
+
+ QuotaProtos.SpaceLimitRequest spaceRequest = settingsToMerge.getProto();
+
+ // The message contained the expect SpaceQuota object
+ if (spaceRequest.hasQuota()) {
+ SpaceQuota quotaToMerge = spaceRequest.getQuota();
+ // Validate that the two settings are for the same target.
+ // SpaceQuotas either apply to a table or a namespace (no user spacequota).
+ if (!Objects.equals(getTableName(), settingsToMerge.getTableName())
+ && !Objects.equals(getNamespace(), settingsToMerge.getNamespace())) {
+ throw new IllegalArgumentException(
+ "Cannot merge " + settingsToMerge + " into " + this);
+ }
+
+ if (quotaToMerge.getRemove()) {
+ // Update the builder to propagate the removal
+ spaceBuilder.setRemove(true).clearSoftLimit().clearViolationPolicy();
+ } else {
+ // Add the new settings to the existing settings
+ spaceBuilder.mergeFrom(quotaToMerge);
+ }
+ }
+ }
+
+ Boolean bypassGlobals = this.bypassGlobals;
+ if (other instanceof QuotaGlobalsSettingsBypass) {
+ bypassGlobals = ((QuotaGlobalsSettingsBypass) other).getBypass();
+ }
+
+ if (throttleBuilder == null &&
+ (spaceBuilder == null || (spaceBuilder.hasRemove() && spaceBuilder.getRemove()))
+ && bypassGlobals == null) {
+ return null;
+ }
+
+ return new GlobalQuotaSettingsImpl(
+ getUserName(), getTableName(), getNamespace(),
+ (throttleBuilder == null ? null : throttleBuilder.build()), bypassGlobals,
+ (spaceBuilder == null ? null : spaceBuilder.build()));
+ }
+
+ 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()));
+ }
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("GlobalQuota: ");
+ if (throttleProto != null) {
+ Map<ThrottleType,TimedQuota> throttleQuotas = buildThrottleQuotas(throttleProto);
+ builder.append(" { TYPE => THROTTLE ");
+ for (Entry<ThrottleType,TimedQuota> entry : throttleQuotas.entrySet()) {
+ final ThrottleType type = entry.getKey();
+ final TimedQuota timedQuota = entry.getValue();
+ builder.append("{THROTTLE_TYPE => ").append(type.name()).append(", LIMIT => ");
+ if (timedQuota.hasSoftLimit()) {
+ switch (type) {
+ case REQUEST_NUMBER:
+ case WRITE_NUMBER:
+ case READ_NUMBER:
+ builder.append(String.format("%dreq", timedQuota.getSoftLimit()));
+ break;
+ case REQUEST_SIZE:
+ case WRITE_SIZE:
+ case READ_SIZE:
+ builder.append(sizeToString(timedQuota.getSoftLimit()));
+ break;
+ }
+ } 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());
+ }
+ }
+ builder.append( "} } ");
+ } else {
+ builder.append(" {} ");
+ }
+ if (bypassGlobals != null) {
+ builder.append(" { GLOBAL_BYPASS => " + bypassGlobals + " } ");
+ }
+ if (spaceProto != null) {
+ builder.append(" { TYPE => SPACE");
+ if (getTableName() != null) {
+ builder.append(", TABLE => ").append(getTableName());
+ }
+ if (getNamespace() != null) {
+ builder.append(", NAMESPACE => ").append(getNamespace());
+ }
+ if (spaceProto.getRemove()) {
+ builder.append(", REMOVE => ").append(spaceProto.getRemove());
+ } else {
+ builder.append(", LIMIT => ").append(spaceProto.getSoftLimit());
+ builder.append(", VIOLATION_POLICY => ").append(spaceProto.getViolationPolicy());
+ }
+ builder.append(" } ");
+ }
+ return builder.toString();
+ }
+
+ private Map<ThrottleType,TimedQuota> buildThrottleQuotas(Throttle proto) {
+ HashMap<ThrottleType,TimedQuota> quotas = new HashMap<>();
+ if (proto.hasReadNum()) {
+ quotas.put(ThrottleType.READ_NUMBER, proto.getReadNum());
+ }
+ if (proto.hasReadSize()) {
+ quotas.put(ThrottleType.READ_SIZE, proto.getReadSize());
+ }
+ if (proto.hasReqNum()) {
+ quotas.put(ThrottleType.REQUEST_NUMBER, proto.getReqNum());
+ }
+ if (proto.hasReqSize()) {
+ quotas.put(ThrottleType.REQUEST_SIZE, proto.getReqSize());
+ }
+ if (proto.hasWriteNum()) {
+ quotas.put(ThrottleType.WRITE_NUMBER, proto.getWriteNum());
+ }
+ if (proto.hasWriteSize()) {
+ quotas.put(ThrottleType.WRITE_SIZE, proto.getWriteSize());
+ }
+ return quotas;
+ }
+
+ private void clearThrottleBuilder(QuotaProtos.Throttle.Builder builder) {
+ builder.clearReadNum();
+ builder.clearReadSize();
+ builder.clearReqNum();
+ builder.clearReqSize();
+ builder.clearWriteNum();
+ builder.clearWriteSize();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/81133f89/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
index 0587cc7..e4fa3ea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
@@ -153,12 +153,12 @@ public class MasterQuotaManager implements RegionStateListener {
throws IOException, InterruptedException {
setQuota(req, new SetQuotaOperations() {
@Override
- public GlobalQuotaSettings fetch() throws IOException {
- return new GlobalQuotaSettings(req.getUserName(), null, null, QuotaUtil.getUserQuota(
+ public GlobalQuotaSettingsImpl fetch() throws IOException {
+ return new GlobalQuotaSettingsImpl(req.getUserName(), null, null, QuotaUtil.getUserQuota(
masterServices.getConnection(), userName));
}
@Override
- public void update(GlobalQuotaSettings quotaPojo) throws IOException {
+ public void update(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
QuotaUtil.addUserQuota(masterServices.getConnection(), userName, quotaPojo.toQuotas());
}
@Override
@@ -166,11 +166,11 @@ public class MasterQuotaManager implements RegionStateListener {
QuotaUtil.deleteUserQuota(masterServices.getConnection(), userName);
}
@Override
- public void preApply(GlobalQuotaSettings quotaPojo) throws IOException {
+ public void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
masterServices.getMasterCoprocessorHost().preSetUserQuota(userName, quotaPojo);
}
@Override
- public void postApply(GlobalQuotaSettings quotaPojo) throws IOException {
+ public void postApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
masterServices.getMasterCoprocessorHost().postSetUserQuota(userName, quotaPojo);
}
});
@@ -180,12 +180,12 @@ public class MasterQuotaManager implements RegionStateListener {
final SetQuotaRequest req) throws IOException, InterruptedException {
setQuota(req, new SetQuotaOperations() {
@Override
- public GlobalQuotaSettings fetch() throws IOException {
- return new GlobalQuotaSettings(userName, table, null, QuotaUtil.getUserQuota(
+ public GlobalQuotaSettingsImpl fetch() throws IOException {
+ return new GlobalQuotaSettingsImpl(userName, table, null, QuotaUtil.getUserQuota(
masterServices.getConnection(), userName, table));
}
@Override
- public void update(GlobalQuotaSettings quotaPojo) throws IOException {
+ public void update(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
QuotaUtil.addUserQuota(masterServices.getConnection(), userName, table,
quotaPojo.toQuotas());
}
@@ -194,11 +194,11 @@ public class MasterQuotaManager implements RegionStateListener {
QuotaUtil.deleteUserQuota(masterServices.getConnection(), userName, table);
}
@Override
- public void preApply(GlobalQuotaSettings quotaPojo) throws IOException {
+ public void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
masterServices.getMasterCoprocessorHost().preSetUserQuota(userName, table, quotaPojo);
}
@Override
- public void postApply(GlobalQuotaSettings quotaPojo) throws IOException {
+ public void postApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
masterServices.getMasterCoprocessorHost().postSetUserQuota(userName, table, quotaPojo);
}
});
@@ -208,12 +208,12 @@ public class MasterQuotaManager implements RegionStateListener {
final SetQuotaRequest req) throws IOException, InterruptedException {
setQuota(req, new SetQuotaOperations() {
@Override
- public GlobalQuotaSettings fetch() throws IOException {
- return new GlobalQuotaSettings(userName, null, namespace, QuotaUtil.getUserQuota(
+ public GlobalQuotaSettingsImpl fetch() throws IOException {
+ return new GlobalQuotaSettingsImpl(userName, null, namespace, QuotaUtil.getUserQuota(
masterServices.getConnection(), userName, namespace));
}
@Override
- public void update(GlobalQuotaSettings quotaPojo) throws IOException {
+ public void update(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
QuotaUtil.addUserQuota(masterServices.getConnection(), userName, namespace,
quotaPojo.toQuotas());
}
@@ -222,12 +222,12 @@ public class MasterQuotaManager implements RegionStateListener {
QuotaUtil.deleteUserQuota(masterServices.getConnection(), userName, namespace);
}
@Override
- public void preApply(GlobalQuotaSettings quotaPojo) throws IOException {
+ public void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
masterServices.getMasterCoprocessorHost().preSetUserQuota(
userName, namespace, quotaPojo);
}
@Override
- public void postApply(GlobalQuotaSettings quotaPojo) throws IOException {
+ public void postApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
masterServices.getMasterCoprocessorHost().postSetUserQuota(
userName, namespace, quotaPojo);
}
@@ -238,12 +238,12 @@ public class MasterQuotaManager implements RegionStateListener {
throws IOException, InterruptedException {
setQuota(req, new SetQuotaOperations() {
@Override
- public GlobalQuotaSettings fetch() throws IOException {
- return new GlobalQuotaSettings(null, table, null, QuotaUtil.getTableQuota(
+ public GlobalQuotaSettingsImpl fetch() throws IOException {
+ return new GlobalQuotaSettingsImpl(null, table, null, QuotaUtil.getTableQuota(
masterServices.getConnection(), table));
}
@Override
- public void update(GlobalQuotaSettings quotaPojo) throws IOException {
+ public void update(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
QuotaUtil.addTableQuota(masterServices.getConnection(), table, quotaPojo.toQuotas());
}
@Override
@@ -251,11 +251,11 @@ public class MasterQuotaManager implements RegionStateListener {
QuotaUtil.deleteTableQuota(masterServices.getConnection(), table);
}
@Override
- public void preApply(GlobalQuotaSettings quotaPojo) throws IOException {
+ public void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
masterServices.getMasterCoprocessorHost().preSetTableQuota(table, quotaPojo);
}
@Override
- public void postApply(GlobalQuotaSettings quotaPojo) throws IOException {
+ public void postApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
masterServices.getMasterCoprocessorHost().postSetTableQuota(table, quotaPojo);
}
});
@@ -265,25 +265,25 @@ public class MasterQuotaManager implements RegionStateListener {
throws IOException, InterruptedException {
setQuota(req, new SetQuotaOperations() {
@Override
- public GlobalQuotaSettings fetch() throws IOException {
- return new GlobalQuotaSettings(null, null, namespace, QuotaUtil.getNamespaceQuota(
+ public GlobalQuotaSettingsImpl fetch() throws IOException {
+ return new GlobalQuotaSettingsImpl(null, null, namespace, QuotaUtil.getNamespaceQuota(
masterServices.getConnection(), namespace));
}
@Override
- public void update(GlobalQuotaSettings quotaPojo) throws IOException {
+ public void update(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
QuotaUtil.addNamespaceQuota(masterServices.getConnection(), namespace,
- ((GlobalQuotaSettings) quotaPojo).toQuotas());
+ ((GlobalQuotaSettingsImpl) quotaPojo).toQuotas());
}
@Override
public void delete() throws IOException {
QuotaUtil.deleteNamespaceQuota(masterServices.getConnection(), namespace);
}
@Override
- public void preApply(GlobalQuotaSettings quotaPojo) throws IOException {
+ public void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
masterServices.getMasterCoprocessorHost().preSetNamespaceQuota(namespace, quotaPojo);
}
@Override
- public void postApply(GlobalQuotaSettings quotaPojo) throws IOException {
+ public void postApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
masterServices.getMasterCoprocessorHost().postSetNamespaceQuota(namespace, quotaPojo);
}
});
@@ -311,7 +311,7 @@ public class MasterQuotaManager implements RegionStateListener {
}
// Apply quota changes
- GlobalQuotaSettings currentQuota = quotaOps.fetch();
+ GlobalQuotaSettingsImpl currentQuota = quotaOps.fetch();
if (LOG.isTraceEnabled()) {
LOG.trace(
"Current quota for request(" + TextFormat.shortDebugString(req)
@@ -329,7 +329,7 @@ public class MasterQuotaManager implements RegionStateListener {
//
// NB: while SetQuotaRequest technically allows for multi types of quotas to be set in one
// message, the Java API (in Admin/AsyncAdmin) does not. Assume there is only one type.
- GlobalQuotaSettings mergedQuota = currentQuota.merge(newQuota);
+ GlobalQuotaSettingsImpl mergedQuota = currentQuota.merge(newQuota);
if (LOG.isTraceEnabled()) {
LOG.trace("Computed merged quota from current quota and user request: " + mergedQuota);
}
@@ -403,7 +403,7 @@ public class MasterQuotaManager implements RegionStateListener {
/**
* Fetches the current quota settings for the subject.
*/
- GlobalQuotaSettings fetch() throws IOException;
+ GlobalQuotaSettingsImpl fetch() throws IOException;
/**
* Deletes the quota for the subject.
*/
@@ -411,17 +411,17 @@ public class MasterQuotaManager implements RegionStateListener {
/**
* Persist the given quota for the subject.
*/
- void update(GlobalQuotaSettings quotaPojo) throws IOException;
+ void update(GlobalQuotaSettingsImpl quotaPojo) throws IOException;
/**
- * Performs some action before {@link #update(GlobalQuotaSettings)} with the current quota
- * for the subject.
+ * Performs some action before {@link #update(GlobalQuotaSettingsImpl)} with the current
+ * quota for the subject.
*/
- void preApply(GlobalQuotaSettings quotaPojo) throws IOException;
+ void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException;
/**
- * Performs some action after {@link #update(GlobalQuotaSettings)} with the resulting quota
- * from the request action for the subject.
+ * Performs some action after {@link #update(GlobalQuotaSettingsImpl)} with the resulting
+ * quota from the request action for the subject.
*/
- void postApply(GlobalQuotaSettings quotaPojo) throws IOException;
+ void postApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException;
}
/* ==========================================================================
http://git-wip-us.apache.org/repos/asf/hbase/blob/81133f89/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestGlobalQuotaSettings.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestGlobalQuotaSettings.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestGlobalQuotaSettings.java
deleted file mode 100644
index c4213cb..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestGlobalQuotaSettings.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * 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.assertTrue;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(SmallTests.class)
-public class TestGlobalQuotaSettings {
-
- QuotaProtos.TimedQuota REQUEST_THROTTLE = QuotaProtos.TimedQuota.newBuilder()
- .setScope(QuotaProtos.QuotaScope.MACHINE).setSoftLimit(100)
- .setTimeUnit(HBaseProtos.TimeUnit.MINUTES).build();
- QuotaProtos.Throttle THROTTLE = QuotaProtos.Throttle.newBuilder()
- .setReqNum(REQUEST_THROTTLE).build();
-
- QuotaProtos.SpaceQuota SPACE_QUOTA = QuotaProtos.SpaceQuota.newBuilder()
- .setSoftLimit(1024L * 1024L).setViolationPolicy(QuotaProtos.SpaceViolationPolicy.NO_WRITES)
- .build();
-
- @Test
- public void testMergeThrottle() throws IOException {
- QuotaProtos.Quotas quota = QuotaProtos.Quotas.newBuilder()
- .setThrottle(THROTTLE).build();
- QuotaProtos.TimedQuota writeQuota = REQUEST_THROTTLE.toBuilder()
- .setSoftLimit(500).build();
- // Unset the req throttle, set a write throttle
- QuotaProtos.ThrottleRequest writeThrottle = QuotaProtos.ThrottleRequest.newBuilder()
- .setTimedQuota(writeQuota).setType(QuotaProtos.ThrottleType.WRITE_NUMBER).build();
-
- GlobalQuotaSettings settings = new GlobalQuotaSettings("joe", null, null, quota);
- GlobalQuotaSettings merged = settings.merge(
- new ThrottleSettings("joe", null, null, writeThrottle));
-
- QuotaProtos.Throttle mergedThrottle = merged.getThrottleProto();
- // Verify the request throttle is in place
- assertTrue(mergedThrottle.hasReqNum());
- QuotaProtos.TimedQuota actualReqNum = mergedThrottle.getReqNum();
- assertEquals(REQUEST_THROTTLE.getSoftLimit(), actualReqNum.getSoftLimit());
-
- // Verify the write throttle is in place
- assertTrue(mergedThrottle.hasWriteNum());
- QuotaProtos.TimedQuota actualWriteNum = mergedThrottle.getWriteNum();
- assertEquals(writeQuota.getSoftLimit(), actualWriteNum.getSoftLimit());
- }
-
- @Test
- public void testMergeSpace() throws IOException {
- TableName tn = TableName.valueOf("foo");
- QuotaProtos.Quotas quota = QuotaProtos.Quotas.newBuilder()
- .setSpace(SPACE_QUOTA).build();
-
- GlobalQuotaSettings settings = new GlobalQuotaSettings(null, tn, null, quota);
- // Switch the violation policy to DISABLE
- GlobalQuotaSettings merged = settings.merge(
- new SpaceLimitSettings(tn, SPACE_QUOTA.getSoftLimit(), SpaceViolationPolicy.DISABLE));
-
- QuotaProtos.SpaceQuota mergedSpaceQuota = merged.getSpaceProto();
- assertEquals(SPACE_QUOTA.getSoftLimit(), mergedSpaceQuota.getSoftLimit());
- assertEquals(
- QuotaProtos.SpaceViolationPolicy.DISABLE, mergedSpaceQuota.getViolationPolicy());
- }
-
- @Test
- public void testMergeThrottleAndSpace() throws IOException {
- final String ns = "org1";
- QuotaProtos.Quotas quota = QuotaProtos.Quotas.newBuilder()
- .setThrottle(THROTTLE).setSpace(SPACE_QUOTA).build();
- GlobalQuotaSettings settings = new GlobalQuotaSettings(null, null, ns, quota);
-
- QuotaProtos.TimedQuota writeQuota = REQUEST_THROTTLE.toBuilder()
- .setSoftLimit(500).build();
- // Add a write throttle
- QuotaProtos.ThrottleRequest writeThrottle = QuotaProtos.ThrottleRequest.newBuilder()
- .setTimedQuota(writeQuota).setType(QuotaProtos.ThrottleType.WRITE_NUMBER).build();
-
- GlobalQuotaSettings merged = settings.merge(
- new ThrottleSettings(null, null, ns, writeThrottle));
- GlobalQuotaSettings finalQuota = merged.merge(new SpaceLimitSettings(
- ns, SPACE_QUOTA.getSoftLimit(), SpaceViolationPolicy.NO_WRITES_COMPACTIONS));
-
- // Verify both throttle quotas
- QuotaProtos.Throttle throttle = finalQuota.getThrottleProto();
- assertTrue(throttle.hasReqNum());
- QuotaProtos.TimedQuota reqNumQuota = throttle.getReqNum();
- assertEquals(REQUEST_THROTTLE.getSoftLimit(), reqNumQuota.getSoftLimit());
-
- assertTrue(throttle.hasWriteNum());
- QuotaProtos.TimedQuota writeNumQuota = throttle.getWriteNum();
- assertEquals(writeQuota.getSoftLimit(), writeNumQuota.getSoftLimit());
-
- // Verify space quota
- QuotaProtos.SpaceQuota finalSpaceQuota = finalQuota.getSpaceProto();
- assertEquals(SPACE_QUOTA.getSoftLimit(), finalSpaceQuota.getSoftLimit());
- assertEquals(
- QuotaProtos.SpaceViolationPolicy.NO_WRITES_COMPACTIONS,
- finalSpaceQuota.getViolationPolicy());
- }
-}
http://git-wip-us.apache.org/repos/asf/hbase/blob/81133f89/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestGlobalQuotaSettingsImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestGlobalQuotaSettingsImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestGlobalQuotaSettingsImpl.java
new file mode 100644
index 0000000..9e5434b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestGlobalQuotaSettingsImpl.java
@@ -0,0 +1,122 @@
+/*
+ * 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.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestGlobalQuotaSettingsImpl {
+
+ QuotaProtos.TimedQuota REQUEST_THROTTLE = QuotaProtos.TimedQuota.newBuilder()
+ .setScope(QuotaProtos.QuotaScope.MACHINE).setSoftLimit(100)
+ .setTimeUnit(HBaseProtos.TimeUnit.MINUTES).build();
+ QuotaProtos.Throttle THROTTLE = QuotaProtos.Throttle.newBuilder()
+ .setReqNum(REQUEST_THROTTLE).build();
+
+ QuotaProtos.SpaceQuota SPACE_QUOTA = QuotaProtos.SpaceQuota.newBuilder()
+ .setSoftLimit(1024L * 1024L).setViolationPolicy(QuotaProtos.SpaceViolationPolicy.NO_WRITES)
+ .build();
+
+ @Test
+ public void testMergeThrottle() throws IOException {
+ QuotaProtos.Quotas quota = QuotaProtos.Quotas.newBuilder()
+ .setThrottle(THROTTLE).build();
+ QuotaProtos.TimedQuota writeQuota = REQUEST_THROTTLE.toBuilder()
+ .setSoftLimit(500).build();
+ // Unset the req throttle, set a write throttle
+ QuotaProtos.ThrottleRequest writeThrottle = QuotaProtos.ThrottleRequest.newBuilder()
+ .setTimedQuota(writeQuota).setType(QuotaProtos.ThrottleType.WRITE_NUMBER).build();
+
+ GlobalQuotaSettingsImpl settings = new GlobalQuotaSettingsImpl("joe", null, null, quota);
+ GlobalQuotaSettingsImpl merged = settings.merge(
+ new ThrottleSettings("joe", null, null, writeThrottle));
+
+ QuotaProtos.Throttle mergedThrottle = merged.getThrottleProto();
+ // Verify the request throttle is in place
+ assertTrue(mergedThrottle.hasReqNum());
+ QuotaProtos.TimedQuota actualReqNum = mergedThrottle.getReqNum();
+ assertEquals(REQUEST_THROTTLE.getSoftLimit(), actualReqNum.getSoftLimit());
+
+ // Verify the write throttle is in place
+ assertTrue(mergedThrottle.hasWriteNum());
+ QuotaProtos.TimedQuota actualWriteNum = mergedThrottle.getWriteNum();
+ assertEquals(writeQuota.getSoftLimit(), actualWriteNum.getSoftLimit());
+ }
+
+ @Test
+ public void testMergeSpace() throws IOException {
+ TableName tn = TableName.valueOf("foo");
+ QuotaProtos.Quotas quota = QuotaProtos.Quotas.newBuilder()
+ .setSpace(SPACE_QUOTA).build();
+
+ GlobalQuotaSettingsImpl settings = new GlobalQuotaSettingsImpl(null, tn, null, quota);
+ // Switch the violation policy to DISABLE
+ GlobalQuotaSettingsImpl merged = settings.merge(
+ new SpaceLimitSettings(tn, SPACE_QUOTA.getSoftLimit(), SpaceViolationPolicy.DISABLE));
+
+ QuotaProtos.SpaceQuota mergedSpaceQuota = merged.getSpaceProto();
+ assertEquals(SPACE_QUOTA.getSoftLimit(), mergedSpaceQuota.getSoftLimit());
+ assertEquals(
+ QuotaProtos.SpaceViolationPolicy.DISABLE, mergedSpaceQuota.getViolationPolicy());
+ }
+
+ @Test
+ public void testMergeThrottleAndSpace() throws IOException {
+ final String ns = "org1";
+ QuotaProtos.Quotas quota = QuotaProtos.Quotas.newBuilder()
+ .setThrottle(THROTTLE).setSpace(SPACE_QUOTA).build();
+ GlobalQuotaSettingsImpl settings = new GlobalQuotaSettingsImpl(null, null, ns, quota);
+
+ QuotaProtos.TimedQuota writeQuota = REQUEST_THROTTLE.toBuilder()
+ .setSoftLimit(500).build();
+ // Add a write throttle
+ QuotaProtos.ThrottleRequest writeThrottle = QuotaProtos.ThrottleRequest.newBuilder()
+ .setTimedQuota(writeQuota).setType(QuotaProtos.ThrottleType.WRITE_NUMBER).build();
+
+ GlobalQuotaSettingsImpl merged = settings.merge(
+ new ThrottleSettings(null, null, ns, writeThrottle));
+ GlobalQuotaSettingsImpl finalQuota = merged.merge(new SpaceLimitSettings(
+ ns, SPACE_QUOTA.getSoftLimit(), SpaceViolationPolicy.NO_WRITES_COMPACTIONS));
+
+ // Verify both throttle quotas
+ QuotaProtos.Throttle throttle = finalQuota.getThrottleProto();
+ assertTrue(throttle.hasReqNum());
+ QuotaProtos.TimedQuota reqNumQuota = throttle.getReqNum();
+ assertEquals(REQUEST_THROTTLE.getSoftLimit(), reqNumQuota.getSoftLimit());
+
+ assertTrue(throttle.hasWriteNum());
+ QuotaProtos.TimedQuota writeNumQuota = throttle.getWriteNum();
+ assertEquals(writeQuota.getSoftLimit(), writeNumQuota.getSoftLimit());
+
+ // Verify space quota
+ QuotaProtos.SpaceQuota finalSpaceQuota = finalQuota.getSpaceProto();
+ assertEquals(SPACE_QUOTA.getSoftLimit(), finalSpaceQuota.getSoftLimit());
+ assertEquals(
+ QuotaProtos.SpaceViolationPolicy.NO_WRITES_COMPACTIONS,
+ finalSpaceQuota.getViolationPolicy());
+ }
+}
|