HADOOP-19119. Spotbugs: possible NPE in org.apache.hadoop.crypto.key.kms.ValueQueue.getSize() (#6642)
Spotbugs is mistaken here as it doesn't observer the read/write locks used to manage exclusive access to the maps. * cache the value between checks * tag as @VisibleForTesting Contributed by Steve Loughran
This commit is contained in:
parent
ff3f2255d2
commit
705fb8323b
@ -33,6 +33,7 @@
|
|||||||
import java.util.concurrent.locks.ReadWriteLock;
|
import java.util.concurrent.locks.ReadWriteLock;
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
|
|
||||||
|
import org.apache.hadoop.classification.VisibleForTesting;
|
||||||
import org.apache.hadoop.util.Preconditions;
|
import org.apache.hadoop.util.Preconditions;
|
||||||
import org.apache.hadoop.thirdparty.com.google.common.cache.CacheBuilder;
|
import org.apache.hadoop.thirdparty.com.google.common.cache.CacheBuilder;
|
||||||
import org.apache.hadoop.thirdparty.com.google.common.cache.CacheLoader;
|
import org.apache.hadoop.thirdparty.com.google.common.cache.CacheLoader;
|
||||||
@ -317,8 +318,9 @@ public void drain(String keyName) {
|
|||||||
/**
|
/**
|
||||||
* Get size of the Queue for keyName. This is only used in unit tests.
|
* Get size of the Queue for keyName. This is only used in unit tests.
|
||||||
* @param keyName the key name
|
* @param keyName the key name
|
||||||
* @return int queue size
|
* @return int queue size. Zero means the queue is empty or the key does not exist.
|
||||||
*/
|
*/
|
||||||
|
@VisibleForTesting
|
||||||
public int getSize(String keyName) {
|
public int getSize(String keyName) {
|
||||||
readLock(keyName);
|
readLock(keyName);
|
||||||
try {
|
try {
|
||||||
@ -326,10 +328,12 @@ public int getSize(String keyName) {
|
|||||||
// since that will have the side effect of populating the cache.
|
// since that will have the side effect of populating the cache.
|
||||||
Map<String, LinkedBlockingQueue<E>> map =
|
Map<String, LinkedBlockingQueue<E>> map =
|
||||||
keyQueues.getAllPresent(Arrays.asList(keyName));
|
keyQueues.getAllPresent(Arrays.asList(keyName));
|
||||||
if (map.get(keyName) == null) {
|
final LinkedBlockingQueue<E> linkedQueue = map.get(keyName);
|
||||||
|
if (linkedQueue == null) {
|
||||||
return 0;
|
return 0;
|
||||||
|
} else {
|
||||||
|
return linkedQueue.size();
|
||||||
}
|
}
|
||||||
return map.get(keyName).size();
|
|
||||||
} finally {
|
} finally {
|
||||||
readUnlock(keyName);
|
readUnlock(keyName);
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user