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