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.metrics.impl; 019 020import java.util.Collection; 021import java.util.Set; 022import java.util.concurrent.ConcurrentHashMap; 023import java.util.function.Supplier; 024import java.util.stream.Collectors; 025import org.apache.yetus.audience.InterfaceAudience; 026 027/** 028 * A map of K to V, but does ref counting for added and removed values. The values are not added 029 * directly, but instead requested from the given Supplier if ref count == 0. Each put() call will 030 * increment the ref count, and each remove() will decrement it. The values are removed from the map 031 * iff ref count == 0. 032 */ 033@InterfaceAudience.Private 034class RefCountingMap<K, V> { 035 036 private ConcurrentHashMap<K, Payload<V>> map = new ConcurrentHashMap<>(); 037 038 private static class Payload<V> { 039 V v; 040 int refCount; 041 042 Payload(V v) { 043 this.v = v; 044 this.refCount = 1; // create with ref count = 1 045 } 046 } 047 048 V put(K k, Supplier<V> supplier) { 049 return ((Payload<V>) map.compute(k, (k1, oldValue) -> { 050 if (oldValue != null) { 051 oldValue.refCount++; 052 return oldValue; 053 } else { 054 return new Payload(supplier.get()); 055 } 056 })).v; 057 } 058 059 V get(K k) { 060 Payload<V> p = map.get(k); 061 return p == null ? null : p.v; 062 } 063 064 /** 065 * Decrements the ref count of k, and removes from map if ref count == 0. 066 * @param k the key to remove 067 * @return the value associated with the specified key or null if key is removed from map. 068 */ 069 V remove(K k) { 070 Payload<V> p = map.computeIfPresent(k, (k1, v) -> --v.refCount <= 0 ? null : v); 071 return p == null ? null : p.v; 072 } 073 074 void clear() { 075 map.clear(); 076 } 077 078 Set<K> keySet() { 079 return map.keySet(); 080 } 081 082 Collection<V> values() { 083 return map.values().stream().map(v -> v.v).collect(Collectors.toList()); 084 } 085 086 int size() { 087 return map.size(); 088 } 089}