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:
parent
536b9ee6d6
commit
9534e533dc
|
@ -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)
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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];
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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())));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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];
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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()));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
|
|
Loading…
Reference in New Issue