HBASE-17001 Enforce quota violation policies in the RegionServer

The nuts-and-bolts of filesystem quotas. The Master must inform
RegionServers of the violation of a quota by a table. The RegionServer
must apply the violation policy as configured. Need to ensure
that the proper interfaces exist to satisfy all necessary policies.

This required a massive rewrite of the internal tracking by
the general space quota feature. Instead of tracking "violations",
we need to start tracking "usage". This allows us to make the decision
at the RegionServer level as to when the files in a bulk load request
should be accept or rejected which ultimately lets us avoid bulk loads
dramatically exceeding a configured space quota.
This commit is contained in:
Josh Elser 2016-12-15 13:27:56 -05:00
parent 98b4181f43
commit 34ba143fc8
50 changed files with 6041 additions and 1031 deletions

View File

@ -29,4 +29,8 @@ public class QuotaExceededException extends DoNotRetryIOException {
public QuotaExceededException(String msg) {
super(msg);
}
public QuotaExceededException(String msg, Throwable cause) {
super(msg, cause);
}
}

View File

@ -35,7 +35,6 @@ 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.Put;
import org.apache.hadoop.hbase.client.Result;
@ -52,6 +51,7 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
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.util.Bytes;
@ -62,8 +62,9 @@ import org.apache.hadoop.hbase.util.Strings;
* <pre>
* ROW-KEY FAM/QUAL DATA
* n.&lt;namespace&gt; q:s &lt;global-quotas&gt;
* t.&lt;namespace&gt; u:p &lt;namespace-quota policy&gt;
* t.&lt;table&gt; q:s &lt;global-quotas&gt;
* t.&lt;table&gt; u:v &lt;space violation policy&gt;
* t.&lt;table&gt; u:p &lt;table-quota policy&gt;
* u.&lt;user&gt; q:s &lt;global-quotas&gt;
* u.&lt;user&gt; q:s.&lt;table&gt; &lt;table-quotas&gt;
* u.&lt;user&gt; q:s.&lt;ns&gt;: &lt;namespace-quotas&gt;
@ -82,7 +83,9 @@ public class QuotaTableUtil {
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_QUALIFIER_VIOLATION = Bytes.toBytes("v");
protected static final byte[] QUOTA_QUALIFIER_POLICY = Bytes.toBytes("p");
protected static final String QUOTA_POLICY_COLUMN =
Bytes.toString(QUOTA_FAMILY_USAGE) + ":" + Bytes.toString(QUOTA_QUALIFIER_POLICY);
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.");
@ -214,10 +217,10 @@ public class QuotaTableUtil {
/**
* Creates a {@link Scan} which returns only quota violations from the quota table.
*/
public static Scan makeQuotaViolationScan() {
public static Scan makeQuotaSnapshotScan() {
Scan s = new Scan();
// Limit to "u:v" column
s.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_VIOLATION);
s.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_POLICY);
// Limit rowspace to the "t:" prefix
s.setRowPrefixFilter(QUOTA_TABLE_ROW_KEY_PREFIX);
return s;
@ -230,26 +233,25 @@ public class QuotaTableUtil {
* will throw an {@link IllegalArgumentException}.
*
* @param result A row from the quota table.
* @param policies A map of policies to add the result of this method into.
* @param snapshots A map of violations to add the result of this method into.
*/
public static void extractViolationPolicy(
Result result, Map<TableName,SpaceViolationPolicy> policies) {
public static void extractQuotaSnapshot(
Result result, Map<TableName,SpaceQuotaSnapshot> snapshots) {
byte[] row = Objects.requireNonNull(result).getRow();
if (null == row) {
throw new IllegalArgumentException("Provided result had a null row");
}
final TableName targetTableName = getTableFromRowKey(row);
Cell c = result.getColumnLatestCell(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_VIOLATION);
Cell c = result.getColumnLatestCell(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_POLICY);
if (null == c) {
throw new IllegalArgumentException("Result did not contain the expected column "
+ Bytes.toString(QUOTA_FAMILY_USAGE) + ":" + Bytes.toString(QUOTA_QUALIFIER_VIOLATION)
+ ", " + result.toString());
+ QUOTA_POLICY_COLUMN + ", " + result.toString());
}
ByteString buffer = UnsafeByteOperations.unsafeWrap(
c.getValueArray(), c.getValueOffset(), c.getValueLength());
try {
SpaceQuota quota = SpaceQuota.parseFrom(buffer);
policies.put(targetTableName, getViolationPolicy(quota));
QuotaProtos.SpaceQuotaSnapshot snapshot = QuotaProtos.SpaceQuotaSnapshot.parseFrom(buffer);
snapshots.put(targetTableName, SpaceQuotaSnapshot.toSpaceQuotaSnapshot(snapshot));
} catch (InvalidProtocolBufferException e) {
throw new IllegalArgumentException(
"Result did not contain a valid SpaceQuota protocol buffer message", e);
@ -385,23 +387,14 @@ public class QuotaTableUtil {
/**
* Creates a {@link Put} to enable the given <code>policy</code> on the <code>table</code>.
*/
public static Put createEnableViolationPolicyUpdate(
TableName tableName, SpaceViolationPolicy policy) {
public static Put createPutSpaceSnapshot(TableName tableName, SpaceQuotaSnapshot snapshot) {
Put p = new Put(getTableRowKey(tableName));
SpaceQuota quota = getProtoViolationPolicy(policy);
p.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_VIOLATION, quota.toByteArray());
p.addColumn(
QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_POLICY,
SpaceQuotaSnapshot.toProtoSnapshot(snapshot).toByteArray());
return p;
}
/**
* Creates a {@link Delete} to remove a policy on the given <code>table</code>.
*/
public static Delete createRemoveViolationPolicyUpdate(TableName tableName) {
Delete d = new Delete(getTableRowKey(tableName));
d.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_VIOLATION);
return d;
}
/* =========================================================================
* Quotas protobuf helpers
*/
@ -536,4 +529,4 @@ public class QuotaTableUtil {
}
return ProtobufUtil.toViolationPolicy(proto.getViolationPolicy());
}
}
}

View File

@ -0,0 +1,192 @@
/*
* 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.Objects;
import org.apache.commons.lang.builder.HashCodeBuilder;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
/**
* A point-in-time view of a space quota on a table.
*/
@InterfaceAudience.Private
public class SpaceQuotaSnapshot {
private static final SpaceQuotaSnapshot NO_SUCH_SNAPSHOT = new SpaceQuotaSnapshot(
SpaceQuotaStatus.notInViolation(), 0, Long.MAX_VALUE);
private final SpaceQuotaStatus quotaStatus;
private final long usage;
private final long limit;
/**
* Encapsulates the state of a quota on a table. The quota may or may not be in violation.
* If it is in violation, there will be a non-null violation policy.
*/
@InterfaceAudience.Private
public static class SpaceQuotaStatus {
private static final SpaceQuotaStatus NOT_IN_VIOLATION = new SpaceQuotaStatus(null, false);
final SpaceViolationPolicy policy;
final boolean inViolation;
public SpaceQuotaStatus(SpaceViolationPolicy policy) {
this.policy = Objects.requireNonNull(policy);
this.inViolation = true;
}
private SpaceQuotaStatus(SpaceViolationPolicy policy, boolean inViolation) {
this.policy = policy;
this.inViolation = inViolation;
}
/**
* The violation policy which may be null. Is guaranteed to be non-null if
* {@link #isInViolation()} is <code>true</code>, and <code>false</code>
* otherwise.
*/
public SpaceViolationPolicy getPolicy() {
return policy;
}
/**
* <code>true</code> if the quota is being violated, <code>false</code> otherwise.
*/
public boolean isInViolation() {
return inViolation;
}
/**
* Returns a singleton referring to a quota which is not in violation.
*/
public static SpaceQuotaStatus notInViolation() {
return NOT_IN_VIOLATION;
}
@Override
public int hashCode() {
return new HashCodeBuilder().append(policy == null ? 0 : policy.hashCode())
.append(inViolation).toHashCode();
}
@Override
public boolean equals(Object o) {
if (o instanceof SpaceQuotaStatus) {
SpaceQuotaStatus other = (SpaceQuotaStatus) o;
return Objects.equals(policy, other.policy) && inViolation == other.inViolation;
}
return false;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder(getClass().getSimpleName());
sb.append("[policy=").append(policy);
sb.append(", inViolation=").append(inViolation).append("]");
return sb.toString();
}
public static QuotaProtos.SpaceQuotaStatus toProto(SpaceQuotaStatus status) {
QuotaProtos.SpaceQuotaStatus.Builder builder = QuotaProtos.SpaceQuotaStatus.newBuilder();
builder.setInViolation(status.inViolation);
if (status.isInViolation()) {
builder.setPolicy(ProtobufUtil.toProtoViolationPolicy(status.getPolicy()));
}
return builder.build();
}
public static SpaceQuotaStatus toStatus(QuotaProtos.SpaceQuotaStatus proto) {
if (proto.getInViolation()) {
return new SpaceQuotaStatus(ProtobufUtil.toViolationPolicy(proto.getPolicy()));
} else {
return NOT_IN_VIOLATION;
}
}
}
public SpaceQuotaSnapshot(SpaceQuotaStatus quotaStatus, long usage, long limit) {
this.quotaStatus = Objects.requireNonNull(quotaStatus);
this.usage = usage;
this.limit = limit;
}
/**
* Returns the status of the quota.
*/
public SpaceQuotaStatus getQuotaStatus() {
return quotaStatus;
}
/**
* Returns the current usage, in bytes, of the target (e.g. table, namespace).
*/
public long getUsage() {
return usage;
}
/**
* Returns the limit, in bytes, of the target (e.g. table, namespace).
*/
public long getLimit() {
return limit;
}
@Override
public int hashCode() {
return new HashCodeBuilder()
.append(quotaStatus.hashCode())
.append(usage)
.append(limit)
.toHashCode();
}
@Override
public boolean equals(Object o) {
if (o instanceof SpaceQuotaSnapshot) {
SpaceQuotaSnapshot other = (SpaceQuotaSnapshot) o;
return quotaStatus.equals(other.quotaStatus) && usage == other.usage && limit == other.limit;
}
return false;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder(32);
sb.append("SpaceQuotaSnapshot[policy=").append(quotaStatus).append(", use=");
sb.append(usage).append("bytes/").append(limit).append("bytes]");
return sb.toString();
}
// ProtobufUtil is in hbase-client, and this doesn't need to be public.
public static SpaceQuotaSnapshot toSpaceQuotaSnapshot(QuotaProtos.SpaceQuotaSnapshot proto) {
return new SpaceQuotaSnapshot(SpaceQuotaStatus.toStatus(proto.getStatus()),
proto.getUsage(), proto.getLimit());
}
public static QuotaProtos.SpaceQuotaSnapshot toProtoSnapshot(SpaceQuotaSnapshot snapshot) {
return QuotaProtos.SpaceQuotaSnapshot.newBuilder()
.setStatus(SpaceQuotaStatus.toProto(snapshot.getQuotaStatus()))
.setUsage(snapshot.getUsage()).setLimit(snapshot.getLimit()).build();
}
/**
* Returns a singleton that corresponds to no snapshot information.
*/
public static SpaceQuotaSnapshot getNoSuchSnapshot() {
return NO_SUCH_SNAPSHOT;
}
}

View File

@ -95,3 +95,18 @@ message SpaceQuota {
message SpaceLimitRequest {
optional SpaceQuota quota = 1;
}
// Represents the state of a quota on a table. Either the quota is not in violation
// or it is in violatino there is a violation policy which should be in effect.
message SpaceQuotaStatus {
optional SpaceViolationPolicy policy = 1;
optional bool in_violation = 2;
}
// Message stored in the value of hbase:quota table to denote the status of a table WRT
// the quota applicable to it.
message SpaceQuotaSnapshot {
optional SpaceQuotaStatus status = 1;
optional uint64 usage = 2;
optional uint64 limit = 3;
}

View File

@ -95,3 +95,18 @@ message SpaceQuota {
message SpaceLimitRequest {
optional SpaceQuota quota = 1;
}
// Represents the state of a quota on a table. Either the quota is not in violation
// or it is in violatino there is a violation policy which should be in effect.
message SpaceQuotaStatus {
optional SpaceViolationPolicy policy = 1;
optional bool in_violation = 2;
}
// Message stored in the value of hbase:quota table to denote the status of a table WRT
// the quota applicable to it.
message SpaceQuotaSnapshot {
optional SpaceQuotaStatus status = 1;
optional uint64 usage = 2;
optional uint64 limit = 3;
}

View File

@ -136,8 +136,8 @@ import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
import org.apache.hadoop.hbase.quotas.QuotaUtil;
import org.apache.hadoop.hbase.quotas.SpaceQuotaViolationNotifier;
import org.apache.hadoop.hbase.quotas.SpaceQuotaViolationNotifierFactory;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifier;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifierFactory;
import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.HStore;
@ -379,7 +379,7 @@ public class HMaster extends HRegionServer implements MasterServices {
// it is assigned after 'initialized' guard set to true, so should be volatile
private volatile MasterQuotaManager quotaManager;
private SpaceQuotaViolationNotifier spaceQuotaViolationNotifier;
private SpaceQuotaSnapshotNotifier spaceQuotaSnapshotNotifier;
private QuotaObserverChore quotaObserverChore;
private ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
@ -908,10 +908,14 @@ public class HMaster extends HRegionServer implements MasterServices {
status.setStatus("Starting quota manager");
initQuotaManager();
this.spaceQuotaViolationNotifier = createQuotaViolationNotifier();
this.quotaObserverChore = new QuotaObserverChore(this);
// Start the chore to read the region FS space reports and act on them
getChoreService().scheduleChore(quotaObserverChore);
if (QuotaUtil.isQuotaEnabled(conf)) {
// Create the quota snapshot notifier
spaceQuotaSnapshotNotifier = createQuotaSnapshotNotifier();
spaceQuotaSnapshotNotifier.initialize(getClusterConnection());
this.quotaObserverChore = new QuotaObserverChore(this);
// Start the chore to read the region FS space reports and act on them
getChoreService().scheduleChore(quotaObserverChore);
}
// clear the dead servers with same host name and port of online server because we are not
// removing dead server with same hostname and port of rs which is trying to check in before
@ -999,10 +1003,9 @@ public class HMaster extends HRegionServer implements MasterServices {
this.quotaManager = quotaManager;
}
SpaceQuotaViolationNotifier createQuotaViolationNotifier() {
SpaceQuotaViolationNotifier notifier =
SpaceQuotaViolationNotifierFactory.getInstance().create(getConfiguration());
notifier.initialize(getClusterConnection());
SpaceQuotaSnapshotNotifier createQuotaSnapshotNotifier() {
SpaceQuotaSnapshotNotifier notifier =
SpaceQuotaSnapshotNotifierFactory.getInstance().create(getConfiguration());
return notifier;
}
@ -3394,7 +3397,7 @@ public class HMaster extends HRegionServer implements MasterServices {
return this.quotaObserverChore;
}
public SpaceQuotaViolationNotifier getSpaceQuotaViolationNotifier() {
return this.spaceQuotaViolationNotifier;
public SpaceQuotaSnapshotNotifier getSpaceQuotaSnapshotNotifier() {
return this.spaceQuotaSnapshotNotifier;
}
}

View File

@ -0,0 +1,86 @@
/*
* 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.Map;
import java.util.Objects;
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.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
/**
* A class to ease dealing with tables that have and do not have violation policies
* being enforced in a uniform manner. Immutable.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class ActivePolicyEnforcement {
private final Map<TableName,SpaceViolationPolicyEnforcement> activePolicies;
private final Map<TableName,SpaceQuotaSnapshot> snapshots;
private final RegionServerServices rss;
public ActivePolicyEnforcement(Map<TableName,SpaceViolationPolicyEnforcement> activePolicies,
Map<TableName,SpaceQuotaSnapshot> snapshots, RegionServerServices rss) {
this.activePolicies = activePolicies;
this.snapshots = snapshots;
this.rss = rss;
}
/**
* Returns the proper {@link SpaceViolationPolicyEnforcement} implementation for the given table.
* If the given table does not have a violation policy enforced, a "no-op" policy will
* be returned which always allows an action.
*
* @see #getPolicyEnforcement(TableName)
*/
public SpaceViolationPolicyEnforcement getPolicyEnforcement(Region r) {
return getPolicyEnforcement(Objects.requireNonNull(r).getTableDesc().getTableName());
}
/**
* Returns the proper {@link SpaceViolationPolicyEnforcement} implementation for the given table.
* If the given table does not have a violation policy enforced, a "no-op" policy will
* be returned which always allows an action.
*
* @param tableName The table to fetch the policy for.
* @return A non-null {@link SpaceViolationPolicyEnforcement} instance.
*/
public SpaceViolationPolicyEnforcement getPolicyEnforcement(TableName tableName) {
SpaceViolationPolicyEnforcement policy = activePolicies.get(Objects.requireNonNull(tableName));
if (null == policy) {
synchronized (activePolicies) {
// If we've never seen a snapshot, assume no use, and infinite limit
SpaceQuotaSnapshot snapshot = snapshots.get(tableName);
if (null == snapshot) {
snapshot = SpaceQuotaSnapshot.getNoSuchSnapshot();
}
// Create the default policy and cache it
return SpaceViolationPolicyEnforcementFactory.getInstance().createWithoutViolation(
rss, tableName, snapshot);
}
}
return policy;
}
@Override
public String toString() {
return getClass().getSimpleName() + ": " + activePolicies;
}
}

View File

@ -27,6 +27,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
@ -34,10 +36,10 @@ import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
/**
* {@link QuotaViolationStore} implementation for namespaces.
* {@link QuotaSnapshotStore} implementation for namespaces.
*/
@InterfaceAudience.Private
public class NamespaceQuotaViolationStore implements QuotaViolationStore<String> {
public class NamespaceQuotaSnapshotStore implements QuotaSnapshotStore<String> {
private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
private final ReadLock rlock = lock.readLock();
private final WriteLock wlock = lock.writeLock();
@ -46,7 +48,7 @@ public class NamespaceQuotaViolationStore implements QuotaViolationStore<String>
private final QuotaObserverChore chore;
private Map<HRegionInfo,Long> regionUsage;
public NamespaceQuotaViolationStore(Connection conn, QuotaObserverChore chore, Map<HRegionInfo,Long> regionUsage) {
public NamespaceQuotaSnapshotStore(Connection conn, QuotaObserverChore chore, Map<HRegionInfo,Long> regionUsage) {
this.conn = Objects.requireNonNull(conn);
this.chore = Objects.requireNonNull(chore);
this.regionUsage = Objects.requireNonNull(regionUsage);
@ -69,26 +71,24 @@ public class NamespaceQuotaViolationStore implements QuotaViolationStore<String>
}
@Override
public ViolationState getCurrentState(String namespace) {
public SpaceQuotaSnapshot getCurrentState(String namespace) {
// Defer the "current state" to the chore
return this.chore.getNamespaceQuotaViolation(namespace);
return this.chore.getNamespaceQuotaSnapshot(namespace);
}
@Override
public ViolationState getTargetState(String subject, SpaceQuota spaceQuota) {
public SpaceQuotaSnapshot getTargetState(String subject, SpaceQuota spaceQuota) {
rlock.lock();
try {
final long sizeLimitInBytes = spaceQuota.getSoftLimit();
long sum = 0L;
for (Entry<HRegionInfo,Long> entry : filterBySubject(subject)) {
sum += entry.getValue();
if (sum > sizeLimitInBytes) {
// Short-circuit early
return ViolationState.IN_VIOLATION;
}
}
// Observance is defined as the size of the table being less than the limit
return sum <= sizeLimitInBytes ? ViolationState.IN_OBSERVANCE : ViolationState.IN_VIOLATION;
SpaceQuotaStatus status = sum <= sizeLimitInBytes ? SpaceQuotaStatus.notInViolation()
: new SpaceQuotaStatus(ProtobufUtil.toViolationPolicy(spaceQuota.getViolationPolicy()));
return new SpaceQuotaSnapshot(status, sum, sizeLimitInBytes);
} finally {
rlock.unlock();
}
@ -110,9 +110,9 @@ public class NamespaceQuotaViolationStore implements QuotaViolationStore<String>
}
@Override
public void setCurrentState(String namespace, ViolationState state) {
public void setCurrentState(String namespace, SpaceQuotaSnapshot snapshot) {
// Defer the "current state" to the chore
this.chore.setNamespaceQuotaViolation(namespace, state);
this.chore.setNamespaceQuotaSnapshot(namespace, snapshot);
}
@Override

View File

@ -37,9 +37,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.quotas.QuotaViolationStore.ViolationState;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
import com.google.common.annotations.VisibleForTesting;
@ -54,51 +53,51 @@ import com.google.common.collect.Multimap;
@InterfaceAudience.Private
public class QuotaObserverChore extends ScheduledChore {
private static final Log LOG = LogFactory.getLog(QuotaObserverChore.class);
static final String VIOLATION_OBSERVER_CHORE_PERIOD_KEY =
"hbase.master.quotas.violation.observer.chore.period";
static final int VIOLATION_OBSERVER_CHORE_PERIOD_DEFAULT = 1000 * 60 * 5; // 5 minutes in millis
static final String QUOTA_OBSERVER_CHORE_PERIOD_KEY =
"hbase.master.quotas.observer.chore.period";
static final int QUOTA_OBSERVER_CHORE_PERIOD_DEFAULT = 1000 * 60 * 5; // 5 minutes in millis
static final String VIOLATION_OBSERVER_CHORE_DELAY_KEY =
"hbase.master.quotas.violation.observer.chore.delay";
static final long VIOLATION_OBSERVER_CHORE_DELAY_DEFAULT = 1000L * 60L; // 1 minute
static final String QUOTA_OBSERVER_CHORE_DELAY_KEY =
"hbase.master.quotas.observer.chore.delay";
static final long QUOTA_OBSERVER_CHORE_DELAY_DEFAULT = 1000L * 60L; // 1 minute
static final String VIOLATION_OBSERVER_CHORE_TIMEUNIT_KEY =
"hbase.master.quotas.violation.observer.chore.timeunit";
static final String VIOLATION_OBSERVER_CHORE_TIMEUNIT_DEFAULT = TimeUnit.MILLISECONDS.name();
static final String QUOTA_OBSERVER_CHORE_TIMEUNIT_KEY =
"hbase.master.quotas.observer.chore.timeunit";
static final String QUOTA_OBSERVER_CHORE_TIMEUNIT_DEFAULT = TimeUnit.MILLISECONDS.name();
static final String VIOLATION_OBSERVER_CHORE_REPORT_PERCENT_KEY =
"hbase.master.quotas.violation.observer.report.percent";
static final double VIOLATION_OBSERVER_CHORE_REPORT_PERCENT_DEFAULT= 0.95;
static final String QUOTA_OBSERVER_CHORE_REPORT_PERCENT_KEY =
"hbase.master.quotas.observer.report.percent";
static final double QUOTA_OBSERVER_CHORE_REPORT_PERCENT_DEFAULT= 0.95;
private final Connection conn;
private final Configuration conf;
private final MasterQuotaManager quotaManager;
/*
* Callback that changes in quota violation are passed to.
* Callback that changes in quota snapshots are passed to.
*/
private final SpaceQuotaViolationNotifier violationNotifier;
private final SpaceQuotaSnapshotNotifier snapshotNotifier;
/*
* Preserves the state of quota violations for tables and namespaces
* Preserves the state of quota snapshots for tables and namespaces
*/
private final Map<TableName,ViolationState> tableQuotaViolationStates;
private final Map<String,ViolationState> namespaceQuotaViolationStates;
private final Map<TableName,SpaceQuotaSnapshot> tableQuotaSnapshots;
private final Map<String,SpaceQuotaSnapshot> namespaceQuotaSnapshots;
/*
* Encapsulates logic for moving tables/namespaces into or out of quota violation
* Encapsulates logic for tracking the state of a table/namespace WRT space quotas
*/
private QuotaViolationStore<TableName> tableViolationStore;
private QuotaViolationStore<String> namespaceViolationStore;
private QuotaSnapshotStore<TableName> tableSnapshotStore;
private QuotaSnapshotStore<String> namespaceSnapshotStore;
public QuotaObserverChore(HMaster master) {
this(
master.getConnection(), master.getConfiguration(),
master.getSpaceQuotaViolationNotifier(), master.getMasterQuotaManager(),
master.getSpaceQuotaSnapshotNotifier(), master.getMasterQuotaManager(),
master);
}
QuotaObserverChore(
Connection conn, Configuration conf, SpaceQuotaViolationNotifier violationNotifier,
Connection conn, Configuration conf, SpaceQuotaSnapshotNotifier snapshotNotifier,
MasterQuotaManager quotaManager, Stoppable stopper) {
super(
QuotaObserverChore.class.getSimpleName(), stopper, getPeriod(conf),
@ -106,17 +105,20 @@ public class QuotaObserverChore extends ScheduledChore {
this.conn = conn;
this.conf = conf;
this.quotaManager = quotaManager;
this.violationNotifier = violationNotifier;
this.tableQuotaViolationStates = new HashMap<>();
this.namespaceQuotaViolationStates = new HashMap<>();
this.snapshotNotifier = Objects.requireNonNull(snapshotNotifier);
this.tableQuotaSnapshots = new HashMap<>();
this.namespaceQuotaSnapshots = new HashMap<>();
}
@Override
protected void chore() {
try {
if (LOG.isTraceEnabled()) {
LOG.trace("Refreshing space quotas in RegionServer");
}
_chore();
} catch (IOException e) {
LOG.warn("Failed to process quota reports and update quota violation state. Will retry.", e);
LOG.warn("Failed to process quota reports and update quota state. Will retry.", e);
}
}
@ -134,12 +136,12 @@ public class QuotaObserverChore extends ScheduledChore {
LOG.trace("Using " + reportedRegionSpaceUse.size() + " region space use reports");
}
// Create the stores to track table and namespace violations
initializeViolationStores(reportedRegionSpaceUse);
// Create the stores to track table and namespace snapshots
initializeSnapshotStores(reportedRegionSpaceUse);
// Filter out tables for which we don't have adequate regionspace reports yet.
// Important that we do this after we instantiate the stores above
tablesWithQuotas.filterInsufficientlyReportedTables(tableViolationStore);
tablesWithQuotas.filterInsufficientlyReportedTables(tableSnapshotStore);
if (LOG.isTraceEnabled()) {
LOG.trace("Filtered insufficiently reported tables, left with " +
@ -158,18 +160,18 @@ public class QuotaObserverChore extends ScheduledChore {
processNamespacesWithQuotas(namespacesWithQuotas, tablesByNamespace);
}
void initializeViolationStores(Map<HRegionInfo,Long> regionSizes) {
void initializeSnapshotStores(Map<HRegionInfo,Long> regionSizes) {
Map<HRegionInfo,Long> immutableRegionSpaceUse = Collections.unmodifiableMap(regionSizes);
if (null == tableViolationStore) {
tableViolationStore = new TableQuotaViolationStore(conn, this, immutableRegionSpaceUse);
if (null == tableSnapshotStore) {
tableSnapshotStore = new TableQuotaSnapshotStore(conn, this, immutableRegionSpaceUse);
} else {
tableViolationStore.setRegionUsage(immutableRegionSpaceUse);
tableSnapshotStore.setRegionUsage(immutableRegionSpaceUse);
}
if (null == namespaceViolationStore) {
namespaceViolationStore = new NamespaceQuotaViolationStore(
if (null == namespaceSnapshotStore) {
namespaceSnapshotStore = new NamespaceQuotaSnapshotStore(
conn, this, immutableRegionSpaceUse);
} else {
namespaceViolationStore.setRegionUsage(immutableRegionSpaceUse);
namespaceSnapshotStore.setRegionUsage(immutableRegionSpaceUse);
}
}
@ -181,7 +183,7 @@ public class QuotaObserverChore extends ScheduledChore {
*/
void processTablesWithQuotas(final Set<TableName> tablesWithTableQuotas) throws IOException {
for (TableName table : tablesWithTableQuotas) {
final SpaceQuota spaceQuota = tableViolationStore.getSpaceQuota(table);
final SpaceQuota spaceQuota = tableSnapshotStore.getSpaceQuota(table);
if (null == spaceQuota) {
if (LOG.isDebugEnabled()) {
LOG.debug("Unexpectedly did not find a space quota for " + table
@ -189,32 +191,12 @@ public class QuotaObserverChore extends ScheduledChore {
}
continue;
}
final ViolationState currentState = tableViolationStore.getCurrentState(table);
final ViolationState targetState = tableViolationStore.getTargetState(table, spaceQuota);
if (currentState == ViolationState.IN_VIOLATION) {
if (targetState == ViolationState.IN_OBSERVANCE) {
LOG.info(table + " moving into observance of table space quota.");
transitionTableToObservance(table);
tableViolationStore.setCurrentState(table, ViolationState.IN_OBSERVANCE);
} else if (targetState == ViolationState.IN_VIOLATION) {
if (LOG.isTraceEnabled()) {
LOG.trace(table + " remains in violation of quota.");
}
tableViolationStore.setCurrentState(table, ViolationState.IN_VIOLATION);
}
} else if (currentState == ViolationState.IN_OBSERVANCE) {
if (targetState == ViolationState.IN_VIOLATION) {
LOG.info(table + " moving into violation of table space quota.");
transitionTableToViolation(table, getViolationPolicy(spaceQuota));
tableViolationStore.setCurrentState(table, ViolationState.IN_VIOLATION);
} else if (targetState == ViolationState.IN_OBSERVANCE) {
if (LOG.isTraceEnabled()) {
LOG.trace(table + " remains in observance of quota.");
}
tableViolationStore.setCurrentState(table, ViolationState.IN_OBSERVANCE);
}
final SpaceQuotaSnapshot currentSnapshot = tableSnapshotStore.getCurrentState(table);
final SpaceQuotaSnapshot targetSnapshot = tableSnapshotStore.getTargetState(table, spaceQuota);
if (LOG.isTraceEnabled()) {
LOG.trace("Processing " + table + " with current=" + currentSnapshot + ", target=" + targetSnapshot);
}
updateTableQuota(table, currentSnapshot, targetSnapshot);
}
}
@ -233,7 +215,7 @@ public class QuotaObserverChore extends ScheduledChore {
final Multimap<String,TableName> tablesByNamespace) throws IOException {
for (String namespace : namespacesWithQuotas) {
// Get the quota definition for the namespace
final SpaceQuota spaceQuota = namespaceViolationStore.getSpaceQuota(namespace);
final SpaceQuota spaceQuota = namespaceSnapshotStore.getSpaceQuota(namespace);
if (null == spaceQuota) {
if (LOG.isDebugEnabled()) {
LOG.debug("Could not get Namespace space quota for " + namespace
@ -241,50 +223,117 @@ public class QuotaObserverChore extends ScheduledChore {
}
continue;
}
final ViolationState currentState = namespaceViolationStore.getCurrentState(namespace);
final ViolationState targetState = namespaceViolationStore.getTargetState(namespace, spaceQuota);
// When in observance, check if we need to move to violation.
if (ViolationState.IN_OBSERVANCE == currentState) {
if (ViolationState.IN_VIOLATION == targetState) {
for (TableName tableInNS : tablesByNamespace.get(namespace)) {
if (ViolationState.IN_VIOLATION == tableViolationStore.getCurrentState(tableInNS)) {
// Table-level quota violation policy is being applied here.
if (LOG.isTraceEnabled()) {
LOG.trace("Not activating Namespace violation policy because Table violation"
+ " policy is already in effect for " + tableInNS);
}
continue;
} else {
LOG.info(tableInNS + " moving into violation of namespace space quota");
transitionTableToViolation(tableInNS, getViolationPolicy(spaceQuota));
final SpaceQuotaSnapshot currentSnapshot = namespaceSnapshotStore.getCurrentState(namespace);
final SpaceQuotaSnapshot targetSnapshot = namespaceSnapshotStore.getTargetState(namespace, spaceQuota);
updateNamespaceQuota(namespace, currentSnapshot, targetSnapshot, tablesByNamespace);
}
}
/**
* Updates the hbase:quota table with the new quota policy for this <code>table</code>
* if necessary.
*
* @param table The table being checked
* @param currentSnapshot The state of the quota on this table from the previous invocation.
* @param targetSnapshot The state the quota should be in for this table.
*/
void updateTableQuota(
TableName table, SpaceQuotaSnapshot currentSnapshot, SpaceQuotaSnapshot targetSnapshot)
throws IOException {
final SpaceQuotaStatus currentStatus = currentSnapshot.getQuotaStatus();
final SpaceQuotaStatus targetStatus = targetSnapshot.getQuotaStatus();
// If we're changing something, log it.
if (!currentSnapshot.equals(targetSnapshot)) {
// If the target is none, we're moving out of violation. Update the hbase:quota table
if (!targetStatus.isInViolation()) {
if (LOG.isDebugEnabled()) {
LOG.debug(table + " moving into observance of table space quota.");
}
} else if (LOG.isDebugEnabled()) {
// We're either moving into violation or changing violation policies
LOG.debug(table + " moving into violation of table space quota with policy of " + targetStatus.getPolicy());
}
this.snapshotNotifier.transitionTable(table, targetSnapshot);
// Update it in memory
tableSnapshotStore.setCurrentState(table, targetSnapshot);
} else if (LOG.isTraceEnabled()) {
// Policies are the same, so we have nothing to do except log this. Don't need to re-update the quota table
if (!currentStatus.isInViolation()) {
LOG.trace(table + " remains in observance of quota.");
} else {
LOG.trace(table + " remains in violation of quota.");
}
}
}
/**
* Updates the hbase:quota table with the target quota policy for this <code>namespace</code>
* if necessary.
*
* @param namespace The namespace being checked
* @param currentSnapshot The state of the quota on this namespace from the previous invocation
* @param targetSnapshot The state the quota should be in for this namespace
* @param tablesByNamespace A mapping of tables in namespaces.
*/
void updateNamespaceQuota(
String namespace, SpaceQuotaSnapshot currentSnapshot, SpaceQuotaSnapshot targetSnapshot,
final Multimap<String,TableName> tablesByNamespace) throws IOException {
final SpaceQuotaStatus targetStatus = targetSnapshot.getQuotaStatus();
// When the policies differ, we need to move into or out of violatino
if (!currentSnapshot.equals(targetSnapshot)) {
// We want to have a policy of "NONE", moving out of violation
if (!targetStatus.isInViolation()) {
for (TableName tableInNS : tablesByNamespace.get(namespace)) {
if (!tableSnapshotStore.getCurrentState(tableInNS).getQuotaStatus().isInViolation()) {
// Table-level quota violation policy is being applied here.
if (LOG.isTraceEnabled()) {
LOG.trace("Not activating Namespace violation policy because a Table violation"
+ " policy is already in effect for " + tableInNS);
}
}
} else {
// still in observance
if (LOG.isTraceEnabled()) {
LOG.trace(namespace + " remains in observance of quota.");
} else {
LOG.info(tableInNS + " moving into observance of namespace space quota");
this.snapshotNotifier.transitionTable(tableInNS, targetSnapshot);
}
}
} else if (ViolationState.IN_VIOLATION == currentState) {
// When in violation, check if we need to move to observance.
if (ViolationState.IN_OBSERVANCE == targetState) {
for (TableName tableInNS : tablesByNamespace.get(namespace)) {
if (ViolationState.IN_VIOLATION == tableViolationStore.getCurrentState(tableInNS)) {
// Table-level quota violation policy is being applied here.
if (LOG.isTraceEnabled()) {
LOG.trace("Not activating Namespace violation policy because Table violation"
+ " policy is already in effect for " + tableInNS);
}
continue;
} else {
LOG.info(tableInNS + " moving into observance of namespace space quota");
transitionTableToObservance(tableInNS);
} else {
// Moving tables in the namespace into violation or to a different violation policy
for (TableName tableInNS : tablesByNamespace.get(namespace)) {
if (tableSnapshotStore.getCurrentState(tableInNS).getQuotaStatus().isInViolation()) {
// Table-level quota violation policy is being applied here.
if (LOG.isTraceEnabled()) {
LOG.trace("Not activating Namespace violation policy because a Table violation"
+ " policy is already in effect for " + tableInNS);
}
} else {
LOG.info(tableInNS + " moving into violation of namespace space quota with policy " + targetStatus.getPolicy());
this.snapshotNotifier.transitionTable(tableInNS, targetSnapshot);
}
} else {
// Remains in violation
if (LOG.isTraceEnabled()) {
LOG.trace(namespace + " remains in violation of quota.");
}
}
} else {
// Policies are the same
if (!targetStatus.isInViolation()) {
// Both are NONE, so we remain in observance
if (LOG.isTraceEnabled()) {
LOG.trace(namespace + " remains in observance of quota.");
}
} else {
// Namespace quota is still in violation, need to enact if the table quota is not taking priority.
for (TableName tableInNS : tablesByNamespace.get(namespace)) {
// Does a table policy exist
if (tableSnapshotStore.getCurrentState(tableInNS).getQuotaStatus().isInViolation()) {
// Table-level quota violation policy is being applied here.
if (LOG.isTraceEnabled()) {
LOG.trace("Not activating Namespace violation policy because Table violation"
+ " policy is already in effect for " + tableInNS);
}
} else {
// No table policy, so enact namespace policy
LOG.info(tableInNS + " moving into violation of namespace space quota");
this.snapshotNotifier.transitionTable(tableInNS, targetSnapshot);
}
}
}
@ -340,39 +389,24 @@ public class QuotaObserverChore extends ScheduledChore {
}
@VisibleForTesting
QuotaViolationStore<TableName> getTableViolationStore() {
return tableViolationStore;
QuotaSnapshotStore<TableName> getTableSnapshotStore() {
return tableSnapshotStore;
}
@VisibleForTesting
QuotaViolationStore<String> getNamespaceViolationStore() {
return namespaceViolationStore;
QuotaSnapshotStore<String> getNamespaceSnapshotStore() {
return namespaceSnapshotStore;
}
/**
* Transitions the given table to violation of its quota, enabling the violation policy.
* Fetches the {@link SpaceQuotaSnapshot} for the given table.
*/
private void transitionTableToViolation(TableName table, SpaceViolationPolicy violationPolicy)
throws IOException {
this.violationNotifier.transitionTableToViolation(table, violationPolicy);
}
/**
* Transitions the given table to observance of its quota, disabling the violation policy.
*/
private void transitionTableToObservance(TableName table) throws IOException {
this.violationNotifier.transitionTableToObservance(table);
}
/**
* Fetch the {@link ViolationState} for the given table.
*/
ViolationState getTableQuotaViolation(TableName table) {
SpaceQuotaSnapshot getTableQuotaSnapshot(TableName table) {
// TODO Can one instance of a Chore be executed concurrently?
ViolationState state = this.tableQuotaViolationStates.get(table);
SpaceQuotaSnapshot state = this.tableQuotaSnapshots.get(table);
if (null == state) {
// No tracked state implies observance.
return ViolationState.IN_OBSERVANCE;
return QuotaSnapshotStore.NO_QUOTA;
}
return state;
}
@ -380,19 +414,19 @@ public class QuotaObserverChore extends ScheduledChore {
/**
* Stores the quota violation state for the given table.
*/
void setTableQuotaViolation(TableName table, ViolationState state) {
this.tableQuotaViolationStates.put(table, state);
void setTableQuotaViolation(TableName table, SpaceQuotaSnapshot snapshot) {
this.tableQuotaSnapshots.put(table, snapshot);
}
/**
* Fetches the {@link ViolationState} for the given namespace.
* Fetches the {@link SpaceQuotaSnapshot} for the given namespace.
*/
ViolationState getNamespaceQuotaViolation(String namespace) {
SpaceQuotaSnapshot getNamespaceQuotaSnapshot(String namespace) {
// TODO Can one instance of a Chore be executed concurrently?
ViolationState state = this.namespaceQuotaViolationStates.get(namespace);
SpaceQuotaSnapshot state = this.namespaceQuotaSnapshots.get(namespace);
if (null == state) {
// No tracked state implies observance.
return ViolationState.IN_OBSERVANCE;
return QuotaSnapshotStore.NO_QUOTA;
}
return state;
}
@ -400,20 +434,8 @@ public class QuotaObserverChore extends ScheduledChore {
/**
* Stores the quota violation state for the given namespace.
*/
void setNamespaceQuotaViolation(String namespace, ViolationState state) {
this.namespaceQuotaViolationStates.put(namespace, state);
}
/**
* Extracts the {@link SpaceViolationPolicy} from the serialized {@link Quotas} protobuf.
* @throws IllegalArgumentException If the SpaceQuota lacks a ViolationPolicy
*/
SpaceViolationPolicy getViolationPolicy(SpaceQuota spaceQuota) {
if (!spaceQuota.hasViolationPolicy()) {
throw new IllegalArgumentException("SpaceQuota had no associated violation policy: "
+ spaceQuota);
}
return ProtobufUtil.toViolationPolicy(spaceQuota.getViolationPolicy());
void setNamespaceQuotaSnapshot(String namespace, SpaceQuotaSnapshot snapshot) {
this.namespaceQuotaSnapshots.put(namespace, snapshot);
}
/**
@ -423,8 +445,8 @@ public class QuotaObserverChore extends ScheduledChore {
* @return The configured chore period or the default value.
*/
static int getPeriod(Configuration conf) {
return conf.getInt(VIOLATION_OBSERVER_CHORE_PERIOD_KEY,
VIOLATION_OBSERVER_CHORE_PERIOD_DEFAULT);
return conf.getInt(QUOTA_OBSERVER_CHORE_PERIOD_KEY,
QUOTA_OBSERVER_CHORE_PERIOD_DEFAULT);
}
/**
@ -434,21 +456,21 @@ public class QuotaObserverChore extends ScheduledChore {
* @return The configured chore initial delay or the default value.
*/
static long getInitialDelay(Configuration conf) {
return conf.getLong(VIOLATION_OBSERVER_CHORE_DELAY_KEY,
VIOLATION_OBSERVER_CHORE_DELAY_DEFAULT);
return conf.getLong(QUOTA_OBSERVER_CHORE_DELAY_KEY,
QUOTA_OBSERVER_CHORE_DELAY_DEFAULT);
}
/**
* Extracts the time unit for the chore period and initial delay from the configuration. The
* configuration value for {@link #VIOLATION_OBSERVER_CHORE_TIMEUNIT_KEY} must correspond to
* configuration value for {@link #QUOTA_OBSERVER_CHORE_TIMEUNIT_KEY} must correspond to
* a {@link TimeUnit} value.
*
* @param conf The configuration object.
* @return The configured time unit for the chore period and initial delay or the default value.
*/
static TimeUnit getTimeUnit(Configuration conf) {
return TimeUnit.valueOf(conf.get(VIOLATION_OBSERVER_CHORE_TIMEUNIT_KEY,
VIOLATION_OBSERVER_CHORE_TIMEUNIT_DEFAULT));
return TimeUnit.valueOf(conf.get(QUOTA_OBSERVER_CHORE_TIMEUNIT_KEY,
QUOTA_OBSERVER_CHORE_TIMEUNIT_DEFAULT));
}
/**
@ -459,8 +481,8 @@ public class QuotaObserverChore extends ScheduledChore {
* @return The percent of regions reported to use.
*/
static Double getRegionReportPercent(Configuration conf) {
return conf.getDouble(VIOLATION_OBSERVER_CHORE_REPORT_PERCENT_KEY,
VIOLATION_OBSERVER_CHORE_REPORT_PERCENT_DEFAULT);
return conf.getDouble(QUOTA_OBSERVER_CHORE_REPORT_PERCENT_KEY,
QUOTA_OBSERVER_CHORE_REPORT_PERCENT_DEFAULT);
}
/**
@ -549,7 +571,7 @@ public class QuotaObserverChore extends ScheduledChore {
* Filters out all tables for which the Master currently doesn't have enough region space
* reports received from RegionServers yet.
*/
public void filterInsufficientlyReportedTables(QuotaViolationStore<TableName> tableStore)
public void filterInsufficientlyReportedTables(QuotaSnapshotStore<TableName> tableStore)
throws IOException {
final double percentRegionsReportedThreshold = getRegionReportPercent(getConfiguration());
Set<TableName> tablesToRemove = new HashSet<>();
@ -572,12 +594,12 @@ public class QuotaObserverChore extends ScheduledChore {
if (ratioReported < percentRegionsReportedThreshold) {
if (LOG.isTraceEnabled()) {
LOG.trace("Filtering " + table + " because " + reportedRegionsInQuota + " of " +
numRegionsInTable + " were reported.");
numRegionsInTable + " regions were reported.");
}
tablesToRemove.add(table);
} else if (LOG.isTraceEnabled()) {
LOG.trace("Retaining " + table + " because " + reportedRegionsInQuota + " of " +
numRegionsInTable + " were reported.");
numRegionsInTable + " regions were reported.");
}
}
for (TableName tableToRemove : tablesToRemove) {
@ -600,7 +622,7 @@ public class QuotaObserverChore extends ScheduledChore {
/**
* Computes the number of regions reported for a table.
*/
int getNumReportedRegions(TableName table, QuotaViolationStore<TableName> tableStore)
int getNumReportedRegions(TableName table, QuotaSnapshotStore<TableName> tableStore)
throws IOException {
return Iterables.size(tableStore.filterBySubject(table));
}

View File

@ -22,6 +22,7 @@ import java.util.Map.Entry;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
/**
@ -30,7 +31,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
* An entity is presently a table or a namespace.
*/
@InterfaceAudience.Private
public interface QuotaViolationStore<T> {
public interface QuotaSnapshotStore<T> {
/**
* The current state of a table with respect to the policy set forth by a quota.
@ -41,6 +42,12 @@ public interface QuotaViolationStore<T> {
IN_OBSERVANCE,
}
/**
* Singleton to represent a table without a quota defined. It is never in violation.
*/
public static final SpaceQuotaSnapshot NO_QUOTA = new SpaceQuotaSnapshot(
SpaceQuotaStatus.notInViolation(), -1, -1);
/**
* Fetch the Quota for the given {@code subject}. May be null.
*
@ -49,20 +56,20 @@ public interface QuotaViolationStore<T> {
SpaceQuota getSpaceQuota(T subject) throws IOException;
/**
* Returns the current {@link ViolationState} for the given {@code subject}.
* Returns the current {@link SpaceQuotaSnapshot} for the given {@code subject}.
*
* @param subject The object which the quota violation state should be fetched
* @param subject The object which the quota snapshot should be fetched
*/
ViolationState getCurrentState(T subject);
SpaceQuotaSnapshot getCurrentState(T subject);
/**
* Computes the target {@link ViolationState} for the given {@code subject} and
* Computes the target {@link SpaceQuotaSnapshot} for the given {@code subject} and
* {@code spaceQuota}.
*
* @param subject The object which to determine the target quota violation state of
* @param subject The object which to determine the target SpaceQuotaSnapshot of
* @param spaceQuota The quota "definition" for the {@code subject}
*/
ViolationState getTargetState(T subject, SpaceQuota spaceQuota);
SpaceQuotaSnapshot getTargetState(T subject, SpaceQuota spaceQuota);
/**
* Filters the provided <code>regions</code>, returning those which match the given
@ -73,12 +80,12 @@ public interface QuotaViolationStore<T> {
Iterable<Entry<HRegionInfo,Long>> filterBySubject(T subject);
/**
* Persists the current {@link ViolationState} for the {@code subject}.
* Persists the current {@link SpaceQuotaSnapshot} for the {@code subject}.
*
* @param subject The object which the {@link ViolationState} is being persisted for
* @param state The current {@link ViolationState} of the {@code subject}
* @param subject The object which the {@link SpaceQuotaSnapshot} is being persisted for
* @param state The current state of the {@code subject}
*/
void setCurrentState(T subject, ViolationState state);
void setCurrentState(T subject, SpaceQuotaSnapshot state);
/**
* Updates {@code this} with the latest snapshot of filesystem use by region.

View File

@ -20,24 +20,29 @@ import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicReference;
import java.util.Map.Entry;
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.client.Connection;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.annotations.VisibleForTesting;
/**
* A manager for filesystem space quotas in the RegionServer.
*
* This class is responsible for reading quota violation policies from the quota
* table and then enacting them on the given table.
* This class is the centralized point for what a RegionServer knows about space quotas
* on tables. For each table, it tracks two different things: the {@link SpaceQuotaSnapshot}
* and a {@link SpaceViolationPolicyEnforcement} (which may be null when a quota is not
* being violated). Both of these are sensitive on when they were last updated. The
* {link SpaceQutoaViolationPolicyRefresherChore} periodically runs and updates
* the state on <code>this</code>.
*/
@InterfaceAudience.Private
public class RegionServerSpaceQuotaManager {
@ -45,12 +50,23 @@ public class RegionServerSpaceQuotaManager {
private final RegionServerServices rsServices;
private SpaceQuotaViolationPolicyRefresherChore spaceQuotaRefresher;
private Map<TableName,SpaceViolationPolicy> enforcedPolicies;
private SpaceQuotaRefresherChore spaceQuotaRefresher;
private AtomicReference<Map<TableName, SpaceQuotaSnapshot>> currentQuotaSnapshots;
private boolean started = false;
private ConcurrentHashMap<TableName,SpaceViolationPolicyEnforcement> enforcedPolicies;
private SpaceViolationPolicyEnforcementFactory factory;
public RegionServerSpaceQuotaManager(RegionServerServices rsServices) {
this(rsServices, SpaceViolationPolicyEnforcementFactory.getInstance());
}
@VisibleForTesting
RegionServerSpaceQuotaManager(
RegionServerServices rsServices, SpaceViolationPolicyEnforcementFactory factory) {
this.rsServices = Objects.requireNonNull(rsServices);
this.factory = factory;
this.enforcedPolicies = new ConcurrentHashMap<>();
this.currentQuotaSnapshots = new AtomicReference<>(new HashMap<>());
}
public synchronized void start() throws IOException {
@ -59,8 +75,12 @@ public class RegionServerSpaceQuotaManager {
return;
}
spaceQuotaRefresher = new SpaceQuotaViolationPolicyRefresherChore(this);
enforcedPolicies = new HashMap<>();
if (started) {
LOG.warn("RegionServerSpaceQuotaManager has already been started!");
return;
}
this.spaceQuotaRefresher = new SpaceQuotaRefresherChore(this, rsServices.getClusterConnection());
rsServices.getChoreService().scheduleChore(spaceQuotaRefresher);
started = true;
}
@ -79,91 +99,136 @@ public class RegionServerSpaceQuotaManager {
return started;
}
Connection getConnection() {
return rsServices.getConnection();
/**
* Copies the last {@link SpaceQuotaSnapshot}s that were recorded. The current view
* of what the RegionServer thinks the table's utilization is.
*/
public Map<TableName,SpaceQuotaSnapshot> copyQuotaSnapshots() {
return new HashMap<>(currentQuotaSnapshots.get());
}
/**
* Updates the current {@link SpaceQuotaSnapshot}s for the RegionServer.
*
* @param newSnapshots The space quota snapshots.
*/
public void updateQuotaSnapshot(Map<TableName,SpaceQuotaSnapshot> newSnapshots) {
currentQuotaSnapshots.set(Objects.requireNonNull(newSnapshots));
}
/**
* Creates an object well-suited for the RegionServer to use in verifying active policies.
*/
public ActivePolicyEnforcement getActiveEnforcements() {
return new ActivePolicyEnforcement(copyActiveEnforcements(), copyQuotaSnapshots(), rsServices);
}
/**
* Converts a map of table to {@link SpaceViolationPolicyEnforcement}s into
* {@link SpaceViolationPolicy}s.
*/
public Map<TableName, SpaceQuotaSnapshot> getActivePoliciesAsMap() {
final Map<TableName, SpaceViolationPolicyEnforcement> enforcements =
copyActiveEnforcements();
final Map<TableName, SpaceQuotaSnapshot> policies = new HashMap<>();
for (Entry<TableName, SpaceViolationPolicyEnforcement> entry : enforcements.entrySet()) {
final SpaceQuotaSnapshot snapshot = entry.getValue().getQuotaSnapshot();
if (null != snapshot) {
policies.put(entry.getKey(), snapshot);
}
}
return policies;
}
/**
* Enforces the given violationPolicy on the given table in this RegionServer.
*/
public void enforceViolationPolicy(TableName tableName, SpaceQuotaSnapshot snapshot) {
SpaceQuotaStatus status = snapshot.getQuotaStatus();
if (!status.isInViolation()) {
throw new IllegalStateException(
tableName + " is not in violation. Violation policy should not be enabled.");
}
if (LOG.isTraceEnabled()) {
LOG.trace(
"Enabling violation policy enforcement on " + tableName
+ " with policy " + status.getPolicy());
}
// Construct this outside of the lock
final SpaceViolationPolicyEnforcement enforcement = getFactory().create(
getRegionServerServices(), tableName, snapshot);
// "Enables" the policy
// TODO Should this synchronize on the actual table name instead of the map? That would allow
// policy enable/disable on different tables to happen concurrently. As written now, only one
// table will be allowed to transition at a time.
synchronized (enforcedPolicies) {
try {
enforcement.enable();
} catch (IOException e) {
LOG.error("Failed to enable space violation policy for " + tableName
+ ". This table will not enter violation.", e);
return;
}
enforcedPolicies.put(tableName, enforcement);
}
}
/**
* Disables enforcement on any violation policy on the given <code>tableName</code>.
*/
public void disableViolationPolicyEnforcement(TableName tableName) {
if (LOG.isTraceEnabled()) {
LOG.trace("Disabling violation policy enforcement on " + tableName);
}
// "Disables" the policy
// TODO Should this synchronize on the actual table name instead of the map?
synchronized (enforcedPolicies) {
SpaceViolationPolicyEnforcement enforcement = enforcedPolicies.remove(tableName);
if (null != enforcement) {
try {
enforcement.disable();
} catch (IOException e) {
LOG.error("Failed to disable space violation policy for " + tableName
+ ". This table will remain in violation.", e);
enforcedPolicies.put(tableName, enforcement);
}
}
}
}
/**
* Returns whether or not compactions should be disabled for the given <code>tableName</code> per
* a space quota violation policy. A convenience method.
*
* @param tableName The table to check
* @return True if compactions should be disabled for the table, false otherwise.
*/
public boolean areCompactionsDisabled(TableName tableName) {
SpaceViolationPolicyEnforcement enforcement = this.enforcedPolicies.get(Objects.requireNonNull(tableName));
if (null != enforcement) {
return enforcement.areCompactionsDisabled();
}
return false;
}
/**
* Returns the collection of tables which have quota violation policies enforced on
* this RegionServer.
*/
public synchronized Map<TableName,SpaceViolationPolicy> getActiveViolationPolicyEnforcements()
throws IOException {
Map<TableName,SpaceViolationPolicyEnforcement> copyActiveEnforcements() {
// Allows reads to happen concurrently (or while the map is being updated)
return new HashMap<>(this.enforcedPolicies);
}
/**
* Wrapper around {@link QuotaTableUtil#extractViolationPolicy(Result, Map)} for testing.
*/
void extractViolationPolicy(Result result, Map<TableName,SpaceViolationPolicy> activePolicies) {
QuotaTableUtil.extractViolationPolicy(result, activePolicies);
}
/**
* Reads all quota violation policies which are to be enforced from the quota table.
*
* @return The collection of tables which are in violation of their quota and the policy which
* should be enforced.
*/
public Map<TableName, SpaceViolationPolicy> getViolationPoliciesToEnforce() throws IOException {
try (Table quotaTable = getConnection().getTable(QuotaUtil.QUOTA_TABLE_NAME);
ResultScanner scanner = quotaTable.getScanner(QuotaTableUtil.makeQuotaViolationScan())) {
Map<TableName,SpaceViolationPolicy> activePolicies = new HashMap<>();
for (Result result : scanner) {
try {
extractViolationPolicy(result, activePolicies);
} catch (IllegalArgumentException e) {
final String msg = "Failed to parse result for row " + Bytes.toString(result.getRow());
LOG.error(msg, e);
throw new IOException(msg, e);
}
}
return activePolicies;
}
}
/**
* Enforces the given violationPolicy on the given table in this RegionServer.
*/
synchronized void enforceViolationPolicy(
TableName tableName, SpaceViolationPolicy violationPolicy) {
if (LOG.isTraceEnabled()) {
LOG.trace(
"Enabling violation policy enforcement on " + tableName
+ " with policy " + violationPolicy);
}
// Enact the policy
enforceOnRegionServer(tableName, violationPolicy);
// Publicize our enacting of the policy
enforcedPolicies.put(tableName, violationPolicy);
}
/**
* Enacts the given violation policy on this table in the RegionServer.
*/
void enforceOnRegionServer(TableName tableName, SpaceViolationPolicy violationPolicy) {
throw new UnsupportedOperationException("TODO");
}
/**
* Disables enforcement on any violation policy on the given <code>tableName</code>.
*/
synchronized void disableViolationPolicyEnforcement(TableName tableName) {
if (LOG.isTraceEnabled()) {
LOG.trace("Disabling violation policy enforcement on " + tableName);
}
disableOnRegionServer(tableName);
enforcedPolicies.remove(tableName);
}
/**
* Disables any violation policy on this table in the RegionServer.
*/
void disableOnRegionServer(TableName tableName) {
throw new UnsupportedOperationException("TODO");
}
RegionServerServices getRegionServerServices() {
return rsServices;
}
Connection getConnection() {
return rsServices.getConnection();
}
SpaceViolationPolicyEnforcementFactory getFactory() {
return factory;
}
}

View File

@ -0,0 +1,95 @@
/*
* 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.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* An Exception that is thrown when a space quota is in violation.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class SpaceLimitingException extends QuotaExceededException {
private static final long serialVersionUID = 2319438922387583600L;
private static final Log LOG = LogFactory.getLog(SpaceLimitingException.class);
private static final String MESSAGE_PREFIX = SpaceLimitingException.class.getName() + ": ";
private final String policyName;
public SpaceLimitingException(String msg) {
super(parseMessage(msg));
// Hack around ResponseConverter expecting to invoke a single-arg String constructor
// on this class
if (null != msg) {
for (SpaceViolationPolicy definedPolicy : SpaceViolationPolicy.values()) {
if (msg.indexOf(definedPolicy.name()) != -1) {
policyName = definedPolicy.name();
return;
}
}
}
policyName = null;
}
public SpaceLimitingException(String policyName, String msg) {
super(msg);
this.policyName = policyName;
}
public SpaceLimitingException(String policyName, String msg, Throwable e) {
super(msg, e);
this.policyName = policyName;
}
/**
* Returns the violation policy in effect.
*
* @return The violation policy in effect.
*/
public String getViolationPolicy() {
return this.policyName;
}
private static String parseMessage(String originalMessage) {
// Serialization of the exception places a duplicate class name. Try to strip that off if it
// exists. Best effort... Looks something like:
// "org.apache.hadoop.hbase.quotas.SpaceLimitingException: NO_INSERTS A Put is disallowed due
// to a space quota."
if (null != originalMessage && originalMessage.startsWith(MESSAGE_PREFIX)) {
// If it starts with the class name, rip off the policy too.
try {
int index = originalMessage.indexOf(' ', MESSAGE_PREFIX.length());
return originalMessage.substring(index + 1);
} catch (Exception e) {
if (LOG.isTraceEnabled()) {
LOG.trace("Failed to trim exception message", e);
}
}
}
return originalMessage;
}
@Override
public String getMessage() {
return (null == policyName ? "(unknown policy)" : policyName) + " " + super.getMessage();
}
}

View File

@ -0,0 +1,225 @@
/*
* 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.Map;
import java.util.Map.Entry;
import java.util.concurrent.TimeUnit;
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.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.Bytes;
/**
* A {@link ScheduledChore} which periodically updates the {@link RegionServerSpaceQuotaManager}
* with information from the hbase:quota.
*/
@InterfaceAudience.Private
public class SpaceQuotaRefresherChore extends ScheduledChore {
private static final Log LOG = LogFactory.getLog(SpaceQuotaRefresherChore.class);
static final String POLICY_REFRESHER_CHORE_PERIOD_KEY =
"hbase.regionserver.quotas.policy.refresher.chore.period";
static final int POLICY_REFRESHER_CHORE_PERIOD_DEFAULT = 1000 * 60 * 5; // 5 minutes in millis
static final String POLICY_REFRESHER_CHORE_DELAY_KEY =
"hbase.regionserver.quotas.policy.refresher.chore.delay";
static final long POLICY_REFRESHER_CHORE_DELAY_DEFAULT = 1000L * 60L; // 1 minute
static final String POLICY_REFRESHER_CHORE_TIMEUNIT_KEY =
"hbase.regionserver.quotas.policy.refresher.chore.timeunit";
static final String POLICY_REFRESHER_CHORE_TIMEUNIT_DEFAULT = TimeUnit.MILLISECONDS.name();
static final String POLICY_REFRESHER_CHORE_REPORT_PERCENT_KEY =
"hbase.regionserver.quotas.policy.refresher.report.percent";
static final double POLICY_REFRESHER_CHORE_REPORT_PERCENT_DEFAULT= 0.95;
private final RegionServerSpaceQuotaManager manager;
private final Connection conn;
public SpaceQuotaRefresherChore(RegionServerSpaceQuotaManager manager, Connection conn) {
super(SpaceQuotaRefresherChore.class.getSimpleName(),
manager.getRegionServerServices(),
getPeriod(manager.getRegionServerServices().getConfiguration()),
getInitialDelay(manager.getRegionServerServices().getConfiguration()),
getTimeUnit(manager.getRegionServerServices().getConfiguration()));
this.manager = manager;
this.conn = conn;
}
@Override
protected void chore() {
try {
if (LOG.isTraceEnabled()) {
LOG.trace("Reading current quota snapshots from hbase:quota.");
}
// Get the snapshots that the quota manager is currently aware of
final Map<TableName, SpaceQuotaSnapshot> currentSnapshots =
getManager().copyQuotaSnapshots();
// Read the new snapshots from the quota table
final Map<TableName, SpaceQuotaSnapshot> newSnapshots = fetchSnapshotsFromQuotaTable();
if (LOG.isTraceEnabled()) {
LOG.trace(currentSnapshots.size() + " table quota snapshots are collected, "
+ "read " + newSnapshots.size() + " from the quota table.");
}
// Iterate over each new quota snapshot
for (Entry<TableName, SpaceQuotaSnapshot> entry : newSnapshots.entrySet()) {
final TableName tableName = entry.getKey();
final SpaceQuotaSnapshot newSnapshot = entry.getValue();
// May be null!
final SpaceQuotaSnapshot currentSnapshot = currentSnapshots.get(tableName);
if (LOG.isTraceEnabled()) {
LOG.trace(tableName + ": current=" + currentSnapshot + ", new=" + newSnapshot);
}
if (!newSnapshot.equals(currentSnapshot)) {
// We have a new snapshot. We might need to enforce it or disable the enforcement
if (!isInViolation(currentSnapshot) && newSnapshot.getQuotaStatus().isInViolation()) {
if (LOG.isTraceEnabled()) {
LOG.trace("Enabling " + newSnapshot + " on " + tableName);
}
getManager().enforceViolationPolicy(tableName, newSnapshot);
}
if (isInViolation(currentSnapshot) && !newSnapshot.getQuotaStatus().isInViolation()) {
if (LOG.isTraceEnabled()) {
LOG.trace("Removing quota violation policy on " + tableName);
}
getManager().disableViolationPolicyEnforcement(tableName);
}
}
}
// We're intentionally ignoring anything extra with the currentSnapshots. If we were missing
// information from the RegionServers to create an accurate SpaceQuotaSnapshot in the Master,
// the Master will generate a new SpaceQuotaSnapshot which represents this state. This lets
// us avoid having to do anything special with currentSnapshots here.
// Update the snapshots in the manager
getManager().updateQuotaSnapshot(newSnapshots);
} catch (IOException e) {
LOG.warn(
"Caught exception while refreshing enforced quota violation policies, will retry.", e);
}
}
/**
* Checks if the given <code>snapshot</code> is in violation, allowing the snapshot to be null.
* If the snapshot is null, this is interpreted as no snapshot which implies not in violation.
*
* @param snapshot The snapshot to operate on.
* @return true if the snapshot is in violation, false otherwise.
*/
boolean isInViolation(SpaceQuotaSnapshot snapshot) {
if (null == snapshot) {
return false;
}
return snapshot.getQuotaStatus().isInViolation();
}
/**
* Reads all quota snapshots from the quota table.
*
* @return The current "view" of space use by each table.
*/
public Map<TableName, SpaceQuotaSnapshot> fetchSnapshotsFromQuotaTable() throws IOException {
try (Table quotaTable = getConnection().getTable(QuotaUtil.QUOTA_TABLE_NAME);
ResultScanner scanner = quotaTable.getScanner(QuotaTableUtil.makeQuotaSnapshotScan())) {
Map<TableName,SpaceQuotaSnapshot> snapshots = new HashMap<>();
for (Result result : scanner) {
try {
extractQuotaSnapshot(result, snapshots);
} catch (IllegalArgumentException e) {
final String msg = "Failed to parse result for row " + Bytes.toString(result.getRow());
LOG.error(msg, e);
throw new IOException(msg, e);
}
}
return snapshots;
}
}
/**
* Wrapper around {@link QuotaTableUtil#extractQuotaSnapshot(Result, Map)} for testing.
*/
void extractQuotaSnapshot(Result result, Map<TableName,SpaceQuotaSnapshot> snapshots) {
QuotaTableUtil.extractQuotaSnapshot(result, snapshots);
}
Connection getConnection() {
return conn;
}
RegionServerSpaceQuotaManager getManager() {
return manager;
}
/**
* Extracts the period for the chore from the configuration.
*
* @param conf The configuration object.
* @return The configured chore period or the default value.
*/
static int getPeriod(Configuration conf) {
return conf.getInt(POLICY_REFRESHER_CHORE_PERIOD_KEY,
POLICY_REFRESHER_CHORE_PERIOD_DEFAULT);
}
/**
* Extracts the initial delay for the chore from the configuration.
*
* @param conf The configuration object.
* @return The configured chore initial delay or the default value.
*/
static long getInitialDelay(Configuration conf) {
return conf.getLong(POLICY_REFRESHER_CHORE_DELAY_KEY,
POLICY_REFRESHER_CHORE_DELAY_DEFAULT);
}
/**
* Extracts the time unit for the chore period and initial delay from the configuration. The
* configuration value for {@link #POLICY_REFRESHER_CHORE_TIMEUNIT_KEY} must correspond to
* a {@link TimeUnit} value.
*
* @param conf The configuration object.
* @return The configured time unit for the chore period and initial delay or the default value.
*/
static TimeUnit getTimeUnit(Configuration conf) {
return TimeUnit.valueOf(conf.get(POLICY_REFRESHER_CHORE_TIMEUNIT_KEY,
POLICY_REFRESHER_CHORE_TIMEUNIT_DEFAULT));
}
/**
* Extracts the percent of Regions for a table to have been reported to enable quota violation
* state change.
*
* @param conf The configuration object.
* @return The percent of regions reported to use.
*/
static Double getRegionReportPercent(Configuration conf) {
return conf.getDouble(POLICY_REFRESHER_CHORE_REPORT_PERCENT_KEY,
POLICY_REFRESHER_CHORE_REPORT_PERCENT_DEFAULT);
}
}

View File

@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.client.Connection;
* must have a no-args constructor.
*/
@InterfaceAudience.Private
public interface SpaceQuotaViolationNotifier {
public interface SpaceQuotaSnapshotNotifier {
/**
* Initializes the notifier.
@ -36,19 +36,10 @@ public interface SpaceQuotaViolationNotifier {
void initialize(Connection conn);
/**
* Instructs the cluster that the given table is in violation of a space quota. The
* provided violation policy is the action which should be taken on the table.
* Informs the cluster of the current state of a space quota for a table.
*
* @param tableName The name of the table in violation of the quota.
* @param violationPolicy The policy which should be enacted on the table.
* @param tableName The name of the table.
* @param snapshot The details of the space quota utilization.
*/
void transitionTableToViolation(
TableName tableName, SpaceViolationPolicy violationPolicy) throws IOException;
/**
* Instructs the cluster that the given table is in observance of any applicable space quota.
*
* @param tableName The name of the table in observance.
*/
void transitionTableToObservance(TableName tableName) throws IOException;
void transitionTable(TableName tableName, SpaceQuotaSnapshot snapshot) throws IOException;
}

View File

@ -22,37 +22,37 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* Factory for creating {@link SpaceQuotaViolationNotifier} implementations. Implementations
* Factory for creating {@link SpaceQuotaSnapshotNotifier} implementations. Implementations
* must have a no-args constructor.
*/
@InterfaceAudience.Private
public class SpaceQuotaViolationNotifierFactory {
private static final SpaceQuotaViolationNotifierFactory INSTANCE =
new SpaceQuotaViolationNotifierFactory();
public class SpaceQuotaSnapshotNotifierFactory {
private static final SpaceQuotaSnapshotNotifierFactory INSTANCE =
new SpaceQuotaSnapshotNotifierFactory();
public static final String VIOLATION_NOTIFIER_KEY = "hbase.master.quota.violation.notifier.impl";
public static final Class<? extends SpaceQuotaViolationNotifier> VIOLATION_NOTIFIER_DEFAULT =
SpaceQuotaViolationNotifierForTest.class;
public static final String SNAPSHOT_NOTIFIER_KEY = "hbase.master.quota.snapshot.notifier.impl";
public static final Class<? extends SpaceQuotaSnapshotNotifier> SNAPSHOT_NOTIFIER_DEFAULT =
TableSpaceQuotaSnapshotNotifier.class;
// Private
private SpaceQuotaViolationNotifierFactory() {}
private SpaceQuotaSnapshotNotifierFactory() {}
public static SpaceQuotaViolationNotifierFactory getInstance() {
public static SpaceQuotaSnapshotNotifierFactory getInstance() {
return INSTANCE;
}
/**
* Instantiates the {@link SpaceQuotaViolationNotifier} implementation as defined in the
* Instantiates the {@link SpaceQuotaSnapshotNotifier} implementation as defined in the
* configuration provided.
*
* @param conf Configuration object
* @return The SpaceQuotaViolationNotifier implementation
* @return The SpaceQuotaSnapshotNotifier implementation
* @throws IllegalArgumentException if the class could not be instantiated
*/
public SpaceQuotaViolationNotifier create(Configuration conf) {
Class<? extends SpaceQuotaViolationNotifier> clz = Objects.requireNonNull(conf)
.getClass(VIOLATION_NOTIFIER_KEY, VIOLATION_NOTIFIER_DEFAULT,
SpaceQuotaViolationNotifier.class);
public SpaceQuotaSnapshotNotifier create(Configuration conf) {
Class<? extends SpaceQuotaSnapshotNotifier> clz = Objects.requireNonNull(conf)
.getClass(SNAPSHOT_NOTIFIER_KEY, SNAPSHOT_NOTIFIER_DEFAULT,
SpaceQuotaSnapshotNotifier.class);
try {
return clz.newInstance();
} catch (InstantiationException | IllegalAccessException e) {

View File

@ -1,154 +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 java.io.IOException;
import java.util.Iterator;
import java.util.Map;
import java.util.Map.Entry;
import java.util.concurrent.TimeUnit;
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.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* A {@link ScheduledChore} which periodically updates a local copy of tables which have
* space quota violation policies enacted on them.
*/
@InterfaceAudience.Private
public class SpaceQuotaViolationPolicyRefresherChore extends ScheduledChore {
private static final Log LOG = LogFactory.getLog(SpaceQuotaViolationPolicyRefresherChore.class);
static final String POLICY_REFRESHER_CHORE_PERIOD_KEY =
"hbase.regionserver.quotas.policy.refresher.chore.period";
static final int POLICY_REFRESHER_CHORE_PERIOD_DEFAULT = 1000 * 60 * 5; // 5 minutes in millis
static final String POLICY_REFRESHER_CHORE_DELAY_KEY =
"hbase.regionserver.quotas.policy.refresher.chore.delay";
static final long POLICY_REFRESHER_CHORE_DELAY_DEFAULT = 1000L * 60L; // 1 minute
static final String POLICY_REFRESHER_CHORE_TIMEUNIT_KEY =
"hbase.regionserver.quotas.policy.refresher.chore.timeunit";
static final String POLICY_REFRESHER_CHORE_TIMEUNIT_DEFAULT = TimeUnit.MILLISECONDS.name();
static final String POLICY_REFRESHER_CHORE_REPORT_PERCENT_KEY =
"hbase.regionserver.quotas.policy.refresher.report.percent";
static final double POLICY_REFRESHER_CHORE_REPORT_PERCENT_DEFAULT= 0.95;
private final RegionServerSpaceQuotaManager manager;
public SpaceQuotaViolationPolicyRefresherChore(RegionServerSpaceQuotaManager manager) {
super(SpaceQuotaViolationPolicyRefresherChore.class.getSimpleName(),
manager.getRegionServerServices(),
getPeriod(manager.getRegionServerServices().getConfiguration()),
getInitialDelay(manager.getRegionServerServices().getConfiguration()),
getTimeUnit(manager.getRegionServerServices().getConfiguration()));
this.manager = manager;
}
@Override
protected void chore() {
// Tables with a policy currently enforced
final Map<TableName, SpaceViolationPolicy> activeViolationPolicies;
// Tables with policies that should be enforced
final Map<TableName, SpaceViolationPolicy> violationPolicies;
try {
// Tables with a policy currently enforced
activeViolationPolicies = manager.getActiveViolationPolicyEnforcements();
// Tables with policies that should be enforced
violationPolicies = manager.getViolationPoliciesToEnforce();
} catch (IOException e) {
LOG.warn("Failed to fetch enforced quota violation policies, will retry.", e);
return;
}
// Ensure each policy which should be enacted is enacted.
for (Entry<TableName, SpaceViolationPolicy> entry : violationPolicies.entrySet()) {
final TableName tableName = entry.getKey();
final SpaceViolationPolicy policyToEnforce = entry.getValue();
final SpaceViolationPolicy currentPolicy = activeViolationPolicies.get(tableName);
if (currentPolicy != policyToEnforce) {
if (LOG.isTraceEnabled()) {
LOG.trace("Enabling " + policyToEnforce + " on " + tableName);
}
manager.enforceViolationPolicy(tableName, policyToEnforce);
}
}
// Remove policies which should no longer be enforced
Iterator<TableName> iter = activeViolationPolicies.keySet().iterator();
while (iter.hasNext()) {
final TableName localTableWithPolicy = iter.next();
if (!violationPolicies.containsKey(localTableWithPolicy)) {
if (LOG.isTraceEnabled()) {
LOG.trace("Removing quota violation policy on " + localTableWithPolicy);
}
manager.disableViolationPolicyEnforcement(localTableWithPolicy);
iter.remove();
}
}
}
/**
* Extracts the period for the chore from the configuration.
*
* @param conf The configuration object.
* @return The configured chore period or the default value.
*/
static int getPeriod(Configuration conf) {
return conf.getInt(POLICY_REFRESHER_CHORE_PERIOD_KEY,
POLICY_REFRESHER_CHORE_PERIOD_DEFAULT);
}
/**
* Extracts the initial delay for the chore from the configuration.
*
* @param conf The configuration object.
* @return The configured chore initial delay or the default value.
*/
static long getInitialDelay(Configuration conf) {
return conf.getLong(POLICY_REFRESHER_CHORE_DELAY_KEY,
POLICY_REFRESHER_CHORE_DELAY_DEFAULT);
}
/**
* Extracts the time unit for the chore period and initial delay from the configuration. The
* configuration value for {@link #POLICY_REFRESHER_CHORE_TIMEUNIT_KEY} must correspond to
* a {@link TimeUnit} value.
*
* @param conf The configuration object.
* @return The configured time unit for the chore period and initial delay or the default value.
*/
static TimeUnit getTimeUnit(Configuration conf) {
return TimeUnit.valueOf(conf.get(POLICY_REFRESHER_CHORE_TIMEUNIT_KEY,
POLICY_REFRESHER_CHORE_TIMEUNIT_DEFAULT));
}
/**
* Extracts the percent of Regions for a table to have been reported to enable quota violation
* state change.
*
* @param conf The configuration object.
* @return The percent of regions reported to use.
*/
static Double getRegionReportPercent(Configuration conf) {
return conf.getDouble(POLICY_REFRESHER_CHORE_REPORT_PERCENT_KEY,
POLICY_REFRESHER_CHORE_REPORT_PERCENT_DEFAULT);
}
}

View File

@ -0,0 +1,91 @@
/*
* 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.hadoop.fs.FileSystem;
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.Mutation;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
/**
* RegionServer implementation of {@link SpaceViolationPolicy}.
*
* Implementations must have a public, no-args constructor.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public interface SpaceViolationPolicyEnforcement {
/**
* Initializes this policy instance.
*/
void initialize(RegionServerServices rss, TableName tableName, SpaceQuotaSnapshot snapshot);
/**
* Enables this policy. Not all policies have enable actions.
*/
void enable() throws IOException;
/**
* Disables this policy. Not all policies have disable actions.
*/
void disable() throws IOException;
/**
* Checks the given {@link Mutation} against <code>this</code> policy. If the
* {@link Mutation} violates the policy, this policy should throw a
* {@link SpaceLimitingException}.
*
* @throws SpaceLimitingException When the given mutation violates this policy.
*/
void check(Mutation m) throws SpaceLimitingException;
/**
* Returns a logical name for the {@link SpaceViolationPolicy} that this enforcement is for.
*/
String getPolicyName();
/**
* Returns whether or not compactions on this table should be disabled for this policy.
*/
boolean areCompactionsDisabled();
/**
* Returns the {@link SpaceQuotaSnapshot} <code>this</code> was initialized with.
*/
SpaceQuotaSnapshot getQuotaSnapshot();
/**
* Returns whether thet caller should verify any bulk loads against <code>this</code>.
*/
boolean shouldCheckBulkLoads();
/**
* Checks the file at the given path against <code>this</code> policy and the current
* {@link SpaceQuotaSnapshot}. If the file would violate the policy, a
* {@link SpaceLimitingException} will be thrown.
*
* @param paths The paths in HDFS to files to be bulk loaded.
*/
void checkBulkLoad(FileSystem fs, List<String> paths) throws SpaceLimitingException;
}

View File

@ -0,0 +1,95 @@
/*
* 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.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
import org.apache.hadoop.hbase.quotas.policies.BulkLoadVerifyingViolationPolicyEnforcement;
import org.apache.hadoop.hbase.quotas.policies.DisableTableViolationPolicyEnforcement;
import org.apache.hadoop.hbase.quotas.policies.NoInsertsViolationPolicyEnforcement;
import org.apache.hadoop.hbase.quotas.policies.NoWritesCompactionsViolationPolicyEnforcement;
import org.apache.hadoop.hbase.quotas.policies.NoWritesViolationPolicyEnforcement;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
/**
* A factory class for instantiating {@link SpaceViolationPolicyEnforcement} instances.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class SpaceViolationPolicyEnforcementFactory {
private static final SpaceViolationPolicyEnforcementFactory INSTANCE =
new SpaceViolationPolicyEnforcementFactory();
private SpaceViolationPolicyEnforcementFactory() {}
/**
* Returns an instance of this factory.
*/
public static SpaceViolationPolicyEnforcementFactory getInstance() {
return INSTANCE;
}
/**
* Constructs the appropriate {@link SpaceViolationPolicyEnforcement} for tables that are
* in violation of their space quota.
*/
public SpaceViolationPolicyEnforcement create(
RegionServerServices rss, TableName tableName, SpaceQuotaSnapshot snapshot) {
SpaceViolationPolicyEnforcement enforcement;
SpaceQuotaStatus status = snapshot.getQuotaStatus();
if (!status.isInViolation()) {
throw new IllegalArgumentException(tableName + " is not in violation. Snapshot=" + snapshot);
}
switch (status.getPolicy()) {
case DISABLE:
enforcement = new DisableTableViolationPolicyEnforcement();
break;
case NO_WRITES_COMPACTIONS:
enforcement = new NoWritesCompactionsViolationPolicyEnforcement();
break;
case NO_WRITES:
enforcement = new NoWritesViolationPolicyEnforcement();
break;
case NO_INSERTS:
enforcement = new NoInsertsViolationPolicyEnforcement();
break;
default:
throw new IllegalArgumentException("Unhandled SpaceViolationPolicy: " + status.getPolicy());
}
enforcement.initialize(rss, tableName, snapshot);
return enforcement;
}
/**
* Creates the "default" {@link SpaceViolationPolicyEnforcement} for a table that isn't in
* violation. This is used to have uniform policy checking for tables in and not quotas.
*/
public SpaceViolationPolicyEnforcement createWithoutViolation(
RegionServerServices rss, TableName tableName, SpaceQuotaSnapshot snapshot) {
SpaceQuotaStatus status = snapshot.getQuotaStatus();
if (status.isInViolation()) {
throw new IllegalArgumentException(
tableName + " is in violation. Logic error. Snapshot=" + snapshot);
}
BulkLoadVerifyingViolationPolicyEnforcement enforcement = new BulkLoadVerifyingViolationPolicyEnforcement();
enforcement.initialize(rss, tableName, snapshot);
return enforcement;
}
}

View File

@ -28,6 +28,8 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
@ -35,10 +37,10 @@ import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
/**
* {@link QuotaViolationStore} for tables.
* {@link QuotaSnapshotStore} for tables.
*/
@InterfaceAudience.Private
public class TableQuotaViolationStore implements QuotaViolationStore<TableName> {
public class TableQuotaSnapshotStore implements QuotaSnapshotStore<TableName> {
private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
private final ReadLock rlock = lock.readLock();
private final WriteLock wlock = lock.writeLock();
@ -47,7 +49,7 @@ public class TableQuotaViolationStore implements QuotaViolationStore<TableName>
private final QuotaObserverChore chore;
private Map<HRegionInfo,Long> regionUsage;
public TableQuotaViolationStore(Connection conn, QuotaObserverChore chore, Map<HRegionInfo,Long> regionUsage) {
public TableQuotaSnapshotStore(Connection conn, QuotaObserverChore chore, Map<HRegionInfo,Long> regionUsage) {
this.conn = Objects.requireNonNull(conn);
this.chore = Objects.requireNonNull(chore);
this.regionUsage = Objects.requireNonNull(regionUsage);
@ -69,26 +71,24 @@ public class TableQuotaViolationStore implements QuotaViolationStore<TableName>
}
@Override
public ViolationState getCurrentState(TableName table) {
public SpaceQuotaSnapshot getCurrentState(TableName table) {
// Defer the "current state" to the chore
return chore.getTableQuotaViolation(table);
return chore.getTableQuotaSnapshot(table);
}
@Override
public ViolationState getTargetState(TableName table, SpaceQuota spaceQuota) {
public SpaceQuotaSnapshot getTargetState(TableName table, SpaceQuota spaceQuota) {
rlock.lock();
try {
final long sizeLimitInBytes = spaceQuota.getSoftLimit();
long sum = 0L;
for (Entry<HRegionInfo,Long> entry : filterBySubject(table)) {
sum += entry.getValue();
if (sum > sizeLimitInBytes) {
// Short-circuit early
return ViolationState.IN_VIOLATION;
}
}
// Observance is defined as the size of the table being less than the limit
return sum <= sizeLimitInBytes ? ViolationState.IN_OBSERVANCE : ViolationState.IN_VIOLATION;
SpaceQuotaStatus status = sum <= sizeLimitInBytes ? SpaceQuotaStatus.notInViolation()
: new SpaceQuotaStatus(ProtobufUtil.toViolationPolicy(spaceQuota.getViolationPolicy()));
return new SpaceQuotaSnapshot(status, sum, sizeLimitInBytes);
} finally {
rlock.unlock();
}
@ -110,9 +110,9 @@ public class TableQuotaViolationStore implements QuotaViolationStore<TableName>
}
@Override
public void setCurrentState(TableName table, ViolationState state) {
public void setCurrentState(TableName table, SpaceQuotaSnapshot snapshot) {
// Defer the "current state" to the chore
this.chore.setTableQuotaViolation(table, state);
this.chore.setTableQuotaViolation(table, snapshot);
}
@Override

View File

@ -18,36 +18,33 @@ package org.apache.hadoop.hbase.quotas;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
/**
* A {@link SpaceQuotaViolationNotifier} which uses the hbase:quota table.
* A {@link SpaceQuotaSnapshotNotifier} which uses the hbase:quota table.
*/
public class TableSpaceQuotaViolationNotifier implements SpaceQuotaViolationNotifier {
public class TableSpaceQuotaSnapshotNotifier implements SpaceQuotaSnapshotNotifier {
private static final Log LOG = LogFactory.getLog(TableSpaceQuotaSnapshotNotifier.class);
private Connection conn;
@Override
public void transitionTableToViolation(
TableName tableName, SpaceViolationPolicy violationPolicy) throws IOException {
final Put p = QuotaTableUtil.createEnableViolationPolicyUpdate(tableName, violationPolicy);
public void transitionTable(
TableName tableName, SpaceQuotaSnapshot snapshot) throws IOException {
final Put p = QuotaTableUtil.createPutSpaceSnapshot(tableName, snapshot);
try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
if (LOG.isTraceEnabled()) {
LOG.trace("Persisting a space quota snapshot " + snapshot + " for " + tableName);
}
quotaTable.put(p);
}
}
@Override
public void transitionTableToObservance(TableName tableName) throws IOException {
final Delete d = QuotaTableUtil.createRemoveViolationPolicyUpdate(tableName);
try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
quotaTable.delete(d);
}
}
@Override
public void initialize(Connection conn) {
this.conn = conn;

View File

@ -0,0 +1,118 @@
/*
* 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.policies;
import java.io.IOException;
import java.util.List;
import java.util.Objects;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
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.quotas.SpaceLimitingException;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
/**
* Abstract implementation for {@link SpaceViolationPolicyEnforcement}.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public abstract class AbstractViolationPolicyEnforcement
implements SpaceViolationPolicyEnforcement {
RegionServerServices rss;
TableName tableName;
SpaceQuotaSnapshot quotaSnapshot;
public void setRegionServerServices(RegionServerServices rss) {
this.rss = Objects.requireNonNull(rss);
}
public void setTableName(TableName tableName) {
this.tableName = tableName;
}
public RegionServerServices getRegionServerServices() {
return this.rss;
}
public TableName getTableName() {
return this.tableName;
}
public void setQuotaSnapshot(SpaceQuotaSnapshot snapshot) {
this.quotaSnapshot = Objects.requireNonNull(snapshot);
}
@Override
public SpaceQuotaSnapshot getQuotaSnapshot() {
return this.quotaSnapshot;
}
@Override
public void initialize(RegionServerServices rss, TableName tableName, SpaceQuotaSnapshot snapshot) {
setRegionServerServices(rss);
setTableName(tableName);
setQuotaSnapshot(snapshot);
}
@Override
public boolean areCompactionsDisabled() {
return false;
}
@Override
public boolean shouldCheckBulkLoads() {
// Reference check. The singleton is used when no quota exists to check against
return SpaceQuotaSnapshot.getNoSuchSnapshot() != quotaSnapshot;
}
@Override
public void checkBulkLoad(FileSystem fs, List<String> paths) throws SpaceLimitingException {
// Compute the amount of space that could be used to save some arithmetic in the for-loop
final long sizeAvailableForBulkLoads = quotaSnapshot.getLimit() - quotaSnapshot.getUsage();
long size = 0L;
for (String path : paths) {
size += addSingleFile(fs, path);
if (size > sizeAvailableForBulkLoads) {
break;
}
}
if (size > sizeAvailableForBulkLoads) {
throw new SpaceLimitingException(getPolicyName(), "Bulk load of " + paths
+ " is disallowed because the file(s) exceed the limits of a space quota.");
}
}
private long addSingleFile(FileSystem fs, String path) throws SpaceLimitingException {
final FileStatus status;
try {
status = fs.getFileStatus(new Path(Objects.requireNonNull(path)));
} catch (IOException e) {
throw new SpaceLimitingException(getPolicyName(), "Could not verify length of file to bulk load", e);
}
if (!status.isFile()) {
throw new IllegalArgumentException(path + " is not a file.");
}
return status.getLen();
}
}

View File

@ -0,0 +1,50 @@
/*
* 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.policies;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
/**
* A {@link SpaceViolationPolicyEnforcement} instance which only checks for bulk loads. Useful for tables
* which have no violation policy. This is the default case for tables, as we want to make sure that
* a single bulk load call would violate the quota.
*/
@InterfaceAudience.Private
public class BulkLoadVerifyingViolationPolicyEnforcement extends AbstractViolationPolicyEnforcement {
@Override
public void enable() {}
@Override
public void disable() {}
@Override
public String getPolicyName() {
return "BulkLoadVerifying";
}
@Override
public boolean areCompactionsDisabled() {
return false;
}
@Override
public void check(Mutation m) throws SpaceLimitingException {}
}

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.policies;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
/**
* A {@link SpaceViolationPolicyEnforcement} which disables the table. The enforcement
* counterpart to {@link SpaceViolationPolicy#DISABLE}.
*/
@InterfaceAudience.Private
public class DisableTableViolationPolicyEnforcement extends AbstractViolationPolicyEnforcement {
private static final Log LOG = LogFactory.getLog(DisableTableViolationPolicyEnforcement.class);
@Override
public void enable() throws IOException {
try {
if (LOG.isTraceEnabled()) {
LOG.trace("Starting disable of " + getTableName());
}
getRegionServerServices().getClusterConnection().getAdmin().disableTable(getTableName());
if (LOG.isTraceEnabled()) {
LOG.trace("Disable is complete for " + getTableName());
}
} catch (TableNotEnabledException tnee) {
// The state we wanted it to be in.
}
}
@Override
public void disable() throws IOException {
try {
if (LOG.isTraceEnabled()) {
LOG.trace("Starting enable of " + getTableName());
}
getRegionServerServices().getClusterConnection().getAdmin().enableTable(getTableName());
if (LOG.isTraceEnabled()) {
LOG.trace("Enable is complete for " + getTableName());
}
} catch (TableNotDisabledException tnde) {
// The state we wanted it to be in
}
}
@Override
public void check(Mutation m) throws SpaceLimitingException {
// If this policy is enacted, then the table is (or should be) disabled.
throw new SpaceLimitingException(
getPolicyName(), "This table is disabled due to violating a space quota.");
}
@Override
public String getPolicyName() {
return SpaceViolationPolicy.DISABLE.name();
}
}

View File

@ -0,0 +1,55 @@
/*
* 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.policies;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
/**
* A {@link SpaceViolationPolicyEnforcement} which disallows any inserts to the table. The
* enforcement counterpart to {@link SpaceViolationPolicy#NO_INSERTS}.
*/
@InterfaceAudience.Private
public class NoInsertsViolationPolicyEnforcement extends AbstractViolationPolicyEnforcement {
@Override
public void enable() {}
@Override
public void disable() {}
@Override
public void check(Mutation m) throws SpaceLimitingException {
// Disallow all "new" data flowing into HBase, but allow Deletes (even though we know they will
// temporarily increase utilization).
if (m instanceof Append || m instanceof Increment || m instanceof Put) {
throw new SpaceLimitingException(getPolicyName(),
m.getClass().getSimpleName() + "s are disallowed due to a space quota.");
}
}
@Override
public String getPolicyName() {
return SpaceViolationPolicy.NO_INSERTS.name();
}
}

View File

@ -0,0 +1,64 @@
/*
* 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.policies;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
/**
* A {@link SpaceViolationPolicyEnforcement} implementation which disables all updates and
* compactions. The enforcement counterpart to {@link SpaceViolationPolicy#NO_WRITES_COMPACTIONS}.
*/
@InterfaceAudience.Private
public class NoWritesCompactionsViolationPolicyEnforcement
extends NoWritesViolationPolicyEnforcement {
private static final Log LOG = LogFactory.getLog(
NoWritesCompactionsViolationPolicyEnforcement.class);
private AtomicBoolean disableCompactions = new AtomicBoolean(false);
@Override
public synchronized void enable() {
boolean ret = disableCompactions.compareAndSet(false, true);
if (!ret && LOG.isTraceEnabled()) {
LOG.trace("Compactions were already disabled upon enabling the policy");
}
}
@Override
public synchronized void disable() {
boolean ret = disableCompactions.compareAndSet(true, false);
if (!ret && LOG.isTraceEnabled()) {
LOG.trace("Compactions were already enabled upon disabling the policy");
}
}
@Override
public String getPolicyName() {
return SpaceViolationPolicy.NO_WRITES_COMPACTIONS.name();
}
@Override
public boolean areCompactionsDisabled() {
return disableCompactions.get();
}
}

View File

@ -0,0 +1,54 @@
/*
* 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.policies;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
/**
* A {@link SpaceViolationPolicyEnforcement} implementation which disables all writes flowing
* into HBase. The enforcement counterpart to {@link SpaceViolationPolicy#NO_WRITES}.
*/
@InterfaceAudience.Private
public class NoWritesViolationPolicyEnforcement extends AbstractViolationPolicyEnforcement {
@Override
public void enable() {}
@Override
public void disable() {}
@Override
public void check(Mutation m) throws SpaceLimitingException {
if (m instanceof Append || m instanceof Delete || m instanceof Increment || m instanceof Put) {
throw new SpaceLimitingException(getPolicyName(),
m.getClass().getSimpleName() + "s are disallowed due to a space quota.");
}
}
@Override
public String getPolicyName() {
return SpaceViolationPolicy.NO_WRITES.name();
}
}

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.conf.ConfigurationManager;
import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
@ -313,6 +314,17 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
if (compaction == null) return null; // message logged inside
}
final RegionServerSpaceQuotaManager spaceQuotaManager =
this.server.getRegionServerSpaceQuotaManager();
if (null != spaceQuotaManager && spaceQuotaManager.areCompactionsDisabled(
r.getTableDesc().getTableName())) {
if (LOG.isDebugEnabled()) {
LOG.debug("Ignoring compaction request for " + r + " as an active space quota violation "
+ " policy disallows compactions.");
}
return null;
}
// We assume that most compactions are small. So, put system compactions into small
// pool; we will do selection there, and move to large pool if necessary.
ThreadPoolExecutor pool = (selectNow && s.throttleCompaction(compaction.getRequest().getSize()))

View File

@ -89,8 +89,12 @@ import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.master.MasterRpcServices;
import org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement;
import org.apache.hadoop.hbase.quotas.OperationQuota;
import org.apache.hadoop.hbase.quotas.QuotaUtil;
import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
import org.apache.hadoop.hbase.regionserver.Leases.Lease;
import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
@ -190,7 +194,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescr
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.DNS;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@ -560,8 +563,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
*/
private boolean checkAndRowMutate(final Region region, final List<ClientProtos.Action> actions,
final CellScanner cellScanner, byte[] row, byte[] family, byte[] qualifier,
CompareOp compareOp, ByteArrayComparable comparator,
RegionActionResult.Builder builder) throws IOException {
CompareOp compareOp, ByteArrayComparable comparator, RegionActionResult.Builder builder,
ActivePolicyEnforcement spaceQuotaEnforcement) throws IOException {
if (!region.getRegionInfo().isMetaTable()) {
regionServer.cacheFlusher.reclaimMemStoreMemory();
}
@ -582,10 +585,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
case PUT:
Put put = ProtobufUtil.toPut(action.getMutation(), cellScanner);
checkCellSizeLimit(region, put);
spaceQuotaEnforcement.getPolicyEnforcement(region).check(put);
rm.add(put);
break;
case DELETE:
rm.add(ProtobufUtil.toDelete(action.getMutation(), cellScanner));
Delete del = ProtobufUtil.toDelete(action.getMutation(), cellScanner);
spaceQuotaEnforcement.getPolicyEnforcement(region).check(del);
rm.add(del);
break;
default:
throw new DoNotRetryIOException("Atomic put and/or delete only, not " + type.name());
@ -612,11 +618,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
* @throws IOException
*/
private Result append(final Region region, final OperationQuota quota,
final MutationProto mutation, final CellScanner cellScanner, long nonceGroup)
final MutationProto mutation, final CellScanner cellScanner, long nonceGroup,
ActivePolicyEnforcement spaceQuota)
throws IOException {
long before = EnvironmentEdgeManager.currentTime();
Append append = ProtobufUtil.toAppend(mutation, cellScanner);
checkCellSizeLimit(region, append);
spaceQuota.getPolicyEnforcement(region).check(append);
quota.addMutation(append);
Result r = null;
if (region.getCoprocessorHost() != null) {
@ -661,11 +669,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
* @throws IOException
*/
private Result increment(final Region region, final OperationQuota quota,
final MutationProto mutation, final CellScanner cells, long nonceGroup)
final MutationProto mutation, final CellScanner cells, long nonceGroup,
ActivePolicyEnforcement spaceQuota)
throws IOException {
long before = EnvironmentEdgeManager.currentTime();
Increment increment = ProtobufUtil.toIncrement(mutation, cells);
checkCellSizeLimit(region, increment);
spaceQuota.getPolicyEnforcement(region).check(increment);
quota.addMutation(increment);
Result r = null;
if (region.getCoprocessorHost() != null) {
@ -717,7 +727,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
private List<CellScannable> doNonAtomicRegionMutation(final Region region,
final OperationQuota quota, final RegionAction actions, final CellScanner cellScanner,
final RegionActionResult.Builder builder, List<CellScannable> cellsToReturn, long nonceGroup,
final RegionScannersCloseCallBack closeCallBack, RpcCallContext context) {
final RegionScannersCloseCallBack closeCallBack, RpcCallContext context,
ActivePolicyEnforcement spaceQuotaEnforcement) {
// 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
// ResultOrException instance that matches each Put or Delete is then added down in the
@ -810,15 +821,17 @@ 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, quota, mutations, cellScanner);
doBatchOp(builder, region, quota, mutations, cellScanner, spaceQuotaEnforcement);
mutations.clear();
}
switch (type) {
case APPEND:
r = append(region, quota, action.getMutation(), cellScanner, nonceGroup);
r = append(region, quota, action.getMutation(), cellScanner, nonceGroup,
spaceQuotaEnforcement);
break;
case INCREMENT:
r = increment(region, quota, action.getMutation(), cellScanner, nonceGroup);
r = increment(region, quota, action.getMutation(), cellScanner, nonceGroup,
spaceQuotaEnforcement);
break;
case PUT:
case DELETE:
@ -869,7 +882,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
// Finish up any outstanding mutations
if (mutations != null && !mutations.isEmpty()) {
doBatchOp(builder, region, quota, mutations, cellScanner);
doBatchOp(builder, region, quota, mutations, cellScanner, spaceQuotaEnforcement);
}
return cellsToReturn;
}
@ -903,7 +916,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
*/
private void doBatchOp(final RegionActionResult.Builder builder, final Region region,
final OperationQuota quota, final List<ClientProtos.Action> mutations,
final CellScanner cells) {
final CellScanner cells, ActivePolicyEnforcement spaceQuotaEnforcement) {
Mutation[] mArray = new Mutation[mutations.size()];
long before = EnvironmentEdgeManager.currentTime();
boolean batchContainsPuts = false, batchContainsDelete = false;
@ -930,6 +943,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
mutationActionMap.put(mutation, action);
mArray[i++] = mutation;
checkCellSizeLimit(region, mutation);
// Check if a space quota disallows this mutation
spaceQuotaEnforcement.getPolicyEnforcement(region).check(mutation);
quota.addMutation(mutation);
}
@ -1306,10 +1321,14 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
return regionServer.getConfiguration();
}
private RegionServerRpcQuotaManager getQuotaManager() {
private RegionServerRpcQuotaManager getRpcQuotaManager() {
return regionServer.getRegionServerRpcQuotaManager();
}
private RegionServerSpaceQuotaManager getSpaceQuotaManager() {
return regionServer.getRegionServerSpaceQuotaManager();
}
void start() {
rpcServer.start();
}
@ -1485,6 +1504,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
checkOpen();
requestCount.increment();
Region region = getRegion(request.getRegion());
if (QuotaUtil.isQuotaEnabled(getConfiguration()) &&
this.regionServer.getRegionServerSpaceQuotaManager().areCompactionsDisabled(
region.getTableDesc().getTableName())) {
throw new DoNotRetryIOException("Compactions on this region are "
+ "disabled due to a space quota violation.");
}
region.startRegionOperation(Operation.COMPACT_REGION);
LOG.info("Compacting " + region.getRegionInfo().getRegionNameAsString());
boolean major = false;
@ -2212,6 +2237,21 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
boolean loaded = false;
Map<byte[], List<Path>> map = null;
// Check to see if this bulk load would exceed the space quota for this table
if (QuotaUtil.isQuotaEnabled(getConfiguration())) {
ActivePolicyEnforcement activeSpaceQuotas = getSpaceQuotaManager().getActiveEnforcements();
SpaceViolationPolicyEnforcement enforcement = activeSpaceQuotas.getPolicyEnforcement(region);
if (null != enforcement) {
// Bulk loads must still be atomic. We must enact all or none.
List<String> filePaths = new ArrayList<>(request.getFamilyPathCount());
for (FamilyPath familyPath : request.getFamilyPathList()) {
filePaths.add(familyPath.getPath());
}
// Check if the batch of files exceeds the current quota
enforcement.checkBulkLoad(regionServer.getFileSystem(), filePaths);
}
}
if (!request.hasBulkToken()) {
// Old style bulk load. This will not be supported in future releases
List<Pair<byte[], String>> familyPaths = new ArrayList<>(request.getFamilyPathCount());
@ -2340,7 +2380,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
Boolean existence = null;
Result r = null;
RpcCallContext context = RpcServer.getCurrentCall();
quota = getQuotaManager().checkQuota(region, OperationQuota.OperationType.GET);
quota = getRpcQuotaManager().checkQuota(region, OperationQuota.OperationType.GET);
Get clientGet = ProtobufUtil.toGet(get);
if (get.getExistenceOnly() && region.getCoprocessorHost() != null) {
@ -2478,6 +2518,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
this.rpcMultiRequestCount.increment();
Map<RegionSpecifier, ClientProtos.RegionLoadStats> regionStats = new HashMap<>(request
.getRegionActionCount());
ActivePolicyEnforcement spaceQuotaEnforcement = getSpaceQuotaManager().getActiveEnforcements();
for (RegionAction regionAction : request.getRegionActionList()) {
this.requestCount.add(regionAction.getActionCount());
OperationQuota quota;
@ -2486,7 +2527,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
RegionSpecifier regionSpecifier = regionAction.getRegion();
try {
region = getRegion(regionSpecifier);
quota = getQuotaManager().checkQuota(region, regionAction.getActionList());
quota = getRpcQuotaManager().checkQuota(region, regionAction.getActionList());
} catch (IOException e) {
rpcServer.getMetrics().exception(e);
regionActionResultBuilder.setException(ResponseConverter.buildException(e));
@ -2514,7 +2555,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
ProtobufUtil.toComparator(condition.getComparator());
processed = checkAndRowMutate(region, regionAction.getActionList(),
cellScanner, row, family, qualifier, compareOp,
comparator, regionActionResultBuilder);
comparator, regionActionResultBuilder, spaceQuotaEnforcement);
} else {
mutateRows(region, regionAction.getActionList(), cellScanner,
regionActionResultBuilder);
@ -2535,7 +2576,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
context.setCallBack(closeCallBack);
}
cellsToReturn = doNonAtomicRegionMutation(region, quota, regionAction, cellScanner,
regionActionResultBuilder, cellsToReturn, nonceGroup, closeCallBack, context);
regionActionResultBuilder, cellsToReturn, nonceGroup, closeCallBack, context,
spaceQuotaEnforcement);
}
responseBuilder.addRegionActionResult(regionActionResultBuilder.build());
quota.close();
@ -2602,6 +2644,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
CellScanner cellScanner = controller != null ? controller.cellScanner() : null;
OperationQuota quota = null;
RpcCallContext context = RpcServer.getCurrentCall();
ActivePolicyEnforcement spaceQuotaEnforcement = null;
// Clear scanner so we are not holding on to reference across call.
if (controller != null) {
controller.setCellScanner(null);
@ -2621,20 +2664,23 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
Boolean processed = null;
MutationType type = mutation.getMutateType();
quota = getQuotaManager().checkQuota(region, OperationQuota.OperationType.MUTATE);
quota = getRpcQuotaManager().checkQuota(region, OperationQuota.OperationType.MUTATE);
spaceQuotaEnforcement = getSpaceQuotaManager().getActiveEnforcements();
switch (type) {
case APPEND:
// TODO: this doesn't actually check anything.
r = append(region, quota, mutation, cellScanner, nonceGroup);
r = append(region, quota, mutation, cellScanner, nonceGroup, spaceQuotaEnforcement);
break;
case INCREMENT:
// TODO: this doesn't actually check anything.
r = increment(region, quota, mutation, cellScanner, nonceGroup);
r = increment(region, quota, mutation, cellScanner, nonceGroup, spaceQuotaEnforcement);
break;
case PUT:
Put put = ProtobufUtil.toPut(mutation, cellScanner);
checkCellSizeLimit(region, put);
// Throws an exception when violated
spaceQuotaEnforcement.getPolicyEnforcement(region).check(put);
quota.addMutation(put);
if (request.hasCondition()) {
Condition condition = request.getCondition();
@ -2665,6 +2711,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
case DELETE:
Delete delete = ProtobufUtil.toDelete(mutation, cellScanner);
checkCellSizeLimit(region, delete);
spaceQuotaEnforcement.getPolicyEnforcement(region).check(delete);
quota.addMutation(delete);
if (request.hasCondition()) {
Condition condition = request.getCondition();
@ -3127,7 +3174,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
OperationQuota quota;
try {
quota = getQuotaManager().checkQuota(region, OperationQuota.OperationType.SCAN);
quota = getRpcQuotaManager().checkQuota(region, OperationQuota.OperationType.SCAN);
} catch (IOException e) {
addScannerLeaseBack(lease);
throw new ServiceException(e);

View File

@ -0,0 +1,228 @@
/*
* 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.fail;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map.Entry;
import java.util.Objects;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.rules.TestName;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.Multimap;
@InterfaceAudience.Private
public class SpaceQuotaHelperForTests {
private static final Log LOG = LogFactory.getLog(SpaceQuotaHelperForTests.class);
public static final int SIZE_PER_VALUE = 256;
public static final String F1 = "f1";
public static final long ONE_KILOBYTE = 1024L;
public static final long ONE_MEGABYTE = ONE_KILOBYTE * ONE_KILOBYTE;
private final HBaseTestingUtility testUtil;
private final TestName testName;
private final AtomicLong counter;
public SpaceQuotaHelperForTests(
HBaseTestingUtility testUtil, TestName testName, AtomicLong counter) {
this.testUtil = Objects.requireNonNull(testUtil);
this.testName = Objects.requireNonNull(testName);
this.counter = Objects.requireNonNull(counter);
}
//
// Helpers
//
void writeData(TableName tn, long sizeInBytes) throws IOException {
final Connection conn = testUtil.getConnection();
final Table table = conn.getTable(tn);
try {
List<Put> updates = new ArrayList<>();
long bytesToWrite = sizeInBytes;
long rowKeyId = 0L;
final StringBuilder sb = new StringBuilder();
final Random r = new Random();
while (bytesToWrite > 0L) {
sb.setLength(0);
sb.append(Long.toString(rowKeyId));
// Use the reverse counter as the rowKey to get even spread across all regions
Put p = new Put(Bytes.toBytes(sb.reverse().toString()));
byte[] value = new byte[SIZE_PER_VALUE];
r.nextBytes(value);
p.addColumn(Bytes.toBytes(F1), Bytes.toBytes("q1"), value);
updates.add(p);
// Batch ~13KB worth of updates
if (updates.size() > 50) {
table.put(updates);
updates.clear();
}
// Just count the value size, ignore the size of rowkey + column
bytesToWrite -= SIZE_PER_VALUE;
rowKeyId++;
}
// Write the final batch
if (!updates.isEmpty()) {
table.put(updates);
}
LOG.debug("Data was written to HBase");
// Push the data to disk.
testUtil.getAdmin().flush(tn);
LOG.debug("Data flushed to disk");
} finally {
table.close();
}
}
Multimap<TableName, QuotaSettings> createTablesWithSpaceQuotas() throws Exception {
final Admin admin = testUtil.getAdmin();
final Multimap<TableName, QuotaSettings> tablesWithQuotas = HashMultimap.create();
final TableName tn1 = createTable();
final TableName tn2 = createTable();
NamespaceDescriptor nd = NamespaceDescriptor.create("ns" + counter.getAndIncrement()).build();
admin.createNamespace(nd);
final TableName tn3 = createTableInNamespace(nd);
final TableName tn4 = createTableInNamespace(nd);
final TableName tn5 = createTableInNamespace(nd);
final long sizeLimit1 = 1024L * 1024L * 1024L * 1024L * 5L; // 5TB
final SpaceViolationPolicy violationPolicy1 = SpaceViolationPolicy.NO_WRITES;
QuotaSettings qs1 = QuotaSettingsFactory.limitTableSpace(tn1, sizeLimit1, violationPolicy1);
tablesWithQuotas.put(tn1, qs1);
admin.setQuota(qs1);
final long sizeLimit2 = 1024L * 1024L * 1024L * 200L; // 200GB
final SpaceViolationPolicy violationPolicy2 = SpaceViolationPolicy.NO_WRITES_COMPACTIONS;
QuotaSettings qs2 = QuotaSettingsFactory.limitTableSpace(tn2, sizeLimit2, violationPolicy2);
tablesWithQuotas.put(tn2, qs2);
admin.setQuota(qs2);
final long sizeLimit3 = 1024L * 1024L * 1024L * 1024L * 100L; // 100TB
final SpaceViolationPolicy violationPolicy3 = SpaceViolationPolicy.NO_INSERTS;
QuotaSettings qs3 = QuotaSettingsFactory.limitNamespaceSpace(
nd.getName(), sizeLimit3, violationPolicy3);
tablesWithQuotas.put(tn3, qs3);
tablesWithQuotas.put(tn4, qs3);
tablesWithQuotas.put(tn5, qs3);
admin.setQuota(qs3);
final long sizeLimit4 = 1024L * 1024L * 1024L * 5L; // 5GB
final SpaceViolationPolicy violationPolicy4 = SpaceViolationPolicy.NO_INSERTS;
QuotaSettings qs4 = QuotaSettingsFactory.limitTableSpace(tn5, sizeLimit4, violationPolicy4);
// Override the ns quota for tn5, import edge-case to catch table quota taking
// precedence over ns quota.
tablesWithQuotas.put(tn5, qs4);
admin.setQuota(qs4);
return tablesWithQuotas;
}
TableName createTable() throws Exception {
return createTableWithRegions(1);
}
TableName createTableWithRegions(int numRegions) throws Exception {
return createTableWithRegions(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR, numRegions);
}
TableName createTableWithRegions(String namespace, int numRegions) throws Exception {
final Admin admin = testUtil.getAdmin();
final TableName tn = TableName.valueOf(
namespace, testName.getMethodName() + counter.getAndIncrement());
// Delete the old table
if (admin.tableExists(tn)) {
admin.disableTable(tn);
admin.deleteTable(tn);
}
// Create the table
HTableDescriptor tableDesc = new HTableDescriptor(tn);
tableDesc.addFamily(new HColumnDescriptor(F1));
if (numRegions == 1) {
admin.createTable(tableDesc);
} else {
admin.createTable(tableDesc, Bytes.toBytes("0"), Bytes.toBytes("9"), numRegions);
}
return tn;
}
TableName createTableInNamespace(NamespaceDescriptor nd) throws Exception {
final Admin admin = testUtil.getAdmin();
final TableName tn = TableName.valueOf(nd.getName(),
testName.getMethodName() + counter.getAndIncrement());
// Delete the old table
if (admin.tableExists(tn)) {
admin.disableTable(tn);
admin.deleteTable(tn);
}
// Create the table
HTableDescriptor tableDesc = new HTableDescriptor(tn);
tableDesc.addFamily(new HColumnDescriptor(F1));
admin.createTable(tableDesc);
return tn;
}
void partitionTablesByQuotaTarget(Multimap<TableName,QuotaSettings> quotas,
Set<TableName> tablesWithTableQuota, Set<TableName> tablesWithNamespaceQuota) {
// Partition the tables with quotas by table and ns quota
for (Entry<TableName, QuotaSettings> entry : quotas.entries()) {
SpaceLimitSettings settings = (SpaceLimitSettings) entry.getValue();
TableName tn = entry.getKey();
if (null != settings.getTableName()) {
tablesWithTableQuota.add(tn);
}
if (null != settings.getNamespace()) {
tablesWithNamespaceQuota.add(tn);
}
if (null == settings.getTableName() && null == settings.getNamespace()) {
fail("Unexpected table name with null tableName and namespace: " + tn);
}
}
}
}

View File

@ -19,36 +19,37 @@ package org.apache.hadoop.hbase.quotas;
import java.util.HashMap;
import java.util.Map;
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.client.Connection;
/**
* A SpaceQuotaViolationNotifier implementation for verifying testing.
* A SpaceQuotaSnapshotNotifier implementation for testing.
*/
@InterfaceAudience.Private
public class SpaceQuotaViolationNotifierForTest implements SpaceQuotaViolationNotifier {
public class SpaceQuotaSnapshotNotifierForTest implements SpaceQuotaSnapshotNotifier {
private static final Log LOG = LogFactory.getLog(SpaceQuotaSnapshotNotifierForTest.class);
private final Map<TableName,SpaceViolationPolicy> tablesInViolation = new HashMap<>();
private final Map<TableName,SpaceQuotaSnapshot> tableQuotaSnapshots = new HashMap<>();
@Override
public void initialize(Connection conn) {}
@Override
public void transitionTableToViolation(TableName tableName, SpaceViolationPolicy violationPolicy) {
tablesInViolation.put(tableName, violationPolicy);
public synchronized void transitionTable(TableName tableName, SpaceQuotaSnapshot snapshot) {
if (LOG.isTraceEnabled()) {
LOG.trace("Persisting " + tableName + "=>" + snapshot);
}
tableQuotaSnapshots.put(tableName, snapshot);
}
@Override
public void transitionTableToObservance(TableName tableName) {
tablesInViolation.remove(tableName);
public synchronized Map<TableName,SpaceQuotaSnapshot> copySnapshots() {
return new HashMap<>(this.tableQuotaSnapshots);
}
public Map<TableName,SpaceViolationPolicy> snapshotTablesInViolation() {
return new HashMap<>(this.tablesInViolation);
}
public void clearTableViolations() {
this.tablesInViolation.clear();
public synchronized void clearSnapshots() {
this.tableQuotaSnapshots.clear();
}
}

View File

@ -0,0 +1,74 @@
/*
* 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.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.quotas.policies.NoWritesViolationPolicyEnforcement;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.quotas.policies.BulkLoadVerifyingViolationPolicyEnforcement;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test class for {@link ActivePolicyEnforcement}.
*/
@Category(SmallTests.class)
public class TestActivePolicyEnforcement {
@Test
public void testGetter() {
final TableName tableName = TableName.valueOf("table");
Map<TableName, SpaceViolationPolicyEnforcement> map = new HashMap<>();
map.put(tableName, new NoWritesViolationPolicyEnforcement());
ActivePolicyEnforcement ape = new ActivePolicyEnforcement(map, Collections.emptyMap(), null);
assertEquals(map.get(tableName), ape.getPolicyEnforcement(tableName));
}
@Test
public void testNoPolicyReturnsNoopEnforcement() {
ActivePolicyEnforcement ape = new ActivePolicyEnforcement(
new HashMap<>(), Collections.emptyMap(), mock(RegionServerServices.class));
SpaceViolationPolicyEnforcement enforcement = ape.getPolicyEnforcement(
TableName.valueOf("nonexistent"));
assertNotNull(enforcement);
assertTrue(
"Expected an instance of NoopViolationPolicyEnforcement",
enforcement instanceof BulkLoadVerifyingViolationPolicyEnforcement);
}
@Test
public void testNoBulkLoadChecksOnNoSnapshot() {
ActivePolicyEnforcement ape = new ActivePolicyEnforcement(
new HashMap<TableName, SpaceViolationPolicyEnforcement>(),
Collections.<TableName,SpaceQuotaSnapshot> emptyMap(),
mock(RegionServerServices.class));
SpaceViolationPolicyEnforcement enforcement = ape.getPolicyEnforcement(
TableName.valueOf("nonexistent"));
assertFalse("Should not check bulkloads", enforcement.shouldCheckBulkLoads());
}
}

View File

@ -144,6 +144,7 @@ public class TestFileSystemUtilizationChore {
assertEquals(timeUnit, chore.getTimeUnit());
}
@SuppressWarnings("unchecked")
@Test
public void testProcessingLeftoverRegions() {
final Configuration conf = getDefaultHBaseConfiguration();
@ -176,6 +177,7 @@ public class TestFileSystemUtilizationChore {
chore.chore();
}
@SuppressWarnings("unchecked")
@Test
public void testProcessingNowOfflineLeftoversAreIgnored() {
final Configuration conf = getDefaultHBaseConfiguration();
@ -185,7 +187,6 @@ public class TestFileSystemUtilizationChore {
final List<Long> leftover1Sizes = Arrays.asList(1024L, 4096L);
final long leftover1Sum = sum(leftover1Sizes);
final List<Long> leftover2Sizes = Arrays.asList(2048L);
final long leftover2Sum = sum(leftover2Sizes);
final Region lr1 = mockRegionWithSize(leftover1Sizes);
final Region lr2 = mockRegionWithSize(leftover2Sizes);

View File

@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.quotas.QuotaViolationStore.ViolationState;
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;
@ -45,7 +44,7 @@ import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
/**
* Test class for {@link NamespaceQuotaViolationStore}.
* Test class for {@link NamespaceQuotaSnapshotStore}.
*/
@Category(SmallTests.class)
public class TestNamespaceQuotaViolationStore {
@ -54,19 +53,19 @@ public class TestNamespaceQuotaViolationStore {
private Connection conn;
private QuotaObserverChore chore;
private Map<HRegionInfo, Long> regionReports;
private NamespaceQuotaViolationStore store;
private NamespaceQuotaSnapshotStore store;
@Before
public void setup() {
conn = mock(Connection.class);
chore = mock(QuotaObserverChore.class);
regionReports = new HashMap<>();
store = new NamespaceQuotaViolationStore(conn, chore, regionReports);
store = new NamespaceQuotaSnapshotStore(conn, chore, regionReports);
}
@Test
public void testGetSpaceQuota() throws Exception {
NamespaceQuotaViolationStore mockStore = mock(NamespaceQuotaViolationStore.class);
NamespaceQuotaSnapshotStore mockStore = mock(NamespaceQuotaSnapshotStore.class);
when(mockStore.getSpaceQuota(any(String.class))).thenCallRealMethod();
Quotas quotaWithSpace = Quotas.newBuilder().setSpace(
@ -113,17 +112,18 @@ public class TestNamespaceQuotaViolationStore {
regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(1), Bytes.toBytes(2)), 1024L * 256L);
// Below the quota
assertEquals(ViolationState.IN_OBSERVANCE, store.getTargetState(NS, quota));
assertEquals(false, store.getTargetState(NS, quota).getQuotaStatus().isInViolation());
regionReports.put(new HRegionInfo(tn2, Bytes.toBytes(2), Bytes.toBytes(3)), 1024L * 256L);
// Equal to the quota is still in observance
assertEquals(ViolationState.IN_OBSERVANCE, store.getTargetState(NS, quota));
assertEquals(false, store.getTargetState(NS, quota).getQuotaStatus().isInViolation());
regionReports.put(new HRegionInfo(tn2, Bytes.toBytes(3), Bytes.toBytes(4)), 1024L);
// Exceeds the quota, should be in violation
assertEquals(ViolationState.IN_VIOLATION, store.getTargetState(NS, quota));
assertEquals(true, store.getTargetState(NS, quota).getQuotaStatus().isInViolation());
assertEquals(SpaceViolationPolicy.DISABLE, store.getTargetState(NS, quota).getQuotaStatus().getPolicy());
}
@Test

View File

@ -17,8 +17,6 @@
package org.apache.hadoop.hbase.quotas;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@ -28,8 +26,6 @@ import java.util.Map;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Before;
@ -50,8 +46,6 @@ public class TestQuotaObserverChore {
public void setup() throws Exception {
conn = mock(Connection.class);
chore = mock(QuotaObserverChore.class);
// Set up some rules to call the real method on the mock.
when(chore.getViolationPolicy(any(SpaceQuota.class))).thenCallRealMethod();
}
@Test
@ -76,31 +70,11 @@ public class TestQuotaObserverChore {
regionReports.put(new HRegionInfo(tn3, Bytes.toBytes(i), Bytes.toBytes(i + 1)), 0L);
}
TableQuotaViolationStore store = new TableQuotaViolationStore(conn, chore, regionReports);
when(chore.getTableViolationStore()).thenReturn(store);
TableQuotaSnapshotStore store = new TableQuotaSnapshotStore(conn, chore, regionReports);
when(chore.getTableSnapshotStore()).thenReturn(store);
assertEquals(numTable1Regions, Iterables.size(store.filterBySubject(tn1)));
assertEquals(numTable2Regions, Iterables.size(store.filterBySubject(tn2)));
assertEquals(numTable3Regions, Iterables.size(store.filterBySubject(tn3)));
}
@Test
public void testExtractViolationPolicy() {
for (SpaceViolationPolicy policy : SpaceViolationPolicy.values()) {
SpaceQuota spaceQuota = SpaceQuota.newBuilder()
.setSoftLimit(1024L)
.setViolationPolicy(ProtobufUtil.toProtoViolationPolicy(policy))
.build();
assertEquals(policy, chore.getViolationPolicy(spaceQuota));
}
SpaceQuota malformedQuota = SpaceQuota.newBuilder()
.setSoftLimit(1024L)
.build();
try {
chore.getViolationPolicy(malformedQuota);
fail("Should have thrown an IllegalArgumentException.");
} catch (IllegalArgumentException e) {
// Pass
}
}
}

View File

@ -22,16 +22,12 @@ import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
@ -40,20 +36,15 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.quotas.QuotaObserverChore.TablesWithQuotas;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
@ -62,7 +53,6 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Multimap;
@ -72,11 +62,8 @@ import com.google.common.collect.Multimap;
@Category(LargeTests.class)
public class TestQuotaObserverChoreWithMiniCluster {
private static final Log LOG = LogFactory.getLog(TestQuotaObserverChoreWithMiniCluster.class);
private static final int SIZE_PER_VALUE = 256;
private static final String F1 = "f1";
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static final AtomicLong COUNTER = new AtomicLong(0);
private static final long ONE_MEGABYTE = 1024L * 1024L;
private static final long DEFAULT_WAIT_MILLIS = 500;
@Rule
@ -84,18 +71,19 @@ public class TestQuotaObserverChoreWithMiniCluster {
private HMaster master;
private QuotaObserverChore chore;
private SpaceQuotaViolationNotifierForTest violationNotifier;
private SpaceQuotaSnapshotNotifierForTest snapshotNotifier;
private SpaceQuotaHelperForTests helper;
@BeforeClass
public static void setUp() throws Exception {
Configuration conf = TEST_UTIL.getConfiguration();
conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_DELAY_KEY, 1000);
conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_PERIOD_KEY, 1000);
conf.setInt(QuotaObserverChore.VIOLATION_OBSERVER_CHORE_DELAY_KEY, 1000);
conf.setInt(QuotaObserverChore.VIOLATION_OBSERVER_CHORE_PERIOD_KEY, 1000);
conf.setInt(QuotaObserverChore.QUOTA_OBSERVER_CHORE_DELAY_KEY, 1000);
conf.setInt(QuotaObserverChore.QUOTA_OBSERVER_CHORE_PERIOD_KEY, 1000);
conf.setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
conf.setClass(SpaceQuotaViolationNotifierFactory.VIOLATION_NOTIFIER_KEY,
SpaceQuotaViolationNotifierForTest.class, SpaceQuotaViolationNotifier.class);
conf.setClass(SpaceQuotaSnapshotNotifierFactory.SNAPSHOT_NOTIFIER_KEY,
SpaceQuotaSnapshotNotifierForTest.class, SpaceQuotaSnapshotNotifier.class);
TEST_UTIL.startMiniCluster(1);
}
@ -131,40 +119,55 @@ public class TestQuotaObserverChoreWithMiniCluster {
}
master = TEST_UTIL.getMiniHBaseCluster().getMaster();
violationNotifier =
(SpaceQuotaViolationNotifierForTest) master.getSpaceQuotaViolationNotifier();
violationNotifier.clearTableViolations();
snapshotNotifier =
(SpaceQuotaSnapshotNotifierForTest) master.getSpaceQuotaSnapshotNotifier();
snapshotNotifier.clearSnapshots();
chore = master.getQuotaObserverChore();
helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, COUNTER);
}
@Test
public void testTableViolatesQuota() throws Exception {
TableName tn = createTableWithRegions(10);
TableName tn = helper.createTableWithRegions(10);
final long sizeLimit = 2L * ONE_MEGABYTE;
final long sizeLimit = 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
final SpaceViolationPolicy violationPolicy = SpaceViolationPolicy.NO_INSERTS;
QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(tn, sizeLimit, violationPolicy);
TEST_UTIL.getAdmin().setQuota(settings);
// Write more data than should be allowed
writeData(tn, 3L * ONE_MEGABYTE);
helper.writeData(tn, 3L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
Map<TableName,SpaceViolationPolicy> violatedQuotas = violationNotifier.snapshotTablesInViolation();
while (violatedQuotas.isEmpty()) {
LOG.info("Found no violated quotas, sleeping and retrying. Current reports: "
+ master.getMasterQuotaManager().snapshotRegionSizes());
try {
Thread.sleep(DEFAULT_WAIT_MILLIS);
} catch (InterruptedException e) {
LOG.debug("Interrupted while sleeping.", e);
Thread.currentThread().interrupt();
Map<TableName,SpaceQuotaSnapshot> quotaSnapshots = snapshotNotifier.copySnapshots();
boolean foundSnapshot = false;
while (!foundSnapshot) {
if (quotaSnapshots.isEmpty()) {
LOG.info("Found no violated quotas, sleeping and retrying. Current reports: "
+ master.getMasterQuotaManager().snapshotRegionSizes());
sleepWithInterrupt(DEFAULT_WAIT_MILLIS);
quotaSnapshots = snapshotNotifier.copySnapshots();
} else {
Entry<TableName,SpaceQuotaSnapshot> entry = Iterables.getOnlyElement(quotaSnapshots.entrySet());
assertEquals(tn, entry.getKey());
final SpaceQuotaSnapshot snapshot = entry.getValue();
if (!snapshot.getQuotaStatus().isInViolation()) {
LOG.info("Found a snapshot, but it was not yet in violation. " + snapshot);
sleepWithInterrupt(DEFAULT_WAIT_MILLIS);
quotaSnapshots = snapshotNotifier.copySnapshots();
} else {
foundSnapshot = true;
}
}
violatedQuotas = violationNotifier.snapshotTablesInViolation();
}
Entry<TableName,SpaceViolationPolicy> entry = Iterables.getOnlyElement(violatedQuotas.entrySet());
Entry<TableName,SpaceQuotaSnapshot> entry = Iterables.getOnlyElement(quotaSnapshots.entrySet());
assertEquals(tn, entry.getKey());
assertEquals(violationPolicy, entry.getValue());
final SpaceQuotaSnapshot snapshot = entry.getValue();
assertEquals("Snapshot was " + snapshot, violationPolicy, snapshot.getQuotaStatus().getPolicy());
assertEquals(sizeLimit, snapshot.getLimit());
assertTrue(
"The usage should be greater than the limit, but were " + snapshot.getUsage() + " and "
+ snapshot.getLimit() + ", respectively", snapshot.getUsage() > snapshot.getLimit());
}
@Test
@ -179,18 +182,18 @@ public class TestQuotaObserverChoreWithMiniCluster {
admin.createNamespace(desc);
}
TableName tn1 = createTableWithRegions(namespace, 5);
TableName tn2 = createTableWithRegions(namespace, 5);
TableName tn3 = createTableWithRegions(namespace, 5);
TableName tn1 = helper.createTableWithRegions(namespace, 5);
TableName tn2 = helper.createTableWithRegions(namespace, 5);
TableName tn3 = helper.createTableWithRegions(namespace, 5);
final long sizeLimit = 5L * ONE_MEGABYTE;
final long sizeLimit = 5L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
final SpaceViolationPolicy violationPolicy = SpaceViolationPolicy.DISABLE;
QuotaSettings settings = QuotaSettingsFactory.limitNamespaceSpace(namespace, sizeLimit, violationPolicy);
admin.setQuota(settings);
writeData(tn1, 2L * ONE_MEGABYTE);
helper.writeData(tn1, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
admin.flush(tn1);
Map<TableName,SpaceViolationPolicy> violatedQuotas = violationNotifier.snapshotTablesInViolation();
Map<TableName,SpaceQuotaSnapshot> violatedQuotas = snapshotNotifier.copySnapshots();
for (int i = 0; i < 5; i++) {
// Check a few times to make sure we don't prematurely move to violation
assertEquals("Should not see any quota violations after writing 2MB of data", 0, violatedQuotas.size());
@ -199,12 +202,12 @@ public class TestQuotaObserverChoreWithMiniCluster {
} catch (InterruptedException e) {
LOG.debug("Interrupted while sleeping." , e);
}
violatedQuotas = violationNotifier.snapshotTablesInViolation();
violatedQuotas = snapshotNotifier.copySnapshots();
}
writeData(tn2, 2L * ONE_MEGABYTE);
helper.writeData(tn2, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
admin.flush(tn2);
violatedQuotas = violationNotifier.snapshotTablesInViolation();
violatedQuotas = snapshotNotifier.copySnapshots();
for (int i = 0; i < 5; i++) {
// Check a few times to make sure we don't prematurely move to violation
assertEquals("Should not see any quota violations after writing 4MB of data", 0,
@ -214,14 +217,14 @@ public class TestQuotaObserverChoreWithMiniCluster {
} catch (InterruptedException e) {
LOG.debug("Interrupted while sleeping." , e);
}
violatedQuotas = violationNotifier.snapshotTablesInViolation();
violatedQuotas = snapshotNotifier.copySnapshots();
}
// Writing the final 2MB of data will push the namespace over the 5MB limit (6MB in total)
// and should push all three tables in the namespace into violation.
writeData(tn3, 2L * ONE_MEGABYTE);
helper.writeData(tn3, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
admin.flush(tn3);
violatedQuotas = violationNotifier.snapshotTablesInViolation();
violatedQuotas = snapshotNotifier.copySnapshots();
while (violatedQuotas.size() < 3) {
LOG.debug("Saw fewer violations than desired (expected 3): " + violatedQuotas
+ ". Current reports: " + master.getMasterQuotaManager().snapshotRegionSizes());
@ -231,18 +234,18 @@ public class TestQuotaObserverChoreWithMiniCluster {
LOG.debug("Interrupted while sleeping.", e);
Thread.currentThread().interrupt();
}
violatedQuotas = violationNotifier.snapshotTablesInViolation();
violatedQuotas = snapshotNotifier.copySnapshots();
}
SpaceViolationPolicy vp1 = violatedQuotas.remove(tn1);
assertNotNull("tn1 should be in violation", vp1);
assertEquals(violationPolicy, vp1);
SpaceViolationPolicy vp2 = violatedQuotas.remove(tn2);
assertNotNull("tn2 should be in violation", vp2);
assertEquals(violationPolicy, vp2);
SpaceViolationPolicy vp3 = violatedQuotas.remove(tn3);
assertNotNull("tn3 should be in violation", vp3);
assertEquals(violationPolicy, vp3);
SpaceQuotaSnapshot snapshot1 = violatedQuotas.remove(tn1);
assertNotNull("tn1 should be in violation", snapshot1);
assertEquals(violationPolicy, snapshot1.getQuotaStatus().getPolicy());
SpaceQuotaSnapshot snapshot2 = violatedQuotas.remove(tn2);
assertNotNull("tn2 should be in violation", snapshot2);
assertEquals(violationPolicy, snapshot2.getQuotaStatus().getPolicy());
SpaceQuotaSnapshot snapshot3 = violatedQuotas.remove(tn3);
assertNotNull("tn3 should be in violation", snapshot3);
assertEquals(violationPolicy, snapshot3.getQuotaStatus().getPolicy());
assertTrue("Unexpected additional quota violations: " + violatedQuotas, violatedQuotas.isEmpty());
}
@ -258,18 +261,18 @@ public class TestQuotaObserverChoreWithMiniCluster {
admin.createNamespace(desc);
}
TableName tn1 = createTableWithRegions(namespace, 5);
TableName tn2 = createTableWithRegions(namespace, 5);
TableName tn1 = helper.createTableWithRegions(namespace, 5);
TableName tn2 = helper.createTableWithRegions(namespace, 5);
final long namespaceSizeLimit = 3L * ONE_MEGABYTE;
final long namespaceSizeLimit = 3L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
final SpaceViolationPolicy namespaceViolationPolicy = SpaceViolationPolicy.DISABLE;
QuotaSettings namespaceSettings = QuotaSettingsFactory.limitNamespaceSpace(namespace,
namespaceSizeLimit, namespaceViolationPolicy);
admin.setQuota(namespaceSettings);
writeData(tn1, 2L * ONE_MEGABYTE);
helper.writeData(tn1, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
admin.flush(tn1);
Map<TableName,SpaceViolationPolicy> violatedQuotas = violationNotifier.snapshotTablesInViolation();
Map<TableName,SpaceQuotaSnapshot> violatedQuotas = snapshotNotifier.copySnapshots();
for (int i = 0; i < 5; i++) {
// Check a few times to make sure we don't prematurely move to violation
assertEquals("Should not see any quota violations after writing 2MB of data", 0,
@ -279,12 +282,12 @@ public class TestQuotaObserverChoreWithMiniCluster {
} catch (InterruptedException e) {
LOG.debug("Interrupted while sleeping." , e);
}
violatedQuotas = violationNotifier.snapshotTablesInViolation();
violatedQuotas = snapshotNotifier.copySnapshots();
}
writeData(tn2, 2L * ONE_MEGABYTE);
helper.writeData(tn2, 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
admin.flush(tn2);
violatedQuotas = violationNotifier.snapshotTablesInViolation();
violatedQuotas = snapshotNotifier.copySnapshots();
while (violatedQuotas.size() < 2) {
LOG.debug("Saw fewer violations than desired (expected 2): " + violatedQuotas
+ ". Current reports: " + master.getMasterQuotaManager().snapshotRegionSizes());
@ -294,18 +297,18 @@ public class TestQuotaObserverChoreWithMiniCluster {
LOG.debug("Interrupted while sleeping.", e);
Thread.currentThread().interrupt();
}
violatedQuotas = violationNotifier.snapshotTablesInViolation();
violatedQuotas = snapshotNotifier.copySnapshots();
}
SpaceViolationPolicy actualPolicyTN1 = violatedQuotas.get(tn1);
SpaceQuotaSnapshot actualPolicyTN1 = violatedQuotas.get(tn1);
assertNotNull("Expected to see violation policy for tn1", actualPolicyTN1);
assertEquals(namespaceViolationPolicy, actualPolicyTN1);
SpaceViolationPolicy actualPolicyTN2 = violatedQuotas.get(tn2);
assertEquals(namespaceViolationPolicy, actualPolicyTN1.getQuotaStatus().getPolicy());
SpaceQuotaSnapshot actualPolicyTN2 = violatedQuotas.get(tn2);
assertNotNull("Expected to see violation policy for tn2", actualPolicyTN2);
assertEquals(namespaceViolationPolicy, actualPolicyTN2);
assertEquals(namespaceViolationPolicy, actualPolicyTN2.getQuotaStatus().getPolicy());
// Override the namespace quota with a table quota
final long tableSizeLimit = ONE_MEGABYTE;
final long tableSizeLimit = SpaceQuotaHelperForTests.ONE_MEGABYTE;
final SpaceViolationPolicy tableViolationPolicy = SpaceViolationPolicy.NO_INSERTS;
QuotaSettings tableSettings = QuotaSettingsFactory.limitTableSpace(tn1, tableSizeLimit,
tableViolationPolicy);
@ -313,10 +316,10 @@ public class TestQuotaObserverChoreWithMiniCluster {
// Keep checking for the table quota policy to override the namespace quota
while (true) {
violatedQuotas = violationNotifier.snapshotTablesInViolation();
SpaceViolationPolicy actualTableViolationPolicy = violatedQuotas.get(tn1);
assertNotNull("Violation policy should never be null", actualTableViolationPolicy);
if (tableViolationPolicy != actualTableViolationPolicy) {
violatedQuotas = snapshotNotifier.copySnapshots();
SpaceQuotaSnapshot actualTableSnapshot = violatedQuotas.get(tn1);
assertNotNull("Violation policy should never be null", actualTableSnapshot);
if (tableViolationPolicy != actualTableSnapshot.getQuotaStatus().getPolicy()) {
LOG.debug("Saw unexpected table violation policy, waiting and re-checking.");
try {
Thread.sleep(DEFAULT_WAIT_MILLIS);
@ -326,23 +329,23 @@ public class TestQuotaObserverChoreWithMiniCluster {
}
continue;
}
assertEquals(tableViolationPolicy, actualTableViolationPolicy);
assertEquals(tableViolationPolicy, actualTableSnapshot.getQuotaStatus().getPolicy());
break;
}
// This should not change with the introduction of the table quota for tn1
actualPolicyTN2 = violatedQuotas.get(tn2);
assertNotNull("Expected to see violation policy for tn2", actualPolicyTN2);
assertEquals(namespaceViolationPolicy, actualPolicyTN2);
assertEquals(namespaceViolationPolicy, actualPolicyTN2.getQuotaStatus().getPolicy());
}
@Test
public void testGetAllTablesWithQuotas() throws Exception {
final Multimap<TableName, QuotaSettings> quotas = createTablesWithSpaceQuotas();
final Multimap<TableName, QuotaSettings> quotas = helper.createTablesWithSpaceQuotas();
Set<TableName> tablesWithQuotas = new HashSet<>();
Set<TableName> namespaceTablesWithQuotas = new HashSet<>();
// Partition the tables with quotas by table and ns quota
partitionTablesByQuotaTarget(quotas, tablesWithQuotas, namespaceTablesWithQuotas);
helper.partitionTablesByQuotaTarget(quotas, tablesWithQuotas, namespaceTablesWithQuotas);
TablesWithQuotas tables = chore.fetchAllTablesWithQuotasDefined();
assertEquals("Found tables: " + tables, tablesWithQuotas, tables.getTableQuotaTables());
@ -351,13 +354,13 @@ public class TestQuotaObserverChoreWithMiniCluster {
@Test
public void testRpcQuotaTablesAreFiltered() throws Exception {
final Multimap<TableName, QuotaSettings> quotas = createTablesWithSpaceQuotas();
final Multimap<TableName, QuotaSettings> quotas = helper.createTablesWithSpaceQuotas();
Set<TableName> tablesWithQuotas = new HashSet<>();
Set<TableName> namespaceTablesWithQuotas = new HashSet<>();
// Partition the tables with quotas by table and ns quota
partitionTablesByQuotaTarget(quotas, tablesWithQuotas, namespaceTablesWithQuotas);
helper.partitionTablesByQuotaTarget(quotas, tablesWithQuotas, namespaceTablesWithQuotas);
TableName rpcQuotaTable = createTable();
TableName rpcQuotaTable = helper.createTable();
TEST_UTIL.getAdmin().setQuota(QuotaSettingsFactory
.throttleTable(rpcQuotaTable, ThrottleType.READ_NUMBER, 6, TimeUnit.MINUTES));
@ -375,7 +378,7 @@ public class TestQuotaObserverChoreWithMiniCluster {
TablesWithQuotas tables = new TablesWithQuotas(TEST_UTIL.getConnection(),
TEST_UTIL.getConfiguration()) {
@Override
int getNumReportedRegions(TableName table, QuotaViolationStore<TableName> tableStore) {
int getNumReportedRegions(TableName table, QuotaSnapshotStore<TableName> tableStore) {
Integer i = mockReportedRegions.get(table);
if (null == i) {
return 0;
@ -385,9 +388,9 @@ public class TestQuotaObserverChoreWithMiniCluster {
};
// Create the tables
TableName tn1 = createTableWithRegions(20);
TableName tn2 = createTableWithRegions(20);
TableName tn3 = createTableWithRegions(20);
TableName tn1 = helper.createTableWithRegions(20);
TableName tn2 = helper.createTableWithRegions(20);
TableName tn3 = helper.createTableWithRegions(20);
// Add them to the Tables with Quotas object
tables.addTableQuotaTable(tn1);
@ -407,9 +410,9 @@ public class TestQuotaObserverChoreWithMiniCluster {
@Test
public void testFetchSpaceQuota() throws Exception {
Multimap<TableName,QuotaSettings> tables = createTablesWithSpaceQuotas();
Multimap<TableName,QuotaSettings> tables = helper.createTablesWithSpaceQuotas();
// Can pass in an empty map, we're not consulting it.
chore.initializeViolationStores(Collections.emptyMap());
chore.initializeSnapshotStores(Collections.emptyMap());
// All tables that were created should have a quota defined.
for (Entry<TableName,QuotaSettings> entry : tables.entries()) {
final TableName table = entry.getKey();
@ -420,10 +423,10 @@ public class TestQuotaObserverChoreWithMiniCluster {
SpaceQuota spaceQuota = null;
if (null != qs.getTableName()) {
spaceQuota = chore.getTableViolationStore().getSpaceQuota(table);
spaceQuota = chore.getTableSnapshotStore().getSpaceQuota(table);
assertNotNull("Could not find table space quota for " + table, spaceQuota);
} else if (null != qs.getNamespace()) {
spaceQuota = chore.getNamespaceViolationStore().getSpaceQuota(table.getNamespaceAsString());
spaceQuota = chore.getNamespaceSnapshotStore().getSpaceQuota(table.getNamespaceAsString());
assertNotNull("Could not find namespace space quota for " + table.getNamespaceAsString(), spaceQuota);
} else {
fail("Expected table or namespace space quota");
@ -433,166 +436,16 @@ public class TestQuotaObserverChoreWithMiniCluster {
assertEquals(sls.getProto().getQuota(), spaceQuota);
}
TableName tableWithoutQuota = createTable();
assertNull(chore.getTableViolationStore().getSpaceQuota(tableWithoutQuota));
TableName tableWithoutQuota = helper.createTable();
assertNull(chore.getTableSnapshotStore().getSpaceQuota(tableWithoutQuota));
}
//
// Helpers
//
void writeData(TableName tn, long sizeInBytes) throws IOException {
final Connection conn = TEST_UTIL.getConnection();
final Table table = conn.getTable(tn);
private void sleepWithInterrupt(long millis) {
try {
List<Put> updates = new ArrayList<>();
long bytesToWrite = sizeInBytes;
long rowKeyId = 0L;
final StringBuilder sb = new StringBuilder();
final Random r = new Random();
while (bytesToWrite > 0L) {
sb.setLength(0);
sb.append(Long.toString(rowKeyId));
// Use the reverse counter as the rowKey to get even spread across all regions
Put p = new Put(Bytes.toBytes(sb.reverse().toString()));
byte[] value = new byte[SIZE_PER_VALUE];
r.nextBytes(value);
p.addColumn(Bytes.toBytes(F1), Bytes.toBytes("q1"), value);
updates.add(p);
// Batch 50K worth of updates
if (updates.size() > 50) {
table.put(updates);
updates.clear();
}
// Just count the value size, ignore the size of rowkey + column
bytesToWrite -= SIZE_PER_VALUE;
rowKeyId++;
}
// Write the final batch
if (!updates.isEmpty()) {
table.put(updates);
}
LOG.debug("Data was written to HBase");
// Push the data to disk.
TEST_UTIL.getAdmin().flush(tn);
LOG.debug("Data flushed to disk");
} finally {
table.close();
}
}
Multimap<TableName, QuotaSettings> createTablesWithSpaceQuotas() throws Exception {
final Admin admin = TEST_UTIL.getAdmin();
final Multimap<TableName, QuotaSettings> tablesWithQuotas = HashMultimap.create();
final TableName tn1 = createTable();
final TableName tn2 = createTable();
NamespaceDescriptor nd = NamespaceDescriptor.create("ns" + COUNTER.getAndIncrement()).build();
admin.createNamespace(nd);
final TableName tn3 = createTableInNamespace(nd);
final TableName tn4 = createTableInNamespace(nd);
final TableName tn5 = createTableInNamespace(nd);
final long sizeLimit1 = 1024L * 1024L * 1024L * 1024L * 5L; // 5TB
final SpaceViolationPolicy violationPolicy1 = SpaceViolationPolicy.NO_WRITES;
QuotaSettings qs1 = QuotaSettingsFactory.limitTableSpace(tn1, sizeLimit1, violationPolicy1);
tablesWithQuotas.put(tn1, qs1);
admin.setQuota(qs1);
final long sizeLimit2 = 1024L * 1024L * 1024L * 200L; // 200GB
final SpaceViolationPolicy violationPolicy2 = SpaceViolationPolicy.NO_WRITES_COMPACTIONS;
QuotaSettings qs2 = QuotaSettingsFactory.limitTableSpace(tn2, sizeLimit2, violationPolicy2);
tablesWithQuotas.put(tn2, qs2);
admin.setQuota(qs2);
final long sizeLimit3 = 1024L * 1024L * 1024L * 1024L * 100L; // 100TB
final SpaceViolationPolicy violationPolicy3 = SpaceViolationPolicy.NO_INSERTS;
QuotaSettings qs3 = QuotaSettingsFactory.limitNamespaceSpace(nd.getName(), sizeLimit3, violationPolicy3);
tablesWithQuotas.put(tn3, qs3);
tablesWithQuotas.put(tn4, qs3);
tablesWithQuotas.put(tn5, qs3);
admin.setQuota(qs3);
final long sizeLimit4 = 1024L * 1024L * 1024L * 5L; // 5GB
final SpaceViolationPolicy violationPolicy4 = SpaceViolationPolicy.NO_INSERTS;
QuotaSettings qs4 = QuotaSettingsFactory.limitTableSpace(tn5, sizeLimit4, violationPolicy4);
// Override the ns quota for tn5, import edge-case to catch table quota taking
// precedence over ns quota.
tablesWithQuotas.put(tn5, qs4);
admin.setQuota(qs4);
return tablesWithQuotas;
}
TableName createTable() throws Exception {
return createTableWithRegions(1);
}
TableName createTableWithRegions(int numRegions) throws Exception {
return createTableWithRegions(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR, numRegions);
}
TableName createTableWithRegions(String namespace, int numRegions) throws Exception {
final Admin admin = TEST_UTIL.getAdmin();
final TableName tn = TableName.valueOf(namespace, testName.getMethodName() + COUNTER.getAndIncrement());
// Delete the old table
if (admin.tableExists(tn)) {
admin.disableTable(tn);
admin.deleteTable(tn);
}
// Create the table
HTableDescriptor tableDesc = new HTableDescriptor(tn);
tableDesc.addFamily(new HColumnDescriptor(F1));
if (numRegions == 1) {
admin.createTable(tableDesc);
} else {
admin.createTable(tableDesc, Bytes.toBytes("0"), Bytes.toBytes("9"), numRegions);
}
return tn;
}
TableName createTableInNamespace(NamespaceDescriptor nd) throws Exception {
final Admin admin = TEST_UTIL.getAdmin();
final TableName tn = TableName.valueOf(nd.getName(),
testName.getMethodName() + COUNTER.getAndIncrement());
// Delete the old table
if (admin.tableExists(tn)) {
admin.disableTable(tn);
admin.deleteTable(tn);
}
// Create the table
HTableDescriptor tableDesc = new HTableDescriptor(tn);
tableDesc.addFamily(new HColumnDescriptor(F1));
admin.createTable(tableDesc);
return tn;
}
void partitionTablesByQuotaTarget(Multimap<TableName,QuotaSettings> quotas,
Set<TableName> tablesWithTableQuota, Set<TableName> tablesWithNamespaceQuota) {
// Partition the tables with quotas by table and ns quota
for (Entry<TableName, QuotaSettings> entry : quotas.entries()) {
SpaceLimitSettings settings = (SpaceLimitSettings) entry.getValue();
TableName tn = entry.getKey();
if (null != settings.getTableName()) {
tablesWithTableQuota.add(tn);
}
if (null != settings.getNamespace()) {
tablesWithNamespaceQuota.add(tn);
}
if (null == settings.getTableName() && null == settings.getNamespace()) {
fail("Unexpected table name with null tableName and namespace: " + tn);
}
Thread.sleep(millis);
} catch (InterruptedException e) {
LOG.debug("Interrupted while sleeping");
Thread.currentThread().interrupt();
}
}
}

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle;
@ -201,26 +202,29 @@ public class TestQuotaTableUtil {
@Test
public void testSerDeViolationPolicies() throws Exception {
final TableName tn1 = getUniqueTableName();
final SpaceViolationPolicy policy1 = SpaceViolationPolicy.DISABLE;
final SpaceQuotaSnapshot snapshot1 = new SpaceQuotaSnapshot(
new SpaceQuotaStatus(SpaceViolationPolicy.DISABLE), 512L, 1024L);
final TableName tn2 = getUniqueTableName();
final SpaceViolationPolicy policy2 = SpaceViolationPolicy.NO_INSERTS;
final SpaceQuotaSnapshot snapshot2 = new SpaceQuotaSnapshot(
new SpaceQuotaStatus(SpaceViolationPolicy.NO_INSERTS), 512L, 1024L);
final TableName tn3 = getUniqueTableName();
final SpaceViolationPolicy policy3 = SpaceViolationPolicy.NO_WRITES;
final SpaceQuotaSnapshot snapshot3 = new SpaceQuotaSnapshot(
new SpaceQuotaStatus(SpaceViolationPolicy.NO_WRITES), 512L, 1024L);
List<Put> puts = new ArrayList<>();
puts.add(QuotaTableUtil.createEnableViolationPolicyUpdate(tn1, policy1));
puts.add(QuotaTableUtil.createEnableViolationPolicyUpdate(tn2, policy2));
puts.add(QuotaTableUtil.createEnableViolationPolicyUpdate(tn3, policy3));
final Map<TableName,SpaceViolationPolicy> expectedPolicies = new HashMap<>();
expectedPolicies.put(tn1, policy1);
expectedPolicies.put(tn2, policy2);
expectedPolicies.put(tn3, policy3);
puts.add(QuotaTableUtil.createPutSpaceSnapshot(tn1, snapshot1));
puts.add(QuotaTableUtil.createPutSpaceSnapshot(tn2, snapshot2));
puts.add(QuotaTableUtil.createPutSpaceSnapshot(tn3, snapshot3));
final Map<TableName,SpaceQuotaSnapshot> expectedPolicies = new HashMap<>();
expectedPolicies.put(tn1, snapshot1);
expectedPolicies.put(tn2, snapshot2);
expectedPolicies.put(tn3, snapshot3);
final Map<TableName,SpaceViolationPolicy> actualPolicies = new HashMap<>();
final Map<TableName,SpaceQuotaSnapshot> actualPolicies = new HashMap<>();
try (Table quotaTable = connection.getTable(QuotaUtil.QUOTA_TABLE_NAME)) {
quotaTable.put(puts);
ResultScanner scanner = quotaTable.getScanner(QuotaTableUtil.makeQuotaViolationScan());
ResultScanner scanner = quotaTable.getScanner(QuotaTableUtil.makeQuotaSnapshotScan());
for (Result r : scanner) {
QuotaTableUtil.extractViolationPolicy(r, actualPolicies);
QuotaTableUtil.extractQuotaSnapshot(r, actualPolicies);
}
scanner.close();
}
@ -231,4 +235,4 @@ public class TestQuotaTableUtil {
private TableName getUniqueTableName() {
return TableName.valueOf(testName.getMethodName() + "_" + tableNameCounter++);
}
}
}

View File

@ -16,33 +16,29 @@
*/
package org.apache.hadoop.hbase.quotas;
import static org.apache.hadoop.hbase.util.Bytes.toBytes;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.doAnswer;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.HashMap;
import java.util.Map;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
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.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
import org.apache.hadoop.hbase.quotas.policies.DisableTableViolationPolicyEnforcement;
import org.apache.hadoop.hbase.quotas.policies.NoInsertsViolationPolicyEnforcement;
import org.apache.hadoop.hbase.quotas.policies.NoWritesCompactionsViolationPolicyEnforcement;
import org.apache.hadoop.hbase.quotas.policies.NoWritesViolationPolicyEnforcement;
import org.apache.hadoop.hbase.quotas.policies.BulkLoadVerifyingViolationPolicyEnforcement;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
/**
* Test class for {@link RegionServerSpaceQuotaManager}.
@ -51,77 +47,105 @@ import org.mockito.stubbing.Answer;
public class TestRegionServerSpaceQuotaManager {
private RegionServerSpaceQuotaManager quotaManager;
private Connection conn;
private Table quotaTable;
private ResultScanner scanner;
private RegionServerServices rss;
@Before
@SuppressWarnings("unchecked")
public void setup() throws Exception {
quotaManager = mock(RegionServerSpaceQuotaManager.class);
conn = mock(Connection.class);
quotaTable = mock(Table.class);
scanner = mock(ResultScanner.class);
// Call the real getViolationPoliciesToEnforce()
when(quotaManager.getViolationPoliciesToEnforce()).thenCallRealMethod();
// Mock out creating a scanner
when(quotaManager.getConnection()).thenReturn(conn);
when(conn.getTable(QuotaUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
when(quotaTable.getScanner(any(Scan.class))).thenReturn(scanner);
// Mock out the static method call with some indirection
doAnswer(new Answer<Void>(){
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
Result result = invocation.getArgumentAt(0, Result.class);
Map<TableName,SpaceViolationPolicy> policies = invocation.getArgumentAt(1, Map.class);
QuotaTableUtil.extractViolationPolicy(result, policies);
return null;
}
}).when(quotaManager).extractViolationPolicy(any(Result.class), any(Map.class));
rss = mock(RegionServerServices.class);
}
@Test
public void testMissingAllColumns() {
List<Result> results = new ArrayList<>();
results.add(Result.create(Collections.emptyList()));
when(scanner.iterator()).thenReturn(results.iterator());
try {
quotaManager.getViolationPoliciesToEnforce();
fail("Expected an IOException, but did not receive one.");
} catch (IOException e) {
// Expected an error because we had no cells in the row.
// This should only happen due to programmer error.
}
public void testSpacePoliciesFromEnforcements() {
final Map<TableName, SpaceViolationPolicyEnforcement> enforcements = new HashMap<>();
final Map<TableName, SpaceQuotaSnapshot> expectedPolicies = new HashMap<>();
when(quotaManager.copyActiveEnforcements()).thenReturn(enforcements);
when(quotaManager.getActivePoliciesAsMap()).thenCallRealMethod();
NoInsertsViolationPolicyEnforcement noInsertsPolicy = new NoInsertsViolationPolicyEnforcement();
SpaceQuotaSnapshot noInsertsSnapshot = new SpaceQuotaSnapshot(
new SpaceQuotaStatus(SpaceViolationPolicy.NO_INSERTS), 256L, 1024L);
noInsertsPolicy.initialize(rss, TableName.valueOf("no_inserts"), noInsertsSnapshot);
enforcements.put(noInsertsPolicy.getTableName(), noInsertsPolicy);
expectedPolicies.put(noInsertsPolicy.getTableName(), noInsertsSnapshot);
NoWritesViolationPolicyEnforcement noWritesPolicy = new NoWritesViolationPolicyEnforcement();
SpaceQuotaSnapshot noWritesSnapshot = new SpaceQuotaSnapshot(
new SpaceQuotaStatus(SpaceViolationPolicy.NO_WRITES), 512L, 2048L);
noWritesPolicy.initialize(rss, TableName.valueOf("no_writes"), noWritesSnapshot);
enforcements.put(noWritesPolicy.getTableName(), noWritesPolicy);
expectedPolicies.put(noWritesPolicy.getTableName(), noWritesSnapshot);
NoWritesCompactionsViolationPolicyEnforcement noWritesCompactionsPolicy =
new NoWritesCompactionsViolationPolicyEnforcement();
SpaceQuotaSnapshot noWritesCompactionsSnapshot = new SpaceQuotaSnapshot(
new SpaceQuotaStatus(SpaceViolationPolicy.NO_WRITES_COMPACTIONS), 1024L, 4096L);
noWritesCompactionsPolicy.initialize(
rss, TableName.valueOf("no_writes_compactions"), noWritesCompactionsSnapshot);
enforcements.put(noWritesCompactionsPolicy.getTableName(), noWritesCompactionsPolicy);
expectedPolicies.put(noWritesCompactionsPolicy.getTableName(),
noWritesCompactionsSnapshot);
DisableTableViolationPolicyEnforcement disablePolicy =
new DisableTableViolationPolicyEnforcement();
SpaceQuotaSnapshot disableSnapshot = new SpaceQuotaSnapshot(
new SpaceQuotaStatus(SpaceViolationPolicy.DISABLE), 2048L, 8192L);
disablePolicy.initialize(rss, TableName.valueOf("disable"), disableSnapshot);
enforcements.put(disablePolicy.getTableName(), disablePolicy);
expectedPolicies.put(disablePolicy.getTableName(), disableSnapshot);
enforcements.put(
TableName.valueOf("no_policy"), new BulkLoadVerifyingViolationPolicyEnforcement());
Map<TableName, SpaceQuotaSnapshot> actualPolicies = quotaManager.getActivePoliciesAsMap();
assertEquals(expectedPolicies, actualPolicies);
}
@Test
public void testMissingDesiredColumn() {
List<Result> results = new ArrayList<>();
// Give a column that isn't the one we want
Cell c = new KeyValue(toBytes("t:inviolation"), toBytes("q"), toBytes("s"), new byte[0]);
results.add(Result.create(Collections.singletonList(c)));
when(scanner.iterator()).thenReturn(results.iterator());
try {
quotaManager.getViolationPoliciesToEnforce();
fail("Expected an IOException, but did not receive one.");
} catch (IOException e) {
// Expected an error because we were missing the column we expected in this row.
// This should only happen due to programmer error.
}
public void testExceptionOnPolicyEnforcementEnable() throws Exception {
final TableName tableName = TableName.valueOf("foo");
final SpaceQuotaSnapshot snapshot = new SpaceQuotaSnapshot(
new SpaceQuotaStatus(SpaceViolationPolicy.DISABLE), 1024L, 2048L);
RegionServerServices rss = mock(RegionServerServices.class);
SpaceViolationPolicyEnforcementFactory factory = mock(
SpaceViolationPolicyEnforcementFactory.class);
SpaceViolationPolicyEnforcement enforcement = mock(SpaceViolationPolicyEnforcement.class);
RegionServerSpaceQuotaManager realManager = new RegionServerSpaceQuotaManager(rss, factory);
when(factory.create(rss, tableName, snapshot)).thenReturn(enforcement);
doThrow(new IOException("Failed for test!")).when(enforcement).enable();
realManager.enforceViolationPolicy(tableName, snapshot);
Map<TableName, SpaceViolationPolicyEnforcement> enforcements =
realManager.copyActiveEnforcements();
assertTrue("Expected active enforcements to be empty, but were " + enforcements,
enforcements.isEmpty());
}
@Test
public void testParsingError() {
List<Result> results = new ArrayList<>();
Cell c = new KeyValue(toBytes("t:inviolation"), toBytes("u"), toBytes("v"), new byte[0]);
results.add(Result.create(Collections.singletonList(c)));
when(scanner.iterator()).thenReturn(results.iterator());
try {
quotaManager.getViolationPoliciesToEnforce();
fail("Expected an IOException, but did not receive one.");
} catch (IOException e) {
// We provided a garbage serialized protobuf message (empty byte array), this should
// in turn throw an IOException
}
public void testExceptionOnPolicyEnforcementDisable() throws Exception {
final TableName tableName = TableName.valueOf("foo");
final SpaceQuotaSnapshot snapshot = new SpaceQuotaSnapshot(
new SpaceQuotaStatus(SpaceViolationPolicy.DISABLE), 1024L, 2048L);
RegionServerServices rss = mock(RegionServerServices.class);
SpaceViolationPolicyEnforcementFactory factory = mock(
SpaceViolationPolicyEnforcementFactory.class);
SpaceViolationPolicyEnforcement enforcement = mock(SpaceViolationPolicyEnforcement.class);
RegionServerSpaceQuotaManager realManager = new RegionServerSpaceQuotaManager(rss, factory);
when(factory.create(rss, tableName, snapshot)).thenReturn(enforcement);
doNothing().when(enforcement).enable();
doThrow(new IOException("Failed for test!")).when(enforcement).disable();
// Enabling should work
realManager.enforceViolationPolicy(tableName, snapshot);
Map<TableName, SpaceViolationPolicyEnforcement> enforcements =
realManager.copyActiveEnforcements();
assertEquals(1, enforcements.size());
// If the disable fails, we should still treat it as "active"
realManager.disableViolationPolicyEnforcement(tableName);
enforcements = realManager.copyActiveEnforcements();
assertEquals(1, enforcements.size());
}
}

View File

@ -16,20 +16,34 @@
*/
package org.apache.hadoop.hbase.quotas;
import static org.apache.hadoop.hbase.util.Bytes.toBytes;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.doCallRealMethod;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
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.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.Before;
@ -37,42 +51,62 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test class for {@link SpaceQuotaViolationPolicyRefresherChore}.
* Test class for {@link SpaceQuotaRefresherChore}.
*/
@Category(SmallTests.class)
public class TestSpaceQuotaViolationPolicyRefresherChore {
private RegionServerSpaceQuotaManager manager;
private RegionServerServices rss;
private SpaceQuotaViolationPolicyRefresherChore chore;
private SpaceQuotaRefresherChore chore;
private Configuration conf;
private Connection conn;
@SuppressWarnings("unchecked")
@Before
public void setup() {
public void setup() throws IOException {
conf = HBaseConfiguration.create();
rss = mock(RegionServerServices.class);
manager = mock(RegionServerSpaceQuotaManager.class);
conn = mock(Connection.class);
when(manager.getRegionServerServices()).thenReturn(rss);
when(rss.getConfiguration()).thenReturn(conf);
chore = new SpaceQuotaViolationPolicyRefresherChore(manager);
chore = mock(SpaceQuotaRefresherChore.class);
when(chore.getConnection()).thenReturn(conn);
when(chore.getManager()).thenReturn(manager);
doCallRealMethod().when(chore).chore();
when(chore.isInViolation(any(SpaceQuotaSnapshot.class))).thenCallRealMethod();
doCallRealMethod().when(chore).extractQuotaSnapshot(any(Result.class), any(Map.class));
}
@Test
public void testPoliciesAreEnforced() throws IOException {
final Map<TableName,SpaceViolationPolicy> policiesToEnforce = new HashMap<>();
policiesToEnforce.put(TableName.valueOf("table1"), SpaceViolationPolicy.DISABLE);
policiesToEnforce.put(TableName.valueOf("table2"), SpaceViolationPolicy.NO_INSERTS);
policiesToEnforce.put(TableName.valueOf("table3"), SpaceViolationPolicy.NO_WRITES);
policiesToEnforce.put(TableName.valueOf("table4"), SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
// Create a number of policies that should be enforced (usage > limit)
final Map<TableName,SpaceQuotaSnapshot> policiesToEnforce = new HashMap<>();
policiesToEnforce.put(
TableName.valueOf("table1"),
new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.DISABLE), 1024L, 512L));
policiesToEnforce.put(
TableName.valueOf("table2"),
new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.NO_INSERTS), 2048L, 512L));
policiesToEnforce.put(
TableName.valueOf("table3"),
new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.NO_WRITES), 4096L, 512L));
policiesToEnforce.put(
TableName.valueOf("table4"),
new SpaceQuotaSnapshot(
new SpaceQuotaStatus(SpaceViolationPolicy.NO_WRITES_COMPACTIONS), 8192L, 512L));
// No active enforcements
when(manager.getActiveViolationPolicyEnforcements()).thenReturn(Collections.emptyMap());
when(manager.copyQuotaSnapshots()).thenReturn(Collections.emptyMap());
// Policies to enforce
when(manager.getViolationPoliciesToEnforce()).thenReturn(policiesToEnforce);
when(chore.fetchSnapshotsFromQuotaTable()).thenReturn(policiesToEnforce);
chore.chore();
for (Entry<TableName,SpaceViolationPolicy> entry : policiesToEnforce.entrySet()) {
for (Entry<TableName,SpaceQuotaSnapshot> entry : policiesToEnforce.entrySet()) {
// Ensure we enforce the policy
verify(manager).enforceViolationPolicy(entry.getKey(), entry.getValue());
// Don't disable any policies
@ -82,50 +116,135 @@ public class TestSpaceQuotaViolationPolicyRefresherChore {
@Test
public void testOldPoliciesAreRemoved() throws IOException {
final Map<TableName,SpaceViolationPolicy> policiesToEnforce = new HashMap<>();
policiesToEnforce.put(TableName.valueOf("table1"), SpaceViolationPolicy.DISABLE);
policiesToEnforce.put(TableName.valueOf("table2"), SpaceViolationPolicy.NO_INSERTS);
final Map<TableName,SpaceQuotaSnapshot> previousPolicies = new HashMap<>();
previousPolicies.put(
TableName.valueOf("table3"),
new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.NO_WRITES), 4096L, 512L));
previousPolicies.put(
TableName.valueOf("table4"),
new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.NO_WRITES), 8192L, 512L));
final Map<TableName,SpaceViolationPolicy> previousPolicies = new HashMap<>();
previousPolicies.put(TableName.valueOf("table3"), SpaceViolationPolicy.NO_WRITES);
previousPolicies.put(TableName.valueOf("table4"), SpaceViolationPolicy.NO_WRITES);
final Map<TableName,SpaceQuotaSnapshot> policiesToEnforce = new HashMap<>();
policiesToEnforce.put(
TableName.valueOf("table1"),
new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.DISABLE), 1024L, 512L));
policiesToEnforce.put(
TableName.valueOf("table2"),
new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.NO_INSERTS), 2048L, 512L));
policiesToEnforce.put(
TableName.valueOf("table3"),
new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 256L, 512L));
policiesToEnforce.put(
TableName.valueOf("table4"),
new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 128L, 512L));
// No active enforcements
when(manager.getActiveViolationPolicyEnforcements()).thenReturn(previousPolicies);
when(manager.copyQuotaSnapshots()).thenReturn(previousPolicies);
// Policies to enforce
when(manager.getViolationPoliciesToEnforce()).thenReturn(policiesToEnforce);
when(chore.fetchSnapshotsFromQuotaTable()).thenReturn(policiesToEnforce);
chore.chore();
for (Entry<TableName,SpaceViolationPolicy> entry : policiesToEnforce.entrySet()) {
verify(manager).enforceViolationPolicy(entry.getKey(), entry.getValue());
}
verify(manager).enforceViolationPolicy(
TableName.valueOf("table1"), policiesToEnforce.get(TableName.valueOf("table1")));
verify(manager).enforceViolationPolicy(
TableName.valueOf("table2"), policiesToEnforce.get(TableName.valueOf("table2")));
for (Entry<TableName,SpaceViolationPolicy> entry : previousPolicies.entrySet()) {
verify(manager).disableViolationPolicyEnforcement(entry.getKey());
}
verify(manager).disableViolationPolicyEnforcement(TableName.valueOf("table3"));
verify(manager).disableViolationPolicyEnforcement(TableName.valueOf("table4"));
}
@Test
public void testNewPolicyOverridesOld() throws IOException {
final Map<TableName,SpaceViolationPolicy> policiesToEnforce = new HashMap<>();
policiesToEnforce.put(TableName.valueOf("table1"), SpaceViolationPolicy.DISABLE);
policiesToEnforce.put(TableName.valueOf("table2"), SpaceViolationPolicy.NO_WRITES);
policiesToEnforce.put(TableName.valueOf("table3"), SpaceViolationPolicy.NO_INSERTS);
final Map<TableName,SpaceQuotaSnapshot> policiesToEnforce = new HashMap<>();
policiesToEnforce.put(
TableName.valueOf("table1"),
new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.DISABLE), 1024L, 512L));
policiesToEnforce.put(
TableName.valueOf("table2"),
new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.NO_WRITES), 2048L, 512L));
policiesToEnforce.put(
TableName.valueOf("table3"),
new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.NO_INSERTS), 4096L, 512L));
final Map<TableName,SpaceViolationPolicy> previousPolicies = new HashMap<>();
previousPolicies.put(TableName.valueOf("table1"), SpaceViolationPolicy.NO_WRITES);
final Map<TableName,SpaceQuotaSnapshot> previousPolicies = new HashMap<>();
previousPolicies.put(
TableName.valueOf("table1"),
new SpaceQuotaSnapshot(new SpaceQuotaStatus(SpaceViolationPolicy.NO_WRITES), 8192L, 512L));
// No active enforcements
when(manager.getActiveViolationPolicyEnforcements()).thenReturn(previousPolicies);
when(manager.getActivePoliciesAsMap()).thenReturn(previousPolicies);
// Policies to enforce
when(manager.getViolationPoliciesToEnforce()).thenReturn(policiesToEnforce);
when(chore.fetchSnapshotsFromQuotaTable()).thenReturn(policiesToEnforce);
chore.chore();
for (Entry<TableName,SpaceViolationPolicy> entry : policiesToEnforce.entrySet()) {
for (Entry<TableName,SpaceQuotaSnapshot> entry : policiesToEnforce.entrySet()) {
verify(manager).enforceViolationPolicy(entry.getKey(), entry.getValue());
}
verify(manager, never()).disableViolationPolicyEnforcement(TableName.valueOf("table1"));
}
@Test
public void testMissingAllColumns() throws IOException {
when(chore.fetchSnapshotsFromQuotaTable()).thenCallRealMethod();
ResultScanner scanner = mock(ResultScanner.class);
Table quotaTable = mock(Table.class);
when(conn.getTable(QuotaUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
when(quotaTable.getScanner(any(Scan.class))).thenReturn(scanner);
List<Result> results = new ArrayList<>();
results.add(Result.create(Collections.emptyList()));
when(scanner.iterator()).thenReturn(results.iterator());
try {
chore.fetchSnapshotsFromQuotaTable();
fail("Expected an IOException, but did not receive one.");
} catch (IOException e) {
// Expected an error because we had no cells in the row.
// This should only happen due to programmer error.
}
}
@Test
public void testMissingDesiredColumn() throws IOException {
when(chore.fetchSnapshotsFromQuotaTable()).thenCallRealMethod();
ResultScanner scanner = mock(ResultScanner.class);
Table quotaTable = mock(Table.class);
when(conn.getTable(QuotaUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
when(quotaTable.getScanner(any(Scan.class))).thenReturn(scanner);
List<Result> results = new ArrayList<>();
// Give a column that isn't the one we want
Cell c = new KeyValue(toBytes("t:inviolation"), toBytes("q"), toBytes("s"), new byte[0]);
results.add(Result.create(Collections.singletonList(c)));
when(scanner.iterator()).thenReturn(results.iterator());
try {
chore.fetchSnapshotsFromQuotaTable();
fail("Expected an IOException, but did not receive one.");
} catch (IOException e) {
// Expected an error because we were missing the column we expected in this row.
// This should only happen due to programmer error.
}
}
@Test
public void testParsingError() throws IOException {
when(chore.fetchSnapshotsFromQuotaTable()).thenCallRealMethod();
ResultScanner scanner = mock(ResultScanner.class);
Table quotaTable = mock(Table.class);
when(conn.getTable(QuotaUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
when(quotaTable.getScanner(any(Scan.class))).thenReturn(scanner);
List<Result> results = new ArrayList<>();
Cell c = new KeyValue(toBytes("t:inviolation"), toBytes("u"), toBytes("v"), new byte[0]);
results.add(Result.create(Collections.singletonList(c)));
when(scanner.iterator()).thenReturn(results.iterator());
try {
chore.fetchSnapshotsFromQuotaTable();
fail("Expected an IOException, but did not receive one.");
} catch (IOException e) {
// We provided a garbage serialized protobuf message (empty byte array), this should
// in turn throw an IOException
}
}
}

View File

@ -0,0 +1,452 @@
/*
* 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.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.ClientServiceCallable;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Increment;
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.ResultScanner;
import org.apache.hadoop.hbase.client.RpcRetryingCaller;
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.SecureBulkLoadClient;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.quotas.policies.BulkLoadVerifyingViolationPolicyEnforcement;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.util.StringUtils;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
/**
* End-to-end test class for filesystem space quotas.
*/
@Category(LargeTests.class)
public class TestSpaceQuotas {
private static final Log LOG = LogFactory.getLog(TestSpaceQuotas.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static final AtomicLong COUNTER = new AtomicLong(0);
private static final int NUM_RETRIES = 10;
@Rule
public TestName testName = new TestName();
private SpaceQuotaHelperForTests helper;
@BeforeClass
public static void setUp() throws Exception {
Configuration conf = TEST_UTIL.getConfiguration();
// Increase the frequency of some of the chores for responsiveness of the test
conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_DELAY_KEY, 1000);
conf.setInt(FileSystemUtilizationChore.FS_UTILIZATION_CHORE_PERIOD_KEY, 1000);
conf.setInt(QuotaObserverChore.QUOTA_OBSERVER_CHORE_DELAY_KEY, 1000);
conf.setInt(QuotaObserverChore.QUOTA_OBSERVER_CHORE_PERIOD_KEY, 1000);
conf.setInt(SpaceQuotaRefresherChore.POLICY_REFRESHER_CHORE_DELAY_KEY, 1000);
conf.setInt(SpaceQuotaRefresherChore.POLICY_REFRESHER_CHORE_PERIOD_KEY, 1000);
conf.setBoolean(QuotaUtil.QUOTA_CONF_KEY, true);
TEST_UTIL.startMiniCluster(1);
}
@AfterClass
public static void tearDown() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
@Before
public void removeAllQuotas() throws Exception {
final Connection conn = TEST_UTIL.getConnection();
// Wait for the quota table to be created
if (!conn.getAdmin().tableExists(QuotaUtil.QUOTA_TABLE_NAME)) {
do {
LOG.debug("Quota table does not yet exist");
Thread.sleep(1000);
} while (!conn.getAdmin().tableExists(QuotaUtil.QUOTA_TABLE_NAME));
} else {
// Or, clean up any quotas from previous test runs.
QuotaRetriever scanner = QuotaRetriever.open(TEST_UTIL.getConfiguration());
for (QuotaSettings quotaSettings : scanner) {
final String namespace = quotaSettings.getNamespace();
final TableName tableName = quotaSettings.getTableName();
if (null != namespace) {
LOG.debug("Deleting quota for namespace: " + namespace);
QuotaUtil.deleteNamespaceQuota(conn, namespace);
} else {
assert null != tableName;
LOG.debug("Deleting quota for table: "+ tableName);
QuotaUtil.deleteTableQuota(conn, tableName);
}
}
}
helper = new SpaceQuotaHelperForTests(TEST_UTIL, testName, COUNTER);
}
@Test
public void testNoInsertsWithPut() throws Exception {
Put p = new Put(Bytes.toBytes("to_reject"));
p.addColumn(
Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_INSERTS, p);
}
@Test
public void testNoInsertsWithAppend() throws Exception {
Append a = new Append(Bytes.toBytes("to_reject"));
a.add(
Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_INSERTS, a);
}
@Test
public void testNoInsertsWithIncrement() throws Exception {
Increment i = new Increment(Bytes.toBytes("to_reject"));
i.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("count"), 0);
writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_INSERTS, i);
}
@Test
public void testDeletesAfterNoInserts() throws Exception {
final TableName tn = writeUntilViolation(SpaceViolationPolicy.NO_INSERTS);
// Try a couple of times to verify that the quota never gets enforced, same as we
// do when we're trying to catch the failure.
Delete d = new Delete(Bytes.toBytes("should_not_be_rejected"));
for (int i = 0; i < NUM_RETRIES; i++) {
try (Table t = TEST_UTIL.getConnection().getTable(tn)) {
t.delete(d);
}
}
}
@Test
public void testNoWritesWithPut() throws Exception {
Put p = new Put(Bytes.toBytes("to_reject"));
p.addColumn(
Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, p);
}
@Test
public void testNoWritesWithAppend() throws Exception {
Append a = new Append(Bytes.toBytes("to_reject"));
a.add(
Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, a);
}
@Test
public void testNoWritesWithIncrement() throws Exception {
Increment i = new Increment(Bytes.toBytes("to_reject"));
i.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("count"), 0);
writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, i);
}
@Test
public void testNoWritesWithDelete() throws Exception {
Delete d = new Delete(Bytes.toBytes("to_reject"));
writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, d);
}
@Test
public void testNoCompactions() throws Exception {
Put p = new Put(Bytes.toBytes("to_reject"));
p.addColumn(
Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
final TableName tn = writeUntilViolationAndVerifyViolation(
SpaceViolationPolicy.NO_WRITES_COMPACTIONS, p);
// We know the policy is active at this point
// Major compactions should be rejected
try {
TEST_UTIL.getAdmin().majorCompact(tn);
fail("Expected that invoking the compaction should throw an Exception");
} catch (DoNotRetryIOException e) {
// Expected!
}
// Minor compactions should also be rejected.
try {
TEST_UTIL.getAdmin().compact(tn);
fail("Expected that invoking the compaction should throw an Exception");
} catch (DoNotRetryIOException e) {
// Expected!
}
}
@Test
public void testNoEnableAfterDisablePolicy() throws Exception {
Put p = new Put(Bytes.toBytes("to_reject"));
p.addColumn(
Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
final TableName tn = writeUntilViolation(SpaceViolationPolicy.DISABLE);
final Admin admin = TEST_UTIL.getAdmin();
// Disabling a table relies on some external action (over the other policies), so wait a bit
// more than the other tests.
for (int i = 0; i < NUM_RETRIES * 2; i++) {
if (admin.isTableEnabled(tn)) {
LOG.info(tn + " is still enabled, expecting it to be disabled. Will wait and re-check.");
Thread.sleep(2000);
}
}
assertFalse(tn + " is still enabled but it should be disabled", admin.isTableEnabled(tn));
try {
admin.enableTable(tn);
} catch (AccessDeniedException e) {
String exceptionContents = StringUtils.stringifyException(e);
final String expectedText = "violated space quota";
assertTrue("Expected the exception to contain " + expectedText + ", but was: "
+ exceptionContents, exceptionContents.contains(expectedText));
}
}
@Test(timeout=120000)
public void testNoBulkLoadsWithNoWrites() throws Exception {
Put p = new Put(Bytes.toBytes("to_reject"));
p.addColumn(
Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"), Bytes.toBytes("reject"));
TableName tableName = writeUntilViolationAndVerifyViolation(SpaceViolationPolicy.NO_WRITES, p);
// The table is now in violation. Try to do a bulk load
ClientServiceCallable<Void> callable = generateFileToLoad(tableName, 1, 50);
RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(TEST_UTIL.getConfiguration());
RpcRetryingCaller<Void> caller = factory.<Void> newCaller();
try {
caller.callWithRetries(callable, Integer.MAX_VALUE);
fail("Expected the bulk load call to fail!");
} catch (SpaceLimitingException e) {
// Pass
LOG.trace("Caught expected exception", e);
}
}
@Test(timeout=120000)
public void testAtomicBulkLoadUnderQuota() throws Exception {
// Need to verify that if the batch of hfiles cannot be loaded, none are loaded.
TableName tn = helper.createTableWithRegions(10);
final long sizeLimit = 50L * SpaceQuotaHelperForTests.ONE_KILOBYTE;
QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(
tn, sizeLimit, SpaceViolationPolicy.NO_INSERTS);
TEST_UTIL.getAdmin().setQuota(settings);
HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
RegionServerSpaceQuotaManager spaceQuotaManager = rs.getRegionServerSpaceQuotaManager();
Map<TableName,SpaceQuotaSnapshot> snapshots = spaceQuotaManager.copyQuotaSnapshots();
Map<HRegionInfo,Long> regionSizes = getReportedSizesForTable(tn);
while (true) {
SpaceQuotaSnapshot snapshot = snapshots.get(tn);
if (null != snapshot && snapshot.getLimit() > 0) {
break;
}
LOG.debug(
"Snapshot does not yet realize quota limit: " + snapshots + ", regionsizes: " +
regionSizes);
Thread.sleep(3000);
snapshots = spaceQuotaManager.copyQuotaSnapshots();
regionSizes = getReportedSizesForTable(tn);
}
// Our quota limit should be reflected in the latest snapshot
SpaceQuotaSnapshot snapshot = snapshots.get(tn);
assertEquals(0L, snapshot.getUsage());
assertEquals(sizeLimit, snapshot.getLimit());
// We would also not have a "real" policy in violation
ActivePolicyEnforcement activePolicies = spaceQuotaManager.getActiveEnforcements();
SpaceViolationPolicyEnforcement enforcement = activePolicies.getPolicyEnforcement(tn);
assertTrue(
"Expected to find Noop policy, but got " + enforcement.getClass().getSimpleName(),
enforcement instanceof BulkLoadVerifyingViolationPolicyEnforcement);
// Should generate two files, each of which is over 25KB each
ClientServiceCallable<Void> callable = generateFileToLoad(tn, 2, 500);
FileSystem fs = TEST_UTIL.getTestFileSystem();
FileStatus[] files = fs.listStatus(
new Path(fs.getHomeDirectory(), testName.getMethodName() + "_files"));
for (FileStatus file : files) {
assertTrue(
"Expected the file, " + file.getPath() + ", length to be larger than 25KB, but was "
+ file.getLen(),
file.getLen() > 25 * SpaceQuotaHelperForTests.ONE_KILOBYTE);
LOG.debug(file.getPath() + " -> " + file.getLen() +"B");
}
RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(TEST_UTIL.getConfiguration());
RpcRetryingCaller<Void> caller = factory.<Void> newCaller();
try {
caller.callWithRetries(callable, Integer.MAX_VALUE);
fail("Expected the bulk load call to fail!");
} catch (SpaceLimitingException e) {
// Pass
LOG.trace("Caught expected exception", e);
}
// Verify that we have no data in the table because neither file should have been
// loaded even though one of the files could have.
Table table = TEST_UTIL.getConnection().getTable(tn);
ResultScanner scanner = table.getScanner(new Scan());
try {
assertNull("Expected no results", scanner.next());
} finally{
scanner.close();
}
}
private Map<HRegionInfo,Long> getReportedSizesForTable(TableName tn) {
HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
MasterQuotaManager quotaManager = master.getMasterQuotaManager();
Map<HRegionInfo,Long> filteredRegionSizes = new HashMap<>();
for (Entry<HRegionInfo,Long> entry : quotaManager.snapshotRegionSizes().entrySet()) {
if (entry.getKey().getTable().equals(tn)) {
filteredRegionSizes.put(entry.getKey(), entry.getValue());
}
}
return filteredRegionSizes;
}
private TableName writeUntilViolation(SpaceViolationPolicy policyToViolate) throws Exception {
TableName tn = helper.createTableWithRegions(10);
final long sizeLimit = 2L * SpaceQuotaHelperForTests.ONE_MEGABYTE;
QuotaSettings settings = QuotaSettingsFactory.limitTableSpace(tn, sizeLimit, policyToViolate);
TEST_UTIL.getAdmin().setQuota(settings);
// Write more data than should be allowed and flush it to disk
helper.writeData(tn, 3L * SpaceQuotaHelperForTests.ONE_MEGABYTE);
// This should be sufficient time for the chores to run and see the change.
Thread.sleep(5000);
return tn;
}
private TableName writeUntilViolationAndVerifyViolation(
SpaceViolationPolicy policyToViolate, Mutation m) throws Exception {
final TableName tn = writeUntilViolation(policyToViolate);
// But let's try a few times to get the exception before failing
boolean sawError = false;
for (int i = 0; i < NUM_RETRIES && !sawError; i++) {
try (Table table = TEST_UTIL.getConnection().getTable(tn)) {
if (m instanceof Put) {
table.put((Put) m);
} else if (m instanceof Delete) {
table.delete((Delete) m);
} else if (m instanceof Append) {
table.append((Append) m);
} else if (m instanceof Increment) {
table.increment((Increment) m);
} else {
fail(
"Failed to apply " + m.getClass().getSimpleName() +
" to the table. Programming error");
}
LOG.info("Did not reject the " + m.getClass().getSimpleName() + ", will sleep and retry");
Thread.sleep(2000);
} catch (Exception e) {
String msg = StringUtils.stringifyException(e);
assertTrue("Expected exception message to contain the word '" + policyToViolate.name() +
"', but was " + msg, msg.contains(policyToViolate.name()));
sawError = true;
}
}
if (!sawError) {
try (Table quotaTable = TEST_UTIL.getConnection().getTable(QuotaUtil.QUOTA_TABLE_NAME)) {
ResultScanner scanner = quotaTable.getScanner(new Scan());
Result result = null;
LOG.info("Dumping contents of hbase:quota table");
while ((result = scanner.next()) != null) {
LOG.info(Bytes.toString(result.getRow()) + " => " + result.toString());
}
scanner.close();
}
}
assertTrue(
"Expected to see an exception writing data to a table exceeding its quota", sawError);
return tn;
}
private ClientServiceCallable<Void> generateFileToLoad(
TableName tn, int numFiles, int numRowsPerFile) throws Exception {
Connection conn = TEST_UTIL.getConnection();
FileSystem fs = TEST_UTIL.getTestFileSystem();
Configuration conf = TEST_UTIL.getConfiguration();
Path baseDir = new Path(fs.getHomeDirectory(), testName.getMethodName() + "_files");
fs.mkdirs(baseDir);
final List<Pair<byte[], String>> famPaths = new ArrayList<Pair<byte[], String>>();
for (int i = 1; i <= numFiles; i++) {
Path hfile = new Path(baseDir, "file" + i);
TestHRegionServerBulkLoad.createHFile(
fs, hfile, Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),
Bytes.toBytes("reject"), numRowsPerFile);
famPaths.add(new Pair<>(Bytes.toBytes(SpaceQuotaHelperForTests.F1), hfile.toString()));
}
// bulk load HFiles
Table table = conn.getTable(tn);
final String bulkToken = new SecureBulkLoadClient(conf, table).prepareBulkLoad(conn);
return new ClientServiceCallable<Void>(conn,
tn, Bytes.toBytes("row"), new RpcControllerFactory(conf).newController()) {
@Override
public Void rpcCall() throws Exception {
SecureBulkLoadClient secureClient = null;
byte[] regionName = getLocation().getRegionInfo().getRegionName();
try (Table table = conn.getTable(getTableName())) {
secureClient = new SecureBulkLoadClient(conf, table);
secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
true, null, bulkToken);
}
return null;
}
};
}
}

View File

@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicReference;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.quotas.QuotaViolationStore.ViolationState;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
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;
@ -44,7 +44,7 @@ import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
/**
* Test class for {@link TableQuotaViolationStore}.
* Test class for {@link TableQuotaSnapshotStore}.
*/
@Category(SmallTests.class)
public class TestTableQuotaViolationStore {
@ -53,14 +53,14 @@ public class TestTableQuotaViolationStore {
private Connection conn;
private QuotaObserverChore chore;
private Map<HRegionInfo, Long> regionReports;
private TableQuotaViolationStore store;
private TableQuotaSnapshotStore store;
@Before
public void setup() {
conn = mock(Connection.class);
chore = mock(QuotaObserverChore.class);
regionReports = new HashMap<>();
store = new TableQuotaViolationStore(conn, chore, regionReports);
store = new TableQuotaSnapshotStore(conn, chore, regionReports);
}
@Test
@ -108,23 +108,29 @@ public class TestTableQuotaViolationStore {
regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(0), Bytes.toBytes(1)), 1024L * 512L);
regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(1), Bytes.toBytes(2)), 1024L * 256L);
SpaceQuotaSnapshot tn1Snapshot = new SpaceQuotaSnapshot(
SpaceQuotaStatus.notInViolation(), 1024L * 768L, 1024L * 1024L);
// Below the quota
assertEquals(ViolationState.IN_OBSERVANCE, store.getTargetState(tn1, quota));
assertEquals(tn1Snapshot, store.getTargetState(tn1, quota));
regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(2), Bytes.toBytes(3)), 1024L * 256L);
tn1Snapshot = new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 1024L * 1024L, 1024L * 1024L);
// Equal to the quota is still in observance
assertEquals(ViolationState.IN_OBSERVANCE, store.getTargetState(tn1, quota));
assertEquals(tn1Snapshot, store.getTargetState(tn1, quota));
regionReports.put(new HRegionInfo(tn1, Bytes.toBytes(3), Bytes.toBytes(4)), 1024L);
tn1Snapshot = new SpaceQuotaSnapshot(
new SpaceQuotaStatus(SpaceViolationPolicy.DISABLE), 1024L * 1024L + 1024L, 1024L * 1024L);
// Exceeds the quota, should be in violation
assertEquals(ViolationState.IN_VIOLATION, store.getTargetState(tn1, quota));
assertEquals(tn1Snapshot, store.getTargetState(tn1, quota));
}
@Test
public void testGetSpaceQuota() throws Exception {
TableQuotaViolationStore mockStore = mock(TableQuotaViolationStore.class);
TableQuotaSnapshotStore mockStore = mock(TableQuotaSnapshotStore.class);
when(mockStore.getSpaceQuota(any(TableName.class))).thenCallRealMethod();
Quotas quotaWithSpace = Quotas.newBuilder().setSpace(

View File

@ -30,12 +30,11 @@ import java.util.Objects;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Before;
@ -44,17 +43,17 @@ import org.junit.experimental.categories.Category;
import org.mockito.ArgumentMatcher;
/**
* Test case for {@link TableSpaceQuotaViolationNotifier}.
* Test case for {@link TableSpaceQuotaSnapshotNotifier}.
*/
@Category(SmallTests.class)
public class TestTableSpaceQuotaViolationNotifier {
private TableSpaceQuotaViolationNotifier notifier;
private TableSpaceQuotaSnapshotNotifier notifier;
private Connection conn;
@Before
public void setup() throws Exception {
notifier = new TableSpaceQuotaViolationNotifier();
notifier = new TableSpaceQuotaSnapshotNotifier();
conn = mock(Connection.class);
notifier.initialize(conn);
}
@ -62,35 +61,25 @@ public class TestTableSpaceQuotaViolationNotifier {
@Test
public void testToViolation() throws Exception {
final TableName tn = TableName.valueOf("inviolation");
final SpaceViolationPolicy policy = SpaceViolationPolicy.NO_INSERTS;
final SpaceQuotaSnapshot snapshot = new SpaceQuotaSnapshot(
new SpaceQuotaStatus(SpaceViolationPolicy.NO_INSERTS), 1024L, 512L);
final Table quotaTable = mock(Table.class);
when(conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
final Put expectedPut = new Put(Bytes.toBytes("t." + tn.getNameAsString()));
final SpaceQuota protoQuota = SpaceQuota.newBuilder()
.setViolationPolicy(ProtobufUtil.toProtoViolationPolicy(policy))
final QuotaProtos.SpaceQuotaSnapshot protoQuota = QuotaProtos.SpaceQuotaSnapshot.newBuilder()
.setStatus(QuotaProtos.SpaceQuotaStatus.newBuilder().setInViolation(true).setPolicy(
org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy.NO_INSERTS))
.setLimit(512L)
.setUsage(1024L)
.build();
expectedPut.addColumn(Bytes.toBytes("u"), Bytes.toBytes("v"), protoQuota.toByteArray());
expectedPut.addColumn(Bytes.toBytes("u"), Bytes.toBytes("p"), protoQuota.toByteArray());
notifier.transitionTableToViolation(tn, policy);
notifier.transitionTable(tn, snapshot);
verify(quotaTable).put(argThat(new SingleCellPutMatcher(expectedPut)));
}
@Test
public void testToObservance() throws Exception {
final TableName tn = TableName.valueOf("notinviolation");
final Table quotaTable = mock(Table.class);
when(conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)).thenReturn(quotaTable);
final Delete expectedDelete = new Delete(Bytes.toBytes("t." + tn.getNameAsString()));
expectedDelete.addColumn(Bytes.toBytes("u"), Bytes.toBytes("v"));
notifier.transitionTableToObservance(tn);
verify(quotaTable).delete(argThat(new SingleCellDeleteMatcher(expectedDelete)));
}
/**
* Parameterized for Puts.
*/
@ -100,15 +89,6 @@ public class TestTableSpaceQuotaViolationNotifier {
}
}
/**
* Parameterized for Deletes.
*/
private static class SingleCellDeleteMatcher extends SingleCellMutationMatcher<Delete> {
private SingleCellDeleteMatcher(Delete expected) {
super(expected);
}
}
/**
* Quick hack to verify a Mutation with one column.
*/

View File

@ -96,7 +96,7 @@ public class TestTablesWithQuotas {
final Map<TableName,Integer> reportedRegions = new HashMap<>();
final Map<TableName,Integer> actualRegions = new HashMap<>();
final Configuration conf = HBaseConfiguration.create();
conf.setDouble(QuotaObserverChore.VIOLATION_OBSERVER_CHORE_REPORT_PERCENT_KEY, 0.95);
conf.setDouble(QuotaObserverChore.QUOTA_OBSERVER_CHORE_REPORT_PERCENT_KEY, 0.95);
TableName tooFewRegionsTable = TableName.valueOf("tn1");
TableName sufficientRegionsTable = TableName.valueOf("tn2");
@ -114,7 +114,7 @@ public class TestTablesWithQuotas {
}
@Override
int getNumReportedRegions(TableName table, QuotaViolationStore<TableName> tableStore) {
int getNumReportedRegions(TableName table, QuotaSnapshotStore<TableName> tableStore) {
return reportedRegions.get(table);
}
};
@ -177,13 +177,13 @@ public class TestTablesWithQuotas {
QuotaObserverChore chore = mock(QuotaObserverChore.class);
Map<HRegionInfo,Long> regionUsage = new HashMap<>();
TableQuotaViolationStore store = new TableQuotaViolationStore(conn, chore, regionUsage);
TableQuotaSnapshotStore store = new TableQuotaSnapshotStore(conn, chore, regionUsage);
// A super dirty hack to verify that, after getting no regions for our table,
// we bail out and start processing the next element (which there is none).
final TablesWithQuotas tables = new TablesWithQuotas(conn, conf) {
@Override
int getNumReportedRegions(TableName table, QuotaViolationStore<TableName> tableStore) {
int getNumReportedRegions(TableName table, QuotaSnapshotStore<TableName> tableStore) {
throw new RuntimeException("Should should not reach here");
}
};

View File

@ -0,0 +1,31 @@
/*
* 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.policies;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.util.Bytes;
public class BaseViolationPolicyEnforcement {
static final Append APPEND = new Append(Bytes.toBytes("foo"));
static final Delete DELETE = new Delete(Bytes.toBytes("foo"));
static final Increment INCREMENT = new Increment(Bytes.toBytes("foo"));
static final Put PUT = new Put(Bytes.toBytes("foo"));
}

View File

@ -0,0 +1,142 @@
/*
* 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.policies;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(SmallTests.class)
public class TestBulkLoadCheckingViolationPolicyEnforcement {
FileSystem fs;
RegionServerServices rss;
TableName tableName;
SpaceViolationPolicyEnforcement policy;
@Before
public void setup() {
fs = mock(FileSystem.class);
rss = mock(RegionServerServices.class);
tableName = TableName.valueOf("foo");
policy = new BulkLoadVerifyingViolationPolicyEnforcement();
}
@Test
public void testFilesUnderLimit() throws Exception {
final List<String> paths = new ArrayList<>();
final List<FileStatus> statuses = new ArrayList<>();
final long length = 100L * 1024L;
for (int i = 0; i < 5; i++) {
String path = "/" + i;
FileStatus status = mock(FileStatus.class);
when(fs.getFileStatus(new Path(path))).thenReturn(status);
when(status.getLen()).thenReturn(length);
when(status.isFile()).thenReturn(true);
paths.add(path);
statuses.add(status);
}
// Quota is not in violation now
SpaceQuotaSnapshot snapshot = new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 0, length * 6);
policy.initialize(rss, tableName, snapshot);
policy.checkBulkLoad(fs, paths);
}
@Test(expected = IllegalArgumentException.class)
public void testFileIsNotAFile() throws Exception {
final List<String> paths = new ArrayList<>();
String path = "/1";
FileStatus status = mock(FileStatus.class);
when(fs.getFileStatus(new Path(path))).thenReturn(status);
when(status.getLen()).thenReturn(1000L);
when(status.isFile()).thenReturn(false);
paths.add(path);
// Quota is not in violation now
SpaceQuotaSnapshot snapshot = new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 0, Long.MAX_VALUE);
policy.initialize(rss, tableName, snapshot);
// If the file to bulk load isn't a file, this should throw an exception
policy.checkBulkLoad(fs, paths);
}
@Test(expected = SpaceLimitingException.class)
public void testOneFileInBatchOverLimit() throws Exception {
final List<String> paths = new ArrayList<>();
final List<FileStatus> statuses = new ArrayList<>();
final long length = 1000L * 1024L;
for (int i = 0; i < 5; i++) {
String path = "/" + i;
FileStatus status = mock(FileStatus.class);
when(fs.getFileStatus(new Path(path))).thenReturn(status);
when(status.getLen()).thenReturn(length);
when(status.isFile()).thenReturn(true);
paths.add(path);
statuses.add(status);
}
// Quota is not in violation now
SpaceQuotaSnapshot snapshot = new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 0, 1024L);
policy.initialize(rss, tableName, snapshot);
policy.checkBulkLoad(fs, paths);
}
@Test(expected = SpaceLimitingException.class)
public void testSumOfFilesOverLimit() throws Exception {
final List<String> paths = new ArrayList<>();
final List<FileStatus> statuses = new ArrayList<>();
final long length = 1024L;
for (int i = 0; i < 5; i++) {
String path = "/" + i;
FileStatus status = mock(FileStatus.class);
when(fs.getFileStatus(new Path(path))).thenReturn(status);
when(status.getLen()).thenReturn(length);
when(status.isFile()).thenReturn(true);
paths.add(path);
statuses.add(status);
}
// Quota is not in violation now, but 5*1024 files would push us to violation
SpaceQuotaSnapshot snapshot = new SpaceQuotaSnapshot(SpaceQuotaStatus.notInViolation(), 0, 5000L);
policy.initialize(rss, tableName, snapshot);
policy.checkBulkLoad(fs, paths);
}
}

View File

@ -0,0 +1,59 @@
/*
* 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.policies;
import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test class for {@link DisableTableViolationPolicyEnforcement}.
*/
@Category(SmallTests.class)
public class TestDisableTableViolationPolicyEnforcement extends BaseViolationPolicyEnforcement {
private DisableTableViolationPolicyEnforcement enforcement;
@Before
public void setup() {
enforcement = new DisableTableViolationPolicyEnforcement();
}
@Test(expected = SpaceLimitingException.class)
public void testCheckPut() throws SpaceLimitingException {
// If the policy is enacted, it will always throw an exception
// to avoid constantly re-checking the table state.
enforcement.check(PUT);
}
@Test(expected = SpaceLimitingException.class)
public void testCheckAppend() throws SpaceLimitingException {
enforcement.check(APPEND);
}
@Test(expected = SpaceLimitingException.class)
public void testCheckDelete() throws SpaceLimitingException {
enforcement.check(DELETE);
}
@Test(expected = SpaceLimitingException.class)
public void testCheckIncrement() throws SpaceLimitingException {
enforcement.check(INCREMENT);
}
}

View File

@ -0,0 +1,57 @@
/*
* 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.policies;
import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test class for {@link NoInsertsViolationPolicyEnforcement}.
*/
@Category(SmallTests.class)
public class TestNoInsertsViolationPolicyEnforcement extends BaseViolationPolicyEnforcement {
private NoInsertsViolationPolicyEnforcement enforcement;
@Before
public void setup() {
enforcement = new NoInsertsViolationPolicyEnforcement();
}
@Test(expected = SpaceLimitingException.class)
public void testCheckAppend() throws Exception {
enforcement.check(APPEND);
}
@Test
public void testCheckDelete() throws Exception {
enforcement.check(DELETE);
}
@Test(expected = SpaceLimitingException.class)
public void testCheckIncrement() throws Exception {
enforcement.check(INCREMENT);
}
@Test(expected = SpaceLimitingException.class)
public void testCheckPut() throws Exception {
enforcement.check(PUT);
}
}

View File

@ -0,0 +1,58 @@
/*
* 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.policies;
import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test class for {@link NoWritesCompactionsViolationPolicyEnforcement};
*/
@Category(SmallTests.class)
public class TestNoWritesCompactionsViolationPolicyEnforcement
extends BaseViolationPolicyEnforcement {
private NoWritesCompactionsViolationPolicyEnforcement enforcement;
@Before
public void setup() {
enforcement = new NoWritesCompactionsViolationPolicyEnforcement();
}
@Test(expected = SpaceLimitingException.class)
public void testCheckAppend() throws Exception {
enforcement.check(APPEND);
}
@Test(expected = SpaceLimitingException.class)
public void testCheckDelete() throws Exception {
enforcement.check(DELETE);
}
@Test(expected = SpaceLimitingException.class)
public void testCheckIncrement() throws Exception {
enforcement.check(INCREMENT);
}
@Test(expected = SpaceLimitingException.class)
public void testCheckPut() throws Exception {
enforcement.check(PUT);
}
}

View File

@ -0,0 +1,57 @@
/*
* 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.policies;
import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test class for {@link NoWritesViolationPolicyEnforcement}.
*/
@Category(SmallTests.class)
public class TestNoWritesViolationPolicyEnforcement extends BaseViolationPolicyEnforcement {
private NoWritesViolationPolicyEnforcement enforcement;
@Before
public void setup() {
enforcement = new NoWritesViolationPolicyEnforcement();
}
@Test(expected = SpaceLimitingException.class)
public void testCheckAppend() throws Exception {
enforcement.check(APPEND);
}
@Test(expected = SpaceLimitingException.class)
public void testCheckDelete() throws Exception {
enforcement.check(DELETE);
}
@Test(expected = SpaceLimitingException.class)
public void testCheckIncrement() throws Exception {
enforcement.check(INCREMENT);
}
@Test(expected = SpaceLimitingException.class)
public void testCheckPut() throws Exception {
enforcement.check(PUT);
}
}