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.util; 019 020import java.lang.ref.ReferenceQueue; 021import java.lang.ref.WeakReference; 022import java.util.concurrent.ConcurrentHashMap; 023import java.util.concurrent.atomic.AtomicInteger; 024import java.util.concurrent.locks.Lock; 025import java.util.concurrent.locks.ReentrantLock; 026import org.apache.commons.lang3.mutable.MutableObject; 027import org.apache.yetus.audience.InterfaceAudience; 028 029import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; 030 031/** 032 * A string pool like {@link String#intern()}, but more flexible as we can create multiple instances 033 * and use them in difference places, where {@link String#intern()} is global. 034 * <p> 035 * We use {@link WeakReference} so when there are no actual reference to the String, it will be GCed 036 * to reduce memory pressure. 037 * <p> 038 * The difference between {@link WeakObjectPool} is that, we also need to use {@link WeakReference} 039 * as key, not only value, because the key(a String) is exactly what we want to deduplicate. 040 */ 041@InterfaceAudience.Private 042public class FastStringPool { 043 044 private static final class WeakKey extends WeakReference<String> { 045 046 private final int hash; 047 048 WeakKey(String referent, ReferenceQueue<String> queue) { 049 super(Preconditions.checkNotNull(referent), queue); 050 // must calculate it here, as later the referent may be GCed 051 this.hash = referent.hashCode(); 052 } 053 054 @Override 055 public int hashCode() { 056 return hash; 057 } 058 059 @Override 060 public boolean equals(Object obj) { 061 if (this == obj) { 062 return true; 063 } 064 if (!(obj instanceof WeakKey)) { 065 return false; 066 } 067 068 String a = this.get(); 069 String b = ((WeakKey) obj).get(); 070 // In ConcurrentHashMap, we will always compare references(like entry.key == key) before 071 // calling actual equals method, so this will not cause problems for clean up. And in normal 072 // intern path, the reference will never be null, so there is no problem too. 073 if (a == null || b == null) { 074 return false; 075 } 076 return a.equals(b); 077 } 078 } 079 080 private final ConcurrentHashMap<WeakKey, WeakReference<String>> map = new ConcurrentHashMap<>(); 081 082 private final ReferenceQueue<String> refQueue = new ReferenceQueue<>(); 083 084 private final Lock cleanupLock = new ReentrantLock(); 085 086 // only call cleanup every 256 times 087 private static final int CLEANUP_MASK = 0xFF; 088 private final AtomicInteger counter = new AtomicInteger(); 089 090 public String intern(String s) { 091 Preconditions.checkNotNull(s); 092 maybeCleanup(); 093 094 WeakKey lookupKey = new WeakKey(s, null); 095 WeakReference<String> ref = map.get(lookupKey); 096 if (ref != null) { 097 String v = ref.get(); 098 if (v != null) { 099 return v; 100 } 101 } 102 103 WeakKey storeKey = new WeakKey(s, refQueue); 104 WeakReference<String> storeVal = new WeakReference<>(s); 105 // Used to store the return value. The return value of compute method is a WeakReference, the 106 // value of the WeakReference may be GCed just before we get it for returning. 107 MutableObject<String> ret = new MutableObject<>(); 108 109 map.compute(storeKey, (k, prevVal) -> { 110 if (prevVal == null) { 111 ret.setValue(s); 112 return storeVal; 113 } else { 114 String prevRef = prevVal.get(); 115 if (prevRef != null) { 116 ret.setValue(prevRef); 117 return prevVal; 118 } else { 119 ret.setValue(s); 120 return storeVal; 121 } 122 } 123 }); 124 assert ret.get() != null; 125 return ret.get(); 126 } 127 128 private void cleanup() { 129 if (!cleanupLock.tryLock()) { 130 // a cleanup task is ongoing, give up 131 return; 132 } 133 try { 134 for (;;) { 135 WeakKey k = (WeakKey) refQueue.poll(); 136 if (k == null) { 137 return; 138 } 139 map.remove(k); 140 } 141 } finally { 142 cleanupLock.unlock(); 143 } 144 } 145 146 private void maybeCleanup() { 147 if ((counter.incrementAndGet() & CLEANUP_MASK) != 0) { 148 return; 149 } 150 cleanup(); 151 } 152 153 public int size() { 154 // size method is not on critical path, so always call cleanup here to reduce memory pressure 155 cleanup(); 156 return map.size(); 157 } 158}