001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.hbase.quotas;
020
021import java.io.IOException;
022import java.util.Collections;
023import java.util.HashMap;
024import java.util.HashSet;
025import java.util.Iterator;
026import java.util.Map;
027import java.util.Map.Entry;
028import java.util.Optional;
029import java.util.Set;
030import java.util.concurrent.ConcurrentHashMap;
031
032import org.apache.commons.lang3.builder.HashCodeBuilder;
033import org.apache.hadoop.conf.Configuration;
034import org.apache.hadoop.fs.FileSystem;
035import org.apache.hadoop.hbase.DoNotRetryIOException;
036import org.apache.hadoop.hbase.MetaTableAccessor;
037import org.apache.hadoop.hbase.NamespaceDescriptor;
038import org.apache.hadoop.hbase.RegionStateListener;
039import org.apache.hadoop.hbase.TableName;
040import org.apache.hadoop.hbase.client.Connection;
041import org.apache.hadoop.hbase.client.RegionInfo;
042import org.apache.hadoop.hbase.master.MasterServices;
043import org.apache.hadoop.hbase.namespace.NamespaceAuditor;
044import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
045import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
046import org.apache.yetus.audience.InterfaceAudience;
047import org.apache.yetus.audience.InterfaceStability;
048import org.slf4j.Logger;
049import org.slf4j.LoggerFactory;
050
051import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
052import org.apache.hbase.thirdparty.com.google.common.collect.HashMultimap;
053import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
054import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
055
056import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
057import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
058import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse;
059import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.FileArchiveNotificationRequest;
060import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.FileArchiveNotificationRequest.FileWithSize;
061
062/**
063 * Master Quota Manager.
064 * It is responsible for initialize the quota table on the first-run and
065 * provide the admin operations to interact with the quota table.
066 *
067 * TODO: FUTURE: The master will be responsible to notify each RS of quota changes
068 * and it will do the "quota aggregation" when the QuotaScope is CLUSTER.
069 */
070@InterfaceAudience.Private
071@InterfaceStability.Evolving
072public class MasterQuotaManager implements RegionStateListener {
073  private static final Logger LOG = LoggerFactory.getLogger(MasterQuotaManager.class);
074  private static final Map<RegionInfo, Long> EMPTY_MAP = Collections.unmodifiableMap(
075      new HashMap<>());
076
077  private final MasterServices masterServices;
078  private NamedLock<String> namespaceLocks;
079  private NamedLock<TableName> tableLocks;
080  private NamedLock<String> userLocks;
081  private boolean initialized = false;
082  private NamespaceAuditor namespaceQuotaManager;
083  private ConcurrentHashMap<RegionInfo, SizeSnapshotWithTimestamp> regionSizes;
084
085  public MasterQuotaManager(final MasterServices masterServices) {
086    this.masterServices = masterServices;
087  }
088
089  public void start() throws IOException {
090    // If the user doesn't want the quota support skip all the initializations.
091    if (!QuotaUtil.isQuotaEnabled(masterServices.getConfiguration())) {
092      LOG.info("Quota support disabled");
093      return;
094    }
095
096    // Create the quota table if missing
097    if (!MetaTableAccessor.tableExists(masterServices.getConnection(),
098          QuotaUtil.QUOTA_TABLE_NAME)) {
099      LOG.info("Quota table not found. Creating...");
100      createQuotaTable();
101    }
102
103    LOG.info("Initializing quota support");
104    namespaceLocks = new NamedLock<>();
105    tableLocks = new NamedLock<>();
106    userLocks = new NamedLock<>();
107    regionSizes = new ConcurrentHashMap<>();
108
109    namespaceQuotaManager = new NamespaceAuditor(masterServices);
110    namespaceQuotaManager.start();
111    initialized = true;
112  }
113
114  public void stop() {
115  }
116
117  public boolean isQuotaInitialized() {
118    return initialized && namespaceQuotaManager.isInitialized();
119  }
120
121  /* ==========================================================================
122   *  Admin operations to manage the quota table
123   */
124  public SetQuotaResponse setQuota(final SetQuotaRequest req)
125      throws IOException, InterruptedException {
126    checkQuotaSupport();
127
128    if (req.hasUserName()) {
129      userLocks.lock(req.getUserName());
130      try {
131        if (req.hasTableName()) {
132          setUserQuota(req.getUserName(), ProtobufUtil.toTableName(req.getTableName()), req);
133        } else if (req.hasNamespace()) {
134          setUserQuota(req.getUserName(), req.getNamespace(), req);
135        } else {
136          setUserQuota(req.getUserName(), req);
137        }
138      } finally {
139        userLocks.unlock(req.getUserName());
140      }
141    } else if (req.hasTableName()) {
142      TableName table = ProtobufUtil.toTableName(req.getTableName());
143      tableLocks.lock(table);
144      try {
145        setTableQuota(table, req);
146      } finally {
147        tableLocks.unlock(table);
148      }
149    } else if (req.hasNamespace()) {
150      namespaceLocks.lock(req.getNamespace());
151      try {
152        setNamespaceQuota(req.getNamespace(), req);
153      } finally {
154        namespaceLocks.unlock(req.getNamespace());
155      }
156    } else {
157      throw new DoNotRetryIOException(
158        new UnsupportedOperationException("a user, a table or a namespace must be specified"));
159    }
160    return SetQuotaResponse.newBuilder().build();
161  }
162
163  public void setUserQuota(final String userName, final SetQuotaRequest req)
164      throws IOException, InterruptedException {
165    setQuota(req, new SetQuotaOperations() {
166      @Override
167      public GlobalQuotaSettingsImpl fetch() throws IOException {
168        return new GlobalQuotaSettingsImpl(req.getUserName(), null, null, QuotaUtil.getUserQuota(
169            masterServices.getConnection(), userName));
170      }
171      @Override
172      public void update(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
173        QuotaUtil.addUserQuota(masterServices.getConnection(), userName, quotaPojo.toQuotas());
174      }
175      @Override
176      public void delete() throws IOException {
177        QuotaUtil.deleteUserQuota(masterServices.getConnection(), userName);
178      }
179      @Override
180      public void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
181        masterServices.getMasterCoprocessorHost().preSetUserQuota(userName, quotaPojo);
182      }
183      @Override
184      public void postApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
185        masterServices.getMasterCoprocessorHost().postSetUserQuota(userName, quotaPojo);
186      }
187    });
188  }
189
190  public void setUserQuota(final String userName, final TableName table,
191      final SetQuotaRequest req) throws IOException, InterruptedException {
192    setQuota(req, new SetQuotaOperations() {
193      @Override
194      public GlobalQuotaSettingsImpl fetch() throws IOException {
195        return new GlobalQuotaSettingsImpl(userName, table, null, QuotaUtil.getUserQuota(
196            masterServices.getConnection(), userName, table));
197      }
198      @Override
199      public void update(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
200        QuotaUtil.addUserQuota(masterServices.getConnection(), userName, table,
201            quotaPojo.toQuotas());
202      }
203      @Override
204      public void delete() throws IOException {
205        QuotaUtil.deleteUserQuota(masterServices.getConnection(), userName, table);
206      }
207      @Override
208      public void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
209        masterServices.getMasterCoprocessorHost().preSetUserQuota(userName, table, quotaPojo);
210      }
211      @Override
212      public void postApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
213        masterServices.getMasterCoprocessorHost().postSetUserQuota(userName, table, quotaPojo);
214      }
215    });
216  }
217
218  public void setUserQuota(final String userName, final String namespace,
219      final SetQuotaRequest req) throws IOException, InterruptedException {
220    setQuota(req, new SetQuotaOperations() {
221      @Override
222      public GlobalQuotaSettingsImpl fetch() throws IOException {
223        return new GlobalQuotaSettingsImpl(userName, null, namespace, QuotaUtil.getUserQuota(
224            masterServices.getConnection(), userName, namespace));
225      }
226      @Override
227      public void update(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
228        QuotaUtil.addUserQuota(masterServices.getConnection(), userName, namespace,
229            quotaPojo.toQuotas());
230      }
231      @Override
232      public void delete() throws IOException {
233        QuotaUtil.deleteUserQuota(masterServices.getConnection(), userName, namespace);
234      }
235      @Override
236      public void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
237        masterServices.getMasterCoprocessorHost().preSetUserQuota(
238            userName, namespace, quotaPojo);
239      }
240      @Override
241      public void postApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
242        masterServices.getMasterCoprocessorHost().postSetUserQuota(
243            userName, namespace, quotaPojo);
244      }
245    });
246  }
247
248  public void setTableQuota(final TableName table, final SetQuotaRequest req)
249      throws IOException, InterruptedException {
250    setQuota(req, new SetQuotaOperations() {
251      @Override
252      public GlobalQuotaSettingsImpl fetch() throws IOException {
253        return new GlobalQuotaSettingsImpl(null, table, null, QuotaUtil.getTableQuota(
254            masterServices.getConnection(), table));
255      }
256      @Override
257      public void update(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
258        QuotaUtil.addTableQuota(masterServices.getConnection(), table, quotaPojo.toQuotas());
259      }
260      @Override
261      public void delete() throws IOException {
262        SpaceQuotaSnapshot currSnapshotOfTable =
263            QuotaTableUtil.getCurrentSnapshotFromQuotaTable(masterServices.getConnection(), table);
264        QuotaUtil.deleteTableQuota(masterServices.getConnection(), table);
265        if (currSnapshotOfTable != null) {
266          SpaceQuotaStatus quotaStatus = currSnapshotOfTable.getQuotaStatus();
267          if (SpaceViolationPolicy.DISABLE == quotaStatus.getPolicy()
268              && quotaStatus.isInViolation()) {
269            QuotaUtil.enableTableIfNotEnabled(masterServices.getConnection(), table);
270          }
271        }
272      }
273      @Override
274      public void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
275        masterServices.getMasterCoprocessorHost().preSetTableQuota(table, quotaPojo);
276      }
277      @Override
278      public void postApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
279        masterServices.getMasterCoprocessorHost().postSetTableQuota(table, quotaPojo);
280      }
281    });
282  }
283
284  public void setNamespaceQuota(final String namespace, final SetQuotaRequest req)
285      throws IOException, InterruptedException {
286    setQuota(req, new SetQuotaOperations() {
287      @Override
288      public GlobalQuotaSettingsImpl fetch() throws IOException {
289        return new GlobalQuotaSettingsImpl(null, null, namespace, QuotaUtil.getNamespaceQuota(
290                masterServices.getConnection(), namespace));
291      }
292      @Override
293      public void update(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
294        QuotaUtil.addNamespaceQuota(masterServices.getConnection(), namespace,
295            ((GlobalQuotaSettingsImpl) quotaPojo).toQuotas());
296      }
297      @Override
298      public void delete() throws IOException {
299        QuotaUtil.deleteNamespaceQuota(masterServices.getConnection(), namespace);
300      }
301      @Override
302      public void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
303        masterServices.getMasterCoprocessorHost().preSetNamespaceQuota(namespace, quotaPojo);
304      }
305      @Override
306      public void postApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
307        masterServices.getMasterCoprocessorHost().postSetNamespaceQuota(namespace, quotaPojo);
308      }
309    });
310  }
311
312  public void setNamespaceQuota(NamespaceDescriptor desc) throws IOException {
313    if (initialized) {
314      this.namespaceQuotaManager.addNamespace(desc);
315    }
316  }
317
318  public void removeNamespaceQuota(String namespace) throws IOException {
319    if (initialized) {
320      this.namespaceQuotaManager.deleteNamespace(namespace);
321    }
322  }
323
324  private void setQuota(final SetQuotaRequest req, final SetQuotaOperations quotaOps)
325      throws IOException, InterruptedException {
326    if (req.hasRemoveAll() && req.getRemoveAll() == true) {
327      quotaOps.preApply(null);
328      quotaOps.delete();
329      quotaOps.postApply(null);
330      return;
331    }
332
333    // Apply quota changes
334    GlobalQuotaSettingsImpl currentQuota = quotaOps.fetch();
335    if (LOG.isTraceEnabled()) {
336      LOG.trace(
337          "Current quota for request(" + TextFormat.shortDebugString(req)
338              + "): " + currentQuota);
339    }
340    // Call the appropriate "pre" CP hook with the current quota value (may be null)
341    quotaOps.preApply(currentQuota);
342    // Translate the protobuf request back into a POJO
343    QuotaSettings newQuota = QuotaSettings.buildFromProto(req);
344    if (LOG.isTraceEnabled()) {
345      LOG.trace("Deserialized quota from request: " + newQuota);
346    }
347
348    // Merge the current quota settings with the new quota settings the user provided.
349    //
350    // NB: while SetQuotaRequest technically allows for multi types of quotas to be set in one
351    // message, the Java API (in Admin/AsyncAdmin) does not. Assume there is only one type.
352    GlobalQuotaSettingsImpl mergedQuota = currentQuota.merge(newQuota);
353    if (LOG.isTraceEnabled()) {
354      LOG.trace("Computed merged quota from current quota and user request: " + mergedQuota);
355    }
356
357    // Submit new changes
358    if (mergedQuota == null) {
359      quotaOps.delete();
360    } else {
361      quotaOps.update(mergedQuota);
362    }
363    // Advertise the final result via the "post" CP hook
364    quotaOps.postApply(mergedQuota);
365  }
366
367  public void checkNamespaceTableAndRegionQuota(TableName tName, int regions) throws IOException {
368    if (initialized) {
369      namespaceQuotaManager.checkQuotaToCreateTable(tName, regions);
370    }
371  }
372
373  public void checkAndUpdateNamespaceRegionQuota(TableName tName, int regions) throws IOException {
374    if (initialized) {
375      namespaceQuotaManager.checkQuotaToUpdateRegion(tName, regions);
376    }
377  }
378
379  /**
380   * @return cached region count, or -1 if quota manager is disabled or table status not found
381  */
382  public int getRegionCountOfTable(TableName tName) throws IOException {
383    if (initialized) {
384      return namespaceQuotaManager.getRegionCountOfTable(tName);
385    }
386    return -1;
387  }
388
389  @Override
390  public void onRegionMerged(RegionInfo mergedRegion) throws IOException {
391    if (initialized) {
392      namespaceQuotaManager.updateQuotaForRegionMerge(mergedRegion);
393    }
394  }
395
396  @Override
397  public void onRegionSplit(RegionInfo hri) throws IOException {
398    if (initialized) {
399      namespaceQuotaManager.checkQuotaToSplitRegion(hri);
400    }
401  }
402
403  /**
404   * Remove table from namespace quota.
405   *
406   * @param tName - The table name to update quota usage.
407   * @throws IOException Signals that an I/O exception has occurred.
408   */
409  public void removeTableFromNamespaceQuota(TableName tName) throws IOException {
410    if (initialized) {
411      namespaceQuotaManager.removeFromNamespaceUsage(tName);
412    }
413  }
414
415  public NamespaceAuditor getNamespaceQuotaManager() {
416    return this.namespaceQuotaManager;
417  }
418
419  /**
420   * Encapsulates CRUD quota operations for some subject.
421   */
422  private static interface SetQuotaOperations {
423    /**
424     * Fetches the current quota settings for the subject.
425     */
426    GlobalQuotaSettingsImpl fetch() throws IOException;
427    /**
428     * Deletes the quota for the subject.
429     */
430    void delete() throws IOException;
431    /**
432     * Persist the given quota for the subject.
433     */
434    void update(GlobalQuotaSettingsImpl quotaPojo) throws IOException;
435    /**
436     * Performs some action before {@link #update(GlobalQuotaSettingsImpl)} with the current
437     * quota for the subject.
438     */
439    void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException;
440    /**
441     * Performs some action after {@link #update(GlobalQuotaSettingsImpl)} with the resulting
442     * quota from the request action for the subject.
443     */
444    void postApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException;
445  }
446
447  /* ==========================================================================
448   *  Helpers
449   */
450
451  private void checkQuotaSupport() throws IOException {
452    if (!QuotaUtil.isQuotaEnabled(masterServices.getConfiguration())) {
453      throw new DoNotRetryIOException(
454        new UnsupportedOperationException("quota support disabled"));
455    }
456    if (!initialized) {
457      long maxWaitTime = masterServices.getConfiguration().getLong(
458        "hbase.master.wait.for.quota.manager.init", 30000); // default is 30 seconds.
459      long startTime = EnvironmentEdgeManager.currentTime();
460      do {
461        try {
462          Thread.sleep(100);
463        } catch (InterruptedException e) {
464          LOG.warn("Interrupted while waiting for Quota Manager to be initialized.");
465          break;
466        }
467      } while (!initialized && (EnvironmentEdgeManager.currentTime() - startTime) < maxWaitTime);
468      if (!initialized) {
469        throw new IOException("Quota manager is uninitialized, please retry later.");
470      }
471    }
472  }
473
474  private void createQuotaTable() throws IOException {
475    masterServices.createSystemTable(QuotaUtil.QUOTA_TABLE_DESC);
476  }
477
478  private static class NamedLock<T> {
479    private final HashSet<T> locks = new HashSet<>();
480
481    public void lock(final T name) throws InterruptedException {
482      synchronized (locks) {
483        while (locks.contains(name)) {
484          locks.wait();
485        }
486        locks.add(name);
487      }
488    }
489
490    public void unlock(final T name) {
491      synchronized (locks) {
492        locks.remove(name);
493        locks.notifyAll();
494      }
495    }
496  }
497
498  @Override
499  public void onRegionSplitReverted(RegionInfo hri) throws IOException {
500    if (initialized) {
501      this.namespaceQuotaManager.removeRegionFromNamespaceUsage(hri);
502    }
503  }
504
505  /**
506   * Holds the size of a region at the given time, millis since the epoch.
507   */
508  private static class SizeSnapshotWithTimestamp {
509    private final long size;
510    private final long time;
511
512    public SizeSnapshotWithTimestamp(long size, long time) {
513      this.size = size;
514      this.time = time;
515    }
516
517    public long getSize() {
518      return size;
519    }
520
521    public long getTime() {
522      return time;
523    }
524
525    @Override
526    public boolean equals(Object o) {
527      if (o instanceof SizeSnapshotWithTimestamp) {
528        SizeSnapshotWithTimestamp other = (SizeSnapshotWithTimestamp) o;
529        return size == other.size && time == other.time;
530      }
531      return false;
532    }
533
534    @Override
535    public int hashCode() {
536      HashCodeBuilder hcb = new HashCodeBuilder();
537      return hcb.append(size).append(time).toHashCode();
538    }
539
540    @Override
541    public String toString() {
542      StringBuilder sb = new StringBuilder(32);
543      sb.append("SizeSnapshotWithTimestamp={size=").append(size).append("B, ");
544      sb.append("time=").append(time).append("}");
545      return sb.toString();
546    }
547  }
548
549  @VisibleForTesting
550  void initializeRegionSizes() {
551    assert regionSizes == null;
552    this.regionSizes = new ConcurrentHashMap<>();
553  }
554
555  public void addRegionSize(RegionInfo hri, long size, long time) {
556    if (regionSizes == null) {
557      return;
558    }
559    regionSizes.put(hri, new SizeSnapshotWithTimestamp(size, time));
560  }
561
562  public Map<RegionInfo, Long> snapshotRegionSizes() {
563    if (regionSizes == null) {
564      return EMPTY_MAP;
565    }
566
567    Map<RegionInfo, Long> copy = new HashMap<>();
568    for (Entry<RegionInfo, SizeSnapshotWithTimestamp> entry : regionSizes.entrySet()) {
569      copy.put(entry.getKey(), entry.getValue().getSize());
570    }
571    return copy;
572  }
573
574  int pruneEntriesOlderThan(long timeToPruneBefore, QuotaObserverChore quotaObserverChore) {
575    if (regionSizes == null) {
576      return 0;
577    }
578    int numEntriesRemoved = 0;
579    Iterator<Entry<RegionInfo, SizeSnapshotWithTimestamp>> iterator =
580        regionSizes.entrySet().iterator();
581    while (iterator.hasNext()) {
582      RegionInfo regionInfo = iterator.next().getKey();
583      long currentEntryTime = regionSizes.get(regionInfo).getTime();
584      // do not prune the entries if table is in violation and
585      // violation policy is disable to avoid cycle of enable/disable.
586      // Please refer HBASE-22012 for more details.
587      // prune entries older than time.
588      if (currentEntryTime < timeToPruneBefore && !isInViolationAndPolicyDisable(
589          regionInfo.getTable(), quotaObserverChore)) {
590        iterator.remove();
591        numEntriesRemoved++;
592      }
593    }
594    return numEntriesRemoved;
595  }
596
597  /**
598   * Method to check if a table is in violation and policy set on table is DISABLE.
599   *
600   * @param tableName          tableName to check.
601   * @param quotaObserverChore QuotaObserverChore instance
602   * @return returns true if table is in violation and policy is disable else false.
603   */
604  private boolean isInViolationAndPolicyDisable(TableName tableName,
605      QuotaObserverChore quotaObserverChore) {
606    boolean isInViolationAtTable = false;
607    boolean isInViolationAtNamespace = false;
608    SpaceViolationPolicy tablePolicy = null;
609    SpaceViolationPolicy namespacePolicy = null;
610    // Get Current Snapshot for the given table
611    SpaceQuotaSnapshot tableQuotaSnapshot = quotaObserverChore.getTableQuotaSnapshot(tableName);
612    SpaceQuotaSnapshot namespaceQuotaSnapshot =
613        quotaObserverChore.getNamespaceQuotaSnapshot(tableName.getNamespaceAsString());
614    if (tableQuotaSnapshot != null) {
615      // check if table in violation
616      isInViolationAtTable = tableQuotaSnapshot.getQuotaStatus().isInViolation();
617      if (isInViolationAtTable) {
618        tablePolicy = tableQuotaSnapshot.getQuotaStatus().getPolicy();
619      }
620    }
621    if (namespaceQuotaSnapshot != null) {
622      // check namespace in violation
623      isInViolationAtNamespace = namespaceQuotaSnapshot.getQuotaStatus().isInViolation();
624      if (isInViolationAtNamespace) {
625        namespacePolicy = namespaceQuotaSnapshot.getQuotaStatus().getPolicy();
626      }
627    }
628    return (tablePolicy == SpaceViolationPolicy.DISABLE && isInViolationAtTable) || (
629        namespacePolicy == SpaceViolationPolicy.DISABLE && isInViolationAtNamespace);
630  }
631
632  /**
633   * Removes each region size entry where the RegionInfo references the provided TableName.
634   *
635   * @param tableName tableName.
636   */
637  public void removeRegionSizesForTable(TableName tableName) {
638    regionSizes.keySet().removeIf(regionInfo -> regionInfo.getTable().equals(tableName));
639  }
640
641  public void processFileArchivals(FileArchiveNotificationRequest request, Connection conn,
642      Configuration conf, FileSystem fs) throws IOException {
643    final HashMultimap<TableName,Entry<String,Long>> archivedFilesByTable = HashMultimap.create();
644    // Group the archived files by table
645    for (FileWithSize fileWithSize : request.getArchivedFilesList()) {
646      TableName tn = ProtobufUtil.toTableName(fileWithSize.getTableName());
647      archivedFilesByTable.put(
648          tn, Maps.immutableEntry(fileWithSize.getName(), fileWithSize.getSize()));
649    }
650    if (LOG.isTraceEnabled()) {
651      LOG.trace("Grouped archived files by table: " + archivedFilesByTable);
652    }
653    // Report each set of files to the appropriate object
654    for (TableName tn : archivedFilesByTable.keySet()) {
655      final Set<Entry<String,Long>> filesWithSize = archivedFilesByTable.get(tn);
656      final FileArchiverNotifier notifier = FileArchiverNotifierFactoryImpl.getInstance().get(
657          conn, conf, fs, tn);
658      notifier.addArchivedFiles(filesWithSize);
659    }
660  }
661}
662