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.policies; 018 019import java.io.IOException; 020import java.util.List; 021import org.apache.hadoop.fs.FileSystem; 022import org.apache.hadoop.hbase.client.Mutation; 023import org.apache.hadoop.hbase.quotas.SpaceLimitingException; 024import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement; 025import org.apache.yetus.audience.InterfaceAudience; 026 027/** 028 * A {@link SpaceViolationPolicyEnforcement} which can be treated as a singleton. When a quota is 029 * not defined on a table or we lack quota information, we want to avoid creating a policy, keeping 030 * this path fast. 031 */ 032@InterfaceAudience.Private 033public final class MissingSnapshotViolationPolicyEnforcement 034 extends AbstractViolationPolicyEnforcement { 035 private static final MissingSnapshotViolationPolicyEnforcement SINGLETON = 036 new MissingSnapshotViolationPolicyEnforcement(); 037 038 private MissingSnapshotViolationPolicyEnforcement() {} 039 040 public static SpaceViolationPolicyEnforcement getInstance() { 041 return SINGLETON; 042 } 043 044 @Override 045 public boolean shouldCheckBulkLoads() { 046 return false; 047 } 048 049 @Override 050 public long computeBulkLoadSize(FileSystem fs, List<String> paths) throws SpaceLimitingException { 051 long size = 0; 052 for (String path : paths) { 053 size += getFileSize(fs, path); 054 } 055 return size; 056 } 057 058 @Override 059 public void enable() throws IOException {} 060 061 @Override 062 public void disable() throws IOException {} 063 064 @Override 065 public void check(Mutation m) throws SpaceLimitingException {} 066 067 @Override 068 public String getPolicyName() { 069 return "NoQuota"; 070 } 071}