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