001/*
002 * Licensed to the Apache Software Foundation (ASF) under one or more
003 * contributor license agreements.  See the NOTICE file distributed with
004 * this work for additional information regarding copyright ownership.
005 * The ASF licenses this file to you under the Apache License, Version 2.0
006 * (the "License"); you may not use this file except in compliance with
007 * the License.  You may obtain a copy of the License at
008 *
009 * http://www.apache.org/licenses/LICENSE-2.0
010 *
011 * Unless required by applicable law or agreed to in writing, software
012 * distributed under the License is distributed on an "AS IS" BASIS,
013 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
014 * See the License for the specific language governing permissions and
015 * limitations under the License.
016 */
017package org.apache.hadoop.hbase.quotas;
018
019import java.io.IOException;
020import java.util.Map;
021import java.util.Map.Entry;
022import java.util.Objects;
023import java.util.concurrent.locks.ReentrantReadWriteLock;
024import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
025import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
026
027import org.apache.hadoop.hbase.client.Connection;
028import org.apache.hadoop.hbase.client.RegionInfo;
029import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
030import org.apache.yetus.audience.InterfaceAudience;
031
032import org.apache.hbase.thirdparty.com.google.common.base.Predicate;
033import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
034import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
035import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
036import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota;
037
038/**
039 * {@link QuotaSnapshotStore} implementation for namespaces.
040 */
041@InterfaceAudience.Private
042public class NamespaceQuotaSnapshotStore implements QuotaSnapshotStore<String> {
043  private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
044  private final ReadLock rlock = lock.readLock();
045  private final WriteLock wlock = lock.writeLock();
046
047  private final Connection conn;
048  private final QuotaObserverChore chore;
049  private Map<RegionInfo,Long> regionUsage;
050
051  public NamespaceQuotaSnapshotStore(Connection conn, QuotaObserverChore chore, Map<RegionInfo,Long> regionUsage) {
052    this.conn = Objects.requireNonNull(conn);
053    this.chore = Objects.requireNonNull(chore);
054    this.regionUsage = Objects.requireNonNull(regionUsage);
055  }
056
057  @Override
058  public SpaceQuota getSpaceQuota(String namespace) throws IOException {
059    Quotas quotas = getQuotaForNamespace(namespace);
060    if (quotas != null && quotas.hasSpace()) {
061      return quotas.getSpace();
062    }
063    return null;
064  }
065
066  /**
067   * Fetches the namespace quota. Visible for mocking/testing.
068   */
069  Quotas getQuotaForNamespace(String namespace) throws IOException {
070    return QuotaTableUtil.getNamespaceQuota(conn, namespace);
071  }
072
073  @Override
074  public SpaceQuotaSnapshot getCurrentState(String namespace) {
075    // Defer the "current state" to the chore
076    return this.chore.getNamespaceQuotaSnapshot(namespace);
077  }
078
079  @Override
080  public SpaceQuotaSnapshot getTargetState(
081      String subject, SpaceQuota spaceQuota) throws IOException {
082    rlock.lock();
083    try {
084      final long sizeLimitInBytes = spaceQuota.getSoftLimit();
085      long sum = 0L;
086      for (Entry<RegionInfo,Long> entry : filterBySubject(subject)) {
087        sum += entry.getValue();
088      }
089      // Add in the size for any snapshots against this table
090      sum += QuotaTableUtil.getNamespaceSnapshotSize(conn, subject);
091      // Observance is defined as the size of the table being less than the limit
092      SpaceQuotaStatus status = sum <= sizeLimitInBytes ? SpaceQuotaStatus.notInViolation()
093          : new SpaceQuotaStatus(ProtobufUtil.toViolationPolicy(spaceQuota.getViolationPolicy()));
094      return new SpaceQuotaSnapshot(status, sum, sizeLimitInBytes);
095    } finally {
096      rlock.unlock();
097    }
098  }
099
100  @Override
101  public Iterable<Entry<RegionInfo, Long>> filterBySubject(String namespace) {
102    rlock.lock();
103    try {
104      return Iterables.filter(regionUsage.entrySet(), new Predicate<Entry<RegionInfo,Long>>() {
105        @Override
106        public boolean apply(Entry<RegionInfo,Long> input) {
107          return namespace.equals(input.getKey().getTable().getNamespaceAsString());
108        }
109      });
110    } finally {
111      rlock.unlock();
112    }
113  }
114
115  @Override
116  public void setCurrentState(String namespace, SpaceQuotaSnapshot snapshot) {
117    // Defer the "current state" to the chore
118    this.chore.setNamespaceQuotaSnapshot(namespace, snapshot);
119  }
120
121  @Override
122  public void setRegionUsage(Map<RegionInfo,Long> regionUsage) {
123    wlock.lock();
124    try {
125      this.regionUsage = Objects.requireNonNull(regionUsage);
126    } finally {
127      wlock.unlock();
128    }
129  }
130}