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 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   * @param tn The table to obtain a notifier for
058   * @return The notifier for the given {@code tablename}.
059   */
060  public FileArchiverNotifier get(Connection conn, Configuration conf, FileSystem fs,
061    TableName tn) {
062    // Ensure that only one instance is exposed to callers
063    final FileArchiverNotifier newMapping = new FileArchiverNotifierImpl(conn, conf, fs, tn);
064    final FileArchiverNotifier previousMapping = CACHE.putIfAbsent(tn, newMapping);
065    if (previousMapping == null) {
066      return newMapping;
067    }
068    return previousMapping;
069  }
070
071  public int getCacheSize() {
072    return CACHE.size();
073  }
074
075  static class CacheKey {
076    final Connection conn;
077    final Configuration conf;
078    final FileSystem fs;
079    final TableName tn;
080
081    CacheKey(Connection conn, Configuration conf, FileSystem fs, TableName tn) {
082      this.conn = conn;
083      this.conf = conf;
084      this.fs = fs;
085      this.tn = tn;
086    }
087
088    @Override
089    public boolean equals(Object o) {
090      if (!(o instanceof CacheKey)) {
091        return false;
092      }
093      CacheKey other = (CacheKey) o;
094      // TableName should be the only thing differing..
095      return tn.equals(other.tn) && conn.equals(other.conn) && conf.equals(other.conf)
096        && fs.equals(other.fs);
097    }
098
099    @Override
100    public int hashCode() {
101      return new HashCodeBuilder().append(conn).append(conf).append(fs).append(tn).toHashCode();
102    }
103
104    @Override
105    public String toString() {
106      return "CacheKey[TableName=" + tn + "]";
107    }
108  }
109}