All Downloads are FREE. Search and download functionalities are using the official Maven repository.

org.apache.hadoop.hbase.quotas.MasterQuotaManager Maven / Gradle / Ivy

There is a newer version: 3.0.0-beta-1
Show newest version
/**
 * 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.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Map;
import java.util.Map.Entry;
import java.util.concurrent.ConcurrentHashMap;

import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionStateListener;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.namespace.NamespaceAuditor;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse;

/**
 * Master Quota Manager.
 * It is responsible for initialize the quota table on the first-run and
 * provide the admin operations to interact with the quota table.
 *
 * TODO: FUTURE: The master will be responsible to notify each RS of quota changes
 * and it will do the "quota aggregation" when the QuotaScope is CLUSTER.
 */
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class MasterQuotaManager implements RegionStateListener {
  private static final Logger LOG = LoggerFactory.getLogger(MasterQuotaManager.class);
  private static final Map EMPTY_MAP = Collections.unmodifiableMap(
      new HashMap<>());

  private final MasterServices masterServices;
  private NamedLock namespaceLocks;
  private NamedLock tableLocks;
  private NamedLock userLocks;
  private boolean initialized = false;
  private NamespaceAuditor namespaceQuotaManager;
  private ConcurrentHashMap regionSizes;

  public MasterQuotaManager(final MasterServices masterServices) {
    this.masterServices = masterServices;
  }

  public void start() throws IOException {
    // If the user doesn't want the quota support skip all the initializations.
    if (!QuotaUtil.isQuotaEnabled(masterServices.getConfiguration())) {
      LOG.info("Quota support disabled");
      return;
    }

    // Create the quota table if missing
    if (!MetaTableAccessor.tableExists(masterServices.getConnection(),
          QuotaUtil.QUOTA_TABLE_NAME)) {
      LOG.info("Quota table not found. Creating...");
      createQuotaTable();
    }

    LOG.info("Initializing quota support");
    namespaceLocks = new NamedLock<>();
    tableLocks = new NamedLock<>();
    userLocks = new NamedLock<>();
    regionSizes = new ConcurrentHashMap<>();

    namespaceQuotaManager = new NamespaceAuditor(masterServices);
    namespaceQuotaManager.start();
    initialized = true;
  }

  public void stop() {
  }

  public boolean isQuotaInitialized() {
    return initialized && namespaceQuotaManager.isInitialized();
  }

  /* ==========================================================================
   *  Admin operations to manage the quota table
   */
  public SetQuotaResponse setQuota(final SetQuotaRequest req)
      throws IOException, InterruptedException {
    checkQuotaSupport();

    if (req.hasUserName()) {
      userLocks.lock(req.getUserName());
      try {
        if (req.hasTableName()) {
          setUserQuota(req.getUserName(), ProtobufUtil.toTableName(req.getTableName()), req);
        } else if (req.hasNamespace()) {
          setUserQuota(req.getUserName(), req.getNamespace(), req);
        } else {
          setUserQuota(req.getUserName(), req);
        }
      } finally {
        userLocks.unlock(req.getUserName());
      }
    } else if (req.hasTableName()) {
      TableName table = ProtobufUtil.toTableName(req.getTableName());
      tableLocks.lock(table);
      try {
        setTableQuota(table, req);
      } finally {
        tableLocks.unlock(table);
      }
    } else if (req.hasNamespace()) {
      namespaceLocks.lock(req.getNamespace());
      try {
        setNamespaceQuota(req.getNamespace(), req);
      } finally {
        namespaceLocks.unlock(req.getNamespace());
      }
    } else {
      throw new DoNotRetryIOException(
        new UnsupportedOperationException("a user, a table or a namespace must be specified"));
    }
    return SetQuotaResponse.newBuilder().build();
  }

  public void setUserQuota(final String userName, final SetQuotaRequest req)
      throws IOException, InterruptedException {
    setQuota(req, new SetQuotaOperations() {
      @Override
      public GlobalQuotaSettingsImpl fetch() throws IOException {
        return new GlobalQuotaSettingsImpl(req.getUserName(), null, null, QuotaUtil.getUserQuota(
            masterServices.getConnection(), userName));
      }
      @Override
      public void update(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
        QuotaUtil.addUserQuota(masterServices.getConnection(), userName, quotaPojo.toQuotas());
      }
      @Override
      public void delete() throws IOException {
        QuotaUtil.deleteUserQuota(masterServices.getConnection(), userName);
      }
      @Override
      public void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
        masterServices.getMasterCoprocessorHost().preSetUserQuota(userName, quotaPojo);
      }
      @Override
      public void postApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
        masterServices.getMasterCoprocessorHost().postSetUserQuota(userName, quotaPojo);
      }
    });
  }

  public void setUserQuota(final String userName, final TableName table,
      final SetQuotaRequest req) throws IOException, InterruptedException {
    setQuota(req, new SetQuotaOperations() {
      @Override
      public GlobalQuotaSettingsImpl fetch() throws IOException {
        return new GlobalQuotaSettingsImpl(userName, table, null, QuotaUtil.getUserQuota(
            masterServices.getConnection(), userName, table));
      }
      @Override
      public void update(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
        QuotaUtil.addUserQuota(masterServices.getConnection(), userName, table,
            quotaPojo.toQuotas());
      }
      @Override
      public void delete() throws IOException {
        QuotaUtil.deleteUserQuota(masterServices.getConnection(), userName, table);
      }
      @Override
      public void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
        masterServices.getMasterCoprocessorHost().preSetUserQuota(userName, table, quotaPojo);
      }
      @Override
      public void postApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
        masterServices.getMasterCoprocessorHost().postSetUserQuota(userName, table, quotaPojo);
      }
    });
  }

  public void setUserQuota(final String userName, final String namespace,
      final SetQuotaRequest req) throws IOException, InterruptedException {
    setQuota(req, new SetQuotaOperations() {
      @Override
      public GlobalQuotaSettingsImpl fetch() throws IOException {
        return new GlobalQuotaSettingsImpl(userName, null, namespace, QuotaUtil.getUserQuota(
            masterServices.getConnection(), userName, namespace));
      }
      @Override
      public void update(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
        QuotaUtil.addUserQuota(masterServices.getConnection(), userName, namespace,
            quotaPojo.toQuotas());
      }
      @Override
      public void delete() throws IOException {
        QuotaUtil.deleteUserQuota(masterServices.getConnection(), userName, namespace);
      }
      @Override
      public void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
        masterServices.getMasterCoprocessorHost().preSetUserQuota(
            userName, namespace, quotaPojo);
      }
      @Override
      public void postApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
        masterServices.getMasterCoprocessorHost().postSetUserQuota(
            userName, namespace, quotaPojo);
      }
    });
  }

  public void setTableQuota(final TableName table, final SetQuotaRequest req)
      throws IOException, InterruptedException {
    setQuota(req, new SetQuotaOperations() {
      @Override
      public GlobalQuotaSettingsImpl fetch() throws IOException {
        return new GlobalQuotaSettingsImpl(null, table, null, QuotaUtil.getTableQuota(
            masterServices.getConnection(), table));
      }
      @Override
      public void update(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
        QuotaUtil.addTableQuota(masterServices.getConnection(), table, quotaPojo.toQuotas());
      }
      @Override
      public void delete() throws IOException {
        SpaceQuotaSnapshot currSnapshotOfTable =
            QuotaTableUtil.getCurrentSnapshotFromQuotaTable(masterServices.getConnection(), table);
        QuotaUtil.deleteTableQuota(masterServices.getConnection(), table);
        if (currSnapshotOfTable != null) {
          SpaceQuotaStatus quotaStatus = currSnapshotOfTable.getQuotaStatus();
          if (SpaceViolationPolicy.DISABLE == quotaStatus.getPolicy()
              && quotaStatus.isInViolation()) {
            QuotaUtil.enableTableIfNotEnabled(masterServices.getConnection(), table);
          }
        }
      }
      @Override
      public void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
        masterServices.getMasterCoprocessorHost().preSetTableQuota(table, quotaPojo);
      }
      @Override
      public void postApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
        masterServices.getMasterCoprocessorHost().postSetTableQuota(table, quotaPojo);
      }
    });
  }

  public void setNamespaceQuota(final String namespace, final SetQuotaRequest req)
      throws IOException, InterruptedException {
    setQuota(req, new SetQuotaOperations() {
      @Override
      public GlobalQuotaSettingsImpl fetch() throws IOException {
        return new GlobalQuotaSettingsImpl(null, null, namespace, QuotaUtil.getNamespaceQuota(
                masterServices.getConnection(), namespace));
      }
      @Override
      public void update(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
        QuotaUtil.addNamespaceQuota(masterServices.getConnection(), namespace,
            ((GlobalQuotaSettingsImpl) quotaPojo).toQuotas());
      }
      @Override
      public void delete() throws IOException {
        QuotaUtil.deleteNamespaceQuota(masterServices.getConnection(), namespace);
      }
      @Override
      public void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
        masterServices.getMasterCoprocessorHost().preSetNamespaceQuota(namespace, quotaPojo);
      }
      @Override
      public void postApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
        masterServices.getMasterCoprocessorHost().postSetNamespaceQuota(namespace, quotaPojo);
      }
    });
  }

  public void setNamespaceQuota(NamespaceDescriptor desc) throws IOException {
    if (initialized) {
      this.namespaceQuotaManager.addNamespace(desc);
    }
  }

  public void removeNamespaceQuota(String namespace) throws IOException {
    if (initialized) {
      this.namespaceQuotaManager.deleteNamespace(namespace);
    }
  }

  private void setQuota(final SetQuotaRequest req, final SetQuotaOperations quotaOps)
      throws IOException, InterruptedException {
    if (req.hasRemoveAll() && req.getRemoveAll() == true) {
      quotaOps.preApply(null);
      quotaOps.delete();
      quotaOps.postApply(null);
      return;
    }

    // Apply quota changes
    GlobalQuotaSettingsImpl currentQuota = quotaOps.fetch();
    if (LOG.isTraceEnabled()) {
      LOG.trace(
          "Current quota for request(" + TextFormat.shortDebugString(req)
              + "): " + currentQuota);
    }
    // Call the appropriate "pre" CP hook with the current quota value (may be null)
    quotaOps.preApply(currentQuota);
    // Translate the protobuf request back into a POJO
    QuotaSettings newQuota = QuotaSettings.buildFromProto(req);
    if (LOG.isTraceEnabled()) {
      LOG.trace("Deserialized quota from request: " + newQuota);
    }

    // Merge the current quota settings with the new quota settings the user provided.
    //
    // NB: while SetQuotaRequest technically allows for multi types of quotas to be set in one
    // message, the Java API (in Admin/AsyncAdmin) does not. Assume there is only one type.
    GlobalQuotaSettingsImpl mergedQuota = currentQuota.merge(newQuota);
    if (LOG.isTraceEnabled()) {
      LOG.trace("Computed merged quota from current quota and user request: " + mergedQuota);
    }

    // Submit new changes
    if (mergedQuota == null) {
      quotaOps.delete();
    } else {
      quotaOps.update(mergedQuota);
    }
    // Advertise the final result via the "post" CP hook
    quotaOps.postApply(mergedQuota);
  }

  public void checkNamespaceTableAndRegionQuota(TableName tName, int regions) throws IOException {
    if (initialized) {
      namespaceQuotaManager.checkQuotaToCreateTable(tName, regions);
    }
  }

  public void checkAndUpdateNamespaceRegionQuota(TableName tName, int regions) throws IOException {
    if (initialized) {
      namespaceQuotaManager.checkQuotaToUpdateRegion(tName, regions);
    }
  }

  /**
   * @return cached region count, or -1 if quota manager is disabled or table status not found
  */
  public int getRegionCountOfTable(TableName tName) throws IOException {
    if (initialized) {
      return namespaceQuotaManager.getRegionCountOfTable(tName);
    }
    return -1;
  }

  @Override
  public void onRegionMerged(RegionInfo mergedRegion) throws IOException {
    if (initialized) {
      namespaceQuotaManager.updateQuotaForRegionMerge(mergedRegion);
    }
  }

  @Override
  public void onRegionSplit(RegionInfo hri) throws IOException {
    if (initialized) {
      namespaceQuotaManager.checkQuotaToSplitRegion(hri);
    }
  }

  /**
   * Remove table from namespace quota.
   *
   * @param tName - The table name to update quota usage.
   * @throws IOException Signals that an I/O exception has occurred.
   */
  public void removeTableFromNamespaceQuota(TableName tName) throws IOException {
    if (initialized) {
      namespaceQuotaManager.removeFromNamespaceUsage(tName);
    }
  }

  public NamespaceAuditor getNamespaceQuotaManager() {
    return this.namespaceQuotaManager;
  }

  /**
   * Encapsulates CRUD quota operations for some subject.
   */
  private static interface SetQuotaOperations {
    /**
     * Fetches the current quota settings for the subject.
     */
    GlobalQuotaSettingsImpl fetch() throws IOException;
    /**
     * Deletes the quota for the subject.
     */
    void delete() throws IOException;
    /**
     * Persist the given quota for the subject.
     */
    void update(GlobalQuotaSettingsImpl quotaPojo) throws IOException;
    /**
     * Performs some action before {@link #update(GlobalQuotaSettingsImpl)} with the current
     * quota for the subject.
     */
    void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException;
    /**
     * Performs some action after {@link #update(GlobalQuotaSettingsImpl)} with the resulting
     * quota from the request action for the subject.
     */
    void postApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException;
  }

  /* ==========================================================================
   *  Helpers
   */

  private void checkQuotaSupport() throws IOException {
    if (!QuotaUtil.isQuotaEnabled(masterServices.getConfiguration())) {
      throw new DoNotRetryIOException(
        new UnsupportedOperationException("quota support disabled"));
    }
    if (!initialized) {
      long maxWaitTime = masterServices.getConfiguration().getLong(
        "hbase.master.wait.for.quota.manager.init", 30000); // default is 30 seconds.
      long startTime = EnvironmentEdgeManager.currentTime();
      do {
        try {
          Thread.sleep(100);
        } catch (InterruptedException e) {
          LOG.warn("Interrupted while waiting for Quota Manager to be initialized.");
          break;
        }
      } while (!initialized && (EnvironmentEdgeManager.currentTime() - startTime) < maxWaitTime);
      if (!initialized) {
        throw new IOException("Quota manager is uninitialized, please retry later.");
      }
    }
  }

  private void createQuotaTable() throws IOException {
    masterServices.createSystemTable(QuotaUtil.QUOTA_TABLE_DESC);
  }

  private static class NamedLock {
    private final HashSet locks = new HashSet<>();

    public void lock(final T name) throws InterruptedException {
      synchronized (locks) {
        while (locks.contains(name)) {
          locks.wait();
        }
        locks.add(name);
      }
    }

    public void unlock(final T name) {
      synchronized (locks) {
        locks.remove(name);
        locks.notifyAll();
      }
    }
  }

  @Override
  public void onRegionSplitReverted(RegionInfo hri) throws IOException {
    if (initialized) {
      this.namespaceQuotaManager.removeRegionFromNamespaceUsage(hri);
    }
  }

  /**
   * Holds the size of a region at the given time, millis since the epoch.
   */
  private static class SizeSnapshotWithTimestamp {
    private final long size;
    private final long time;

    public SizeSnapshotWithTimestamp(long size, long time) {
      this.size = size;
      this.time = time;
    }

    public long getSize() {
      return size;
    }

    public long getTime() {
      return time;
    }

    @Override
    public boolean equals(Object o) {
      if (o instanceof SizeSnapshotWithTimestamp) {
        SizeSnapshotWithTimestamp other = (SizeSnapshotWithTimestamp) o;
        return size == other.size && time == other.time;
      }
      return false;
    }

    @Override
    public int hashCode() {
      HashCodeBuilder hcb = new HashCodeBuilder();
      return hcb.append(size).append(time).toHashCode();
    }

    @Override
    public String toString() {
      StringBuilder sb = new StringBuilder(32);
      sb.append("SizeSnapshotWithTimestamp={size=").append(size).append("B, ");
      sb.append("time=").append(time).append("}");
      return sb.toString();
    }
  }

  @VisibleForTesting
  void initializeRegionSizes() {
    assert regionSizes == null;
    this.regionSizes = new ConcurrentHashMap<>();
  }

  public void addRegionSize(RegionInfo hri, long size, long time) {
    if (regionSizes == null) {
      return;
    }
    regionSizes.put(hri, new SizeSnapshotWithTimestamp(size, time));
  }

  public Map snapshotRegionSizes() {
    if (regionSizes == null) {
      return EMPTY_MAP;
    }

    Map copy = new HashMap<>();
    for (Entry entry : regionSizes.entrySet()) {
      copy.put(entry.getKey(), entry.getValue().getSize());
    }
    return copy;
  }

  int pruneEntriesOlderThan(long timeToPruneBefore) {
    if (regionSizes == null) {
      return 0;
    }
    int numEntriesRemoved = 0;
    Iterator> iterator =
        regionSizes.entrySet().iterator();
    while (iterator.hasNext()) {
      long currentEntryTime = iterator.next().getValue().getTime();
      if (currentEntryTime < timeToPruneBefore) {
        iterator.remove();
        numEntriesRemoved++;
      }
    }
    return numEntriesRemoved;
  }
}





© 2015 - 2024 Weber Informatics LLC | Privacy Policy