HADOOP-11970. Replace uses of ThreadLocal<Random> with JDK7 ThreadLocalRandom (Sean Busbey via Colin P. McCabe)

(cherry picked from commit 470c87dbc6)

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
        hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
This commit is contained in:
Colin Patrick Mccabe 2015-05-19 10:49:17 -07:00
parent 536b9ee6d6
commit 9534e533dc
22 changed files with 92 additions and 93 deletions

View File

@ -130,6 +130,9 @@ Release 2.8.0 - UNRELEASED
HADOOP-11812. Implement listLocatedStatus for ViewFileSystem to speed up
split calculation (gera)
HADOOP-11970. Replace uses of ThreadLocal<Random> with JDK7
ThreadLocalRandom. (Sean Busbey via Colin P. McCabe)
BUG FIXES
HADOOP-11802: DomainSocketWatcher thread terminates sometimes after there
is an I/O error during requestShortCircuitShm (cmccabe)

View File

@ -28,7 +28,7 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Random;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
@ -47,13 +47,6 @@ public class RetryPolicies {
public static final Log LOG = LogFactory.getLog(RetryPolicies.class);
private static ThreadLocal<Random> RANDOM = new ThreadLocal<Random>() {
@Override
protected Random initialValue() {
return new Random();
}
};
/**
* <p>
* Try once, and fail by re-throwing the exception.
@ -321,7 +314,8 @@ public class RetryPolicies {
}
//calculate sleep time and return.
final double ratio = RANDOM.get().nextDouble() + 0.5;//0.5 <= ratio <=1.5
// ensure 0.5 <= ratio <=1.5
final double ratio = ThreadLocalRandom.current().nextDouble() + 0.5;
final long sleepTime = Math.round(p.sleepMillis * ratio);
return new RetryAction(RetryAction.RetryDecision.RETRY, sleepTime);
}
@ -610,7 +604,7 @@ public class RetryPolicies {
private static long calculateExponentialTime(long time, int retries,
long cap) {
long baseTime = Math.min(time * (1L << retries), cap);
return (long) (baseTime * (RANDOM.get().nextDouble() + 0.5));
return (long) (baseTime * (ThreadLocalRandom.current().nextDouble() + 0.5));
}
private static long calculateExponentialTime(long time, int retries) {

View File

@ -49,6 +49,7 @@ import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
@ -318,7 +319,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
this.authority = nameNodeUri == null? "null": nameNodeUri.getAuthority();
this.clientName = "DFSClient_" + dfsClientConf.getTaskId() + "_" +
DFSUtil.getRandom().nextInt() + "_" + Thread.currentThread().getId();
ThreadLocalRandom.current().nextInt() + "_" +
Thread.currentThread().getId();
int numResponseToDrop = conf.getInt(
DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY,
DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT);

View File

@ -40,6 +40,7 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.Future;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
@ -976,7 +977,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
// expanded to 9000ms.
final int timeWindow = dfsClient.getConf().getTimeWindow();
double waitTime = timeWindow * failures + // grace period for the last round of attempt
timeWindow * (failures + 1) * DFSUtil.getRandom().nextDouble(); // expanding time window for each failure
// expanding time window for each failure
timeWindow * (failures + 1) *
ThreadLocalRandom.current().nextDouble();
DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec.");
Thread.sleep((long)waitTime);
} catch (InterruptedException iex) {

View File

@ -50,8 +50,8 @@ import java.util.Date;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.ThreadLocalRandom;
import javax.net.SocketFactory;
@ -105,12 +105,6 @@ public class DFSUtil {
public static final Log LOG = LogFactory.getLog(DFSUtil.class.getName());
private DFSUtil() { /* Hidden constructor */ }
private static final ThreadLocal<Random> RANDOM = new ThreadLocal<Random>() {
@Override
protected Random initialValue() {
return new Random();
}
};
private static final ThreadLocal<SecureRandom> SECURE_RANDOM = new ThreadLocal<SecureRandom>() {
@Override
@ -119,11 +113,6 @@ public class DFSUtil {
}
};
/** @return a pseudo random number generator. */
public static Random getRandom() {
return RANDOM.get();
}
/** @return a pseudo secure random number generator. */
public static SecureRandom getSecureRandom() {
return SECURE_RANDOM.get();
@ -132,9 +121,8 @@ public class DFSUtil {
/** Shuffle the elements in the given array. */
public static <T> T[] shuffle(final T[] array) {
if (array != null && array.length > 0) {
final Random random = getRandom();
for (int n = array.length; n > 1; ) {
final int randomIndex = random.nextInt(n);
final int randomIndex = ThreadLocalRandom.current().nextInt(n);
n--;
if (n != randomIndex) {
final T tmp = array[randomIndex];

View File

@ -35,6 +35,7 @@ import java.util.Queue;
import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.HadoopIllegalArgumentException;
@ -1026,7 +1027,8 @@ public class BlockManager {
return new BlocksWithLocations(new BlockWithLocations[0]);
}
Iterator<BlockInfoContiguous> iter = node.getBlockIterator();
int startBlock = DFSUtil.getRandom().nextInt(numBlocks); // starting from a random block
// starting from a random block
int startBlock = ThreadLocalRandom.current().nextInt(numBlocks);
// skip blocks
for(int i=0; i<startBlock; i++) {
iter.next();
@ -1671,7 +1673,7 @@ public class BlockManager {
// switch to a different node randomly
// this to prevent from deterministically selecting the same node even
// if the node failed to replicate the block on previous iterations
if(DFSUtil.getRandom().nextBoolean())
if(ThreadLocalRandom.current().nextBoolean())
srcNode = node;
}
if(numReplicas != null)
@ -1927,7 +1929,7 @@ public class BlockManager {
datanodeManager.getBlocksPerPostponedMisreplicatedBlocksRescan();
long base = getPostponedMisreplicatedBlocksCount() - blocksPerRescan;
if (base > 0) {
startIndex = DFSUtil.getRandom().nextLong() % (base+1);
startIndex = ThreadLocalRandom.current().nextLong() % (base+1);
if (startIndex < 0) {
startIndex += (base+1);
}

View File

@ -51,6 +51,7 @@ import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.util.*;
import java.util.concurrent.ThreadLocalRandom;
/**
* Manage datanodes, include decommission and other activities.
@ -457,7 +458,7 @@ public class DatanodeManager {
// Try something rack local.
if (node == null && !rackNodes.isEmpty()) {
node = (DatanodeDescriptor) (rackNodes
.get(DFSUtil.getRandom().nextInt(rackNodes.size())));
.get(ThreadLocalRandom.current().nextInt(rackNodes.size())));
}
}

View File

@ -20,12 +20,12 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.DFSUtil;
/** A map from host names to datanode descriptors. */
@InterfaceAudience.Private
@ -161,7 +161,7 @@ class Host2NodesMap {
return nodes[0];
}
// more than one node
return nodes[DFSUtil.getRandom().nextInt(nodes.length)];
return nodes[ThreadLocalRandom.current().nextInt(nodes.length)];
} finally {
hostmapLock.readLock().unlock();
}

View File

@ -28,6 +28,7 @@ import java.net.InetSocketAddress;
import java.util.Arrays;
import java.util.Comparator;
import java.util.HashMap;
import java.util.concurrent.ThreadLocalRandom;
import javax.servlet.ServletContext;
import javax.servlet.http.HttpServletRequest;
@ -150,7 +151,7 @@ public class JspHelper {
throw new IOException("No active nodes contain this block");
}
int index = doRandom ? DFSUtil.getRandom().nextInt(l) : 0;
int index = doRandom ? ThreadLocalRandom.current().nextInt(l) : 0;
return nodes[index];
}

View File

@ -28,13 +28,13 @@ import java.util.Collection;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ThreadLocalRandom;
import com.google.common.base.Joiner;
import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.hdfs.client.BlockReportOptions;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -122,7 +122,7 @@ class BPServiceActor implements Runnable {
this.dn = bpos.getDataNode();
this.nnAddr = nnAddr;
this.dnConf = dn.getDnConf();
prevBlockReportId = DFSUtil.getRandom().nextLong();
prevBlockReportId = ThreadLocalRandom.current().nextLong();
scheduler = new Scheduler(dnConf.heartBeatInterval, dnConf.blockReportInterval);
}
@ -409,7 +409,7 @@ class BPServiceActor implements Runnable {
// not send a 0 value ourselves.
prevBlockReportId++;
while (prevBlockReportId == 0) {
prevBlockReportId = DFSUtil.getRandom().nextLong();
prevBlockReportId = ThreadLocalRandom.current().nextLong();
}
return prevBlockReportId;
}
@ -1054,7 +1054,7 @@ class BPServiceActor implements Runnable {
if (delay > 0) { // send BR after random delay
// Numerical overflow is possible here and is okay.
nextBlockReportTime =
monotonicNow() + DFSUtil.getRandom().nextInt((int) (delay));
monotonicNow() + ThreadLocalRandom.current().nextInt((int) (delay));
} else { // send at next heartbeat
nextBlockReportTime = monotonicNow();
}
@ -1073,7 +1073,7 @@ class BPServiceActor implements Runnable {
// time before we start the periodic block reports.
if (resetBlockReportTime) {
nextBlockReportTime = monotonicNow() +
DFSUtil.getRandom().nextInt((int)(blockReportIntervalMs));
ThreadLocalRandom.current().nextInt((int)(blockReportIntervalMs));
resetBlockReportTime = false;
} else {
/* say the last block report was at 8:20:14. The current report

View File

@ -31,6 +31,7 @@ import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@ -41,7 +42,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
@ -327,7 +327,8 @@ public class DirectoryScanner implements Runnable {
void start() {
shouldRun = true;
long offset = DFSUtil.getRandom().nextInt((int) (scanPeriodMsecs/1000L)) * 1000L; //msec
long offset = ThreadLocalRandom.current().nextInt(
(int) (scanPeriodMsecs/1000L)) * 1000L; //msec
long firstScanTime = Time.now() + offset;
LOG.info("Periodic Directory Tree Verification scan starting at "
+ firstScanTime + " with interval " + scanPeriodMsecs);

View File

@ -22,7 +22,6 @@ import static org.apache.hadoop.metrics2.impl.MsInfo.SessionId;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.metrics2.MetricsSystem;
import org.apache.hadoop.metrics2.annotation.Metric;
import org.apache.hadoop.metrics2.annotation.Metrics;
@ -33,6 +32,8 @@ import org.apache.hadoop.metrics2.lib.MutableQuantiles;
import org.apache.hadoop.metrics2.lib.MutableRate;
import org.apache.hadoop.metrics2.source.JvmMetrics;
import java.util.concurrent.ThreadLocalRandom;
/**
*
* This class is for maintaining the various DataNode statistics
@ -177,7 +178,7 @@ public class DataNodeMetrics {
MetricsSystem ms = DefaultMetricsSystem.instance();
JvmMetrics jm = JvmMetrics.create("DataNode", sessionId, ms);
String name = "DataNodeActivity-"+ (dnName.isEmpty()
? "UndefinedDataNodeName"+ DFSUtil.getRandom().nextInt()
? "UndefinedDataNodeName"+ ThreadLocalRandom.current().nextInt()
: dnName.replace(':', '-'));
// Percentile measurement is off by default, by watching no intervals

View File

@ -32,6 +32,7 @@ import java.util.List;
import java.util.Properties;
import java.util.UUID;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ThreadLocalRandom;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
@ -599,7 +600,7 @@ public class NNStorage extends Storage implements Closeable,
private static int newNamespaceID() {
int newID = 0;
while(newID == 0)
newID = DFSUtil.getRandom().nextInt(0x7FFFFFFF); // use 31 bits only
newID = ThreadLocalRandom.current().nextInt(0x7FFFFFFF); // use 31 bits
return newID;
}

View File

@ -32,6 +32,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.TreeSet;
import java.util.concurrent.ThreadLocalRandom;
import org.apache.commons.io.IOUtils;
import org.apache.commons.logging.Log;
@ -44,7 +45,6 @@ import org.apache.hadoop.hdfs.BlockReader;
import org.apache.hadoop.hdfs.BlockReaderFactory;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.RemotePeerFactory;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.net.Peer;
@ -909,7 +909,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
}
DatanodeInfo chosenNode;
do {
chosenNode = nodes[DFSUtil.getRandom().nextInt(nodes.length)];
chosenNode = nodes[ThreadLocalRandom.current().nextInt(nodes.length)];
} while (deadNodes.contains(chosenNode));
return chosenNode;
}

View File

@ -29,6 +29,7 @@ import java.util.HashMap;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.Callable;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
@ -214,8 +215,7 @@ public class TestAppendSnapshotTruncate {
@Override
public String call() throws Exception {
final Random r = DFSUtil.getRandom();
final int op = r.nextInt(6);
final int op = ThreadLocalRandom.current().nextInt(6);
if (op <= 1) {
pauseAllFiles();
try {
@ -229,7 +229,8 @@ public class TestAppendSnapshotTruncate {
if (keys.length == 0) {
return "NO-OP";
}
final String snapshot = keys[r.nextInt(keys.length)];
final String snapshot = keys[ThreadLocalRandom.current()
.nextInt(keys.length)];
final String s = checkSnapshot(snapshot);
if (op == 2) {
@ -292,13 +293,13 @@ public class TestAppendSnapshotTruncate {
@Override
public String call() throws IOException {
final Random r = DFSUtil.getRandom();
final int op = r.nextInt(9);
final int op = ThreadLocalRandom.current().nextInt(9);
if (op == 0) {
return checkFullFile();
} else {
final int nBlocks = r.nextInt(4) + 1;
final int lastBlockSize = r.nextInt(BLOCK_SIZE) + 1;
final int nBlocks = ThreadLocalRandom.current().nextInt(4) + 1;
final int lastBlockSize = ThreadLocalRandom.current()
.nextInt(BLOCK_SIZE) + 1;
final int nBytes = nBlocks*BLOCK_SIZE + lastBlockSize;
if (op <= 4) {
@ -316,8 +317,8 @@ public class TestAppendSnapshotTruncate {
.append(n).append(" bytes to ").append(file.getName());
final byte[] bytes = new byte[n];
DFSUtil.getRandom().nextBytes(bytes);
ThreadLocalRandom.current().nextBytes(bytes);
{ // write to local file
final FileOutputStream out = new FileOutputStream(localFile, true);
out.write(bytes, 0, bytes.length);
@ -446,7 +447,6 @@ public class TestAppendSnapshotTruncate {
final Thread t = new Thread(null, new Runnable() {
@Override
public void run() {
final Random r = DFSUtil.getRandom();
for(State s; !(s = checkErrorState()).isTerminated;) {
if (s == State.RUNNING) {
isCalling.set(true);
@ -458,7 +458,7 @@ public class TestAppendSnapshotTruncate {
}
isCalling.set(false);
}
sleep(r.nextInt(100) + 50);
sleep(ThreadLocalRandom.current().nextInt(100) + 50);
}
}
}, name);

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs;
import java.io.File;
import java.io.IOException;
import java.util.concurrent.ThreadLocalRandom;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -272,7 +273,7 @@ public class TestRollingUpgrade {
final Path file = new Path(foo, "file");
final byte[] data = new byte[1024];
DFSUtil.getRandom().nextBytes(data);
ThreadLocalRandom.current().nextBytes(data);
final FSDataOutputStream out = cluster.getFileSystem().create(file);
out.write(data, 0, data.length);
out.close();
@ -320,7 +321,8 @@ public class TestRollingUpgrade {
Assert.assertTrue(dfs.exists(bar));
//truncate a file
final int newLength = DFSUtil.getRandom().nextInt(data.length - 1) + 1;
final int newLength = ThreadLocalRandom.current().nextInt(data.length - 1)
+ 1;
dfs.truncate(file, newLength);
TestFileTruncate.checkBlockRecovery(file, dfs);
AppendTestUtil.checkFullFile(dfs, file, newLength, data);

View File

@ -33,8 +33,8 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.ThreadLocalRandom;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
@ -73,7 +73,6 @@ public class TestReplicationPolicy {
((Log4JLogger)BlockPlacementPolicy.LOG).getLogger().setLevel(Level.ALL);
}
private final Random random = DFSUtil.getRandom();
private static final int BLOCK_SIZE = 1024;
private static final int NUM_OF_DATANODES = 6;
private static NetworkTopology cluster;
@ -848,14 +847,16 @@ public class TestReplicationPolicy {
.getNamesystem().getBlockManager().neededReplications;
for (int i = 0; i < 100; i++) {
// Adding the blocks directly to normal priority
neededReplications.add(new Block(random.nextLong()), 2, 0, 3);
neededReplications.add(new Block(
ThreadLocalRandom.current().nextLong()), 2, 0, 3);
}
// Lets wait for the replication interval, to start process normal
// priority blocks
Thread.sleep(DFS_NAMENODE_REPLICATION_INTERVAL);
// Adding the block directly to high priority list
neededReplications.add(new Block(random.nextLong()), 1, 0, 3);
neededReplications.add(new Block(
ThreadLocalRandom.current().nextLong()), 1, 0, 3);
// Lets wait for the replication interval
Thread.sleep(DFS_NAMENODE_REPLICATION_INTERVAL);
@ -878,19 +879,24 @@ public class TestReplicationPolicy {
for (int i = 0; i < 5; i++) {
// Adding QUEUE_HIGHEST_PRIORITY block
underReplicatedBlocks.add(new Block(random.nextLong()), 1, 0, 3);
underReplicatedBlocks.add(new Block(ThreadLocalRandom.current().
nextLong()), 1, 0, 3);
// Adding QUEUE_VERY_UNDER_REPLICATED block
underReplicatedBlocks.add(new Block(random.nextLong()), 2, 0, 7);
underReplicatedBlocks.add(new Block(ThreadLocalRandom.current().
nextLong()), 2, 0, 7);
// Adding QUEUE_REPLICAS_BADLY_DISTRIBUTED block
underReplicatedBlocks.add(new Block(random.nextLong()), 6, 0, 6);
underReplicatedBlocks.add(new Block(ThreadLocalRandom.current().
nextLong()), 6, 0, 6);
// Adding QUEUE_UNDER_REPLICATED block
underReplicatedBlocks.add(new Block(random.nextLong()), 5, 0, 6);
underReplicatedBlocks.add(new Block(ThreadLocalRandom.current().
nextLong()), 5, 0, 6);
// Adding QUEUE_WITH_CORRUPT_BLOCKS block
underReplicatedBlocks.add(new Block(random.nextLong()), 0, 0, 3);
underReplicatedBlocks.add(new Block(ThreadLocalRandom.current().
nextLong()), 0, 0, 3);
}
// Choose 6 blocks from UnderReplicatedBlocks. Then it should pick 5 blocks
@ -906,7 +912,8 @@ public class TestReplicationPolicy {
assertTheChosenBlocks(chosenBlocks, 0, 4, 5, 1, 0);
// Adding QUEUE_HIGHEST_PRIORITY
underReplicatedBlocks.add(new Block(random.nextLong()), 1, 0, 3);
underReplicatedBlocks.add(new Block(
ThreadLocalRandom.current().nextLong()), 1, 0, 3);
// Choose 10 blocks from UnderReplicatedBlocks. Then it should pick 1 block from
// QUEUE_HIGHEST_PRIORITY, 4 blocks from QUEUE_REPLICAS_BADLY_DISTRIBUTED

View File

@ -31,6 +31,7 @@ import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.concurrent.ThreadLocalRandom;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -46,7 +47,6 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.AppendTestUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
@ -164,11 +164,11 @@ public class TestFileTruncate {
fs.mkdirs(dir);
final Path p = new Path(dir, "file");
final byte[] data = new byte[100 * BLOCK_SIZE];
DFSUtil.getRandom().nextBytes(data);
ThreadLocalRandom.current().nextBytes(data);
writeContents(data, data.length, p);
for(int n = data.length; n > 0; ) {
final int newLength = DFSUtil.getRandom().nextInt(n);
final int newLength = ThreadLocalRandom.current().nextInt(n);
final boolean isReady = fs.truncate(p, newLength);
LOG.info("newLength=" + newLength + ", isReady=" + isReady);
assertEquals("File must be closed for truncating at the block boundary",
@ -193,7 +193,7 @@ public class TestFileTruncate {
fs.allowSnapshot(dir);
final Path p = new Path(dir, "file");
final byte[] data = new byte[BLOCK_SIZE];
DFSUtil.getRandom().nextBytes(data);
ThreadLocalRandom.current().nextBytes(data);
writeContents(data, data.length, p);
final String snapshot = "s0";
fs.createSnapshot(dir, snapshot);
@ -226,7 +226,7 @@ public class TestFileTruncate {
final Path p = new Path(dir, "file");
final byte[] data = new byte[2 * BLOCK_SIZE];
DFSUtil.getRandom().nextBytes(data);
ThreadLocalRandom.current().nextBytes(data);
writeContents(data, data.length, p);
final int newLength = data.length - 1;

View File

@ -24,6 +24,7 @@ import java.io.PrintWriter;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ThreadLocalRandom;
import com.google.common.base.Supplier;
import com.google.common.collect.Lists;
@ -37,7 +38,6 @@ import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.AppendTestUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.protocol.Block;
@ -640,7 +640,7 @@ public class TestDNFencing {
Collection<DatanodeStorageInfo> chooseFrom = !first.isEmpty() ? first : second;
List<DatanodeStorageInfo> l = Lists.newArrayList(chooseFrom);
return l.get(DFSUtil.getRandom().nextInt(l.size()));
return l.get(ThreadLocalRandom.current().nextInt(l.size()));
}
}

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode.ha;
import static org.junit.Assert.assertEquals;
import java.io.IOException;
import java.util.concurrent.ThreadLocalRandom;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -27,7 +28,6 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.AppendTestUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.server.namenode.TestFileTruncate;
@ -73,7 +73,7 @@ public class TestHAAppend {
Path fileToTruncate = new Path("/FileToTruncate");
final byte[] data = new byte[1 << 16];
DFSUtil.getRandom().nextBytes(data);
ThreadLocalRandom.current().nextBytes(data);
final int[] appendPos = AppendTestUtil.randomFilePartition(
data.length, COUNT);
final int[] truncatePos = AppendTestUtil.randomFilePartition(

View File

@ -27,6 +27,7 @@ import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
@ -34,7 +35,6 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.util.ByteArrayManager.Counter;
import org.apache.hadoop.hdfs.util.ByteArrayManager.CounterMap;
@ -72,7 +72,7 @@ public class TestByteArrayManager {
final long countResetTimePeriodMs = 200L;
final Counter c = new Counter(countResetTimePeriodMs);
final int n = DFSUtil.getRandom().nextInt(512) + 512;
final int n = ThreadLocalRandom.current().nextInt(512) + 512;
final List<Future<Integer>> futures = new ArrayList<Future<Integer>>(n);
final ExecutorService pool = Executors.newFixedThreadPool(32);
@ -334,7 +334,7 @@ public class TestByteArrayManager {
public void run() {
LOG.info("randomRecycler start");
for(int i = 0; shouldRun(); i++) {
final int j = DFSUtil.getRandom().nextInt(runners.length);
final int j = ThreadLocalRandom.current().nextInt(runners.length);
try {
runners[j].recycle();
} catch (Exception e) {
@ -440,7 +440,7 @@ public class TestByteArrayManager {
public byte[] call() throws Exception {
final int lower = maxArrayLength == ByteArrayManager.MIN_ARRAY_LENGTH?
0: maxArrayLength >> 1;
final int arrayLength = DFSUtil.getRandom().nextInt(
final int arrayLength = ThreadLocalRandom.current().nextInt(
maxArrayLength - lower) + lower + 1;
final byte[] array = bam.newByteArray(arrayLength);
try {
@ -496,7 +496,8 @@ public class TestByteArrayManager {
@Override
public void run() {
for(int i = 0; i < n; i++) {
final boolean isAllocate = DFSUtil.getRandom().nextInt(NUM_RUNNERS) < p;
final boolean isAllocate = ThreadLocalRandom.current()
.nextInt(NUM_RUNNERS) < p;
if (isAllocate) {
submitAllocate();
} else {
@ -573,7 +574,6 @@ public class TestByteArrayManager {
+ ", nAllocations=" + nAllocations
+ ", maxArrays=" + maxArrays);
final Random ran = DFSUtil.getRandom();
final ByteArrayManager[] impls = {
new ByteArrayManager.NewByteArrayWithoutLimit(),
new NewByteArrayWithLimit(maxArrays),
@ -590,7 +590,7 @@ public class TestByteArrayManager {
for(int j = 0; j < nTrials; j++) {
final int[] sleepTime = new int[nAllocations];
for(int k = 0; k < sleepTime.length; k++) {
sleepTime[k] = ran.nextInt(100);
sleepTime[k] = ThreadLocalRandom.current().nextInt(100);
}
final long elapsed = performanceTest(arrayLength, maxArrays, nThreads,

View File

@ -22,8 +22,8 @@ import java.io.IOException;
import java.io.InputStream;
import java.lang.reflect.UndeclaredThrowableException;
import java.net.URISyntaxException;
import java.util.Random;
import java.util.concurrent.Callable;
import java.util.concurrent.ThreadLocalRandom;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -61,13 +61,6 @@ class SharedCacheUploader implements Callable<Boolean> {
new FsPermission((short)00555);
private static final Log LOG = LogFactory.getLog(SharedCacheUploader.class);
private static final ThreadLocal<Random> randomTl =
new ThreadLocal<Random>() {
@Override
protected Random initialValue() {
return new Random(System.nanoTime());
}
};
private final LocalResource resource;
private final Path localPath;
@ -267,7 +260,7 @@ class SharedCacheUploader implements Callable<Boolean> {
}
private String getTemporaryFileName(Path path) {
return path.getName() + "-" + randomTl.get().nextLong();
return path.getName() + "-" + ThreadLocalRandom.current().nextLong();
}
@VisibleForTesting