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 */ 018package org.apache.hadoop.hbase.quotas; 019 020import java.util.Objects; 021import java.util.concurrent.ConcurrentHashMap; 022import java.util.concurrent.ConcurrentMap; 023import org.apache.commons.lang3.builder.HashCodeBuilder; 024import org.apache.hadoop.conf.Configuration; 025import org.apache.hadoop.fs.FileSystem; 026import org.apache.hadoop.hbase.TableName; 027import org.apache.hadoop.hbase.client.Connection; 028import org.apache.yetus.audience.InterfaceAudience; 029 030/** 031 * A factory for getting instances of {@link FileArchiverNotifier}. 032 */ 033@InterfaceAudience.Private 034public final class FileArchiverNotifierFactoryImpl implements FileArchiverNotifierFactory { 035 private static final FileArchiverNotifierFactoryImpl DEFAULT_INSTANCE = 036 new FileArchiverNotifierFactoryImpl(); 037 private static volatile FileArchiverNotifierFactory CURRENT_INSTANCE = DEFAULT_INSTANCE; 038 private final ConcurrentMap<TableName, FileArchiverNotifier> CACHE; 039 040 private FileArchiverNotifierFactoryImpl() { 041 CACHE = new ConcurrentHashMap<>(); 042 } 043 044 public static FileArchiverNotifierFactory getInstance() { 045 return CURRENT_INSTANCE; 046 } 047 048 static void setInstance(FileArchiverNotifierFactory inst) { 049 CURRENT_INSTANCE = Objects.requireNonNull(inst); 050 } 051 052 static void reset() { 053 CURRENT_INSTANCE = DEFAULT_INSTANCE; 054 } 055 056 /** 057 * Returns the {@link FileArchiverNotifier} instance for the given {@link TableName}. 058 * @param tn The table to obtain a notifier for 059 * @return The notifier for the given {@code tablename}. 060 */ 061 public FileArchiverNotifier get(Connection conn, Configuration conf, FileSystem fs, 062 TableName tn) { 063 // Ensure that only one instance is exposed to callers 064 return CACHE.computeIfAbsent(tn, key -> new FileArchiverNotifierImpl(conn, conf, fs, key)); 065 } 066 067 public int getCacheSize() { 068 return CACHE.size(); 069 } 070 071 static class CacheKey { 072 final Connection conn; 073 final Configuration conf; 074 final FileSystem fs; 075 final TableName tn; 076 077 CacheKey(Connection conn, Configuration conf, FileSystem fs, TableName tn) { 078 this.conn = conn; 079 this.conf = conf; 080 this.fs = fs; 081 this.tn = tn; 082 } 083 084 @Override 085 public boolean equals(Object o) { 086 if (!(o instanceof CacheKey)) { 087 return false; 088 } 089 CacheKey other = (CacheKey) o; 090 // TableName should be the only thing differing.. 091 return tn.equals(other.tn) && conn.equals(other.conn) && conf.equals(other.conf) 092 && fs.equals(other.fs); 093 } 094 095 @Override 096 public int hashCode() { 097 return new HashCodeBuilder().append(conn).append(conf).append(fs).append(tn).toHashCode(); 098 } 099 100 @Override 101 public String toString() { 102 return "CacheKey[TableName=" + tn + "]"; 103 } 104 } 105}