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}