From 10471944bd2afcd423b604c36a57d686cc15a9e9 Mon Sep 17 00:00:00 2001 From: Andrew Purtell Date: Tue, 8 Mar 2022 13:49:02 -0800 Subject: [PATCH] HBASE-26582 Prune use of Random and SecureRandom objects (#4118) Avoid the pattern where a Random object is allocated, used once or twice, and then left for GC. This pattern triggers warnings from some static analysis tools because this pattern leads to poor effective randomness. In a few cases we were legitimately suffering from this issue; in others a change is still good to reduce noise in analysis results. Use ThreadLocalRandom where there is no requirement to set the seed to gain good reuse. Where useful relax use of SecureRandom to simply Random or ThreadLocalRandom, which are unlikely to block if the system entropy pool is low, if we don't need crypographically strong randomness for the use case. The exception to this is normalization of use of Bytes#random to fill byte arrays with randomness. Because Bytes#random may be used to generate key material it must be backed by SecureRandom. Signed-off-by: Duo Zhang --- .../TestFanOutOneBlockAsyncDFSOutput.java | 21 ++++----- .../TestFanOutOneBlockAsyncDFSOutputHang.java | 6 +-- .../master/balancer/BalancerTestBase.java | 22 ++++----- .../TestRegionHDFSBlockLocationFinder.java | 8 ++-- .../client/PerClientRandomNonceGenerator.java | 9 ++-- .../hadoop/hbase/filter/RandomRowFilter.java | 5 +- .../hadoop/hbase/security/EncryptionUtil.java | 8 +--- .../hbase/slowlog/SlowLogTableAccessor.java | 7 ++- .../hbase/security/TestEncryptionUtil.java | 10 ++-- .../hbase/util/TestRoundRobinPoolMap.java | 2 - .../hbase/util/TestThreadLocalPoolMap.java | 2 - .../hadoop/hbase/io/crypto/Encryption.java | 2 +- .../HFileBlockDefaultEncodingContext.java | 3 +- .../org/apache/hadoop/hbase/util/Bytes.java | 45 ++++++++++++++++-- .../hadoop/hbase/HBaseCommonTestingUtil.java | 16 ++----- .../hbase/io/crypto/TestEncryption.java | 16 +++---- .../hbase/io/util/TestLRUDictionary.java | 5 +- .../hbase/util/LoadTestKVGenerator.java | 2 +- .../apache/hadoop/hbase/util/TestAvlUtil.java | 5 +- .../hbase/util/TestByteBufferArray.java | 6 +-- .../apache/hadoop/hbase/util/TestBytes.java | 13 +++--- .../TestCompatibilitySingletonFactory.java | 5 +- .../hadoop/hbase/http/TestServletFilter.java | 6 ++- .../IntegrationTestDDLMasterFailover.java | 46 ++++++++++--------- .../hadoop/hbase/chaos/actions/Action.java | 9 ++-- .../actions/ChangeBloomFilterAction.java | 5 +- .../actions/ChangeCompressionAction.java | 10 ++-- .../chaos/actions/ChangeEncodingAction.java | 7 +-- .../actions/ChangeSplitPolicyAction.java | 8 ++-- .../chaos/actions/ChangeVersionsAction.java | 8 +--- .../hbase/chaos/actions/CompactMobAction.java | 7 ++- .../CompactRandomRegionOfTableAction.java | 10 ++-- .../chaos/actions/CompactTableAction.java | 4 +- .../chaos/actions/CorruptDataFilesAction.java | 7 +-- .../actions/DecreaseMaxHFileSizeAction.java | 7 ++- .../chaos/actions/DeleteDataFilesAction.java | 6 ++- .../GracefulRollingRestartRsAction.java | 8 ++-- ...rgeRandomAdjacentRegionsOfTableAction.java | 4 +- .../actions/MoveRegionsOfTableAction.java | 7 +-- .../chaos/actions/RemoveColumnAction.java | 9 ++-- .../actions/RestartRsHoldingTableAction.java | 7 +-- .../actions/RollingBatchRestartRsAction.java | 10 ++-- .../RollingBatchSuspendResumeRsAction.java | 8 ++-- .../actions/SplitAllRegionOfTableAction.java | 6 +-- .../chaos/actions/TruncateTableAction.java | 6 +-- .../UnbalanceKillAndRebalanceAction.java | 7 +-- .../chaos/actions/UnbalanceRegionsAction.java | 6 ++- .../chaos/monkies/PolicyBasedChaosMonkey.java | 11 ++--- .../hbase/chaos/policies/PeriodicPolicy.java | 4 +- .../hbase/ipc/IntegrationTestRpcClient.java | 14 ++---- .../mapreduce/IntegrationTestBulkLoad.java | 17 +++---- .../test/IntegrationTestBigLinkedList.java | 20 ++++---- .../test/IntegrationTestLoadAndVerify.java | 7 +-- ...TimeBoundedRequestsWithRegionReplicas.java | 4 +- ...onTestWithCellVisibilityLoadAndVerify.java | 6 ++- .../IntegrationTestSendTraceRequests.java | 10 ++-- .../hadoop/hbase/PerformanceEvaluation.java | 8 ++-- .../hbase/TestPerformanceEvaluation.java | 6 ++- .../mapreduce/TestHFileOutputFormat2.java | 21 ++++----- .../hadoop/hbase/util/LoadTestTool.java | 3 +- .../metrics/impl/TestFastLongHistogram.java | 4 +- .../ProcedureStorePerformanceEvaluation.java | 4 +- ...ocedureWALLoaderPerformanceEvaluation.java | 5 +- .../store/wal/TestProcedureStoreTracker.java | 6 +-- .../wal/TestStressWALProcedureStore.java | 5 +- .../TestZKReplicationPeerStorage.java | 16 +++---- .../hbase/rest/TestScannerResource.java | 4 +- .../ZkSplitLogWorkerCoordination.java | 5 +- .../hbase/io/hfile/PrefetchExecutor.java | 7 ++- .../master/RegionPlacementMaintainer.java | 4 +- .../AdaptiveMemStoreCompactionStrategy.java | 8 ++-- .../hbase/regionserver/HRegionServer.java | 11 +++-- .../compactions/SortedCompactionPolicy.java | 18 +++----- .../wal/AbstractProtobufLogWriter.java | 9 +--- .../regionserver/wal/SecureWALCellCodec.java | 3 +- .../regionserver/HFileReplicator.java | 4 +- .../apache/hadoop/hbase/tool/CanaryTool.java | 11 ++--- .../hadoop/hbase/util/EncryptionTest.java | 2 +- .../hadoop/hbase/util/HBaseFsckRepair.java | 6 ++- .../hadoop/hbase/AcidGuaranteesTestTool.java | 8 ++-- .../apache/hadoop/hbase/HBaseTestingUtil.java | 12 ++--- .../hbase/HFilePerformanceEvaluation.java | 15 ++---- .../hbase/RegionReplicationLagEvaluation.java | 11 ++--- .../hadoop/hbase/TestHBaseTestingUtil.java | 2 +- .../hadoop/hbase/TestMetaTableAccessor.java | 17 ++++--- .../hadoop/hbase/client/TestAdmin2.java | 6 +-- .../hbase/client/TestAsyncBufferMutator.java | 4 +- .../client/TestAsyncNonMetaRegionLocator.java | 6 +-- .../client/TestAsyncProcedureAdminApi.java | 6 +-- .../TestAsyncTableBatchRetryImmediately.java | 3 +- .../TestAsyncTableGetMultiThreaded.java | 4 +- .../hbase/client/TestFromClientSide3.java | 8 ++-- .../hbase/client/TestMultiRespectsLimits.java | 3 +- .../client/TestRequestTooBigException.java | 4 +- .../hbase/client/TestTimestampsFilter.java | 3 -- .../hbase/client/locking/TestEntityLocks.java | 4 +- .../coprocessor/TestMetaTableMetrics.java | 4 +- .../hbase/io/compress/HFileTestBase.java | 8 ++-- .../io/encoding/TestChangingEncoding.java | 4 +- .../io/encoding/TestDataBlockEncoders.java | 7 +-- .../hadoop/hbase/io/hfile/CacheTestUtils.java | 7 +-- .../hadoop/hbase/io/hfile/TestHFile.java | 13 +++--- .../hadoop/hbase/io/hfile/TestHFileBlock.java | 32 ++++++------- .../hbase/io/hfile/TestHFileBlockIndex.java | 18 ++++---- .../hbase/io/hfile/TestHFileEncryption.java | 13 ++++-- .../TestHFileScannerImplReferenceCount.java | 9 ++-- .../hbase/io/hfile/TestHFileWriterV3.java | 17 +++---- .../TestHFileWriterV3WithDataEncoders.java | 11 ++--- .../hfile/TestLazyDataBlockDecompression.java | 9 ++-- .../io/hfile/TestLruAdaptiveBlockCache.java | 5 +- .../hbase/io/hfile/TestLruBlockCache.java | 7 +-- .../hadoop/hbase/io/hfile/TestPrefetch.java | 14 +++--- .../hbase/io/hfile/TestTinyLfuBlockCache.java | 5 +- .../io/hfile/bucket/TestBucketCache.java | 6 +-- .../hadoop/hbase/master/MockRegionServer.java | 6 +-- .../hbase/master/TestRegionPlacement.java | 12 ++--- .../assignment/TestAssignmentManagerBase.java | 10 ++-- .../balancer/RSGroupableBalancerTestBase.java | 14 +++--- .../TestBalancerStatusTagInJMXMetrics.java | 4 +- .../TestStochasticBalancerJmxMetrics.java | 4 +- .../master/cleaner/TestCleanerChore.java | 8 ++-- .../master/cleaner/TestHFileCleaner.java | 3 +- .../hbase/master/cleaner/TestLogsCleaner.java | 6 +-- ...ocedureSchedulerPerformanceEvaluation.java | 5 +- .../master/procedure/TestProcedureAdmin.java | 6 +-- .../hbase/mob/FaultyMobStoreCompactor.java | 8 ++-- .../apache/hadoop/hbase/mob/MobTestUtil.java | 5 +- .../hbase/mob/TestExpiredMobFileCleaner.java | 3 +- .../mob/TestMobCompactionWithDefaults.java | 5 +- .../hbase/mob/TestMobDataBlockEncoding.java | 4 +- .../hadoop/hbase/mob/TestMobFileName.java | 10 ++-- .../hbase/mob/TestMobStoreCompaction.java | 4 +- .../hadoop/hbase/mob/TestMobStoreScanner.java | 5 +- .../quotas/SpaceQuotaHelperForTests.java | 4 +- .../hbase/quotas/TestRegionSizeUse.java | 5 +- .../regionserver/CreateRandomStoreFile.java | 12 +++-- .../regionserver/TestAtomicOperation.java | 2 - .../hbase/regionserver/TestBulkloadBase.java | 4 +- .../regionserver/TestCompactionState.java | 9 ++-- .../regionserver/TestDeleteMobTable.java | 5 +- .../TestEncryptionKeyRotation.java | 7 ++- .../TestEndToEndSplitTransaction.java | 3 +- .../hbase/regionserver/TestHMobStore.java | 5 +- .../regionserver/TestHRegionReplayEvents.java | 4 +- .../regionserver/TestJoinedScanners.java | 8 ++-- .../regionserver/TestMemStoreChunkPool.java | 3 +- .../hbase/regionserver/TestMemStoreLAB.java | 6 +-- .../TestMemstoreLABWithoutPool.java | 4 +- .../regionserver/TestMultiColumnScanner.java | 5 +- .../TestMultiVersionConcurrencyControl.java | 5 +- .../hbase/regionserver/TestParallelPut.java | 4 +- .../TestPerColumnFamilyFlush.java | 24 ++++------ .../TestRecoveredEditsReplayAndAbort.java | 5 +- .../TestRegionMergeTransactionOnCluster.java | 7 ++- .../regionserver/TestRegionReplicas.java | 6 +-- .../TestSecureBulkloadListener.java | 27 ++--------- .../regionserver/TestSeekOptimizations.java | 8 ++-- .../regionserver/TestStoreScannerClosure.java | 11 +++-- .../ConstantSizeFileListGenerator.java | 5 +- .../ExplicitFileListGenerator.java | 7 +-- .../GaussianFileListGenerator.java | 9 +--- .../compactions/MockStoreFileGenerator.java | 13 ++---- .../PerfTestCompactionPolicies.java | 1 - .../SemiConstantSizeFileListGenerator.java | 8 ++-- .../SinusoidalFileListGenerator.java | 4 +- .../compactions/SpikyFileListGenerator.java | 13 +++--- .../compactions/StoreFileListGenerator.java | 4 -- .../compactions/TestFIFOCompactionPolicy.java | 4 +- ...estCompactionWithThroughputController.java | 3 +- .../TestFlushWithThroughputController.java | 7 +-- .../replication/TestMasterReplication.java | 7 +-- .../regionserver/TestReplicationSink.java | 6 +-- .../hbase/rsgroup/TestRSGroupsAdmin2.java | 8 ++-- .../util/LoadTestDataGeneratorWithTags.java | 9 ++-- .../hbase/util/MultiThreadedAction.java | 6 +-- .../hbase/util/MultiThreadedReader.java | 10 ++-- .../hbase/util/MultiThreadedUpdater.java | 10 ++-- .../apache/hadoop/hbase/util/TestFSUtils.java | 15 +++--- .../hbase/util/TestHBaseFsckEncryption.java | 4 +- .../apache/hadoop/hbase/util/TestIdLock.java | 3 +- .../TestIdReadWriteLockWithObjectPool.java | 3 +- .../hbase/wal/WALPerformanceEvaluation.java | 4 +- .../hbase/HBaseCommonTestingUtility.java | 15 ++---- .../hadoop/hbase/HBaseTestingUtility.java | 10 ++-- .../hbase/zookeeper/MiniZooKeeperCluster.java | 5 +- .../hbase/zookeeper/TestReadOnlyZKClient.java | 4 +- 186 files changed, 689 insertions(+), 817 deletions(-) diff --git a/hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java b/hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java index 8533d38bae09..3b992a853edf 100644 --- a/hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java +++ b/hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java @@ -34,7 +34,6 @@ import java.util.Random; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -44,6 +43,7 @@ import org.apache.hadoop.hbase.io.asyncfs.monitor.StreamSlowMonitor; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties; import org.apache.hadoop.hdfs.server.datanode.DataNode; @@ -57,7 +57,6 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import org.apache.hbase.thirdparty.io.netty.channel.Channel; import org.apache.hbase.thirdparty.io.netty.channel.EventLoop; import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; @@ -72,13 +71,9 @@ public class TestFanOutOneBlockAsyncDFSOutput extends AsyncFSTestBase { HBaseClassTestRule.forClass(TestFanOutOneBlockAsyncDFSOutput.class); private static final Logger LOG = LoggerFactory.getLogger(TestFanOutOneBlockAsyncDFSOutput.class); - private static DistributedFileSystem FS; - private static EventLoopGroup EVENT_LOOP_GROUP; - private static Class CHANNEL_CLASS; - private static int READ_TIMEOUT_MS = 2000; private static StreamSlowMonitor MONITOR; @@ -104,14 +99,16 @@ public static void tearDown() throws IOException, InterruptedException { shutdownMiniDFSCluster(); } + private static final Random RNG = new Random(); // This test depends on Random#setSeed + static void writeAndVerify(FileSystem fs, Path f, AsyncFSOutput out) throws IOException, InterruptedException, ExecutionException { List> futures = new ArrayList<>(); byte[] b = new byte[10]; - Random rand = new Random(12345); // test pipelined flush + RNG.setSeed(12345); for (int i = 0; i < 10; i++) { - rand.nextBytes(b); + RNG.nextBytes(b); out.write(b); futures.add(out.flush(false)); futures.add(out.flush(false)); @@ -123,11 +120,11 @@ static void writeAndVerify(FileSystem fs, Path f, AsyncFSOutput out) out.close(); assertEquals(b.length * 10, fs.getFileStatus(f).getLen()); byte[] actual = new byte[b.length]; - rand.setSeed(12345); + RNG.setSeed(12345); try (FSDataInputStream in = fs.open(f)) { for (int i = 0; i < 10; i++) { in.readFully(actual); - rand.nextBytes(b); + RNG.nextBytes(b); assertArrayEquals(b, actual); } assertEquals(-1, in.read()); @@ -150,7 +147,7 @@ public void testRecover() throws IOException, InterruptedException, ExecutionExc FanOutOneBlockAsyncDFSOutput out = FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS, f, true, false, (short) 3, FS.getDefaultBlockSize(), eventLoop, CHANNEL_CLASS, MONITOR); byte[] b = new byte[10]; - ThreadLocalRandom.current().nextBytes(b); + Bytes.random(b); out.write(b, 0, b.length); out.flush(false).get(); // restart one datanode which causes one connection broken @@ -260,7 +257,7 @@ public void testWriteLargeChunk() throws IOException, InterruptedException, Exec FanOutOneBlockAsyncDFSOutput out = FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS, f, true, false, (short) 3, 1024 * 1024 * 1024, eventLoop, CHANNEL_CLASS, MONITOR); byte[] b = new byte[50 * 1024 * 1024]; - ThreadLocalRandom.current().nextBytes(b); + Bytes.random(b); out.write(b); out.flush(false); assertEquals(b.length, out.flush(false).get().longValue()); diff --git a/hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutputHang.java b/hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutputHang.java index 496dc127fded..8ee838449e14 100644 --- a/hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutputHang.java +++ b/hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutputHang.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.io.asyncfs; - import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -29,12 +28,12 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.ExecutionException; -import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.io.asyncfs.monitor.StreamSlowMonitor; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; @@ -57,7 +56,6 @@ import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup; import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; - /** * Testcase for HBASE-26679, here we introduce a separate test class and not put the testcase in * {@link TestFanOutOneBlockAsyncDFSOutput} because we will send heartbeat to DN when there is no @@ -191,7 +189,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception }); byte[] b = new byte[10]; - ThreadLocalRandom.current().nextBytes(b); + Bytes.random(b); OUT.write(b, 0, b.length); CompletableFuture future = OUT.flush(false); /** diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java index 8288962874d5..59335079bcda 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java @@ -420,10 +420,9 @@ protected List randomRegions(int numRegions) { protected List createRegions(int numRegions, TableName tableName) { List regions = new ArrayList<>(numRegions); byte[] start = new byte[16]; + Bytes.random(start); byte[] end = new byte[16]; - Random rand = ThreadLocalRandom.current(); - rand.nextBytes(start); - rand.nextBytes(end); + Bytes.random(end); for (int i = 0; i < numRegions; i++) { Bytes.putInt(start, 0, numRegions << 1); Bytes.putInt(end, 0, (numRegions << 1) + 1); @@ -440,10 +439,9 @@ protected List createRegions(int numRegions, TableName tableName) { protected List randomRegions(int numRegions, int numTables) { List regions = new ArrayList<>(numRegions); byte[] start = new byte[16]; + Bytes.random(start); byte[] end = new byte[16]; - Random rand = ThreadLocalRandom.current(); - rand.nextBytes(start); - rand.nextBytes(end); + Bytes.random(end); for (int i = 0; i < numRegions; i++) { if (!regionQueue.isEmpty()) { regions.add(regionQueue.poll()); @@ -451,8 +449,8 @@ protected List randomRegions(int numRegions, int numTables) { } Bytes.putInt(start, 0, numRegions << 1); Bytes.putInt(end, 0, (numRegions << 1) + 1); - TableName tableName = - TableName.valueOf("table" + (numTables > 0 ? rand.nextInt(numTables) : i)); + TableName tableName = TableName.valueOf("table" + + (numTables > 0 ? ThreadLocalRandom.current().nextInt(numTables) : i)); RegionInfo hri = RegionInfoBuilder.newBuilder(tableName) .setStartKey(start) .setEndKey(end) @@ -467,15 +465,13 @@ protected List randomRegions(int numRegions, int numTables) { protected List uniformRegions(int numRegions) { List regions = new ArrayList<>(numRegions); byte[] start = new byte[16]; + Bytes.random(start); byte[] end = new byte[16]; - Random rand = ThreadLocalRandom.current(); - rand.nextBytes(start); - rand.nextBytes(end); + Bytes.random(end); for (int i = 0; i < numRegions; i++) { Bytes.putInt(start, 0, numRegions << 1); Bytes.putInt(end, 0, (numRegions << 1) + 1); - TableName tableName = - TableName.valueOf("table" + i); + TableName tableName = TableName.valueOf("table" + i); RegionInfo hri = RegionInfoBuilder.newBuilder(tableName) .setStartKey(start) .setEndKey(end) diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionHDFSBlockLocationFinder.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionHDFSBlockLocationFinder.java index 807b5ff35f81..11ef8695785f 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionHDFSBlockLocationFinder.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionHDFSBlockLocationFinder.java @@ -63,8 +63,8 @@ public class TestRegionHDFSBlockLocationFinder { public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestRegionHDFSBlockLocationFinder.class); + private static final Random RNG = new Random(); // This test depends on Random#setSeed private static TableDescriptor TD; - private static List REGIONS; private RegionHDFSBlockLocationFinder finder; @@ -72,10 +72,10 @@ public class TestRegionHDFSBlockLocationFinder { private static HDFSBlocksDistribution generate(RegionInfo region) { HDFSBlocksDistribution distribution = new HDFSBlocksDistribution(); int seed = region.hashCode(); - Random rand = new Random(seed); - int size = 1 + rand.nextInt(10); + RNG.setSeed(seed); + int size = 1 + RNG.nextInt(10); for (int i = 0; i < size; i++) { - distribution.addHostsAndBlockWeight(new String[] { "host-" + i }, 1 + rand.nextInt(100)); + distribution.addHostsAndBlockWeight(new String[] { "host-" + i }, 1 + RNG.nextInt(100)); } return distribution; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PerClientRandomNonceGenerator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PerClientRandomNonceGenerator.java index c492282e2a7d..8aedc4d2205c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PerClientRandomNonceGenerator.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PerClientRandomNonceGenerator.java @@ -19,8 +19,7 @@ package org.apache.hadoop.hbase.client; import java.util.Arrays; -import java.util.Random; - +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.HConstants; import org.apache.yetus.audience.InterfaceAudience; @@ -33,12 +32,12 @@ public final class PerClientRandomNonceGenerator implements NonceGenerator { private static final PerClientRandomNonceGenerator INST = new PerClientRandomNonceGenerator(); - private final Random rdm = new Random(); private final long clientId; private PerClientRandomNonceGenerator() { byte[] clientIdBase = ClientIdGenerator.generateClientId(); - this.clientId = (((long) Arrays.hashCode(clientIdBase)) << 32) + rdm.nextInt(); + this.clientId = (((long) Arrays.hashCode(clientIdBase)) << 32) + + ThreadLocalRandom.current().nextInt(); } @Override @@ -50,7 +49,7 @@ public long getNonceGroup() { public long newNonce() { long result = HConstants.NO_NONCE; do { - result = rdm.nextLong(); + result = ThreadLocalRandom.current().nextLong(); } while (result == HConstants.NO_NONCE); return result; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java index e57cff54539c..f89182a704fb 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java @@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.filter; import java.util.Objects; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.Cell; import org.apache.yetus.audience.InterfaceAudience; @@ -35,7 +35,6 @@ */ @InterfaceAudience.Public public class RandomRowFilter extends FilterBase { - protected static final Random random = new Random(); protected float chance; protected boolean filterOutRow; @@ -98,7 +97,7 @@ public boolean filterRowKey(Cell firstRowCell) { filterOutRow = false; } else { // roll the dice - filterOutRow = !(random.nextFloat() < chance); + filterOutRow = !(ThreadLocalRandom.current().nextFloat() < chance); } return filterOutRow; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java index 8d380dc7fe6c..74ad96e2cbda 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.security.Key; import java.security.KeyException; -import java.security.SecureRandom; import java.util.Properties; import javax.crypto.spec.SecretKeySpec; import org.apache.commons.crypto.cipher.CryptoCipherFactory; @@ -37,7 +36,6 @@ import org.apache.yetus.audience.InterfaceStability; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; import org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; @@ -50,8 +48,6 @@ public final class EncryptionUtil { static private final Logger LOG = LoggerFactory.getLogger(EncryptionUtil.class); - static private final SecureRandom RNG = new SecureRandom(); - /** * Private constructor to keep this class from being instantiated. */ @@ -96,7 +92,7 @@ public static byte[] wrapKey(Configuration conf, String subject, Key key) byte[] iv = null; if (cipher.getIvLength() > 0) { iv = new byte[cipher.getIvLength()]; - RNG.nextBytes(iv); + Bytes.secureRandom(iv); builder.setIv(UnsafeByteOperations.unsafeWrap(iv)); } byte[] keyBytes = key.getEncoded(); @@ -286,7 +282,7 @@ public static Key unwrapKey(Configuration conf, byte[] keyBytes) throws IOExcept * @throws IOException if create CryptoAES failed */ public static CryptoAES createCryptoAES(RPCProtos.CryptoCipherMeta cryptoCipherMeta, - Configuration conf) throws IOException { + Configuration conf) throws IOException { Properties properties = new Properties(); // the property for cipher class properties.setProperty(CryptoCipherFactory.CLASSES_KEY, diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/slowlog/SlowLogTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/slowlog/SlowLogTableAccessor.java index bf4cd046755b..771ee8cffbec 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/slowlog/SlowLogTableAccessor.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/slowlog/SlowLogTableAccessor.java @@ -22,7 +22,8 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.NamespaceDescriptor; @@ -49,8 +50,6 @@ public class SlowLogTableAccessor { private static final Logger LOG = LoggerFactory.getLogger(SlowLogTableAccessor.class); - private static final Random RANDOM = new Random(); - private static Connection connection; /** @@ -139,7 +138,7 @@ private static byte[] getRowKey(final TooSlowLog.SlowLogPayload slowLogPayload) String lastFiveDig = hashcode.substring((hashcode.length() > 5) ? (hashcode.length() - 5) : 0); if (lastFiveDig.startsWith("-")) { - lastFiveDig = String.valueOf(RANDOM.nextInt(99999)); + lastFiveDig = String.valueOf(ThreadLocalRandom.current().nextInt(99999)); } final long currentTime = EnvironmentEdgeManager.currentTime(); final String timeAndHashcode = currentTime + lastFiveDig; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestEncryptionUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestEncryptionUtil.java index 9275dc9e763c..cf5939031b02 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestEncryptionUtil.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestEncryptionUtil.java @@ -23,7 +23,7 @@ import java.security.Key; import java.security.KeyException; -import java.security.SecureRandom; + import javax.crypto.spec.SecretKeySpec; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -110,7 +110,7 @@ public void testWALKeyWrappingWithIncorrectKey() throws Exception { // generate a test key byte[] keyBytes = new byte[AES.KEY_LENGTH]; - new SecureRandom().nextBytes(keyBytes); + Bytes.secureRandom(keyBytes); String algorithm = conf.get(HConstants.CRYPTO_WAL_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES); Key key = new SecretKeySpec(keyBytes, algorithm); @@ -152,7 +152,7 @@ private void testKeyWrapping(String hashAlgorithm) throws Exception { // generate a test key byte[] keyBytes = new byte[AES.KEY_LENGTH]; - new SecureRandom().nextBytes(keyBytes); + Bytes.secureRandom(keyBytes); String algorithm = conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES); Key key = new SecretKeySpec(keyBytes, algorithm); @@ -189,7 +189,7 @@ private void testWALKeyWrapping(String hashAlgorithm) throws Exception { // generate a test key byte[] keyBytes = new byte[AES.KEY_LENGTH]; - new SecureRandom().nextBytes(keyBytes); + Bytes.secureRandom(keyBytes); String algorithm = conf.get(HConstants.CRYPTO_WAL_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES); Key key = new SecretKeySpec(keyBytes, algorithm); @@ -214,7 +214,7 @@ private void testKeyWrappingWithMismatchingAlgorithms(Configuration conf) throws // generate a test key byte[] keyBytes = new byte[AES.KEY_LENGTH]; - new SecureRandom().nextBytes(keyBytes); + Bytes.secureRandom(keyBytes); String algorithm = conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES); Key key = new SecretKeySpec(keyBytes, algorithm); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestRoundRobinPoolMap.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestRoundRobinPoolMap.java index ef7cb4e6512a..2fd73caea46a 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestRoundRobinPoolMap.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestRoundRobinPoolMap.java @@ -25,11 +25,9 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.Random; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; -import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestThreadLocalPoolMap.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestThreadLocalPoolMap.java index a1cb610e8544..2f497c6fdfb5 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestThreadLocalPoolMap.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestThreadLocalPoolMap.java @@ -20,11 +20,9 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; -import java.util.Random; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; -import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Encryption.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Encryption.java index 807758958d0a..6cb0b073e0f0 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Encryption.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Encryption.java @@ -317,7 +317,7 @@ public static byte[] generateSecretKey(Configuration conf, String cypherAlg, byt */ private static byte[] generateSecretKey(String algorithm, int keyLengthBytes, char[] password) { byte[] salt = new byte[keyLengthBytes]; - Bytes.random(salt); + Bytes.secureRandom(salt); PBEKeySpec spec = new PBEKeySpec(password, salt, 10000, keyLengthBytes*8); try { return SecretKeyFactory.getInstance(algorithm).generateSecret(spec).getEncoded(); diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDefaultEncodingContext.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDefaultEncodingContext.java index 8d9e6824fa43..25b946a143f0 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDefaultEncodingContext.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/HFileBlockDefaultEncodingContext.java @@ -21,7 +21,6 @@ import java.io.DataOutputStream; import java.io.IOException; import java.io.InputStream; -import java.security.SecureRandom; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.io.ByteArrayOutputStream; @@ -110,7 +109,7 @@ public HFileBlockDefaultEncodingContext(Configuration conf, DataBlockEncoding en if (cryptoContext != Encryption.Context.NONE) { cryptoByteStream = new ByteArrayOutputStream(); iv = new byte[cryptoContext.getCipher().getIvLength()]; - new SecureRandom().nextBytes(iv); + Bytes.secureRandom(iv); } dummyHeader = Preconditions.checkNotNull(headerBytes, diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java index ce24694e7fdc..0c96dcc3c1a8 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java @@ -37,6 +37,8 @@ import java.util.Comparator; import java.util.Iterator; import java.util.List; +import java.util.Random; + import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.io.RawComparator; @@ -2358,11 +2360,16 @@ public static void zero(byte[] b, int offset, int length) { Arrays.fill(b, offset, offset + length, (byte) 0); } - private static final SecureRandom RNG = new SecureRandom(); + // Pseudorandom random number generator, do not use SecureRandom here + private static final Random RNG = new Random(); /** * Fill given array with random bytes. * @param b array which needs to be filled with random bytes + *

+ * If you want random bytes generated by a strong source of randomness use {@link + * Bytes#secureRandom(byte[])}. + * @param b array which needs to be filled with random bytes */ public static void random(byte[] b) { RNG.nextBytes(b); @@ -2370,9 +2377,12 @@ public static void random(byte[] b) { /** * Fill given array with random bytes at the specified position. - * @param b - * @param offset - * @param length + *

+ * If you want random bytes generated by a strong source of randomness use {@link + * Bytes#secureRandom(byte[], int, int)}. + * @param b array which needs to be filled with random bytes + * @param offset staring offset in array + * @param length number of bytes to fill */ public static void random(byte[] b, int offset, int length) { checkPositionIndex(offset, b.length, "offset"); @@ -2383,6 +2393,33 @@ public static void random(byte[] b, int offset, int length) { System.arraycopy(buf, 0, b, offset, length); } + // Bytes.secureRandom may be used to create key material. + private static final SecureRandom SECURE_RNG = new SecureRandom(); + + /** + * Fill given array with random bytes using a strong random number generator. + * @param b array which needs to be filled with random bytes + */ + public static void secureRandom(byte[] b) { + SECURE_RNG.nextBytes(b); + } + + /** + * Fill given array with random bytes at the specified position using a strong random number + * generator. + * @param b array which needs to be filled with random bytes + * @param offset staring offset in array + * @param length number of bytes to fill + */ + public static void secureRandom(byte[] b, int offset, int length) { + checkPositionIndex(offset, b.length, "offset"); + checkArgument(length > 0, "length must be greater than 0"); + checkPositionIndex(offset + length, b.length, "offset + length"); + byte[] buf = new byte[length]; + SECURE_RNG.nextBytes(buf); + System.arraycopy(buf, 0, b, offset, length); + } + /** * Create a max byte array with the specified max byte count * @param maxByteCount the length of returned byte array diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtil.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtil.java index 1c8f8b945335..fa5a8dea3ec2 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtil.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtil.java @@ -23,7 +23,6 @@ import java.util.Arrays; import java.util.HashSet; import java.util.List; -import java.util.Random; import java.util.Set; import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; @@ -255,10 +254,7 @@ public long waitFor(long timeout, long interval, boolean f return Waiter.waitFor(this.conf, timeout, interval, failIfTimeout, predicate); } - // Support for Random Port Generation. - static Random random = new Random(); - - private static final PortAllocator portAllocator = new PortAllocator(random); + private static final PortAllocator portAllocator = new PortAllocator(); public static int randomFreePort() { return portAllocator.randomFreePort(); @@ -271,11 +267,9 @@ static class PortAllocator { /** A set of ports that have been claimed using {@link #randomFreePort()}. */ private final Set takenRandomPorts = new HashSet<>(); - private final Random random; private final AvailablePortChecker portChecker; - public PortAllocator(Random random) { - this.random = random; + public PortAllocator() { this.portChecker = new AvailablePortChecker() { @Override public boolean available(int port) { @@ -290,8 +284,7 @@ public boolean available(int port) { }; } - public PortAllocator(Random random, AvailablePortChecker portChecker) { - this.random = random; + public PortAllocator(AvailablePortChecker portChecker) { this.portChecker = portChecker; } @@ -321,7 +314,8 @@ public int randomFreePort() { * dynamic allocation (see http://bit.ly/dynports). */ private int randomPort() { - return MIN_RANDOM_PORT + random.nextInt(MAX_RANDOM_PORT - MIN_RANDOM_PORT); + return MIN_RANDOM_PORT + + ThreadLocalRandom.current().nextInt(MAX_RANDOM_PORT - MIN_RANDOM_PORT); } interface AvailablePortChecker { diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestEncryption.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestEncryption.java index 8d850a7aa4e2..d8057856f625 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestEncryption.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/crypto/TestEncryption.java @@ -48,9 +48,9 @@ public class TestEncryption { @Test public void testSmallBlocks() throws Exception { byte[] key = new byte[16]; - Bytes.random(key); + Bytes.secureRandom(key); byte[] iv = new byte[16]; - Bytes.random(iv); + Bytes.secureRandom(iv); for (int size: new int[] { 4, 8, 16, 32, 64, 128, 256, 512 }) { checkTransformSymmetry(key, iv, getRandomBlock(size)); } @@ -59,9 +59,9 @@ public void testSmallBlocks() throws Exception { @Test public void testLargeBlocks() throws Exception { byte[] key = new byte[16]; - Bytes.random(key); + Bytes.secureRandom(key); byte[] iv = new byte[16]; - Bytes.random(iv); + Bytes.secureRandom(iv); for (int size: new int[] { 256 * 1024, 512 * 1024, 1024 * 1024 }) { checkTransformSymmetry(key, iv, getRandomBlock(size)); } @@ -70,9 +70,9 @@ public void testLargeBlocks() throws Exception { @Test public void testOddSizedBlocks() throws Exception { byte[] key = new byte[16]; - Bytes.random(key); + Bytes.secureRandom(key); byte[] iv = new byte[16]; - Bytes.random(iv); + Bytes.secureRandom(iv); for (int size: new int[] { 3, 7, 11, 23, 47, 79, 119, 175 }) { checkTransformSymmetry(key, iv, getRandomBlock(size)); } @@ -81,9 +81,9 @@ public void testOddSizedBlocks() throws Exception { @Test public void testTypicalHFileBlocks() throws Exception { byte[] key = new byte[16]; - Bytes.random(key); + Bytes.secureRandom(key); byte[] iv = new byte[16]; - Bytes.random(iv); + Bytes.secureRandom(iv); for (int size: new int[] { 4 * 1024, 8 * 1024, 64 * 1024, 128 * 1024 }) { checkTransformSymmetry(key, iv, getRandomBlock(size)); } diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/util/TestLRUDictionary.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/util/TestLRUDictionary.java index bc8af936164e..8452f3ee8165 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/util/TestLRUDictionary.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/util/TestLRUDictionary.java @@ -23,7 +23,7 @@ import java.math.BigInteger; import java.util.Arrays; -import java.util.Random; + import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -81,9 +81,8 @@ public void testPassingSameArrayToAddEntry() { @Test public void testBasic() { - Random rand = new Random(); byte[] testBytes = new byte[10]; - rand.nextBytes(testBytes); + Bytes.random(testBytes); // Verify that our randomly generated array doesn't exist in the dictionary assertEquals(-1, testee.findEntry(testBytes, 0, testBytes.length)); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/LoadTestKVGenerator.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/LoadTestKVGenerator.java index ad49e5537196..174cd4c37fbf 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/LoadTestKVGenerator.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/LoadTestKVGenerator.java @@ -36,7 +36,7 @@ public class LoadTestKVGenerator { private static int logLimit = 10; /** A random number generator for determining value size */ - private Random randomForValueSize = new Random(); + private Random randomForValueSize = new Random(); // Seed may be set with Random#setSeed private final int minValueSize; private final int maxValueSize; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestAvlUtil.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestAvlUtil.java index 0499a04a9007..b006f638fafb 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestAvlUtil.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestAvlUtil.java @@ -21,8 +21,11 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; + import java.util.Random; import java.util.TreeMap; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -54,7 +57,7 @@ public void testAvlTreeCrud() { final TreeMap treeMap = new TreeMap<>(); TestAvlNode root = null; - final Random rand = new Random(); + Random rand = ThreadLocalRandom.current(); for (int i = 0; i < NELEM; ++i) { int key = rand.nextInt(MAX_KEY); if (AvlTree.get(root, key, KEY_COMPARATOR) != null) { diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java index 8384b055cf27..0934581decf9 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java @@ -24,7 +24,7 @@ import java.io.IOException; import java.nio.ByteBuffer; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.nio.ByteBuff; @@ -39,8 +39,6 @@ @Category({ MiscTests.class, SmallTests.class }) public class TestByteBufferArray { - private static final Random RANDOM = new Random(EnvironmentEdgeManager.currentTime()); - @ClassRule public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestByteBufferArray.class); @@ -87,7 +85,7 @@ private static void fill(ByteBuff buf, byte val) { private ByteBuff createByteBuff(int len) { assert len >= 0; - int pos = len == 0 ? 0 : RANDOM.nextInt(len); + int pos = len == 0 ? 0 : ThreadLocalRandom.current().nextInt(len); ByteBuff b = ByteBuff.wrap(ByteBuffer.allocate(2 * len)); b.position(pos).limit(pos + len); return b; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestBytes.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestBytes.java index 3027e46a9fd3..80e2c354865e 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestBytes.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestBytes.java @@ -38,6 +38,8 @@ import java.util.Arrays; import java.util.List; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -386,14 +388,11 @@ public void testBinarySearch() { @Test public void testToStringBytesBinaryReversible() { - // let's run test with 1000 randomly generated byte arrays - Random rand = new Random(EnvironmentEdgeManager.currentTime()); byte[] randomBytes = new byte[1000]; for (int i = 0; i < 1000; i++) { - rand.nextBytes(randomBytes); + Bytes.random(randomBytes); verifyReversibleForBytes(randomBytes); } - // some specific cases verifyReversibleForBytes(new byte[] {}); verifyReversibleForBytes(new byte[] {'\\', 'x', 'A', 'D'}); @@ -598,10 +597,10 @@ public void testToFromHex() { List testByteData = new ArrayList<>(5); testByteData.addAll(Arrays.asList(new byte[0], new byte[1], new byte[10], new byte[] { 1, 2, 3, 4, 5 }, new byte[] { (byte) 0xFF })); - Random r = new Random(); + Random rand = ThreadLocalRandom.current(); for (int i = 0; i < 20; i++) { - byte[] bytes = new byte[r.nextInt(100)]; - r.nextBytes(bytes); + byte[] bytes = new byte[rand.nextInt(100)]; + Bytes.random(bytes); testByteData.add(bytes); } diff --git a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/TestCompatibilitySingletonFactory.java b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/TestCompatibilitySingletonFactory.java index a57c935651ae..27888db0f6d2 100644 --- a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/TestCompatibilitySingletonFactory.java +++ b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/TestCompatibilitySingletonFactory.java @@ -22,7 +22,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.Random; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -41,13 +40,13 @@ public class TestCompatibilitySingletonFactory { HBaseClassTestRule.forClass(TestCompatibilitySingletonFactory.class); private static final int ITERATIONS = 100000; - private static final Random RANDOM = new Random(); private class TestCompatibilitySingletonFactoryCallable implements Callable { @Override public String call() throws Exception { - Thread.sleep(RANDOM.nextInt(10)); + // XXX: Why is this sleep here? + Thread.sleep(10); RandomStringGenerator instance = CompatibilitySingletonFactory.getInstance(RandomStringGenerator.class); diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestServletFilter.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestServletFilter.java index cddfb1b79202..1e9a2861c9ef 100644 --- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestServletFilter.java +++ b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestServletFilter.java @@ -19,6 +19,8 @@ import java.io.IOException; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import javax.servlet.Filter; import javax.servlet.FilterChain; import javax.servlet.FilterConfig; @@ -114,12 +116,12 @@ public void testServletFilter() throws Exception { final String hadooplogoURL = "/static/hadoop-logo.jpg"; final String[] urls = {fsckURL, stacksURL, ajspURL, logURL, hadooplogoURL}; - final Random ran = new Random(); + final Random rand = ThreadLocalRandom.current(); final int[] sequence = new int[50]; //generate a random sequence and update counts for(int i = 0; i < sequence.length; i++) { - sequence[i] = ran.nextInt(urls.length); + sequence[i] = rand.nextInt(urls.length); } //access the urls as the sequence diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java index b2d90f11bd53..2b0542142b3c 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java @@ -23,10 +23,9 @@ import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; -import org.apache.commons.lang3.RandomStringUtils; -import org.apache.commons.lang3.RandomUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -261,7 +260,8 @@ protected NamespaceDescriptor selectNamespace( return null; } ArrayList namespaceList = new ArrayList<>(namespaceMap.keySet()); - String randomKey = namespaceList.get(RandomUtils.nextInt(0, namespaceList.size())); + String randomKey = namespaceList.get(ThreadLocalRandom.current() + .nextInt(namespaceList.size())); NamespaceDescriptor randomNsd = namespaceMap.get(randomKey); // remove from namespaceMap namespaceMap.remove(randomKey); @@ -310,12 +310,12 @@ void perform() throws IOException { private NamespaceDescriptor createNamespaceDesc() { String namespaceName = "itnamespace" + String.format("%010d", - RandomUtils.nextInt()); + ThreadLocalRandom.current().nextInt()); NamespaceDescriptor nsd = NamespaceDescriptor.create(namespaceName).build(); nsd.setConfiguration( nsTestConfigKey, - String.format("%010d", RandomUtils.nextInt())); + String.format("%010d", ThreadLocalRandom.current().nextInt())); return nsd; } } @@ -335,7 +335,7 @@ void perform() throws IOException { NamespaceDescriptor modifiedNsd = NamespaceDescriptor.create(namespaceName).build(); String nsValueNew; do { - nsValueNew = String.format("%010d", RandomUtils.nextInt()); + nsValueNew = String.format("%010d", ThreadLocalRandom.current().nextInt()); } while (selected.getConfigurationValue(nsTestConfigKey).equals(nsValueNew)); modifiedNsd.setConfiguration(nsTestConfigKey, nsValueNew); admin.modifyNamespace(modifiedNsd); @@ -401,8 +401,8 @@ protected TableDescriptor selectTable(ConcurrentHashMap tableList = new ArrayList<>(tableMap.keySet()); - TableName randomKey = tableList.get(RandomUtils.nextInt(0, tableList.size())); - TableDescriptor randomTd = tableMap.remove(randomKey); + TableName key = tableList.get(ThreadLocalRandom.current().nextInt(tableList.size())); + TableDescriptor randomTd = tableMap.remove(key); return randomTd; } } @@ -440,8 +440,9 @@ void perform() throws IOException { } private TableDescriptor createTableDesc() { - String tableName = String.format("ittable-%010d", RandomUtils.nextInt()); - String familyName = "cf-" + Math.abs(RandomUtils.nextInt()); + String tableName = String.format("ittable-%010d", + ThreadLocalRandom.current().nextInt(Integer.MAX_VALUE)); + String familyName = "cf-" + ThreadLocalRandom.current().nextInt(Integer.MAX_VALUE); return TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName)) .setColumnFamily(ColumnFamilyDescriptorBuilder.of(familyName)) .build(); @@ -585,8 +586,7 @@ protected ColumnFamilyDescriptor selectFamily(TableDescriptor td) { LOG.info("No column families in table: " + td); return null; } - ColumnFamilyDescriptor randomCfd = families[RandomUtils.nextInt(0, families.length)]; - return randomCfd; + return families[ThreadLocalRandom.current().nextInt(families.length)]; } } @@ -603,7 +603,7 @@ void perform() throws IOException { try { ColumnFamilyDescriptor cfd = createFamilyDesc(); if (selected.hasColumnFamily(cfd.getName())){ - LOG.info(new String(cfd.getName()) + " already exists in table " + LOG.info(Bytes.toString(cfd.getName()) + " already exists in table " + selected.getTableName()); return; } @@ -628,7 +628,7 @@ void perform() throws IOException { } private ColumnFamilyDescriptor createFamilyDesc() { - String familyName = String.format("cf-%010d", RandomUtils.nextInt()); + String familyName = String.format("cf-%010d", ThreadLocalRandom.current().nextInt()); return ColumnFamilyDescriptorBuilder.of(familyName); } } @@ -647,7 +647,7 @@ void perform() throws IOException { } Admin admin = connection.getAdmin(); - int versions = RandomUtils.nextInt(0, 10) + 3; + int versions = ThreadLocalRandom.current().nextInt(10) + 3; try { TableName tableName = selected.getTableName(); LOG.info("Altering versions of column family: " + columnDesc + " to: " + versions + @@ -703,7 +703,7 @@ void perform() throws IOException { // possible DataBlockEncoding ids DataBlockEncoding[] possibleIds = {DataBlockEncoding.NONE, DataBlockEncoding.PREFIX, DataBlockEncoding.DIFF, DataBlockEncoding.FAST_DIFF, DataBlockEncoding.ROW_INDEX_V1}; - short id = possibleIds[RandomUtils.nextInt(0, possibleIds.length)].getId(); + short id = possibleIds[ThreadLocalRandom.current().nextInt(possibleIds.length)].getId(); LOG.info("Altering encoding of column family: " + columnDesc + " to: " + id + " in table: " + tableName); @@ -791,17 +791,18 @@ void perform() throws IOException { int average_rows = 1; int numRows = average_rows * numRegions; LOG.info("Adding " + numRows + " rows to table: " + selected); + byte[] value = new byte[10]; for (int i = 0; i < numRows; i++){ // nextInt(Integer.MAX_VALUE)) to return positive numbers only byte[] rowKey = Bytes.toBytes( - "row-" + String.format("%010d", RandomUtils.nextInt())); + "row-" + String.format("%010d", ThreadLocalRandom.current().nextInt())); ColumnFamilyDescriptor cfd = selectFamily(selected); if (cfd == null){ return; } byte[] family = cfd.getName(); - byte[] qualifier = Bytes.toBytes("col-" + RandomUtils.nextInt() % 10); - byte[] value = Bytes.toBytes("val-" + RandomStringUtils.randomAlphanumeric(10)); + byte[] qualifier = Bytes.toBytes("col-" + ThreadLocalRandom.current().nextInt(10)); + Bytes.random(value); Put put = new Put(rowKey); put.addColumn(family, qualifier, value); table.put(put); @@ -845,7 +846,8 @@ private class Worker extends Thread { public void run() { while (running.get()) { // select random action - ACTION selectedAction = ACTION.values()[RandomUtils.nextInt() % ACTION.values().length]; + ACTION selectedAction = + ACTION.values()[ThreadLocalRandom.current().nextInt(ACTION.values().length)]; this.action = selectedAction; LOG.info("Performing Action: " + selectedAction); @@ -878,7 +880,7 @@ public void run() { break; case DELETE_TABLE: // reduce probability of deleting table to 20% - if (RandomUtils.nextInt(0, 100) < 20) { + if (ThreadLocalRandom.current().nextInt(100) < 20) { new DeleteTableAction().perform(); } break; @@ -887,7 +889,7 @@ public void run() { break; case DELETE_COLUMNFAMILY: // reduce probability of deleting column family to 20% - if (RandomUtils.nextInt(0, 100) < 20) { + if (ThreadLocalRandom.current().nextInt(100) < 20) { new DeleteColumnFamilyAction().perform(); } break; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java index a4209a94e063..76bc876b9178 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java @@ -26,10 +26,11 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.Random; import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; import java.util.function.BiConsumer; import java.util.function.Consumer; -import org.apache.commons.lang3.RandomUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ClusterMetrics; import org.apache.hadoop.hbase.HBaseClusterInterface; @@ -286,8 +287,9 @@ protected void unbalanceRegions(ClusterMetrics clusterStatus, List regions = new LinkedList<>(serverLoad.getRegionMetrics().keySet()); int victimRegionCount = (int)Math.ceil(fractionOfRegions * regions.size()); getLogger().debug("Removing {} regions from {}", victimRegionCount, sn); + Random rand = ThreadLocalRandom.current(); for (int i = 0; i < victimRegionCount; ++i) { - int victimIx = RandomUtils.nextInt(0, regions.size()); + int victimIx = rand.nextInt(regions.size()); String regionId = RegionInfo.encodeRegionName(regions.remove(victimIx)); victimRegions.add(Bytes.toBytes(regionId)); } @@ -296,13 +298,14 @@ protected void unbalanceRegions(ClusterMetrics clusterStatus, getLogger().info("Moving {} regions from {} servers to {} different servers", victimRegions.size(), fromServers.size(), toServers.size()); Admin admin = this.context.getHBaseIntegrationTestingUtility().getAdmin(); + Random rand = ThreadLocalRandom.current(); for (byte[] victimRegion : victimRegions) { // Don't keep moving regions if we're // trying to stop the monkey. if (context.isStopping()) { break; } - int targetIx = RandomUtils.nextInt(0, toServers.size()); + int targetIx = rand.nextInt(toServers.size()); admin.move(victimRegion, toServers.get(targetIx)); } } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeBloomFilterAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeBloomFilterAction.java index 18ea664eb6b9..773f3588d0a9 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeBloomFilterAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeBloomFilterAction.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.chaos.actions; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.util.BloomFilterUtil; @@ -49,14 +49,13 @@ public ChangeBloomFilterAction(int sleepTime, TableName tableName) { @Override public void perform() throws Exception { - final Random random = new Random(); final BloomType[] bloomArray = BloomType.values(); final int bloomArraySize = bloomArray.length; getLogger().info("Performing action: Change bloom filter on all columns of table " + tableName); modifyAllTableColumns(tableName, (columnName, columnBuilder) -> { - BloomType bloomType = bloomArray[random.nextInt(bloomArraySize)]; + BloomType bloomType = bloomArray[ThreadLocalRandom.current().nextInt(bloomArraySize)]; getLogger().debug("Performing action: About to set bloom filter type to " + bloomType + " on column " + columnName + " of table " + tableName); columnBuilder.setBloomFilterType(bloomType); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeCompressionAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeCompressionAction.java index 8828bc8c1a23..d481d9bfae33 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeCompressionAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeCompressionAction.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; import org.apache.hadoop.io.compress.Compressor; @@ -31,12 +32,10 @@ */ public class ChangeCompressionAction extends Action { private final TableName tableName; - private final Random random; private static final Logger LOG = LoggerFactory.getLogger(ChangeCompressionAction.class); public ChangeCompressionAction(TableName tableName) { this.tableName = tableName; - this.random = new Random(); } @Override protected Logger getLogger() { @@ -48,16 +47,15 @@ public void perform() throws IOException { // Possible compression algorithms. If an algorithm is not supported, // modifyTable will fail, so there is no harm. Algorithm[] possibleAlgos = Algorithm.values(); - // Since not every compression algorithm is supported, // let's use the same algorithm for all column families. - + Random rand = ThreadLocalRandom.current(); // If an unsupported compression algorithm is chosen, pick a different one. // This is to work around the issue that modifyTable() does not throw remote // exception. Algorithm algo; do { - algo = possibleAlgos[random.nextInt(possibleAlgos.length)]; + algo = possibleAlgos[rand.nextInt(possibleAlgos.length)]; try { Compressor c = algo.getCompressor(); @@ -75,7 +73,7 @@ public void perform() throws IOException { getLogger().debug("Performing action: Changing compression algorithms on " + tableName.getNameAsString() + " to " + chosenAlgo); modifyAllTableColumns(tableName, columnFamilyDescriptorBuilder -> { - if (random.nextBoolean()) { + if (rand.nextBoolean()) { columnFamilyDescriptorBuilder.setCompactionCompressionType(chosenAlgo); } else { columnFamilyDescriptorBuilder.setCompressionType(chosenAlgo); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeEncodingAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeEncodingAction.java index afa8a2504771..61a9bd9cfa0f 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeEncodingAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeEncodingAction.java @@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.chaos.actions; import java.io.IOException; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.slf4j.Logger; @@ -30,12 +30,10 @@ */ public class ChangeEncodingAction extends Action { private final TableName tableName; - private final Random random; private static final Logger LOG = LoggerFactory.getLogger(ChangeEncodingAction.class); public ChangeEncodingAction(TableName tableName) { this.tableName = tableName; - this.random = new Random(); } @Override protected Logger getLogger() { @@ -47,9 +45,8 @@ public void perform() throws IOException { getLogger().debug("Performing action: Changing encodings on " + tableName); // possible DataBlockEncoding id's final int[] possibleIds = {0, 2, 3, 4, 7}; - modifyAllTableColumns(tableName, (columnName, columnBuilder) -> { - short id = (short) possibleIds[random.nextInt(possibleIds.length)]; + short id = (short) possibleIds[ThreadLocalRandom.current().nextInt(possibleIds.length)]; DataBlockEncoding encoding = DataBlockEncoding.getEncodingById(id); columnBuilder.setDataBlockEncoding(encoding); getLogger().debug("Set encoding of column family " + columnName + " to: " + encoding); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeSplitPolicyAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeSplitPolicyAction.java index 63553a757d63..c5a6257199e8 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeSplitPolicyAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeSplitPolicyAction.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.chaos.actions; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -33,7 +33,6 @@ public class ChangeSplitPolicyAction extends Action { private static final Logger LOG = LoggerFactory.getLogger(ChangeSplitPolicyAction.class); private final TableName tableName; private final String[] possiblePolicies; - private final Random random; public ChangeSplitPolicyAction(TableName tableName) { this.tableName = tableName; @@ -42,7 +41,6 @@ public ChangeSplitPolicyAction(TableName tableName) { ConstantSizeRegionSplitPolicy.class.getName(), DisabledRegionSplitPolicy.class.getName() }; - this.random = new Random(); } @Override protected Logger getLogger() { @@ -53,11 +51,11 @@ public ChangeSplitPolicyAction(TableName tableName) { public void perform() throws Exception { HBaseTestingUtil util = context.getHBaseIntegrationTestingUtility(); Admin admin = util.getAdmin(); - getLogger().info("Performing action: Change split policy of table " + tableName); TableDescriptor tableDescriptor = admin.getDescriptor(tableName); TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableDescriptor); - String chosenPolicy = possiblePolicies[random.nextInt(possiblePolicies.length)]; + String chosenPolicy = + possiblePolicies[ThreadLocalRandom.current().nextInt(possiblePolicies.length)]; builder.setRegionSplitPolicyClassName(chosenPolicy); getLogger().info("Changing " + tableName + " split policy to " + chosenPolicy); admin.modifyTable(builder.build()); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeVersionsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeVersionsAction.java index 1228db1e4957..14a290012f23 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeVersionsAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeVersionsAction.java @@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.chaos.actions; import java.io.IOException; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.TableName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,11 +33,8 @@ public class ChangeVersionsAction extends Action { private static final Logger LOG = LoggerFactory.getLogger(ChangeVersionsAction.class); private final TableName tableName; - private final Random random; - public ChangeVersionsAction(TableName tableName) { this.tableName = tableName; - this.random = new Random(); } @Override protected Logger getLogger() { @@ -46,8 +43,7 @@ public ChangeVersionsAction(TableName tableName) { @Override public void perform() throws IOException { - final int versions = random.nextInt(3) + 1; - + final int versions = ThreadLocalRandom.current().nextInt(3) + 1; getLogger().debug("Performing action: Changing versions on " + tableName + " to " + versions); modifyAllTableColumns(tableName, columnBuilder -> { columnBuilder.setMinVersions(versions).setMaxVersions(versions); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactMobAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactMobAction.java index fe5f5e923444..c501ee060c43 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactMobAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactMobAction.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.chaos.actions; -import org.apache.commons.lang3.RandomUtils; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -39,8 +39,7 @@ public CompactMobAction(TableName tableName, float majorRatio) { this(-1, tableName, majorRatio); } - public CompactMobAction( - int sleepTime, TableName tableName, float majorRatio) { + public CompactMobAction(int sleepTime, TableName tableName, float majorRatio) { this.tableName = tableName; this.majorRatio = (int) (100 * majorRatio); this.sleepTime = sleepTime; @@ -54,7 +53,7 @@ public CompactMobAction( public void perform() throws Exception { HBaseTestingUtil util = context.getHBaseIntegrationTestingUtility(); Admin admin = util.getAdmin(); - boolean major = RandomUtils.nextInt(0, 100) < majorRatio; + boolean major = ThreadLocalRandom.current().nextInt(100) < majorRatio; // Don't try the modify if we're stopping if (context.isStopping()) { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java index 9ed876dbaeb1..e92b740b003b 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java @@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.chaos.actions; import java.util.List; -import org.apache.commons.lang3.RandomUtils; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey; @@ -38,13 +38,11 @@ public class CompactRandomRegionOfTableAction extends Action { private final long sleepTime; private final TableName tableName; - public CompactRandomRegionOfTableAction( - TableName tableName, float majorRatio) { + public CompactRandomRegionOfTableAction(TableName tableName, float majorRatio) { this(-1, tableName, majorRatio); } - public CompactRandomRegionOfTableAction( - int sleepTime, TableName tableName, float majorRatio) { + public CompactRandomRegionOfTableAction(int sleepTime, TableName tableName, float majorRatio) { this.majorRatio = (int) (100 * majorRatio); this.sleepTime = sleepTime; this.tableName = tableName; @@ -58,7 +56,7 @@ public CompactRandomRegionOfTableAction( public void perform() throws Exception { HBaseTestingUtil util = context.getHBaseIntegrationTestingUtility(); Admin admin = util.getAdmin(); - boolean major = RandomUtils.nextInt(0, 100) < majorRatio; + boolean major = ThreadLocalRandom.current().nextInt(100) < majorRatio; getLogger().info("Performing action: Compact random region of table " + tableName + ", major=" + major); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactTableAction.java index 7d57b3b6b30f..168300f15cc3 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactTableAction.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.chaos.actions; -import org.apache.commons.lang3.RandomUtils; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -54,7 +54,7 @@ public CompactTableAction( public void perform() throws Exception { HBaseTestingUtil util = context.getHBaseIntegrationTestingUtility(); Admin admin = util.getAdmin(); - boolean major = RandomUtils.nextInt(0, 100) < majorRatio; + boolean major = ThreadLocalRandom.current().nextInt(100) < majorRatio; getLogger().info("Performing action: Compact table " + tableName + ", major=" + major); try { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CorruptDataFilesAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CorruptDataFilesAction.java index 3b81f2e617cd..616abb2cca1d 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CorruptDataFilesAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CorruptDataFilesAction.java @@ -18,7 +18,8 @@ package org.apache.hadoop.hbase.chaos.actions; -import org.apache.commons.lang3.RandomUtils; +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; @@ -56,15 +57,15 @@ public void perform() throws Exception { Path rootDir = CommonFSUtils.getRootDir(getConf()); Path defaultDir = rootDir.suffix("/data/default"); RemoteIterator iterator = fs.listFiles(defaultDir, true); + Random rand = ThreadLocalRandom.current(); while (iterator.hasNext()){ LocatedFileStatus status = iterator.next(); if(!HFile.isHFileFormat(fs, status.getPath())){ continue; } - if(RandomUtils.nextFloat(0, 100) > chance){ + if ((100 * rand.nextFloat()) > chance){ continue; } - FSDataOutputStream out = fs.create(status.getPath(), true); try { out.write(0); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DecreaseMaxHFileSizeAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DecreaseMaxHFileSizeAction.java index 0764d008cc34..a384719a51ce 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DecreaseMaxHFileSizeAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DecreaseMaxHFileSizeAction.java @@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.chaos.actions; import java.io.IOException; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -35,13 +35,11 @@ public class DecreaseMaxHFileSizeAction extends Action { private final long sleepTime; private final TableName tableName; - private final Random random; private Admin admin; public DecreaseMaxHFileSizeAction(long sleepTime, TableName tableName) { this.sleepTime = sleepTime; this.tableName = tableName; - this.random = new Random(); } @Override protected Logger getLogger() { @@ -75,7 +73,8 @@ public void perform() throws Exception { // We don't want to go too far below 1gb. // So go to about 1gb +/- 512 on each side. - newValue = Math.max(minFileSize, newValue) - (512 - random.nextInt(1024)); + newValue = Math.max(minFileSize, newValue) - + (512 - ThreadLocalRandom.current().nextInt(1024)); // Change the table descriptor. TableDescriptor modifiedTable = diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DeleteDataFilesAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DeleteDataFilesAction.java index 2b869706785a..ee4f7deb4611 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DeleteDataFilesAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DeleteDataFilesAction.java @@ -18,7 +18,8 @@ package org.apache.hadoop.hbase.chaos.actions; -import org.apache.commons.lang3.RandomUtils; +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; @@ -54,12 +55,13 @@ public void perform() throws Exception { Path rootDir = CommonFSUtils.getRootDir(getConf()); Path defaultDir = rootDir.suffix("/data/default"); RemoteIterator iterator = fs.listFiles(defaultDir, true); + Random rand = ThreadLocalRandom.current(); while (iterator.hasNext()){ LocatedFileStatus status = iterator.next(); if(!HFile.isHFileFormat(fs, status.getPath())){ continue; } - if(RandomUtils.nextFloat(0, 100) > chance){ + if ((100 * rand.nextFloat()) > chance){ continue; } fs.delete(status.getPath(), true); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/GracefulRollingRestartRsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/GracefulRollingRestartRsAction.java index f4ef7978c974..93bcf6b3a1d4 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/GracefulRollingRestartRsAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/GracefulRollingRestartRsAction.java @@ -21,7 +21,8 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; -import org.apache.commons.lang3.RandomUtils; +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.util.RegionMover; import org.apache.hadoop.util.Shell; @@ -47,10 +48,9 @@ public GracefulRollingRestartRsAction(long sleepTime) { public void perform() throws Exception { getLogger().info("Performing action: Rolling restarting non-master region servers"); List selectedServers = selectServers(); - getLogger().info("Disabling balancer to make unloading possible"); setBalancer(false, true); - + Random rand = ThreadLocalRandom.current(); for (ServerName server : selectedServers) { String rsName = server.getAddress().toString(); try (RegionMover rm = @@ -64,7 +64,7 @@ public void perform() throws Exception { } catch (Shell.ExitCodeException e) { getLogger().info("Problem restarting but presume successful; code={}", e.getExitCode(), e); } - sleep(RandomUtils.nextInt(0, (int)sleepTime)); + sleep(rand.nextInt((int)sleepTime)); } getLogger().info("Enabling balancer"); setBalancer(true, true); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java index 2e359ed11617..8594bb40a0d1 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java @@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.chaos.actions; import java.util.List; -import org.apache.commons.lang3.RandomUtils; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -61,7 +61,7 @@ public void perform() throws Exception { return; } - int i = RandomUtils.nextInt(0, regions.size() - 1); + int i = ThreadLocalRandom.current().nextInt(regions.size() - 1); RegionInfo a = regions.get(i++); RegionInfo b = regions.get(i); getLogger().debug("Merging " + a.getRegionNameAsString() + " and " + b.getRegionNameAsString()); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java index a1cab787e390..94b6ae68fe3f 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java @@ -22,7 +22,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; -import org.apache.commons.lang3.RandomUtils; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -93,9 +93,10 @@ public void perform() throws Exception { return serversList.toArray(new ServerName[0]); } - static void moveRegion(Admin admin, ServerName [] servers, RegionInfo regionInfo, Logger logger) { + static void moveRegion(Admin admin, ServerName [] servers, RegionInfo regionInfo, + Logger logger) { try { - ServerName destServerName = servers[RandomUtils.nextInt(0, servers.length)]; + ServerName destServerName = servers[ThreadLocalRandom.current().nextInt(servers.length)]; logger.debug("Moving {} to {}", regionInfo.getRegionNameAsString(), destServerName); admin.move(regionInfo.getEncodedNameAsBytes(), destServerName); } catch (Exception ex) { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RemoveColumnAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RemoveColumnAction.java index a4361f7ce260..ed5cc6044339 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RemoveColumnAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RemoveColumnAction.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Random; import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -39,12 +40,10 @@ public class RemoveColumnAction extends Action { private final TableName tableName; private final Set protectedColumns; private Admin admin; - private final Random random; public RemoveColumnAction(TableName tableName, Set protectedColumns) { this.tableName = tableName; this.protectedColumns = protectedColumns; - random = new Random(); } @Override protected Logger getLogger() { @@ -61,15 +60,15 @@ public void init(ActionContext context) throws IOException { public void perform() throws Exception { TableDescriptor tableDescriptor = admin.getDescriptor(tableName); ColumnFamilyDescriptor[] columnDescriptors = tableDescriptor.getColumnFamilies(); + Random rand = ThreadLocalRandom.current(); if (columnDescriptors.length <= (protectedColumns == null ? 1 : protectedColumns.size())) { return; } - - int index = random.nextInt(columnDescriptors.length); + int index = rand.nextInt(columnDescriptors.length); while(protectedColumns != null && protectedColumns.contains(columnDescriptors[index].getNameAsString())) { - index = random.nextInt(columnDescriptors.length); + index = rand.nextInt(columnDescriptors.length); } byte[] colDescName = columnDescriptors[index].getName(); getLogger().debug("Performing action: Removing " + Bytes.toString(colDescName)+ " from " diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java index fcc533396278..04f8d86793aa 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java @@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.chaos.actions; import java.util.List; -import org.apache.commons.lang3.RandomUtils; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.client.RegionLocator; import org.slf4j.Logger; @@ -46,8 +46,9 @@ public RestartRsHoldingTableAction(long sleepTime, RegionLocator locator) { public void perform() throws Exception { getLogger().info( "Performing action: Restart random RS holding table " + this.locator.getName()); - List locations = locator.getAllRegionLocations(); - restartRs(locations.get(RandomUtils.nextInt(0, locations.size())).getServerName(), sleepTime); + restartRs(locations.get(ThreadLocalRandom.current().nextInt(locations.size())) + .getServerName(), + sleepTime); } } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java index c25a6b31f4b9..f54d5c1216c1 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java @@ -25,7 +25,8 @@ import java.util.List; import java.util.Objects; import java.util.Queue; -import org.apache.commons.lang3.RandomUtils; +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey; import org.slf4j.Logger; @@ -70,10 +71,9 @@ public void perform() throws Exception { getLogger().info("Performing action: Rolling batch restarting {}% of region servers", (int)(ratio * 100)); List selectedServers = selectServers(); - Queue serversToBeKilled = new LinkedList<>(selectedServers); LinkedList deadServers = new LinkedList<>(); - + Random rand = ThreadLocalRandom.current(); // loop while there are servers to be killed or dead servers to be restarted while ((!serversToBeKilled.isEmpty() || !deadServers.isEmpty()) && !context.isStopping()) { @@ -87,7 +87,7 @@ public void perform() throws Exception { action = KillOrStart.START; } else { // do a coin toss - action = RandomUtils.nextBoolean() ? KillOrStart.KILL : KillOrStart.START; + action = rand.nextBoolean() ? KillOrStart.KILL : KillOrStart.START; } ServerName server; @@ -120,7 +120,7 @@ public void perform() throws Exception { break; } - sleep(RandomUtils.nextInt(0, (int)sleepTime)); + sleep(rand.nextInt((int)sleepTime)); } } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchSuspendResumeRsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchSuspendResumeRsAction.java index dfae09af2557..4c15f66f9c4b 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchSuspendResumeRsAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchSuspendResumeRsAction.java @@ -22,7 +22,8 @@ import java.util.LinkedList; import java.util.List; import java.util.Queue; -import org.apache.commons.lang3.RandomUtils; +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey; import org.apache.hadoop.hbase.util.Threads; @@ -66,10 +67,9 @@ public void perform() throws Exception { getLogger().info("Performing action: Rolling batch restarting {}% of region servers", (int) (ratio * 100)); List selectedServers = selectServers(); - Queue serversToBeSuspended = new LinkedList<>(selectedServers); Queue suspendedServers = new LinkedList<>(); - + Random rand = ThreadLocalRandom.current(); // loop while there are servers to be suspended or suspended servers to be resumed while ((!serversToBeSuspended.isEmpty() || !suspendedServers.isEmpty()) && !context .isStopping()) { @@ -84,7 +84,7 @@ public void perform() throws Exception { action = SuspendOrResume.RESUME; } else { // do a coin toss - action = RandomUtils.nextBoolean() ? SuspendOrResume.SUSPEND : SuspendOrResume.RESUME; + action = rand.nextBoolean() ? SuspendOrResume.SUSPEND : SuspendOrResume.RESUME; } ServerName server; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitAllRegionOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitAllRegionOfTableAction.java index dd4e9931f953..06ffa4fbf16e 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitAllRegionOfTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitAllRegionOfTableAction.java @@ -26,8 +26,7 @@ import org.slf4j.LoggerFactory; public class SplitAllRegionOfTableAction extends Action { - private static final Logger LOG = - LoggerFactory.getLogger(SplitAllRegionOfTableAction.class); + private static final Logger LOG = LoggerFactory.getLogger(SplitAllRegionOfTableAction.class); private static final int DEFAULT_MAX_SPLITS = 3; private static final String MAX_SPLIT_KEY = "hbase.chaosmonkey.action.maxFullTableSplits"; @@ -39,7 +38,6 @@ public SplitAllRegionOfTableAction(TableName tableName) { this.tableName = tableName; } - public void init(ActionContext context) throws IOException { super.init(context); this.maxFullTableSplits = getConf().getInt(MAX_SPLIT_KEY, DEFAULT_MAX_SPLITS); @@ -57,8 +55,6 @@ public void perform() throws Exception { if (context.isStopping()) { return; } - - // Don't always split. This should allow splitting of a full table later in the run if (ThreadLocalRandom.current().nextDouble() < (((double) splits) / ((double) maxFullTableSplits)) / ((double) 2)) { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TruncateTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TruncateTableAction.java index 381de95dffaa..a64dbffae92a 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TruncateTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TruncateTableAction.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.chaos.actions; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -31,11 +31,9 @@ public class TruncateTableAction extends Action { private static final Logger LOG = LoggerFactory.getLogger(TruncateTableAction.class); private final TableName tableName; - private final Random random; public TruncateTableAction(String tableName) { this.tableName = TableName.valueOf(tableName); - this.random = new Random(); } @Override protected Logger getLogger() { @@ -52,7 +50,7 @@ public void perform() throws Exception { return; } - boolean preserveSplits = random.nextBoolean(); + boolean preserveSplits = ThreadLocalRandom.current().nextBoolean(); getLogger().info("Performing action: Truncate table {} preserve splits {}", tableName.getNameAsString(), preserveSplits); admin.truncateTable(tableName, preserveSplits); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java index 623d41d5d65b..9dc1cf129100 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java @@ -22,8 +22,9 @@ import java.util.HashSet; import java.util.LinkedList; import java.util.List; +import java.util.Random; import java.util.Set; -import org.apache.commons.lang3.RandomUtils; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.ClusterMetrics; import org.apache.hadoop.hbase.ServerName; import org.junit.Assert; @@ -64,15 +65,15 @@ public void perform() throws Exception { ClusterMetrics status = this.cluster.getClusterMetrics(); List victimServers = new LinkedList<>(status.getLiveServerMetrics().keySet()); Set killedServers = new HashSet<>(); - int liveCount = (int)Math.ceil(FRC_SERVERS_THAT_HOARD_AND_LIVE * victimServers.size()); int deadCount = (int)Math.ceil(FRC_SERVERS_THAT_HOARD_AND_DIE * victimServers.size()); Assert.assertTrue( "There are not enough victim servers: " + victimServers.size(), liveCount + deadCount < victimServers.size()); + Random rand = ThreadLocalRandom.current(); List targetServers = new ArrayList<>(liveCount); for (int i = 0; i < liveCount + deadCount; ++i) { - int victimIx = RandomUtils.nextInt(0, victimServers.size()); + int victimIx = rand.nextInt(victimServers.size()); targetServers.add(victimServers.remove(victimIx)); } unbalanceRegions(status, victimServers, targetServers, HOARD_FRC_OF_REGIONS); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceRegionsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceRegionsAction.java index 3d85e85db3e2..da7607e48de0 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceRegionsAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceRegionsAction.java @@ -21,7 +21,8 @@ import java.util.ArrayList; import java.util.LinkedList; import java.util.List; -import org.apache.commons.lang3.RandomUtils; +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.ClusterMetrics; import org.apache.hadoop.hbase.ServerName; import org.slf4j.Logger; @@ -57,8 +58,9 @@ public void perform() throws Exception { List victimServers = new LinkedList<>(status.getLiveServerMetrics().keySet()); int targetServerCount = (int)Math.ceil(fractionOfServers * victimServers.size()); List targetServers = new ArrayList<>(targetServerCount); + Random rand = ThreadLocalRandom.current(); for (int i = 0; i < targetServerCount; ++i) { - int victimIx = RandomUtils.nextInt(0, victimServers.size()); + int victimIx = rand.nextInt(victimServers.size()); targetServers.add(victimServers.remove(victimIx)); } unbalanceRegions(status, victimServers, targetServers, fractionOfRegions); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/monkies/PolicyBasedChaosMonkey.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/monkies/PolicyBasedChaosMonkey.java index e3e2e49ac268..95c5bc68b576 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/monkies/PolicyBasedChaosMonkey.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/monkies/PolicyBasedChaosMonkey.java @@ -26,13 +26,11 @@ import java.util.Properties; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; -import org.apache.commons.lang3.RandomUtils; import org.apache.hadoop.hbase.IntegrationTestingUtility; import org.apache.hadoop.hbase.chaos.policies.Policy; import org.apache.hadoop.hbase.util.Pair; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; /** @@ -40,7 +38,6 @@ */ public class PolicyBasedChaosMonkey extends ChaosMonkey { - private static final Logger LOG = LoggerFactory.getLogger(PolicyBasedChaosMonkey.class); private static final long ONE_SEC = 1000; private static final long ONE_MIN = 60 * ONE_SEC; @@ -93,7 +90,7 @@ private static ExecutorService buildMonkeyThreadPool(final int size) { /** Selects a random item from the given items */ public static T selectRandomItem(T[] items) { - return items[RandomUtils.nextInt(0, items.length)]; + return items[ThreadLocalRandom.current().nextInt(items.length)]; } /** Selects a random item from the given items with weights*/ @@ -103,7 +100,7 @@ public static T selectWeightedRandomItem(List> items) { totalWeight += pair.getSecond(); } - int cutoff = RandomUtils.nextInt(0, totalWeight); + int cutoff = ThreadLocalRandom.current().nextInt(totalWeight); int cummulative = 0; T item = null; @@ -127,7 +124,7 @@ public static List selectRandomItems(T[] items, float ratio) { List originalItems = Arrays.asList(items); Collections.shuffle(originalItems); - int startIndex = RandomUtils.nextInt(0, items.length - selectedNumber); + int startIndex = ThreadLocalRandom.current().nextInt(items.length - selectedNumber); return originalItems.subList(startIndex, startIndex + selectedNumber); } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/policies/PeriodicPolicy.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/policies/PeriodicPolicy.java index ae1c65e55274..7c69ee720ed1 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/policies/PeriodicPolicy.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/policies/PeriodicPolicy.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.chaos.policies; -import org.apache.commons.lang3.RandomUtils; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Threads; @@ -33,7 +33,7 @@ public PeriodicPolicy(long periodMs) { @Override public void run() { // Add some jitter. - int jitter = RandomUtils.nextInt(0, (int) periodMs); + int jitter = ThreadLocalRandom.current().nextInt((int)periodMs); LOG.info("Sleeping for {} ms to add jitter", jitter); Threads.sleep(jitter); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java index 2dd163305b95..2765c0ed1765 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java @@ -30,13 +30,12 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; -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.AtomicReference; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.codec.Codec; @@ -88,7 +87,6 @@ Codec getCodec() { } class Cluster { - Random random = new Random(); ReadWriteLock lock = new ReentrantReadWriteLock(); HashMap rpcServers = new HashMap<>(); List serverList = new ArrayList<>(); @@ -134,7 +132,7 @@ void stopRandomServer() throws Exception { return; } int size = rpcServers.size(); - int rand = random.nextInt(size); + int rand = ThreadLocalRandom.current().nextInt(size); rpcServer = serverList.remove(rand); InetSocketAddress address = rpcServer.getListenerAddress(); if (address == null) { @@ -176,7 +174,7 @@ RpcServer getRandomServer() { lock.readLock().lock(); try { int size = rpcServers.size(); - int rand = random.nextInt(size); + int rand = ThreadLocalRandom.current().nextInt(size); return serverList.get(rand); } finally { lock.readLock().unlock(); @@ -186,7 +184,6 @@ RpcServer getRandomServer() { static class MiniChaosMonkey extends Thread { AtomicBoolean running = new AtomicBoolean(true); - Random random = new Random(); AtomicReference exception = new AtomicReference<>(null); Cluster cluster; @@ -197,7 +194,7 @@ public MiniChaosMonkey(Cluster cluster) { @Override public void run() { while (running.get()) { - if (random.nextBoolean()) { + if (ThreadLocalRandom.current().nextBoolean()) { //start a server try { cluster.startServer(); @@ -238,7 +235,6 @@ static class SimpleClient extends Thread { Cluster cluster; String id; long numCalls = 0; - Random random = new Random(); public SimpleClient(Cluster cluster, AbstractRpcClient rpcClient, String id) { this.cluster = cluster; @@ -250,7 +246,7 @@ public SimpleClient(Cluster cluster, AbstractRpcClient rpcClient, String id) @Override public void run() { while (running.get()) { - boolean isBigPayload = random.nextBoolean(); + boolean isBigPayload = ThreadLocalRandom.current().nextBoolean(); String message = isBigPayload ? BIG_PAYLOAD : id + numCalls; EchoRequestProto param = EchoRequestProto.newBuilder().setMessage(message).build(); EchoResponseProto ret; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java index 4fcb51ff30b6..3fc733db23cb 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java @@ -27,10 +27,9 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Random; import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; @@ -83,7 +82,6 @@ import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import org.apache.hbase.thirdparty.com.google.common.base.Joiner; import org.apache.hbase.thirdparty.com.google.common.collect.Sets; import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; @@ -155,7 +153,6 @@ public class IntegrationTestBulkLoad extends IntegrationTestBase { public static class SlowMeCoproScanOperations implements RegionCoprocessor, RegionObserver { static final AtomicLong sleepTime = new AtomicLong(2000); - Random r = new Random(); AtomicLong countOfNext = new AtomicLong(0); AtomicLong countOfOpen = new AtomicLong(0); public SlowMeCoproScanOperations() {} @@ -373,7 +370,7 @@ public RecordReader createRecordReader(InputSplit sp taskId = taskId + iteration * numMapTasks; numMapTasks = numMapTasks * numIterations; - long chainId = Math.abs(new Random().nextLong()); + long chainId = Math.abs(ThreadLocalRandom.current().nextLong()); chainId = chainId - (chainId % numMapTasks) + taskId; // ensure that chainId is unique per task and across iterations LongWritable[] keys = new LongWritable[] {new LongWritable(chainId)}; @@ -391,8 +388,6 @@ public RecordReader createRecordReader(InputSplit sp public static class LinkedListCreationMapper extends Mapper { - private Random rand = new Random(); - @Override protected void map(LongWritable key, LongWritable value, Context context) throws IOException, InterruptedException { @@ -404,6 +399,7 @@ protected void map(LongWritable key, LongWritable value, Context context) long chainLength = context.getConfiguration().getLong(CHAIN_LENGTH_KEY, CHAIN_LENGTH); long nextRow = getNextRow(0, chainLength); + byte[] valueBytes = new byte[50]; for (long i = 0; i < chainLength; i++) { byte[] rk = Bytes.toBytes(currentRow); @@ -413,9 +409,8 @@ protected void map(LongWritable key, LongWritable value, Context context) // What link in the chain this is. KeyValue sortKv = new KeyValue(rk, SORT_FAM, chainIdArray, Bytes.toBytes(i)); // Added data so that large stores are created. - KeyValue dataKv = new KeyValue(rk, DATA_FAM, chainIdArray, - Bytes.toBytes(RandomStringUtils.randomAlphabetic(50)) - ); + Bytes.random(valueBytes); + KeyValue dataKv = new KeyValue(rk, DATA_FAM, chainIdArray, valueBytes); // Emit the key values. context.write(new ImmutableBytesWritable(rk), linkKv); @@ -429,7 +424,7 @@ protected void map(LongWritable key, LongWritable value, Context context) /** Returns a unique row id within this chain for this index */ private long getNextRow(long index, long chainLength) { - long nextRow = Math.abs(rand.nextLong()); + long nextRow = ThreadLocalRandom.current().nextLong(Long.MAX_VALUE); // use significant bits from the random number, but pad with index to ensure it is unique // this also ensures that we do not reuse row = 0 // row collisions from multiple mappers are fine, since we guarantee unique chainIds diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java index cd0cf2058272..8075a94fe7e3 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java @@ -22,18 +22,17 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InterruptedIOException; -import java.security.SecureRandom; import java.util.ArrayList; import java.util.Arrays; import java.util.Iterator; import java.util.List; -import java.util.Random; import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.FileSystem; @@ -344,7 +343,8 @@ public void write(DataOutput arg0) throws IOException { static class GeneratorRecordReader extends RecordReader { private long count; private long numNodes; - private Random64 rand; + // Use Random64 to avoid issue described in HBASE-21256. + private Random64 rand = new Random64(); @Override public void close() throws IOException { @@ -371,15 +371,12 @@ public float getProgress() throws IOException, InterruptedException { public void initialize(InputSplit arg0, TaskAttemptContext context) throws IOException, InterruptedException { numNodes = context.getConfiguration().getLong(GENERATOR_NUM_ROWS_PER_MAP_KEY, 25000000); - // Use Random64 to avoid issue described in HBASE-21256. - rand = new Random64(); } @Override public boolean nextKeyValue() throws IOException, InterruptedException { return count++ < numNodes; } - } @Override @@ -457,6 +454,8 @@ static class GeneratorMapper byte[] tinyValue = new byte[] { 't' }; byte[] bigValue = null; Configuration conf; + // Use Random64 to avoid issue described in HBASE-21256. + private Random64 rand = new Random64(); volatile boolean walkersStop; int numWalkers; @@ -494,7 +493,7 @@ protected void setup(Context context) throws IOException, InterruptedException { BIG_FAMILY_VALUE_SIZE_KEY, n, ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY, limit); bigValue = new byte[n]; - ThreadLocalRandom.current().nextBytes(bigValue); + rand.nextBytes(bigValue); LOG.info("Create a bigValue with " + n + " bytes."); } @@ -642,12 +641,10 @@ public class ContinuousConcurrentWalker implements Runnable { ConcurrentWalker walker; Configuration conf; Context context; - Random rand; public ContinuousConcurrentWalker(Configuration conf, Context context) { this.conf = conf; this.context = context; - rand = new Random(); } @Override @@ -681,7 +678,7 @@ private long selectLoop () throws InterruptedException{ if (walkersStop) { throw new InterruptedException(); } - return flushedLoops.get(rand.nextInt(flushedLoops.size())); + return flushedLoops.get(ThreadLocalRandom.current().nextInt(flushedLoops.size())); } } } @@ -1773,7 +1770,6 @@ public int run(String[] args) throws IOException { if (cmd.hasOption('n')) { maxQueries = Long.parseLong(cmd.getOptionValue("n")); } - Random rand = new SecureRandom(); boolean isSpecificStart = cmd.hasOption('s'); byte[] startKey = isSpecificStart ? Bytes.toBytesBinary(cmd.getOptionValue('s')) : null; @@ -1788,7 +1784,7 @@ public int run(String[] args) throws IOException { while (numQueries < maxQueries && (numQueries == 0 || !isSpecificStart)) { if (!isSpecificStart) { startKey = new byte[ROWKEY_LENGTH]; - rand.nextBytes(startKey); + Bytes.random(startKey); } CINode node = findStartNode(table, startKey); if (node == null && isSpecificStart) { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java index 29a71600f97d..e210ca6ae3cf 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java @@ -30,6 +30,7 @@ import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -83,7 +84,6 @@ import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import org.apache.hbase.thirdparty.com.google.common.collect.Sets; import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; @@ -189,10 +189,7 @@ public static class LoadMapper protected BufferedMutator mutator; protected Configuration conf; protected int numBackReferencesPerRow; - protected String shortTaskId; - - protected Random rand = new Random(); protected Counter rowsWritten, refsWritten; @Override @@ -229,8 +226,8 @@ protected void map(NullWritable key, NullWritable value, String suffix = "/" + shortTaskId; byte[] row = Bytes.add(new byte[8], Bytes.toBytes(suffix)); - int BLOCK_SIZE = (int)(recordsToWrite / 100); + Random rand = ThreadLocalRandom.current(); for (long i = 0; i < recordsToWrite;) { long blockStart = i; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestTimeBoundedRequestsWithRegionReplicas.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestTimeBoundedRequestsWithRegionReplicas.java index c5416c9a02e7..e6217cdac6bb 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestTimeBoundedRequestsWithRegionReplicas.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestTimeBoundedRequestsWithRegionReplicas.java @@ -23,9 +23,9 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.lang3.RandomUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HRegionLocation; @@ -331,7 +331,7 @@ protected Get createGet(long keyToRead) throws IOException { @Override protected long getNextKeyToRead() { // always read a random key, assuming that the writer has finished writing all keys - long key = startKey + Math.abs(RandomUtils.nextLong()) + long key = startKey + ThreadLocalRandom.current().nextLong(Long.MAX_VALUE) % (endKey - startKey); return key; } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java index e196aa73f0c5..74f828da53ef 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java @@ -23,6 +23,8 @@ import java.io.IOException; import java.security.PrivilegedExceptionAction; import java.util.List; +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -56,7 +58,6 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.util.ToolRunner; import org.junit.experimental.categories.Category; - import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; /** @@ -165,9 +166,10 @@ protected void map(NullWritable key, NullWritable value, Context context) throws InterruptedException { String suffix = "/" + shortTaskId; int BLOCK_SIZE = (int) (recordsToWrite / 100); + Random rand = ThreadLocalRandom.current(); for (long i = 0; i < recordsToWrite;) { for (long idx = 0; idx < BLOCK_SIZE && i < recordsToWrite; idx++, i++) { - int expIdx = rand.nextInt(BLOCK_SIZE) % VISIBILITY_EXPS_COUNT; + int expIdx = rand.nextInt(VISIBILITY_EXPS_COUNT); String exp = VISIBILITY_EXPS[expIdx]; byte[] row = Bytes.add(Bytes.toBytes(i), Bytes.toBytes(suffix), Bytes.toBytes(exp)); Put p = new Put(row); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java index afe7ba53136e..19ce3b377233 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java @@ -27,6 +27,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -46,7 +47,6 @@ import org.apache.hadoop.util.ToolRunner; import org.junit.Test; import org.junit.experimental.categories.Category; - import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; @Category(IntegrationTests.class) @@ -60,7 +60,6 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool { private TableName tableName = TableName.valueOf(TABLE_NAME_DEFAULT); private byte[] familyName = Bytes.toBytes(COLUMN_FAMILY_DEFAULT); private IntegrationTestingUtility util; - private Random random = new Random(); private Admin admin; public static void main(String[] args) throws Exception { @@ -223,17 +222,18 @@ private void deleteTable() throws IOException { private LinkedBlockingQueue insertData() throws IOException, InterruptedException { LinkedBlockingQueue rowKeys = new LinkedBlockingQueue<>(25000); BufferedMutator ht = util.getConnection().getBufferedMutator(this.tableName); + Random rand = ThreadLocalRandom.current(); byte[] value = new byte[300]; for (int x = 0; x < 5000; x++) { Span span = TraceUtil.getGlobalTracer().spanBuilder("insertData").startSpan(); try (Scope scope = span.makeCurrent()) { for (int i = 0; i < 5; i++) { - long rk = random.nextLong(); + long rk = rand.nextLong(); rowKeys.add(rk); Put p = new Put(Bytes.toBytes(rk)); for (int y = 0; y < 10; y++) { - random.nextBytes(value); - p.addColumn(familyName, Bytes.toBytes(random.nextLong()), value); + Bytes.random(value); + p.addColumn(familyName, Bytes.toBytes(rand.nextLong()), value); } ht.mutate(p); } diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java index 087a0345a721..0a24cbe1edc9 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java @@ -45,6 +45,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.ThreadLocalRandom; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; @@ -1526,7 +1527,6 @@ void onTakedown() throws IOException { static class AsyncRandomReadTest extends AsyncTableTest { private final Consistency consistency; private ArrayList gets; - private Random rd = new Random(); AsyncRandomReadTest(AsyncConnection con, TestOptions options, Status status) { super(con, options, status); @@ -1540,7 +1540,7 @@ static class AsyncRandomReadTest extends AsyncTableTest { @Override boolean testRow(final int i, final long startTime) throws IOException, InterruptedException { if (opts.randomSleep > 0) { - Thread.sleep(rd.nextInt(opts.randomSleep)); + Thread.sleep(ThreadLocalRandom.current().nextInt(opts.randomSleep)); } Get get = new Get(getRandomRow(this.rand, opts.totalRows)); for (int family = 0; family < opts.families; family++) { @@ -1948,8 +1948,6 @@ protected Pair getStartAndStopRow() { static class RandomReadTest extends TableTest { private final Consistency consistency; private ArrayList gets; - private Random rd = new Random(); - private long numOfReplyFromReplica = 0; RandomReadTest(Connection con, TestOptions options, Status status) { super(con, options, status); @@ -1963,7 +1961,7 @@ static class RandomReadTest extends TableTest { @Override boolean testRow(final int i, final long startTime) throws IOException, InterruptedException { if (opts.randomSleep > 0) { - Thread.sleep(rd.nextInt(opts.randomSleep)); + Thread.sleep(ThreadLocalRandom.current().nextInt(opts.randomSleep)); } Get get = new Get(getRandomRow(this.rand, opts.totalRows)); for (int family = 0; family < opts.families; family++) { diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java index 21e5b0a03c61..d1f8cc08b269 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/TestPerformanceEvaluation.java @@ -37,6 +37,8 @@ import java.util.NoSuchElementException; import java.util.Queue; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -155,11 +157,11 @@ public void testRandomReadCalculation() { opts.setNumClientThreads(2); opts = PerformanceEvaluation.calculateRowsAndSize(opts); assertEquals(1000, opts.getPerClientRunRows()); - Random random = new Random(); // assuming we will get one before this loop expires boolean foundValue = false; + Random rand = ThreadLocalRandom.current(); for (int i = 0; i < 10000000; i++) { - int randomRow = PerformanceEvaluation.generateRandomRow(random, opts.totalRows); + int randomRow = PerformanceEvaluation.generateRandomRow(rand, opts.totalRows); if (randomRow > 1000) { foundValue = true; break; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java index f976684d1252..7adbbc62821a 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java @@ -44,6 +44,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; +import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; @@ -102,7 +103,6 @@ import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.TestHRegionFileSystem; import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; -import org.apache.hadoop.hbase.security.SecurityConstants; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests; @@ -207,14 +207,13 @@ protected void map( int taskId = context.getTaskAttemptID().getTaskID().getId(); assert taskId < Byte.MAX_VALUE : "Unit tests dont support > 127 tasks!"; - Random random = new Random(); byte[] key; for (int j = 0; j < tables.length; ++j) { for (int i = 0; i < ROWSPERSPLIT; i++) { - random.nextBytes(keyBytes); + Bytes.random(keyBytes); // Ensure that unique tasks generate unique keys keyBytes[keyLength - 1] = (byte) (taskId & 0xFF); - random.nextBytes(valBytes); + Bytes.random(valBytes); key = keyBytes; if (multiTableMapper) { key = MultiTableHFileOutputFormat.createCompositeKey(tables[j].getName(), keyBytes); @@ -277,14 +276,13 @@ protected void map( int taskId = context.getTaskAttemptID().getTaskID().getId(); assert taskId < Byte.MAX_VALUE : "Unit tests dont support > 127 tasks!"; - Random random = new Random(); byte[] key; for (int j = 0; j < tables.length; ++j) { for (int i = 0; i < ROWSPERSPLIT; i++) { - random.nextBytes(keyBytes); + Bytes.random(keyBytes); // Ensure that unique tasks generate unique keys keyBytes[keyLength - 1] = (byte) (taskId & 0xFF); - random.nextBytes(valBytes); + Bytes.random(valBytes); key = keyBytes; if (multiTableMapper) { key = MultiTableHFileOutputFormat.createCompositeKey(tables[j].getName(), keyBytes); @@ -565,7 +563,7 @@ public void testJobConfiguration() throws Exception { } private byte [][] generateRandomStartKeys(int numKeys) { - Random random = new Random(); + Random random = ThreadLocalRandom.current(); byte[][] ret = new byte[numKeys][]; // first region start key is always empty ret[0] = HConstants.EMPTY_BYTE_ARRAY; @@ -577,7 +575,7 @@ public void testJobConfiguration() throws Exception { } private byte[][] generateRandomSplitKeys(int numKeys) { - Random random = new Random(); + Random random = ThreadLocalRandom.current(); byte[][] ret = new byte[numKeys][]; for (int i = 0; i < numKeys; i++) { ret[i] = @@ -1254,13 +1252,10 @@ private void writeRandomKeyValues(RecordWriter wri int taskId = context.getTaskAttemptID().getTaskID().getId(); assert taskId < Byte.MAX_VALUE : "Unit tests dont support > 127 tasks!"; final byte [] qualifier = Bytes.toBytes("data"); - Random random = new Random(); for (int i = 0; i < numRows; i++) { - Bytes.putInt(keyBytes, 0, i); - random.nextBytes(valBytes); + Bytes.random(valBytes); ImmutableBytesWritable key = new ImmutableBytesWritable(keyBytes); - for (byte[] family : families) { Cell kv = new KeyValue(keyBytes, family, qualifier, valBytes); writer.write(key, kv); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java index c2f22f63a25f..336816e2b49e 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java @@ -19,7 +19,6 @@ import java.io.IOException; import java.io.InterruptedIOException; import java.lang.reflect.Constructor; -import java.security.SecureRandom; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -291,7 +290,7 @@ protected void applyColumnFamilyOptions(TableName tableName, } if (cipher != null) { byte[] keyBytes = new byte[cipher.getKeyLength()]; - new SecureRandom().nextBytes(keyBytes); + Bytes.secureRandom(keyBytes); columnDescBuilder.setEncryptionType(cipher.getName()); columnDescBuilder.setEncryptionKey( EncryptionUtil.wrapKey(conf, diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestFastLongHistogram.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestFastLongHistogram.java index 371f767f74c3..120f91169c5a 100644 --- a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestFastLongHistogram.java +++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestFastLongHistogram.java @@ -21,6 +21,8 @@ import java.util.Arrays; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -71,7 +73,7 @@ public void testAdaptionOfChange() { // assumes the uniform distribution FastLongHistogram hist = new FastLongHistogram(100, 0, 100); - Random rand = new Random(); + Random rand = ThreadLocalRandom.current(); for (int n = 0; n < 10; n++) { for (int i = 0; i < 900; i++) { diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStorePerformanceEvaluation.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStorePerformanceEvaluation.java index 17e4376108fd..d88d93e571f2 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStorePerformanceEvaluation.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStorePerformanceEvaluation.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.procedure2.store; import java.io.IOException; -import java.util.Random; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -31,6 +30,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.procedure2.util.StringUtils; import org.apache.hadoop.hbase.util.AbstractHBaseTool; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; import org.apache.hbase.thirdparty.org.apache.commons.cli.Option; @@ -106,7 +106,7 @@ protected void processOptions(CommandLine cmd) { syncType) : "sync argument can only accept one of these three values: hsync, hflush, nosync"; stateSize = getOptionAsInt(cmd, STATE_SIZE_OPTION.getOpt(), DEFAULT_STATE_SIZE); SERIALIZED_STATE = new byte[stateSize]; - new Random(12345).nextBytes(SERIALIZED_STATE); + Bytes.random(SERIALIZED_STATE); } private void setUpProcedureStore() throws IOException { diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALLoaderPerformanceEvaluation.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALLoaderPerformanceEvaluation.java index 6b8b46e1044b..2866b21518ec 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALLoaderPerformanceEvaluation.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALLoaderPerformanceEvaluation.java @@ -22,8 +22,8 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; -import java.util.Random; import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -136,14 +136,13 @@ public void setUpProcedureStore() throws IOException { * value denotes delete state. */ private List shuffleProcWriteSequence() { - Random rand = new Random(); List procStatesSequence = new ArrayList<>(); Set toBeDeletedProcs = new HashSet<>(); // Add n + 1 entries of the proc id for insert + updates. If proc is chosen for delete, add // extra entry which is marked -ve in the loop after shuffle. for (int procId = 1; procId <= numProcs; ++procId) { procStatesSequence.addAll(Collections.nCopies(updatesPerProc + 1, procId)); - if (rand.nextFloat() < deleteProcsFraction) { + if (ThreadLocalRandom.current().nextFloat() < deleteProcsFraction) { procStatesSequence.add(procId); toBeDeletedProcs.add(procId); } diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestProcedureStoreTracker.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestProcedureStoreTracker.java index 25678e39e0bd..e3064c9ab823 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestProcedureStoreTracker.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestProcedureStoreTracker.java @@ -21,7 +21,8 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.testclassification.MasterTests; @@ -153,13 +154,12 @@ public void testRandLoad() { final ProcedureStoreTracker tracker = new ProcedureStoreTracker(); - Random rand = new Random(1); for (int i = 0; i < NRUNS; ++i) { assertTrue(tracker.isEmpty()); int count = 0; while (count < NPROCEDURES) { - long procId = rand.nextLong(); + long procId = ThreadLocalRandom.current().nextLong(); if (procId < 1) { continue; } diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestStressWALProcedureStore.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestStressWALProcedureStore.java index 2d7d671f33b3..12ea02adf9c6 100644 --- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestStressWALProcedureStore.java +++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestStressWALProcedureStore.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -97,12 +98,12 @@ public void tearDown() throws IOException { public void testInsertUpdateDelete() throws Exception { final long LAST_PROC_ID = 19999; final Thread[] thread = new Thread[PROCEDURE_STORE_SLOTS]; - final AtomicLong procCounter = new AtomicLong((long)Math.round(Math.random() * 100)); + final Random rand = ThreadLocalRandom.current(); + final AtomicLong procCounter = new AtomicLong(rand.nextInt(100)); for (int i = 0; i < thread.length; ++i) { thread[i] = new Thread() { @Override public void run() { - Random rand = new Random(); TestProcedure proc; do { // After HBASE-15579 there may be gap in the procId sequence, trying to simulate that. diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java index 2c20430ab909..211576173308 100644 --- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java +++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java @@ -61,7 +61,7 @@ public class TestZKReplicationPeerStorage { HBaseClassTestRule.forClass(TestZKReplicationPeerStorage.class); private static final HBaseZKTestingUtil UTIL = new HBaseZKTestingUtil(); - + private static final Random RNG = new Random(); // Seed may be set with Random#setSeed private static ZKReplicationPeerStorage STORAGE; @BeforeClass @@ -98,13 +98,13 @@ private Map> randTableCFs(Random rand) { } private ReplicationPeerConfig getConfig(int seed) { - Random rand = new Random(seed); - return ReplicationPeerConfig.newBuilder().setClusterKey(Long.toHexString(rand.nextLong())) - .setReplicationEndpointImpl(Long.toHexString(rand.nextLong())) - .setRemoteWALDir(Long.toHexString(rand.nextLong())).setNamespaces(randNamespaces(rand)) - .setExcludeNamespaces(randNamespaces(rand)).setTableCFsMap(randTableCFs(rand)) - .setExcludeTableCFsMap(randTableCFs(rand)).setReplicateAllUserTables(rand.nextBoolean()) - .setBandwidth(rand.nextInt(1000)).build(); + RNG.setSeed(seed); + return ReplicationPeerConfig.newBuilder().setClusterKey(Long.toHexString(RNG.nextLong())) + .setReplicationEndpointImpl(Long.toHexString(RNG.nextLong())) + .setRemoteWALDir(Long.toHexString(RNG.nextLong())).setNamespaces(randNamespaces(RNG)) + .setExcludeNamespaces(randNamespaces(RNG)).setTableCFsMap(randTableCFs(RNG)) + .setExcludeTableCFsMap(randTableCFs(RNG)).setReplicateAllUserTables(RNG.nextBoolean()) + .setBandwidth(RNG.nextInt(1000)).build(); } private void assertSetEquals(Set expected, Set actual) { diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java index 95d5550d226b..9a2542e70518 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java @@ -29,6 +29,8 @@ import java.util.Iterator; import java.util.List; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBException; import javax.xml.bind.Marshaller; @@ -95,7 +97,7 @@ public class TestScannerResource { static int insertData(Configuration conf, TableName tableName, String column, double prob) throws IOException { - Random rng = new Random(); + Random rng = ThreadLocalRandom.current(); byte[] k = new byte[3]; byte [][] famAndQf = CellUtil.parseColumn(Bytes.toBytes(column)); List puts = new ArrayList<>(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java index e1a49491dc2e..07e751716bf2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java @@ -24,10 +24,9 @@ import java.util.Collections; import java.util.List; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.LongAdder; - -import org.apache.commons.lang3.RandomUtils; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -267,7 +266,7 @@ private boolean grabTask(String path) { // after a successful submit, sleep a little bit to allow other RSs to grab the rest tasks try { - int sleepTime = RandomUtils.nextInt(0, 500) + 500; + int sleepTime = ThreadLocalRandom.current().nextInt(500) + 500; Thread.sleep(sleepTime); } catch (InterruptedException e) { LOG.warn("Interrupted while yielding for other region servers", e); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/PrefetchExecutor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/PrefetchExecutor.java index 62a86d7418d8..80de44915f2e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/PrefetchExecutor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/PrefetchExecutor.java @@ -19,13 +19,13 @@ package org.apache.hadoop.hbase.io.hfile; import java.util.Map; -import java.util.Random; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; import org.apache.hadoop.conf.Configuration; @@ -71,8 +71,6 @@ public Thread newThread(Runnable r) { }); } - private static final Random RNG = new Random(); - // TODO: We want HFile, which is where the blockcache lives, to handle // prefetching of file blocks but the Store level is where path convention // knowledge should be contained @@ -93,7 +91,8 @@ public static void request(Path path, Runnable runnable) { long delay; if (prefetchDelayMillis > 0) { delay = (long)((prefetchDelayMillis * (1.0f - (prefetchDelayVariation/2))) + - (prefetchDelayMillis * (prefetchDelayVariation/2) * RNG.nextFloat())); + (prefetchDelayMillis * (prefetchDelayVariation/2) * + ThreadLocalRandom.current().nextFloat())); } else { delay = 0; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java index 68de855b90c2..a7eb0a54f093 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java @@ -29,6 +29,8 @@ import java.util.Scanner; import java.util.Set; import java.util.TreeMap; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -519,7 +521,7 @@ protected static class RandomizedMatrix { public RandomizedMatrix(int rows, int cols) { this.rows = rows; this.cols = cols; - Random random = new Random(); + Random random = ThreadLocalRandom.current(); rowTransform = new int[rows]; rowInverse = new int[rows]; for (int i = 0; i < rows; i++) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AdaptiveMemStoreCompactionStrategy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AdaptiveMemStoreCompactionStrategy.java index aff329ff4755..3c5c06bfdb05 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AdaptiveMemStoreCompactionStrategy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AdaptiveMemStoreCompactionStrategy.java @@ -18,7 +18,7 @@ */ package org.apache.hadoop.hbase.regionserver; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.conf.Configuration; import org.apache.yetus.audience.InterfaceAudience; @@ -53,7 +53,6 @@ public class AdaptiveMemStoreCompactionStrategy extends MemStoreCompactionStrate private double compactionThreshold; private double initialCompactionProbability; private double compactionProbability; - private Random rand = new Random(); private double numCellsInVersionedList = 0; private boolean compacted = false; @@ -66,9 +65,10 @@ public AdaptiveMemStoreCompactionStrategy(Configuration conf, String cfName) { resetStats(); } - @Override public Action getAction(VersionedSegmentsList versionedList) { + @Override + public Action getAction(VersionedSegmentsList versionedList) { if (versionedList.getEstimatedUniquesFrac() < 1.0 - compactionThreshold) { - double r = rand.nextDouble(); + double r = ThreadLocalRandom.current().nextDouble(); if(r < compactionProbability) { numCellsInVersionedList = versionedList.getNumOfCells(); compacted = true; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 1517c86aff1f..8b64f25c6687 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -49,13 +49,13 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; import javax.management.MalformedObjectNameException; import javax.servlet.http.HttpServlet; -import org.apache.commons.lang3.RandomUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -1654,14 +1654,14 @@ protected void chore() { if (r.shouldFlush(whyFlush)) { FlushRequester requester = server.getFlushRequester(); if (requester != null) { - long randomDelay = RandomUtils.nextLong(0, rangeOfDelayMs) + MIN_DELAY_TIME; + long delay = ThreadLocalRandom.current().nextLong(rangeOfDelayMs) + MIN_DELAY_TIME; //Throttle the flushes by putting a delay. If we don't throttle, and there //is a balanced write-load on the regions in a table, we might end up //overwhelming the filesystem with too many flushes at once. - if (requester.requestDelayedFlush(r, randomDelay)) { + if (requester.requestDelayedFlush(r, delay)) { LOG.info("{} requesting flush of {} because {} after random delay {} ms", getName(), r.getRegionInfo().getRegionNameAsString(), whyFlush.toString(), - randomDelay); + delay); } } } @@ -1963,7 +1963,8 @@ private void initializeThreads() { double brokenStoreFileCleanerDelayJitter = conf.getDouble( BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_DELAY_JITTER, BrokenStoreFileCleaner.DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER); - double jitterRate = (RandomUtils.nextDouble() - 0.5D) * brokenStoreFileCleanerDelayJitter; + double jitterRate = (ThreadLocalRandom.current().nextDouble() - 0.5D) * + brokenStoreFileCleanerDelayJitter; long jitterValue = Math.round(brokenStoreFileCleanerDelay * jitterRate); this.brokenStoreFileCleaner = new BrokenStoreFileCleaner((int) (brokenStoreFileCleanerDelay + jitterValue), diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/SortedCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/SortedCompactionPolicy.java index ef9f3ca2e327..db469c420ca0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/SortedCompactionPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/SortedCompactionPolicy.java @@ -23,7 +23,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @@ -35,6 +34,8 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy { private static final Logger LOG = LoggerFactory.getLogger(SortedCompactionPolicy.class); + private static final Random RNG = new Random(); + public SortedCompactionPolicy(Configuration conf, StoreConfigInformation storeConfigInfo) { super(conf, storeConfigInfo); } @@ -109,11 +110,6 @@ protected abstract CompactionRequestImpl createCompactionRequest( public abstract boolean shouldPerformMajorCompaction(Collection filesToCompact) throws IOException; - /** - * Used calculation jitter - */ - private final Random random = new Random(); - /** * @param filesToCompact * @return When to run next major compaction @@ -137,14 +133,12 @@ public long getNextMajorCompactTime(Collection filesToCompact) { // deterministic jitter avoids a major compaction storm on restart OptionalInt seed = StoreUtils.getDeterministicRandomSeed(filesToCompact); if (seed.isPresent()) { + long jitter = Math.round(period * jitterPct); // Synchronized to ensure one user of random instance at a time. - double rnd; - synchronized (this) { - this.random.setSeed(seed.getAsInt()); - rnd = this.random.nextDouble(); + synchronized (RNG) { + RNG.setSeed(seed.getAsInt()); + return period + jitter - Math.round(2L * jitter * RNG.nextDouble()); } - long jitter = Math.round(period * jitterPct); - return period + jitter - Math.round(2L * jitter * rnd); } else { return 0L; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java index 743369f2175b..42335a52b981 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java @@ -23,9 +23,7 @@ import java.io.IOException; import java.io.OutputStream; import java.security.Key; -import java.security.SecureRandom; import java.util.concurrent.atomic.AtomicLong; -import javax.crypto.spec.SecretKeySpec; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -110,11 +108,8 @@ protected final WALHeader buildSecureWALHeader(Configuration conf, WALHeader.Bui throw new RuntimeException("Cipher '" + cipherName + "' is not available"); } - // Generate an encryption key for this WAL - SecureRandom rng = new SecureRandom(); - byte[] keyBytes = new byte[cipher.getKeyLength()]; - rng.nextBytes(keyBytes); - Key key = new SecretKeySpec(keyBytes, cipher.getName()); + // Generate a random encryption key for this WAL + Key key = cipher.getRandomKey(); builder.setEncryptionKey(UnsafeByteOperations.unsafeWrap(EncryptionUtil.wrapKey(conf, conf.get(HConstants.CRYPTO_WAL_KEY_NAME_CONF_KEY, conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureWALCellCodec.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureWALCellCodec.java index 469d69266eab..6d2bd61a0234 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureWALCellCodec.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureWALCellCodec.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.security.SecureRandom; import org.apache.commons.io.IOUtils; import org.apache.yetus.audience.InterfaceAudience; @@ -151,7 +150,7 @@ static class EncryptedKvEncoder extends KeyValueCodecWithTags.KeyValueEncoder { @Override protected byte[] initialValue() { byte[] iv = new byte[encryptor.getIvLength()]; - new SecureRandom().nextBytes(iv); + Bytes.secureRandom(iv); return iv; } }; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java index b6cc75897b80..209537137d7b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.io.InterruptedIOException; import java.math.BigInteger; -import java.security.SecureRandom; import java.util.ArrayList; import java.util.Deque; import java.util.HashMap; @@ -33,6 +32,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; @@ -307,7 +307,7 @@ private Path createStagingDir(Path baseDir, User user, TableName tableName) thro int RANDOM_RADIX = 32; String doubleUnderScore = UNDERSCORE + UNDERSCORE; String randomDir = user.getShortName() + doubleUnderScore + tblName + doubleUnderScore - + (new BigInteger(RANDOM_WIDTH, new SecureRandom()).toString(RANDOM_RADIX)); + + (new BigInteger(RANDOM_WIDTH, ThreadLocalRandom.current()).toString(RANDOM_RADIX)); return createStagingDir(baseDir, user, randomDir); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java index 4f6638e9c439..38f1ce31c18e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/CanaryTool.java @@ -34,7 +34,6 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.Set; import java.util.TreeSet; import java.util.concurrent.Callable; @@ -100,7 +99,6 @@ import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import org.apache.hbase.thirdparty.com.google.common.collect.Lists; /** @@ -604,14 +602,13 @@ private Void write() { if (rowToCheck.length == 0) { rowToCheck = new byte[]{0x0}; } - int writeValueSize = - connection.getConfiguration().getInt(HConstants.HBASE_CANARY_WRITE_VALUE_SIZE_KEY, 10); + int writeValueSize = connection.getConfiguration() + .getInt(HConstants.HBASE_CANARY_WRITE_VALUE_SIZE_KEY, 10); for (ColumnFamilyDescriptor column : tableDesc.getColumnFamilies()) { Put put = new Put(rowToCheck); byte[] value = new byte[writeValueSize]; Bytes.random(value); put.addColumn(column.getName(), HConstants.EMPTY_BYTE_ARRAY, value); - LOG.debug("Writing to {} {} {} {}", tableDesc.getTableName(), region.getRegionNameAsString(), column.getNameAsString(), Bytes.toStringBinary(rowToCheck)); @@ -1831,7 +1828,6 @@ private void monitorRegionServers(Map> rsAndRMap, RegionServerStdOutSink regionServerSink) { List tasks = new ArrayList<>(); Map successMap = new HashMap<>(); - Random rand = new Random(); for (Map.Entry> entry : rsAndRMap.entrySet()) { String serverName = entry.getKey(); AtomicLong successes = new AtomicLong(0); @@ -1848,7 +1844,8 @@ private void monitorRegionServers(Map> rsAndRMap, } } else { // random select a region if flag not set - RegionInfo region = entry.getValue().get(rand.nextInt(entry.getValue().size())); + RegionInfo region = entry.getValue() + .get(ThreadLocalRandom.current().nextInt(entry.getValue().size())); tasks.add(new RegionServerTask(this.connection, serverName, region, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/EncryptionTest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/EncryptionTest.java index 0ccf0f4d8b44..cee3b56d6f6f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/EncryptionTest.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/EncryptionTest.java @@ -135,7 +135,7 @@ public static void testEncryption(final Configuration conf, final String cipher, byte[] iv = null; if (context.getCipher().getIvLength() > 0) { iv = new byte[context.getCipher().getIvLength()]; - Bytes.random(iv); + Bytes.secureRandom(iv); } byte[] plaintext = new byte[1024]; Bytes.random(plaintext); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java index 7f5c318bb430..c6db715d1a0f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java @@ -23,6 +23,8 @@ import java.util.EnumSet; import java.util.List; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ClusterMetrics.Option; @@ -167,10 +169,10 @@ public static void fixMetaHoleOnlineAndAddReplicas(Configuration conf, Table meta = conn.getTable(TableName.META_TABLE_NAME); Put put = MetaTableAccessor.makePutFromRegionInfo(hri, EnvironmentEdgeManager.currentTime()); if (numReplicas > 1) { - Random r = new Random(); + Random rand = ThreadLocalRandom.current(); ServerName[] serversArr = servers.toArray(new ServerName[servers.size()]); for (int i = 1; i < numReplicas; i++) { - ServerName sn = serversArr[r.nextInt(serversArr.length)]; + ServerName sn = serversArr[rand.nextInt(serversArr.length)]; // the column added here is just to make sure the master is able to // see the additional replicas when it is asked to assign. The // final value of these columns will be different and will be updated diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/AcidGuaranteesTestTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/AcidGuaranteesTestTool.java index 4faf86dd4e60..ae940dc24a17 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/AcidGuaranteesTestTool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/AcidGuaranteesTestTool.java @@ -19,10 +19,10 @@ import java.io.IOException; import java.util.List; -import java.util.Random; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; @@ -137,7 +137,6 @@ protected int doWork() throws Exception { * Thread that does random full-row writes into a table. */ public static class AtomicityWriter extends RepeatingTestThread { - Random rand = new Random(); byte data[] = new byte[10]; byte[][] targetRows; byte[][] targetFamilies; @@ -157,10 +156,9 @@ public AtomicityWriter(TestContext ctx, byte[][] targetRows, byte[][] targetFami @Override public void doAnAction() throws Exception { // Pick a random row to write into - byte[] targetRow = targetRows[rand.nextInt(targetRows.length)]; + byte[] targetRow = targetRows[ThreadLocalRandom.current().nextInt(targetRows.length)]; Put p = new Put(targetRow); - rand.nextBytes(data); - + Bytes.random(data); for (byte[] family : targetFamilies) { for (int i = 0; i < NUM_COLS_TO_CHECK; i++) { byte qualifier[] = Bytes.toBytes("col" + i); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java index a73d4c7516c8..4b4ce9e03a1d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java @@ -49,6 +49,7 @@ import java.util.Random; import java.util.Set; import java.util.TreeSet; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BooleanSupplier; @@ -150,9 +151,7 @@ import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.ZooKeeper.States; - import org.apache.hbase.thirdparty.com.google.common.io.Closeables; - import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; /** @@ -1954,10 +1953,9 @@ public void loadNumericRows(final Table t, final byte[] f, int startRow, int end public void loadRandomRows(final Table t, final byte[] f, int rowSize, int totalRows) throws IOException { - Random r = new Random(); - byte[] row = new byte[rowSize]; for (int i = 0; i < totalRows; i++) { - r.nextBytes(row); + byte[] row = new byte[rowSize]; + Bytes.random(row); Put put = new Put(row); put.addColumn(f, new byte[] { 0 }, new byte[] { 0 }); t.put(put); @@ -2693,7 +2691,7 @@ public HRegion getSplittableRegion(TableName tableName, int maxAttempts) { // There are chances that before we get the region for the table from an RS the region may // be going for CLOSE. This may be because online schema change is enabled if (regCount > 0) { - idx = random.nextInt(regCount); + idx = ThreadLocalRandom.current().nextInt(regCount); // if we have just tried this region, there is no need to try again if (attempted.contains(idx)) { continue; @@ -3297,7 +3295,7 @@ public static int randomFreePort() { } public static String randomMultiCastAddress() { - return "226.1.1." + random.nextInt(254); + return "226.1.1." + ThreadLocalRandom.current().nextInt(254); } public static void waitForHostPort(String host, int port) throws IOException { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java index 3855e4a6d495..b3fb634a1de1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java @@ -19,8 +19,7 @@ package org.apache.hadoop.hbase; import java.io.IOException; -import java.security.SecureRandom; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.commons.math3.random.RandomData; import org.apache.commons.math3.random.RandomDataImpl; @@ -351,7 +350,6 @@ long run() throws Exception { static class SequentialWriteBenchmark extends RowOrientedBenchmark { protected HFile.Writer writer; - private Random random = new Random(); private byte[] bytes = new byte[ROW_LENGTH]; public SequentialWriteBenchmark(Configuration conf, FileSystem fs, Path mf, @@ -368,7 +366,7 @@ void setUp() throws Exception { if (cipher == "aes") { byte[] cipherKey = new byte[AES.KEY_LENGTH]; - new SecureRandom().nextBytes(cipherKey); + Bytes.secureRandom(cipherKey); builder.withEncryptionContext(Encryption.newContext(conf) .setCipher(Encryption.getCipher(conf, cipher)) .setKey(cipherKey)); @@ -390,7 +388,7 @@ void doRow(int i) throws Exception { } private byte[] generateValue() { - random.nextBytes(bytes); + Bytes.random(bytes); return bytes; } @@ -461,8 +459,6 @@ protected int getReportingPeriod() { static class UniformRandomReadBenchmark extends ReadBenchmark { - private Random random = new Random(); - public UniformRandomReadBenchmark(Configuration conf, FileSystem fs, Path mf, int totalRows) { super(conf, fs, mf, totalRows); @@ -483,12 +479,11 @@ void doRow(int i) throws Exception { } private byte [] getRandomRow() { - return format(random.nextInt(totalRows)); + return format(ThreadLocalRandom.current().nextInt(totalRows)); } } static class UniformRandomSmallScan extends ReadBenchmark { - private Random random = new Random(); public UniformRandomSmallScan(Configuration conf, FileSystem fs, Path mf, int totalRows) { @@ -521,7 +516,7 @@ void doRow(int i) throws Exception { } private byte [] getRandomRow() { - return format(random.nextInt(totalRows)); + return format(ThreadLocalRandom.current().nextInt(totalRows)); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/RegionReplicationLagEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/RegionReplicationLagEvaluation.java index 6a54adcaced3..da4101dbb853 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/RegionReplicationLagEvaluation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/RegionReplicationLagEvaluation.java @@ -20,7 +20,6 @@ import com.google.errorprone.annotations.RestrictedApi; import java.io.IOException; import java.util.Arrays; -import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.hbase.client.Admin; @@ -43,7 +42,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; import org.apache.hbase.thirdparty.org.apache.commons.cli.DefaultParser; import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter; @@ -113,15 +111,14 @@ private void createTable(Admin admin, TableName tableName) throws IOException { } private void checkLag(Table table, int rlen, int vlen, int rows) throws IOException { - ThreadLocalRandom rand = ThreadLocalRandom.current(); byte[] family = Bytes.toBytes(FAMILY_NAME); byte[] qualifier = Bytes.toBytes(QUALIFIER_NAME); - byte[] row = new byte[rlen]; - byte[] value = new byte[vlen]; LOG.info("Test replication lag on table {} with {} rows", table.getName(), rows); for (int i = 0; i < rows; i++) { - rand.nextBytes(row); - rand.nextBytes(value); + byte[] row = new byte[rlen]; + Bytes.random(row); + byte[] value = new byte[vlen]; + Bytes.random(value); table.put(new Put(row).addColumn(family, qualifier, value)); // get from secondary replica Get get = new Get(row).setConsistency(Consistency.TIMELINE).setReplicaId(1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtil.java index 7076bf936126..489f6eb374d9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtil.java @@ -429,7 +429,7 @@ public Integer answer(InvocationOnMock invocation) { when(portChecker.available(anyInt())).thenReturn(true); HBaseTestingUtil.PortAllocator portAllocator = - new HBaseTestingUtil.PortAllocator(random, portChecker); + new HBaseTestingUtil.PortAllocator(portChecker); int port1 = portAllocator.randomFreePort(); int port2 = portAllocator.randomFreePort(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java index 9169c22d86c2..94236b7b4cde 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java @@ -32,6 +32,8 @@ import java.util.Collections; import java.util.List; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -79,7 +81,6 @@ public class TestMetaTableAccessor { private static final Logger LOG = LoggerFactory.getLogger(TestMetaTableAccessor.class); private static final HBaseTestingUtil UTIL = new HBaseTestingUtil(); private static Connection connection; - private Random random = new Random(); @Rule public TestName name = new TestName(); @@ -292,9 +293,11 @@ private static void testGetRegion(final Connection connection, final RegionInfo @Test public void testMetaLocationsForRegionReplicas() throws IOException { - ServerName serverName0 = ServerName.valueOf("foo", 60010, random.nextLong()); - ServerName serverName1 = ServerName.valueOf("bar", 60010, random.nextLong()); - ServerName serverName100 = ServerName.valueOf("baz", 60010, random.nextLong()); + Random rand = ThreadLocalRandom.current(); + + ServerName serverName0 = ServerName.valueOf("foo", 60010, rand.nextLong()); + ServerName serverName1 = ServerName.valueOf("bar", 60010, rand.nextLong()); + ServerName serverName100 = ServerName.valueOf("baz", 60010, rand.nextLong()); long regionId = EnvironmentEdgeManager.currentTime(); RegionInfo primary = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName())) @@ -307,9 +310,9 @@ public void testMetaLocationsForRegionReplicas() throws IOException { .setStartKey(HConstants.EMPTY_START_ROW).setEndKey(HConstants.EMPTY_END_ROW).setSplit(false) .setRegionId(regionId).setReplicaId(100).build(); - long seqNum0 = random.nextLong(); - long seqNum1 = random.nextLong(); - long seqNum100 = random.nextLong(); + long seqNum0 = rand.nextLong(); + long seqNum1 = rand.nextLong(); + long seqNum100 = rand.nextLong(); try (Table meta = MetaTableAccessor.getMetaHTable(connection)) { MetaTableAccessor.updateRegionLocation(connection, primary, serverName0, seqNum0, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java index f96770910541..d7916343278f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java @@ -31,7 +31,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.hadoop.hbase.ClusterMetrics.Option; @@ -579,9 +579,7 @@ public void testRegionNormalizer() throws Exception { @Test public void testAbortProcedureFail() throws Exception { - Random randomGenerator = new Random(); - long procId = randomGenerator.nextLong(); - + long procId = ThreadLocalRandom.current().nextLong(); boolean abortResult = ADMIN.abortProcedure(procId, true); assertFalse(abortResult); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncBufferMutator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncBufferMutator.java index 602bc62ea9af..9a2c5505945a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncBufferMutator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncBufferMutator.java @@ -32,7 +32,6 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -47,7 +46,6 @@ import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; - import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer; import org.apache.hbase.thirdparty.io.netty.util.Timeout; @@ -80,7 +78,7 @@ public static void setUp() throws Exception { TEST_UTIL.createTable(TABLE_NAME, CF); TEST_UTIL.createMultiRegionTable(MULTI_REGION_TABLE_NAME, CF); CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get(); - ThreadLocalRandom.current().nextBytes(VALUE); + Bytes.random(VALUE); } @AfterClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java index 576238ca2153..040bc627ab9f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java @@ -198,11 +198,11 @@ public void testSingleRegionTable() throws IOException, InterruptedException, Ex assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, serverName, getDefaultRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType, false).get()); } - byte[] randKey = new byte[ThreadLocalRandom.current().nextInt(128)]; - ThreadLocalRandom.current().nextBytes(randKey); + byte[] key = new byte[ThreadLocalRandom.current().nextInt(128)]; + Bytes.random(key); for (RegionLocateType locateType : RegionLocateType.values()) { assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, serverName, - getDefaultRegionLocation(TABLE_NAME, randKey, locateType, false).get()); + getDefaultRegionLocation(TABLE_NAME, key, locateType, false).get()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java index 103f022c7150..37b2b88c43e6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcedureAdminApi.java @@ -24,7 +24,8 @@ import java.util.HashMap; import java.util.Map; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; @@ -116,8 +117,7 @@ public void isProcedureFinished() throws Exception { @Test public void abortProcedure() throws Exception { - Random randomGenerator = new Random(); - long procId = randomGenerator.nextLong(); + long procId = ThreadLocalRandom.current().nextLong(); boolean abortResult = admin.abortProcedure(procId, true).get(); assertFalse(abortResult); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatchRetryImmediately.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatchRetryImmediately.java index d9c4bc91d173..9f22eba623a5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatchRetryImmediately.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatchRetryImmediately.java @@ -22,7 +22,6 @@ import java.util.Arrays; import java.util.List; -import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -71,7 +70,7 @@ public static void setUp() throws Exception { UTIL.startMiniCluster(1); Table table = UTIL.createTable(TABLE_NAME, FAMILY); UTIL.waitTableAvailable(TABLE_NAME); - ThreadLocalRandom.current().nextBytes(VALUE_PREFIX); + Bytes.random(VALUE_PREFIX); for (int i = 0; i < COUNT; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(FAMILY, QUAL, Bytes.add(VALUE_PREFIX, Bytes.toBytes(i)))); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java index 107d59e1ed00..71e8e3bcdbb9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java @@ -26,11 +26,11 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Random; import java.util.concurrent.ExecutionException; 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.atomic.AtomicBoolean; import java.util.stream.Collectors; @@ -144,7 +144,7 @@ public void test() throws Exception { return null; }))); LOG.info("====== Scheduled {} read threads ======", numThreads); - Collections.shuffle(Arrays.asList(SPLIT_KEYS), new Random(123)); + Collections.shuffle(Arrays.asList(SPLIT_KEYS), ThreadLocalRandom.current()); Admin admin = TEST_UTIL.getAdmin(); for (byte[] splitPoint : SPLIT_KEYS) { int oldRegionCount = admin.getRegions(TABLE_NAME).size(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java index ddfa37515797..d3c055e75457 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java @@ -95,7 +95,6 @@ public class TestFromClientSide3 { = new HBaseTestingUtil(); private static final int WAITTABLE_MILLIS = 10000; private static byte[] FAMILY = Bytes.toBytes("testFamily"); - private static Random random = new Random(); private static int SLAVES = 3; private static final byte[] ROW = Bytes.toBytes("testRow"); private static final byte[] ANOTHERROW = Bytes.toBytes("anotherrow"); @@ -141,9 +140,10 @@ public void tearDown() throws Exception { private void randomCFPuts(Table table, byte[] row, byte[] family, int nPuts) throws Exception { Put put = new Put(row); + Random rand = ThreadLocalRandom.current(); for (int i = 0; i < nPuts; i++) { - byte[] qualifier = Bytes.toBytes(random.nextInt()); - byte[] value = Bytes.toBytes(random.nextInt()); + byte[] qualifier = Bytes.toBytes(rand.nextInt()); + byte[] value = Bytes.toBytes(rand.nextInt()); put.addColumn(family, qualifier, value); } table.put(put); @@ -280,7 +280,7 @@ public void testAdvancedConfigOverride() throws Exception { Admin admin = TEST_UTIL.getAdmin(); // Create 3 store files. - byte[] row = Bytes.toBytes(random.nextInt()); + byte[] row = Bytes.toBytes(ThreadLocalRandom.current().nextInt()); performMultiplePutAndFlush(admin, table, row, FAMILY, 3, 100); try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java index f9ac721f9a7a..d81b5a26f428 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java @@ -21,7 +21,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellBuilderFactory; import org.apache.hadoop.hbase.CellBuilderType; @@ -156,7 +155,7 @@ public void testBlockMultiLimits() throws Exception { // however the block being reference will be larger than MAX_SIZE. // This should cause the regionserver to try and send a result immediately. byte[] value = new byte[MAX_SIZE - 100]; - ThreadLocalRandom.current().nextBytes(value); + Bytes.random(value); for (byte[] col:cols) { Put p = new Put(row); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestTooBigException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestTooBigException.java index 5ea59724e8c4..1ffdf1fae539 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestTooBigException.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestTooBigException.java @@ -20,7 +20,6 @@ import static org.apache.hadoop.hbase.ipc.RpcServer.MAX_REQUEST_SIZE; import static org.junit.Assert.assertThrows; -import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; @@ -33,7 +32,6 @@ import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; - import org.apache.hbase.thirdparty.com.google.common.io.Closeables; @Category({ MediumTests.class, ClientTests.class }) @@ -68,7 +66,7 @@ public static void tearDownAfterClass() throws Exception { @Test public void testHbasePutDeleteCell() throws Exception { byte[] value = new byte[1024]; - ThreadLocalRandom.current().nextBytes(value); + Bytes.random(value); for (int m = 0; m < 100; m++) { Put p = new Put(Bytes.toBytes("bigrow-" + m)); // max request is 10K, big request = 100 * 1K diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java index 06c2de4b4f03..161f3eba4ea6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java @@ -43,8 +43,6 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Run tests related to {@link TimestampsFilter} using HBase client APIs. @@ -58,7 +56,6 @@ public class TestTimestampsFilter { public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestTimestampsFilter.class); - private static final Logger LOG = LoggerFactory.getLogger(TestTimestampsFilter.class); private final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); @Rule diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/locking/TestEntityLocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/locking/TestEntityLocks.java index 2b8c3a927cd4..50b6a278a4a2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/locking/TestEntityLocks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/locking/TestEntityLocks.java @@ -28,7 +28,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; @@ -97,7 +97,7 @@ public void setUp() throws Exception { admin = getAdmin(); lockReqArgCaptor = ArgumentCaptor.forClass(LockRequest.class); lockHeartbeatReqArgCaptor = ArgumentCaptor.forClass(LockHeartbeatRequest.class); - procId = new Random().nextLong(); + procId = ThreadLocalRandom.current().nextLong(); } private boolean waitLockTimeOut(EntityLock lock, long maxWaitTimeMillis) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMetaTableMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMetaTableMetrics.java index 8391d80fff87..d80fd7178858 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMetaTableMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMetaTableMetrics.java @@ -34,6 +34,8 @@ import java.util.Map; import java.util.Random; import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; + import javax.management.MBeanAttributeInfo; import javax.management.MBeanInfo; import javax.management.MBeanServerConnection; @@ -99,7 +101,7 @@ public static void setupBeforeClass() throws Exception { UTIL.getConfiguration().set("hbase.coprocessor.region.classes", MetaTableMetrics.class.getName()); conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY, JMXListener.class.getName()); - Random rand = new Random(); + Random rand = ThreadLocalRandom.current(); for (int i = 0; i < 10; i++) { do { int sign = i % 2 == 0 ? 1 : -1; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/compress/HFileTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/compress/HFileTestBase.java index 9c97493cb38a..43e67d8f0643 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/compress/HFileTestBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/compress/HFileTestBase.java @@ -20,8 +20,10 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import java.security.SecureRandom; import java.util.List; +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -44,7 +46,6 @@ public class HFileTestBase { protected static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); protected static final Logger LOG = LoggerFactory.getLogger(HFileTestBase.class); - protected static final SecureRandom RNG = new SecureRandom(); protected static FileSystem FS; public static void setUpBeforeClass() throws Exception { @@ -105,13 +106,14 @@ public void doTest(Configuration conf, Path path, Compression.Algorithm compress assertEquals("Did not read back as many KVs as written", i, testKvs.size()); // Test random seeks with pread + Random rand = ThreadLocalRandom.current(); LOG.info("Random seeking with " + fileContext); reader = HFile.createReader(FS, path, cacheConf, true, conf); try { scanner = reader.getScanner(conf, false, true); assertTrue("Initial seekTo failed", scanner.seekTo()); for (i = 0; i < 100; i++) { - KeyValue kv = testKvs.get(RNG.nextInt(testKvs.size())); + KeyValue kv = testKvs.get(rand.nextInt(testKvs.size())); assertEquals("Unable to find KV as expected: " + kv, 0, scanner.seekTo(kv)); } } finally { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java index 1b1014fb6c6d..8ac59b94346e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java @@ -25,6 +25,8 @@ import java.util.Collections; import java.util.List; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -254,7 +256,7 @@ private void compactAndWait() throws IOException, InterruptedException { @Test public void testCrazyRandomChanges() throws Exception { prepareTest("RandomChanges"); - Random rand = new Random(2934298742974297L); + Random rand = ThreadLocalRandom.current(); for (int i = 0; i < 10; ++i) { int encodingOrdinal = rand.nextInt(DataBlockEncoding.values().length); DataBlockEncoding encoding = DataBlockEncoding.values()[encodingOrdinal]; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java index da5bd73a9e9b..e4d1aa914c6d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java @@ -30,6 +30,7 @@ import java.util.Collection; import java.util.List; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ArrayBackedTag; @@ -87,7 +88,6 @@ public class TestDataBlockEncoders { private final Configuration conf = HBaseConfiguration.create(); private final RedundantKVGenerator generator = new RedundantKVGenerator(); - private final Random randomizer = new Random(42L); private final boolean includesMemstoreTS; private final boolean includesTags; private final boolean useOffheapData; @@ -217,13 +217,14 @@ public void testSeekingOnSample() throws IOException { LOG.info("Testing it!"); // test it! // try a few random seeks + Random rand = ThreadLocalRandom.current(); for (boolean seekBefore : new boolean[] { false, true }) { for (int i = 0; i < NUM_RANDOM_SEEKS; ++i) { int keyValueId; if (!seekBefore) { - keyValueId = randomizer.nextInt(sampleKv.size()); + keyValueId = rand.nextInt(sampleKv.size()); } else { - keyValueId = randomizer.nextInt(sampleKv.size() - 1) + 1; + keyValueId = rand.nextInt(sampleKv.size() - 1) + 1; } KeyValue keyValue = sampleKv.get(keyValueId); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java index f610ad6f07a5..7daedc45f8df 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java @@ -30,8 +30,8 @@ import java.util.HashSet; import java.util.Random; import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MultithreadedTestUtil; @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; import org.apache.hadoop.hbase.nio.ByteBuff; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ChecksumType; public class CacheTestUtils { @@ -282,11 +283,11 @@ public BlockType getBlockType() { public static HFileBlockPair[] generateHFileBlocks(int blockSize, int numBlocks) { HFileBlockPair[] returnedBlocks = new HFileBlockPair[numBlocks]; - Random rand = new Random(); + Random rand = ThreadLocalRandom.current(); HashSet usedStrings = new HashSet<>(); for (int i = 0; i < numBlocks; i++) { ByteBuffer cachedBuffer = ByteBuffer.allocate(blockSize); - rand.nextBytes(cachedBuffer.array()); + Bytes.random(cachedBuffer.array()); cachedBuffer.rewind(); int onDiskSizeWithoutHeader = blockSize; int uncompressedSizeWithoutHeader = blockSize; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java index 301894a78688..cb9fc4880488 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java @@ -39,6 +39,8 @@ import java.util.List; import java.util.Objects; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -280,16 +282,15 @@ private Path writeStoreFile() throws IOException { StoreFileWriter sfw = new StoreFileWriter.Builder(conf, fs).withOutputDir(storeFileParentDir) .withFileContext(meta).build(); - final int rowLen = 32; - Random RNG = new Random(); + Random rand = ThreadLocalRandom.current(); for (int i = 0; i < 1000; ++i) { - byte[] k = RandomKeyValueUtil.randomOrderedKey(RNG, i); - byte[] v = RandomKeyValueUtil.randomValue(RNG); - int cfLen = RNG.nextInt(k.length - rowLen + 1); + byte[] k = RandomKeyValueUtil.randomOrderedKey(rand, i); + byte[] v = RandomKeyValueUtil.randomValue(rand); + int cfLen = rand.nextInt(k.length - rowLen + 1); KeyValue kv = new KeyValue(k, 0, rowLen, k, rowLen, cfLen, k, rowLen + cfLen, - k.length - rowLen - cfLen, RNG.nextLong(), generateKeyType(RNG), v, 0, v.length); + k.length - rowLen - cfLen, rand.nextLong(), generateKeyType(rand), v, 0, v.length); sfw.append(kv); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java index 4fb0e7b19c5b..873976f05dfa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java @@ -41,7 +41,6 @@ import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.Executors; import java.util.concurrent.Future; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -111,6 +110,7 @@ public class TestHFileBlock { private static float CHANCE_TO_REPEAT = 0.6f; private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private static final Random RNG = new Random(); // This test depends on Random#setSeed private FileSystem fs; private final boolean includesMemstoreTS; @@ -182,9 +182,9 @@ static void writeTestBlockContents(DataOutputStream dos) throws IOException { static int writeTestKeyValues(HFileBlock.Writer hbw, int seed, boolean includesMemstoreTS, boolean useTag) throws IOException { List keyValues = new ArrayList<>(); - Random randomizer = new Random(42L + seed); // just any fixed number // generate keyValues + RNG.setSeed(42); // just any fixed number for (int i = 0; i < NUM_KEYVALUES; ++i) { byte[] row; long timestamp; @@ -193,35 +193,35 @@ static int writeTestKeyValues(HFileBlock.Writer hbw, int seed, boolean includesM byte[] value; // generate it or repeat, it should compress well - if (0 < i && randomizer.nextFloat() < CHANCE_TO_REPEAT) { - row = CellUtil.cloneRow(keyValues.get(randomizer.nextInt(keyValues.size()))); + if (0 < i && RNG.nextFloat() < CHANCE_TO_REPEAT) { + row = CellUtil.cloneRow(keyValues.get(RNG.nextInt(keyValues.size()))); } else { row = new byte[FIELD_LENGTH]; - randomizer.nextBytes(row); + RNG.nextBytes(row); } if (0 == i) { family = new byte[FIELD_LENGTH]; - randomizer.nextBytes(family); + RNG.nextBytes(family); } else { family = CellUtil.cloneFamily(keyValues.get(0)); } - if (0 < i && randomizer.nextFloat() < CHANCE_TO_REPEAT) { - qualifier = CellUtil.cloneQualifier(keyValues.get(randomizer.nextInt(keyValues.size()))); + if (0 < i && RNG.nextFloat() < CHANCE_TO_REPEAT) { + qualifier = CellUtil.cloneQualifier(keyValues.get(RNG.nextInt(keyValues.size()))); } else { qualifier = new byte[FIELD_LENGTH]; - randomizer.nextBytes(qualifier); + RNG.nextBytes(qualifier); } - if (0 < i && randomizer.nextFloat() < CHANCE_TO_REPEAT) { - value = CellUtil.cloneValue(keyValues.get(randomizer.nextInt(keyValues.size()))); + if (0 < i && RNG.nextFloat() < CHANCE_TO_REPEAT) { + value = CellUtil.cloneValue(keyValues.get(RNG.nextInt(keyValues.size()))); } else { value = new byte[FIELD_LENGTH]; - randomizer.nextBytes(value); + RNG.nextBytes(value); } - if (0 < i && randomizer.nextFloat() < CHANCE_TO_REPEAT) { + if (0 < i && RNG.nextFloat() < CHANCE_TO_REPEAT) { timestamp = keyValues.get( - randomizer.nextInt(keyValues.size())).getTimestamp(); + RNG.nextInt(keyValues.size())).getTimestamp(); } else { - timestamp = randomizer.nextLong(); + timestamp = RNG.nextLong(); } if (!useTag) { keyValues.add(new KeyValue(row, family, qualifier, timestamp, value)); @@ -238,7 +238,7 @@ static int writeTestKeyValues(HFileBlock.Writer hbw, int seed, boolean includesM for (KeyValue kv : keyValues) { totalSize += kv.getLength(); if (includesMemstoreTS) { - long memstoreTS = randomizer.nextLong(); + long memstoreTS = RNG.nextLong(); kv.setSequenceId(memstoreTS); totalSize += WritableUtils.getVIntSize(memstoreTS); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java index 08165bbdf442..3673d991f9a9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java @@ -90,7 +90,7 @@ public TestHFileBlockIndex(Compression.Algorithm compr) { } private static final Logger LOG = LoggerFactory.getLogger(TestHFileBlockIndex.class); - + private static final Random RNG = new Random(); // This test depends on Random#setSeed private static final int NUM_DATA_BLOCKS = 1000; private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); @@ -100,7 +100,6 @@ public TestHFileBlockIndex(Compression.Algorithm compr) { private static FileSystem fs; private Path path; - private Random rand; private long rootIndexOffset; private int numRootEntries; private int numLevels; @@ -124,9 +123,9 @@ public TestHFileBlockIndex(Compression.Algorithm compr) { @Before public void setUp() throws IOException { keys.clear(); - rand = new Random(2389757); firstKeyInFile = null; conf = TEST_UTIL.getConfiguration(); + RNG.setSeed(2389757); // This test requires at least HFile format version 2. conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MAX_FORMAT_VERSION); @@ -143,9 +142,9 @@ public void testBlockIndex() throws IOException { private void clear() throws IOException { keys.clear(); - rand = new Random(2389757); firstKeyInFile = null; conf = TEST_UTIL.getConfiguration(); + RNG.setSeed(2389757); // This test requires at least HFile format version 2. conf.setInt(HFile.FORMAT_VERSION_KEY, 3); @@ -275,9 +274,8 @@ private void writeWholeIndex(boolean useTags) throws IOException { FSDataOutputStream outputStream = fs.create(path); HFileBlockIndex.BlockIndexWriter biw = new HFileBlockIndex.BlockIndexWriter(hbw, null, null); - for (int i = 0; i < NUM_DATA_BLOCKS; ++i) { - hbw.startWriting(BlockType.DATA).write(Bytes.toBytes(String.valueOf(rand.nextInt(1000)))); + hbw.startWriting(BlockType.DATA).write(Bytes.toBytes(String.valueOf(RNG.nextInt(1000)))); long blockOffset = outputStream.getPos(); hbw.writeHeaderAndData(outputStream); @@ -286,7 +284,7 @@ private void writeWholeIndex(boolean useTags) throws IOException { byte[] qualifier = Bytes.toBytes("q"); for (int j = 0; j < 16; ++j) { byte[] k = - new KeyValue(RandomKeyValueUtil.randomOrderedKey(rand, i * 16 + j), family, qualifier, + new KeyValue(RandomKeyValueUtil.randomOrderedKey(RNG, i * 16 + j), family, qualifier, EnvironmentEdgeManager.currentTime(), KeyValue.Type.Put).getKey(); keys.add(k); if (j == 8) { @@ -354,7 +352,7 @@ public void testSecondaryIndexBinarySearch() throws IOException { int secondaryIndexEntries[] = new int[numTotalKeys]; for (int i = 0; i < numTotalKeys; ++i) { - byte[] k = RandomKeyValueUtil.randomOrderedKey(rand, i * 2); + byte[] k = RandomKeyValueUtil.randomOrderedKey(RNG, i * 2); KeyValue cell = new KeyValue(k, Bytes.toBytes("f"), Bytes.toBytes("q"), Bytes.toBytes("val")); //KeyValue cell = new KeyValue.KeyOnlyKeyValue(k, 0, k.length); @@ -479,8 +477,8 @@ public void testBlockIndexChunk() throws IOException { c.writeRoot(dos); assertEquals(c.getRootSize(), dos.size()); - byte[] k = RandomKeyValueUtil.randomOrderedKey(rand, i); - numSubEntries += rand.nextInt(5) + 1; + byte[] k = RandomKeyValueUtil.randomOrderedKey(RNG, i); + numSubEntries += RNG.nextInt(5) + 1; keys.add(k); c.add(k, getDummyFileOffset(i), getDummyOnDiskSize(i), numSubEntries); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java index 6b71f495e0c7..acc17acdf969 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileEncryption.java @@ -26,8 +26,10 @@ import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; -import java.security.SecureRandom; import java.util.List; +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -67,7 +69,6 @@ public class TestHFileEncryption { private static final Logger LOG = LoggerFactory.getLogger(TestHFileEncryption.class); private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); - private static final SecureRandom RNG = new SecureRandom(); private static FileSystem fs; private static Encryption.Context cryptoContext; @@ -90,7 +91,7 @@ public static void setUp() throws Exception { assertNotNull(aes); cryptoContext.setCipher(aes); byte[] key = new byte[aes.getKeyLength()]; - RNG.nextBytes(key); + Bytes.secureRandom(key); cryptoContext.setKey(key); } @@ -135,8 +136,9 @@ private long readAndVerifyBlock(long pos, HFileContext ctx, HFileBlock.FSReaderI public void testDataBlockEncryption() throws IOException { final int blocks = 10; final int[] blockSizes = new int[blocks]; + final Random rand = ThreadLocalRandom.current(); for (int i = 0; i < blocks; i++) { - blockSizes[i] = (1024 + RNG.nextInt(1024 * 63)) / Bytes.SIZEOF_INT; + blockSizes[i] = (1024 + rand.nextInt(1024 * 63)) / Bytes.SIZEOF_INT; } for (Compression.Algorithm compression : HBaseCommonTestingUtil.COMPRESSION_ALGORITHMS) { Path path = new Path(TEST_UTIL.getDataTestDir(), "block_v3_" + compression + "_AES"); @@ -271,12 +273,13 @@ public void testHFileEncryption() throws Exception { // Test random seeks with pread LOG.info("Random seeking with " + fileContext); + Random rand = ThreadLocalRandom.current(); reader = HFile.createReader(fs, path, cacheConf, true, conf); try { scanner = reader.getScanner(conf, false, true); assertTrue("Initial seekTo failed", scanner.seekTo()); for (i = 0; i < 100; i++) { - KeyValue kv = testKvs.get(RNG.nextInt(testKvs.size())); + KeyValue kv = testKvs.get(rand.nextInt(testKvs.size())); assertEquals("Unable to find KV as expected: " + kv, 0, scanner.seekTo(kv)); } } finally { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileScannerImplReferenceCount.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileScannerImplReferenceCount.java index 1243d466ca38..c52c517eafc4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileScannerImplReferenceCount.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileScannerImplReferenceCount.java @@ -32,7 +32,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.Random; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -90,15 +89,15 @@ public static Collection data() { private static final Logger LOG = LoggerFactory.getLogger(TestHFileScannerImplReferenceCount.class); private static final HBaseTestingUtil UTIL = new HBaseTestingUtil(); + private static final Random RNG = new Random(9713312); // Just a fixed seed. private static final byte[] FAMILY = Bytes.toBytes("f"); private static final byte[] QUALIFIER = Bytes.toBytes("q"); private static final byte[] SUFFIX = randLongBytes(); private static final int CELL_COUNT = 1000; private static byte[] randLongBytes() { - Random rand = new Random(); byte[] keys = new byte[30]; - rand.nextBytes(keys); + Bytes.random(keys); return keys; } @@ -166,12 +165,10 @@ private void writeHFile(Configuration conf, FileSystem fs, Path hfilePath, Algor try (HFile.Writer writer = new HFile.WriterFactory(conf, new CacheConfig(conf)).withPath(fs, hfilePath) .withFileContext(context).create()) { - Random rand = new Random(9713312); // Just a fixed seed. for (int i = 0; i < cellCount; ++i) { byte[] keyBytes = Bytes.add(Bytes.toBytes(i), SUFFIX); - // A random-length random value. - byte[] valueBytes = RandomKeyValueUtil.randomValue(rand); + byte[] valueBytes = RandomKeyValueUtil.randomValue(RNG); KeyValue keyValue = new KeyValue(keyBytes, FAMILY, QUALIFIER, HConstants.LATEST_TIMESTAMP, valueBytes); if (firstCell == null) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java index d14ba00a532c..d58668352578 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java @@ -28,7 +28,6 @@ import java.util.Collection; import java.util.List; import java.util.Random; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; @@ -76,9 +75,8 @@ public class TestHFileWriterV3 { HBaseClassTestRule.forClass(TestHFileWriterV3.class); private static final Logger LOG = LoggerFactory.getLogger(TestHFileWriterV3.class); - - private static final HBaseTestingUtil TEST_UTIL = - new HBaseTestingUtil(); + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private static final Random RNG = new Random(9713312); // Just a fixed seed. private Configuration conf; private FileSystem fs; @@ -135,20 +133,17 @@ private void writeDataAndReadFromHFile(Path hfilePath, .withFileContext(context) .create(); - Random rand = new Random(9713312); // Just a fixed seed. List keyValues = new ArrayList<>(entryCount); - for (int i = 0; i < entryCount; ++i) { - byte[] keyBytes = RandomKeyValueUtil.randomOrderedKey(rand, i); - + byte[] keyBytes = RandomKeyValueUtil.randomOrderedKey(RNG, i); // A random-length random value. - byte[] valueBytes = RandomKeyValueUtil.randomValue(rand); + byte[] valueBytes = RandomKeyValueUtil.randomValue(RNG); KeyValue keyValue = null; if (useTags) { ArrayList tags = new ArrayList<>(); - for (int j = 0; j < 1 + rand.nextInt(4); j++) { + for (int j = 0; j < 1 + RNG.nextInt(4); j++) { byte[] tagBytes = new byte[16]; - rand.nextBytes(tagBytes); + RNG.nextBytes(tagBytes); tags.add(new ArrayBackedTag((byte) 1, tagBytes)); } keyValue = new KeyValue(keyBytes, null, null, HConstants.LATEST_TIMESTAMP, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3WithDataEncoders.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3WithDataEncoders.java index cbf615e76f97..ce2ad18883ed 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3WithDataEncoders.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3WithDataEncoders.java @@ -24,7 +24,6 @@ import java.util.Collection; import java.util.List; import java.util.Random; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; @@ -73,9 +72,8 @@ public class TestHFileWriterV3WithDataEncoders { private static final Logger LOG = LoggerFactory.getLogger(TestHFileWriterV3WithDataEncoders.class); - - private static final HBaseTestingUtil TEST_UTIL = - new HBaseTestingUtil(); + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private static final Random RNG = new Random(9713312); // Just a fixed seed. private Configuration conf; private FileSystem fs; @@ -150,11 +148,8 @@ private void writeDataAndReadFromHFile(Path hfilePath, .withFileContext(context) .create(); - Random rand = new Random(9713312); // Just a fixed seed. List keyValues = new ArrayList<>(entryCount); - - writeKeyValues(entryCount, useTags, writer, rand, keyValues); - + writeKeyValues(entryCount, useTags, writer, RNG, keyValues); FSDataInputStream fsdis = fs.open(hfilePath); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java index bfb40bc6672a..d3bcba1af876 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java @@ -66,6 +66,7 @@ public class TestLazyDataBlockDecompression { private static final Logger LOG = LoggerFactory.getLogger(TestLazyDataBlockDecompression.class); private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private static final Random RNG = new Random(9713312); // Just a fixed seed. private FileSystem fs; @@ -101,14 +102,12 @@ private static void writeHFile(Configuration conf, CacheConfig cc, FileSystem fs .withFileContext(cxt) .create(); - // write a bunch of random kv's - Random rand = new Random(9713312); // some seed. + // write a bunch of random kvs final byte[] family = Bytes.toBytes("f"); final byte[] qualifier = Bytes.toBytes("q"); - for (int i = 0; i < entryCount; i++) { - byte[] keyBytes = RandomKeyValueUtil.randomOrderedKey(rand, i); - byte[] valueBytes = RandomKeyValueUtil.randomValue(rand); + byte[] keyBytes = RandomKeyValueUtil.randomOrderedKey(RNG, i); + byte[] valueBytes = RandomKeyValueUtil.randomValue(RNG); // make a real keyvalue so that hfile tool can examine it writer.append(new KeyValue(keyBytes, family, qualifier, valueBytes)); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruAdaptiveBlockCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruAdaptiveBlockCache.java index f29d12ac315b..a922b308c005 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruAdaptiveBlockCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruAdaptiveBlockCache.java @@ -27,6 +27,7 @@ import java.util.Random; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -944,9 +945,9 @@ public void testCacheBlockNextBlockMetadataMissing() { private CachedItem [] generateRandomBlocks(int numBlocks, long maxSize) { CachedItem [] blocks = new CachedItem[numBlocks]; - Random r = new Random(); + Random rand = ThreadLocalRandom.current(); for(int i=0;i data() { return Arrays.asList(new Object[][] { @@ -162,7 +160,7 @@ private Path createAndGetTestDir() throws IOException { * Return a random element from {@code a}. */ private static T randFrom(List a) { - return a.get(RAND.nextInt(a.size())); + return a.get(ThreadLocalRandom.current().nextInt(a.size())); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java index 4e5cd2840d8f..f4f63a3badc3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java @@ -27,9 +27,10 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Optional; -import java.util.Random; import java.util.TreeMap; import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.Abortable; @@ -154,7 +155,6 @@ class MockRegionServer implements AdminProtos.AdminService.BlockingInterface, private final ServerName sn; private final ZKWatcher zkw; private final Configuration conf; - private final Random random = new Random(); /** * Map of regions to map of rows and {@link Result}. Used as data source when @@ -251,7 +251,7 @@ public boolean isAborted() { } public long openScanner(byte[] regionName, Scan scan) throws IOException { - long scannerId = this.random.nextLong(); + long scannerId = ThreadLocalRandom.current().nextLong(); this.scannersAndOffsets.put(scannerId, new RegionNameAndIndex(regionName)); return scannerId; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java index 0a71e01d205c..0fc8cddd19a3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java @@ -32,6 +32,7 @@ import java.util.List; import java.util.Map; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CatalogFamilyFormat; @@ -63,7 +64,6 @@ import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Pair; import org.apache.zookeeper.KeeperException; import org.junit.AfterClass; @@ -193,13 +193,13 @@ private void killRandomServerAndVerifyAssignment() throws IOException, InterruptedException, KeeperException { ServerName serverToKill = null; int killIndex = 0; - Random random = new Random(EnvironmentEdgeManager.currentTime()); + Random rand = ThreadLocalRandom.current(); ServerName metaServer = TEST_UTIL.getHBaseCluster().getServerHoldingMeta(); LOG.debug("Server holding meta " + metaServer); boolean isNamespaceServer = false; do { // kill a random non-meta server carrying at least one region - killIndex = random.nextInt(SLAVES); + killIndex = rand.nextInt(SLAVES); serverToKill = TEST_UTIL.getHBaseCluster().getRegionServer(killIndex).getServerName(); Collection regs = TEST_UTIL.getHBaseCluster().getRegionServer(killIndex).getOnlineRegionsLocalContext(); @@ -259,10 +259,10 @@ public void testRandomizedMatrix() { int rows = 100; int cols = 100; float[][] matrix = new float[rows][cols]; - Random random = new Random(); + Random rand = ThreadLocalRandom.current(); for (int i = 0; i < rows; i++) { for (int j = 0; j < cols; j++) { - matrix[i][j] = random.nextFloat(); + matrix[i][j] = rand.nextFloat(); } } @@ -283,7 +283,7 @@ public void testRandomizedMatrix() { // the same values on the original matrix. int[] transformedIndices = new int[rows]; for (int i = 0; i < rows; i++) { - transformedIndices[i] = random.nextInt(cols); + transformedIndices[i] = rand.nextInt(cols); } int[] invertedTransformedIndices = rm.invertIndices(transformedIndices); float[] transformedValues = new float[rows]; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java index 02e8600ae4e4..696e71f0a2c0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java @@ -27,7 +27,6 @@ import java.net.SocketTimeoutException; import java.util.Arrays; import java.util.NavigableMap; -import java.util.Random; import java.util.Set; import java.util.SortedSet; import java.util.concurrent.ConcurrentSkipListMap; @@ -36,6 +35,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CallQueueTooBigException; @@ -582,12 +582,10 @@ public void run() { } protected class RandRsExecutor extends NoopRsExecutor { - private final Random rand = new Random(); - @Override public ExecuteProceduresResponse sendRequest(ServerName server, ExecuteProceduresRequest req) throws IOException { - switch (rand.nextInt(5)) { + switch (ThreadLocalRandom.current().nextInt(5)) { case 0: throw new ServerNotRunningYetException("wait on server startup"); case 1: @@ -606,7 +604,7 @@ protected RegionOpeningState execOpenRegion(final ServerName server, RegionOpenI RegionInfo hri = ProtobufUtil.toRegionInfo(openReq.getRegion()); long previousOpenSeqNum = am.getRegionStates().getOrCreateRegionStateNode(hri).getOpenSeqNum(); - switch (rand.nextInt(3)) { + switch (ThreadLocalRandom.current().nextInt(3)) { case 0: LOG.info("Return OPENED response"); sendTransitionReport(server, openReq.getRegion(), TransitionCode.OPENED, @@ -637,7 +635,7 @@ public void run() { protected CloseRegionResponse execCloseRegion(ServerName server, byte[] regionName) throws IOException { CloseRegionResponse.Builder resp = CloseRegionResponse.newBuilder(); - boolean closed = rand.nextBoolean(); + boolean closed = ThreadLocalRandom.current().nextBoolean(); if (closed) { RegionInfo hri = am.getRegionInfo(regionName); sendTransitionReport(server, ProtobufUtil.toRegionInfo(hri), TransitionCode.CLOSED, -1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/RSGroupableBalancerTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/RSGroupableBalancerTestBase.java index 9ae3e80d881d..964528d2810a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/RSGroupableBalancerTestBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/RSGroupableBalancerTestBase.java @@ -23,15 +23,16 @@ import java.io.FileNotFoundException; import java.io.IOException; -import java.security.SecureRandom; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Random; import java.util.Set; import java.util.TreeMap; import java.util.TreeSet; +import java.util.concurrent.ThreadLocalRandom; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -53,7 +54,6 @@ import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; - import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @@ -62,7 +62,6 @@ */ public class RSGroupableBalancerTestBase extends BalancerTestBase { - static SecureRandom rand = new SecureRandom(); static String[] groups = new String[] { RSGroupInfo.DEFAULT_GROUP, "dg2", "dg3", "dg4" }; static TableName table0 = TableName.valueOf("dt0"); static TableName[] tables = new TableName[] { TableName.valueOf("dt1"), TableName.valueOf("dt2"), @@ -298,10 +297,10 @@ protected Map> mockClusterServers() throws IOExcept protected List randomRegions(int numRegions) { List regions = new ArrayList<>(numRegions); byte[] start = new byte[16]; + Bytes.random(start); byte[] end = new byte[16]; - rand.nextBytes(start); - rand.nextBytes(end); - int regionIdx = rand.nextInt(tables.length); + Bytes.random(end); + int regionIdx = ThreadLocalRandom.current().nextInt(tables.length); for (int i = 0; i < numRegions; i++) { Bytes.putInt(start, 0, numRegions << 1); Bytes.putInt(end, 0, (numRegions << 1) + 1); @@ -344,6 +343,7 @@ protected List assignedRegions(int numRegions, ServerName sn) throws protected static List generateServers(int numServers) { List servers = new ArrayList<>(numServers); + Random rand = ThreadLocalRandom.current(); for (int i = 0; i < numServers; i++) { String host = "server" + rand.nextInt(100000); int port = rand.nextInt(60000); @@ -370,6 +370,7 @@ protected static Map constructGroupInfo(List se groupMap.put(grpName, RSGroupInfo); index++; } + Random rand = ThreadLocalRandom.current(); while (index < servers.size()) { int grpIndex = rand.nextInt(groups.length); groupMap.get(groups[grpIndex]).addServer(servers.get(index).getAddress()); @@ -385,6 +386,7 @@ protected static Map constructGroupInfo(List se */ protected static Map constructTableDesc(boolean hasBogusTable) { Map tds = new HashMap<>(); + Random rand = ThreadLocalRandom.current(); int index = rand.nextInt(groups.length); for (int i = 0; i < tables.length; i++) { int grpIndex = (i + index) % groups.length; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerStatusTagInJMXMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerStatusTagInJMXMetrics.java index 2ff1153cdcd2..a3c566a75c2e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerStatusTagInJMXMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBalancerStatusTagInJMXMetrics.java @@ -20,6 +20,8 @@ import static org.junit.Assert.assertEquals; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; @@ -57,7 +59,7 @@ public class TestBalancerStatusTagInJMXMetrics extends BalancerTestBase { @BeforeClass public static void setupBeforeClass() throws Exception { conf = UTIL.getConfiguration(); - Random rand = new Random(); + Random rand = ThreadLocalRandom.current(); for (int i = 0; i < 10; i++) { do { int sign = i % 2 == 0 ? 1 : -1; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticBalancerJmxMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticBalancerJmxMetrics.java index 727fcb5e2a8a..ca9133961299 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticBalancerJmxMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticBalancerJmxMetrics.java @@ -27,6 +27,8 @@ import java.util.Map; import java.util.Random; import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; + import javax.management.MBeanAttributeInfo; import javax.management.MBeanInfo; import javax.management.MBeanServerConnection; @@ -96,7 +98,7 @@ public static void setupBeforeClass() throws Exception { conf.setClass("hbase.util.ip.to.rack.determiner", MockMapping.class, DNSToSwitchMapping.class); conf.setFloat("hbase.regions.slop", 0.0f); conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY, JMXListener.class.getName()); - Random rand = new Random(); + Random rand = ThreadLocalRandom.current(); for (int i = 0; i < 10; i++) { do { int sign = i % 2 == 0 ? 1 : -1; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java index c3e7741aa2ec..06ad2f043e21 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java @@ -22,7 +22,7 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CommonFSUtils; import org.apache.hadoop.hbase.util.StoppableImplementation; import org.junit.AfterClass; @@ -536,13 +537,12 @@ public void testMinimumNumberOfThreads() throws Exception { } private void createFiles(FileSystem fs, Path parentDir, int numOfFiles) throws IOException { - Random random = new Random(); for (int i = 0; i < numOfFiles; i++) { - int xMega = 1 + random.nextInt(3); // size of each file is between 1~3M + int xMega = 1 + ThreadLocalRandom.current().nextInt(3); // size of each file is between 1~3M try (FSDataOutputStream fsdos = fs.create(new Path(parentDir, "file-" + i))) { for (int m = 0; m < xMega; m++) { byte[] M = new byte[1024 * 1024]; - random.nextBytes(M); + Bytes.random(M); fsdos.write(M); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java index 0408ad1b1685..ca41c559f5a6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.util.List; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; @@ -419,7 +420,7 @@ public void run() { private void createFilesForTesting(int largeFileNum, int smallFileNum, FileSystem fs, Path archivedHfileDir) throws IOException { - final Random rand = new Random(); + final Random rand = ThreadLocalRandom.current(); final byte[] large = new byte[1024 * 1024]; for (int i = 0; i < large.length; i++) { large[i] = (byte) rand.nextInt(128); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java index e924e6449d3f..ff76e2379240 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java @@ -32,9 +32,7 @@ import java.util.List; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicBoolean; - import org.apache.commons.io.FileUtils; -import org.apache.commons.lang3.RandomUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; @@ -54,6 +52,7 @@ import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.MockServer; import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper; @@ -347,8 +346,9 @@ private void createFiles(FileSystem fs, Path parentDir, int numOfFiles) throws I for (int i = 0; i < numOfFiles; i++) { // size of each file is 1M, 2M, or 3M int xMega = 1 + ThreadLocalRandom.current().nextInt(1, 4); + byte[] M = new byte[Math.toIntExact(FileUtils.ONE_MB * xMega)]; + Bytes.random(M); try (FSDataOutputStream fsdos = fs.create(new Path(parentDir, "file-" + i))) { - byte[] M = RandomUtils.nextBytes(Math.toIntExact(FileUtils.ONE_MB * xMega)); fsdos.write(M); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureSchedulerPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureSchedulerPerformanceEvaluation.java index c26d199cf3da..ae0c4c623125 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureSchedulerPerformanceEvaluation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureSchedulerPerformanceEvaluation.java @@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.master.procedure; import java.io.IOException; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.lang3.ArrayUtils; import org.apache.hadoop.hbase.HBaseTestingUtil; @@ -201,11 +201,10 @@ protected void processOptions(CommandLine cmd) { private class AddProcsWorker extends Thread { @Override public void run() { - final Random rand = new Random(EnvironmentEdgeManager.currentTime()); long procId = procIds.incrementAndGet(); int index; while (procId <= numOps) { - index = rand.nextInt(ops.length); + index = ThreadLocalRandom.current().nextInt(ops.length); procedureScheduler.addBack(ops[index].newProcedure(procId)); procId = procIds.incrementAndGet(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java index 01cfa8a501f8..5c225cca0ca4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java @@ -21,7 +21,8 @@ import static org.junit.Assert.assertTrue; import java.util.List; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; @@ -181,11 +182,10 @@ public void testAbortProcedureInterruptedNotAllowed() throws Exception { @Test public void testAbortNonExistProcedure() throws Exception { final ProcedureExecutor procExec = getMasterProcedureExecutor(); - Random randomGenerator = new Random(); long procId; // Generate a non-existing procedure do { - procId = randomGenerator.nextLong(); + procId = ThreadLocalRandom.current().nextLong(); } while (procExec.getResult(procId) != null); boolean abortResult = procExec.abort(procId, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java index d178d564f650..813c288c61dc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java @@ -25,7 +25,7 @@ import java.util.Date; import java.util.List; import java.util.Optional; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.conf.Configuration; @@ -51,6 +51,7 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + /** * This class is used for testing only. The main purpose is to emulate * random failures during MOB compaction process. @@ -80,7 +81,6 @@ public class FaultyMobStoreCompactor extends DefaultMobStoreCompactor { public static AtomicLong totalMajorCompactions = new AtomicLong(); static double failureProb = 0.1d; - static Random rnd = new Random(); public FaultyMobStoreCompactor(Configuration conf, HStore store) { super(conf, store); @@ -108,7 +108,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, boolean mustFail = false; if (compactMOBs) { mobCounter.incrementAndGet(); - double dv = rnd.nextDouble(); + double dv = ThreadLocalRandom.current().nextDouble(); if (dv < failureProb) { mustFail = true; totalFailures.incrementAndGet(); @@ -150,7 +150,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, long counter = 0; long countFailAt = -1; if (mustFail) { - countFailAt = rnd.nextInt(100); // randomly fail fast + countFailAt = ThreadLocalRandom.current().nextInt(100); // randomly fail fast } try { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobTestUtil.java index 3fdbb1ff95b3..4fe33d86ee02 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobTestUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobTestUtil.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.List; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -41,10 +42,10 @@ public class MobTestUtil { protected static String generateRandomString(int demoLength) { String base = "abcdefghijklmnopqrstuvwxyz"; - Random random = new Random(); + Random rand = ThreadLocalRandom.current(); StringBuilder sb = new StringBuilder(); for (int i = 0; i < demoLength; i++) { - int number = random.nextInt(base.length()); + int number = rand.nextInt(base.length()); sb.append(base.charAt(number)); } return sb.toString(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java index 35d62f4ffe72..36acf60dba5f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java @@ -19,7 +19,6 @@ import static org.junit.Assert.assertEquals; -import java.util.Random; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -176,7 +175,7 @@ private int secondsOfDay() { private byte[] makeDummyData(int size) { byte [] dummyData = new byte[size]; - new Random().nextBytes(dummyData); + Bytes.random(dummyData); return dummyData; } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionWithDefaults.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionWithDefaults.java index 5847a448df10..fc96f87cfd83 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionWithDefaults.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionWithDefaults.java @@ -24,7 +24,6 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; -import java.util.Random; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -139,13 +138,11 @@ public void setUp() throws Exception { } private void loadData(TableName tableName, int num) { - - Random r = new Random(); LOG.info("Started loading {} rows into {}", num, tableName); try (final Table table = HTU.getConnection().getTable(tableName)) { for (int i = 0; i < num; i++) { byte[] key = new byte[32]; - r.nextBytes(key); + Bytes.random(key); Put p = new Put(key); p.addColumn(fam, qualifier, mobVal); table.put(p); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java index 1c7d8b0c1034..81376e440b4e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.mob; -import java.util.Random; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; @@ -57,7 +56,6 @@ public class TestMobDataBlockEncoding { private static Admin admin; private static ColumnFamilyDescriptor columnFamilyDescriptor; private static TableDescriptor tableDescriptor; - private static Random random = new Random(); private static long defaultThreshold = 10; @BeforeClass @@ -90,7 +88,7 @@ public void setUp(long threshold, String TN, DataBlockEncoding encoding) */ private static byte[] generateMobValue(int size) { byte[] mobVal = new byte[size]; - random.nextBytes(mobVal); + Bytes.random(mobVal); return mobVal; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java index 1891de8adbb9..426e2d49a314 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java @@ -23,7 +23,8 @@ import static org.junit.Assert.assertTrue; import java.util.Date; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -41,8 +42,6 @@ public class TestMobFileName { public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestMobFileName.class); - private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); - private String uuid; private Date date; private String dateStr; @@ -51,11 +50,10 @@ public class TestMobFileName { @Before public void setUp() { - Random random = new Random(); - uuid = TEST_UTIL.getRandomUUID().toString().replaceAll("-", ""); + uuid = HBaseTestingUtil.getRandomUUID().toString().replaceAll("-", ""); date = new Date(); dateStr = MobUtils.formatDate(date); - startKey = Bytes.toBytes(random.nextInt()); + startKey = Bytes.toBytes(ThreadLocalRandom.current().nextInt()); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java index a037c036ff13..e1a64aa2b1b3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java @@ -30,8 +30,8 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Random; import java.util.Set; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -373,7 +373,7 @@ private int countMobRows() throws IOException { private byte[] makeDummyData(int size) { byte[] dummyData = new byte[size]; - new Random().nextBytes(dummyData); + Bytes.random(dummyData); return dummyData; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java index 04f51402cc7e..f0c442fb0455 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java @@ -19,7 +19,7 @@ import java.io.IOException; import java.util.List; -import java.util.Random; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -78,7 +78,6 @@ public class TestMobStoreScanner { private static Admin admin; private static ColumnFamilyDescriptor familyDescriptor; private static TableDescriptor tableDescriptor; - private static Random random = new Random(); private static long defaultThreshold = 10; private FileSystem fs; private Configuration conf; @@ -119,7 +118,7 @@ public void setUp(long threshold, TableName tn) throws Exception { */ private static byte[] generateMobValue(int size) { byte[] mobVal = new byte[size]; - random.nextBytes(mobVal); + Bytes.random(mobVal); return mobVal; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java index eef0ea563d9e..eada244cf845 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java @@ -27,7 +27,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Objects; -import java.util.Random; import java.util.Set; import java.util.TreeMap; import java.util.concurrent.atomic.AtomicLong; @@ -430,14 +429,13 @@ void writeData(TableName tn, long sizeInBytes, byte[] qual) throws IOException { long bytesToWrite = sizeInBytes; long rowKeyId = 0L; final StringBuilder sb = new StringBuilder(); - final Random r = new Random(); while (bytesToWrite > 0L) { sb.setLength(0); sb.append(Long.toString(rowKeyId)); // Use the reverse counter as the rowKey to get even spread across all regions Put p = new Put(Bytes.toBytes(sb.reverse().toString())); byte[] value = new byte[SIZE_PER_VALUE]; - r.nextBytes(value); + Bytes.random(value); p.addColumn(Bytes.toBytes(F1), qual, value); updates.add(p); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java index 7a3fc74676b5..5b475ae43431 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRegionSizeUse.java @@ -24,7 +24,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.Random; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; @@ -154,14 +154,13 @@ private TableName writeData(long sizeInBytes) throws IOException { long bytesToWrite = sizeInBytes; long rowKeyId = 0L; final StringBuilder sb = new StringBuilder(); - final Random r = new Random(); while (bytesToWrite > 0L) { sb.setLength(0); sb.append(Long.toString(rowKeyId)); // Use the reverse counter as the rowKey to get even spread across all regions Put p = new Put(Bytes.toBytes(sb.reverse().toString())); byte[] value = new byte[SIZE_PER_VALUE]; - r.nextBytes(value); + Bytes.random(value); p.addColumn(Bytes.toBytes(F1), Bytes.toBytes("q1"), value); updates.add(p); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java index 67caf8ea2091..3b27971f878a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -80,7 +81,6 @@ public class CreateRandomStoreFile { private Options options = new Options(); private int keyPrefixLen, keyLen, rowLen, cfLen, valueLen; - private Random rand; /** * Runs the tools. @@ -207,13 +207,12 @@ public boolean run(String[] args) throws IOException { .withFileContext(meta) .build(); - rand = new Random(); LOG.info("Writing " + numKV + " key/value pairs"); for (long i = 0; i < numKV; ++i) { sfw.append(generateKeyValue(i)); } - int numMetaBlocks = rand.nextInt(10) + 1; + int numMetaBlocks = ThreadLocalRandom.current().nextInt(10) + 1; LOG.info("Writing " + numMetaBlocks + " meta blocks"); for (int metaI = 0; metaI < numMetaBlocks; ++metaI) { sfw.getHFileWriter().appendMetaBlock(generateString(), @@ -246,13 +245,13 @@ private void configureKeyValue(long numKV, int keyLen, int valueLen) { } private int nextInRange(int range) { - return rand.nextInt(2 * range + 1) - range; + return ThreadLocalRandom.current().nextInt(2 * range + 1) - range; } public KeyValue generateKeyValue(long i) { byte[] k = generateKey(i); byte[] v = generateValue(); - + Random rand = ThreadLocalRandom.current(); return new KeyValue( k, 0, rowLen, k, rowLen, cfLen, @@ -279,6 +278,7 @@ public static KeyValue.Type generateKeyType(Random rand) { } private String generateString() { + Random rand = ThreadLocalRandom.current(); StringBuilder sb = new StringBuilder(); for (int i = 0; i < rand.nextInt(10); ++i) { sb.append((char) ('A' + rand.nextInt(26))); @@ -287,6 +287,7 @@ private String generateString() { } private byte[] generateKey(long i) { + Random rand = ThreadLocalRandom.current(); byte[] k = new byte[Math.max(keyPrefixLen, keyLen + nextInRange(LEN_VARIATION))]; for (int pos = keyPrefixLen - 1; pos >= 0; --pos) { @@ -300,6 +301,7 @@ private byte[] generateKey(long i) { } private byte[] generateValue() { + Random rand = ThreadLocalRandom.current(); byte[] v = new byte[Math.max(1, valueLen + nextInRange(LEN_VARIATION))]; for (int i = 0; i < v.length; ++i) { v[i] = (byte) rand.nextInt(256); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java index a221b00303a8..49aeae0c1051 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java @@ -29,7 +29,6 @@ import java.util.Arrays; import java.util.List; import java.util.Objects; -import java.util.Random; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -612,7 +611,6 @@ public static class AtomicOperation extends Thread { protected final int numOps; protected final AtomicLong timeStamps; protected final AtomicInteger failures; - protected final Random r = new Random(); public AtomicOperation(HRegion region, int numOps, AtomicLong timeStamps, AtomicInteger failures) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkloadBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkloadBase.java index 774e921bdaae..ffc13c677ddb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkloadBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkloadBase.java @@ -29,7 +29,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; -import java.util.Random; import java.util.UUID; import org.apache.hadoop.conf.Configuration; @@ -75,7 +74,6 @@ public class TestBulkloadBase { private static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); protected final WAL log = mock(WAL.class); protected final Configuration conf = HBaseConfiguration.create(); - private final Random random = new Random(); private final byte[] randomBytes = new byte[100]; protected final byte[] family1 = Bytes.toBytes("family1"); protected final byte[] family2 = Bytes.toBytes("family2"); @@ -98,7 +96,7 @@ public static Collection data() { @Before public void before() throws IOException { - random.nextBytes(randomBytes); + Bytes.random(randomBytes); if(useFileBasedSFT) { conf.set(StoreFileTrackerFactory.TRACKER_IMPL, "org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java index 1a5aa8ee137b..de6c54628f56 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java @@ -24,6 +24,8 @@ import java.util.ArrayList; import java.util.List; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; @@ -43,8 +45,6 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** Unit tests to test retrieving table/region compaction state*/ @Category({VerySlowRegionServerTests.class, LargeTests.class}) @@ -54,9 +54,7 @@ public class TestCompactionState { public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestCompactionState.class); - private static final Logger LOG = LoggerFactory.getLogger(TestCompactionState.class); private final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); - private final static Random random = new Random(); @Rule public TestName name = new TestName(); @@ -265,9 +263,10 @@ private static void loadData(final Table ht, final byte[][] families, final int rows, final int flushes) throws IOException { List puts = new ArrayList<>(rows); byte[] qualifier = Bytes.toBytes("val"); + Random rand = ThreadLocalRandom.current(); for (int i = 0; i < flushes; i++) { for (int k = 0; k < rows; k++) { - byte[] row = Bytes.toBytes(random.nextLong()); + byte[] row = Bytes.toBytes(rand.nextLong()); Put p = new Put(row); for (int j = 0; j < families.length; ++j) { p.addColumn(families[j], qualifier, row); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java index faf61b6e36b9..d685ad5bb9c6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; -import java.util.Random; + import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -59,7 +59,6 @@ public class TestDeleteMobTable { private final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); private final static byte[] FAMILY = Bytes.toBytes("family"); private final static byte[] QF = Bytes.toBytes("qualifier"); - private static Random random = new Random(); @Rule public TestName name = new TestName(); @@ -83,7 +82,7 @@ public static void tearDownAfterClass() throws Exception { */ private static byte[] generateMobValue(int size) { byte[] mobVal = new byte[size]; - random.nextBytes(mobVal); + Bytes.random(mobVal); return mobVal; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java index a082f4ffcc68..fcda02eb0ae0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java @@ -22,10 +22,10 @@ import java.io.IOException; import java.security.Key; -import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collection; import java.util.List; + import javax.crypto.spec.SecretKeySpec; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -76,13 +76,12 @@ public class TestEncryptionKeyRotation { static { // Create the test encryption keys - SecureRandom rng = new SecureRandom(); byte[] keyBytes = new byte[AES.KEY_LENGTH]; - rng.nextBytes(keyBytes); + Bytes.secureRandom(keyBytes); String algorithm = conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES); initialCFKey = new SecretKeySpec(keyBytes, algorithm); - rng.nextBytes(keyBytes); + Bytes.secureRandom(keyBytes); secondCFKey = new SecretKeySpec(keyBytes, algorithm); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java index 9934417ed1ab..e5722d53bcb1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java @@ -29,6 +29,7 @@ import java.util.Random; import java.util.Set; import java.util.TreeSet; +import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CatalogFamilyFormat; @@ -223,7 +224,7 @@ static class RegionSplitter extends Thread { @Override public void run() { try { - Random random = new Random(); + Random random = ThreadLocalRandom.current(); for (int i = 0; i < 5; i++) { List regions = MetaTableAccessor.getTableRegions(connection, tableName, true); if (regions.isEmpty()) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java index 2d23a37a46f3..fb560d114af5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.security.Key; -import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -31,6 +30,7 @@ import java.util.NavigableSet; import java.util.Optional; import java.util.concurrent.ConcurrentSkipListSet; + import javax.crypto.spec.SecretKeySpec; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -484,9 +484,8 @@ public void testMOBStoreEncryption() throws Exception { conf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName()); conf.set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase"); - SecureRandom rng = new SecureRandom(); byte[] keyBytes = new byte[AES.KEY_LENGTH]; - rng.nextBytes(keyBytes); + Bytes.secureRandom(keyBytes); String algorithm = conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES); Key cfKey = new SecretKeySpec(keyBytes, algorithm); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java index 6214b312eb22..2b4dca8bb1d5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java @@ -39,7 +39,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Random; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; @@ -1521,9 +1520,8 @@ public void testReplayBulkLoadEvent() throws IOException { primaryRegion = HRegion.openHRegion(rootDir, primaryHri, htd, walPrimary, CONF, rss, null); // bulk load a file into primary region - Random random = new Random(); byte[] randomValues = new byte[20]; - random.nextBytes(randomValues); + Bytes.random(randomValues); Path testPath = TEST_UTIL.getDataTestDirOnTestFS(); List> familyPaths = new ArrayList<>(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java index d15796c9423c..997410cc03d6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java @@ -21,6 +21,8 @@ import java.util.ArrayList; import java.util.List; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -125,16 +127,14 @@ public void testJoinedScanners() throws Exception { long rows_to_insert = 1000; int insert_batch = 20; - long time = System.nanoTime(); - Random rand = new Random(time); LOG.info("Make " + Long.toString(rows_to_insert) + " rows, total size = " + Float .toString(rows_to_insert * valueWidth / 1024 / 1024) + " MB"); + long time = System.nanoTime(); + Random rand = ThreadLocalRandom.current(); byte[] val_large = new byte[valueWidth]; - List puts = new ArrayList<>(); - for (long i = 0; i < rows_to_insert; i++) { Put put = new Put(Bytes.toBytes(Long.toString(i))); if (rand.nextInt(100) <= selectionRatio) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java index afbbfeabd37a..db708c599626 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java @@ -28,6 +28,7 @@ import java.util.Collections; import java.util.List; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.conf.Configuration; @@ -87,7 +88,6 @@ public void tearDown() throws Exception { @Test public void testReusingChunks() { - Random rand = new Random(); MemStoreLAB mslab = new MemStoreLABImpl(conf); int expectedOff = 0; ByteBuffer lastBuffer = null; @@ -95,6 +95,7 @@ public void testReusingChunks() { final byte[] cf = Bytes.toBytes("f"); final byte[] q = Bytes.toBytes("q"); // Randomly allocate some bytes + final Random rand = ThreadLocalRandom.current(); for (int i = 0; i < 100; i++) { int valSize = rand.nextInt(1000); KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java index 00569bc37da9..032ea4eef608 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java @@ -29,6 +29,7 @@ import java.util.Map; import java.util.Random; import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ByteBufferKeyValue; @@ -87,7 +88,6 @@ public static void tearDownAfterClass() throws Exception { */ @Test public void testLABRandomAllocation() { - Random rand = new Random(); MemStoreLAB mslab = new MemStoreLABImpl(); int expectedOff = 0; ByteBuffer lastBuffer = null; @@ -95,6 +95,7 @@ public void testLABRandomAllocation() { // 100K iterations by 0-1K alloc -> 50MB expected // should be reasonable for unit test and also cover wraparound // behavior + Random rand = ThreadLocalRandom.current(); for (int i = 0; i < 100000; i++) { int valSize = rand.nextInt(3); KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]); @@ -144,10 +145,9 @@ public void testLABThreading() throws Exception { allocations.add(allocsByThisThread); TestThread t = new MultithreadedTestUtil.RepeatingTestThread(ctx) { - private Random r = new Random(); @Override public void doAnAction() throws Exception { - int valSize = r.nextInt(3); + int valSize = ThreadLocalRandom.current().nextInt(3); KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]); int size = kv.getSerializedSize(); ByteBufferKeyValue newCell = (ByteBufferKeyValue) mslab.copyCellInto(kv); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java index fd2ce0ed9da2..1dae13322c3f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java @@ -25,6 +25,8 @@ import java.util.ArrayList; import java.util.List; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ByteBufferKeyValue; import org.apache.hadoop.hbase.Cell; @@ -69,7 +71,6 @@ public static void setUpBeforeClass() throws Exception { */ @Test public void testLABRandomAllocation() { - Random rand = new Random(); MemStoreLAB mslab = new MemStoreLABImpl(); int expectedOff = 0; ByteBuffer lastBuffer = null; @@ -77,6 +78,7 @@ public void testLABRandomAllocation() { // 100K iterations by 0-1K alloc -> 50MB expected // should be reasonable for unit test and also cover wraparound // behavior + Random rand = ThreadLocalRandom.current(); for (int i = 0; i < 100000; i++) { int valSize = rand.nextInt(1000); KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java index bed40bbf59e9..f059884b28d2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java @@ -32,6 +32,8 @@ import java.util.Random; import java.util.Set; import java.util.TreeSet; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.CellUtil; @@ -152,8 +154,7 @@ public void testMultiColumnScanner() throws IOException { // that column. Map lastDelTimeMap = new HashMap<>(); - Random rand = new Random(29372937L); - + Random rand = ThreadLocalRandom.current(); for (int iFlush = 0; iFlush < NUM_FLUSHES; ++iFlush) { for (String qual : qualifiers) { // This is where we decide to include or not include this column into diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java index b08be0f37e77..ca5e83b26eab 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hbase.regionserver; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -50,7 +50,6 @@ static class Writer implements Runnable { this.status = status; } - private Random rnd = new Random(); public boolean failed = false; @Override @@ -60,7 +59,7 @@ public void run() { mvcc.begin(); // System.out.println("Begin write: " + e.getWriteNumber()); // 10 usec - 500usec (including 0) - int sleepTime = rnd.nextInt(500); + int sleepTime = ThreadLocalRandom.current().nextInt(500); // 500 * 1000 = 500,000ns = 500 usec // 1 * 100 = 100ns = 1usec try { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java index 60446ca95032..75b4ad875914 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java @@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; -import java.util.Random; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -202,7 +201,6 @@ public static class Putter extends Thread { private final HRegion region; private final int threadNumber; private final int numOps; - private final Random rand = new Random(); byte [] rowkey = null; public Putter(HRegion region, int threadNumber, int numOps) { @@ -221,7 +219,7 @@ public void run() { // iterate for the specified number of operations for (int i=0; i putTimestamps = new HashSet<>(); private Set delTimestamps = new HashSet<>(); private List expectedKVs = new ArrayList<>(); @@ -128,6 +127,7 @@ public class TestSeekOptimizations { private long totalSeekDiligent, totalSeekLazy; private final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private static final Random RNG = new Random(); // This test depends on Random#setSeed @Parameters public static final Collection parameters() { @@ -142,7 +142,7 @@ public TestSeekOptimizations(Compression.Algorithm comprAlgo, @Before public void setUp() { - rand = new Random(91238123L); + RNG.setSeed(91238123L); expectedKVs.clear(); TEST_UTIL.getConfiguration().setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, 10); } @@ -350,7 +350,7 @@ private void delUpToTimestamp(String qual, long upToTS) { } private long randLong(long n) { - long l = rand.nextLong(); + long l = RNG.nextLong(); if (l == Long.MIN_VALUE) l = Long.MAX_VALUE; return Math.abs(l) % n; @@ -413,7 +413,7 @@ public void createTimestampRange(long minTS, long maxTS, int tsRemaining = putTimestampList.length; del = new Delete(rowBytes); for (long ts : putTimestampList) { - if (rand.nextInt(tsRemaining) < numToDel) { + if (RNG.nextInt(tsRemaining) < numToDel) { delAtTimestamp(qual, ts); putTimestamps.remove(ts); --numToDel; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScannerClosure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScannerClosure.java index 7ddc9b4cfaf7..3b893a87cb32 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScannerClosure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScannerClosure.java @@ -29,6 +29,7 @@ import java.util.Random; import java.util.TreeSet; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -185,13 +186,13 @@ private Path writeStoreFile() throws IOException { .withFileContext(meta).build(); final int rowLen = 32; - Random RNG = new Random(); + Random rand = ThreadLocalRandom.current(); for (int i = 0; i < 1000; ++i) { - byte[] k = RandomKeyValueUtil.randomOrderedKey(RNG, i); - byte[] v = RandomKeyValueUtil.randomValue(RNG); - int cfLen = RNG.nextInt(k.length - rowLen + 1); + byte[] k = RandomKeyValueUtil.randomOrderedKey(rand, i); + byte[] v = RandomKeyValueUtil.randomValue(rand); + int cfLen = rand.nextInt(k.length - rowLen + 1); KeyValue kv = new KeyValue(k, 0, rowLen, k, rowLen, cfLen, k, rowLen + cfLen, - k.length - rowLen - cfLen, RNG.nextLong(), generateKeyType(RNG), v, 0, v.length); + k.length - rowLen - cfLen, rand.nextLong(), generateKeyType(rand), v, 0, v.length); sfw.append(kv); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ConstantSizeFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ConstantSizeFileListGenerator.java index 2e47566a9cb0..124d8f10a9f6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ConstantSizeFileListGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ConstantSizeFileListGenerator.java @@ -31,10 +31,6 @@ class ConstantSizeFileListGenerator extends StoreFileListGenerator { /** How many mb's mock storefiles should be. */ private static final int FILESIZE = 5; - ConstantSizeFileListGenerator() { - super(ConstantSizeFileListGenerator.class); - } - @Override public final Iterator> iterator() { return new Iterator>() { @@ -61,4 +57,5 @@ public void remove() { } }; } + } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ExplicitFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ExplicitFileListGenerator.java index e14f69618b15..a76a8a510b69 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ExplicitFileListGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ExplicitFileListGenerator.java @@ -19,7 +19,6 @@ import java.util.Iterator; import java.util.List; - import org.apache.hadoop.hbase.regionserver.HStoreFile; /** @@ -27,6 +26,7 @@ * This is great for testing edge cases. */ class ExplicitFileListGenerator extends StoreFileListGenerator { + /** The explicit files size lists to return. */ private int[][] fileSizes = new int[][]{ {1000, 350, 200, 100, 20, 10, 10}, @@ -47,10 +47,6 @@ class ExplicitFileListGenerator extends StoreFileListGenerator { {1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 15} }; - ExplicitFileListGenerator() { - super(ExplicitFileListGenerator.class); - } - @Override public final Iterator> iterator() { return new Iterator>() { @@ -72,4 +68,5 @@ public void remove() { } }; } + } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/GaussianFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/GaussianFileListGenerator.java index fe5f9040988c..51930301a8aa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/GaussianFileListGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/GaussianFileListGenerator.java @@ -20,22 +20,16 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; - import org.apache.commons.math3.random.GaussianRandomGenerator; import org.apache.commons.math3.random.MersenneTwister; import org.apache.hadoop.hbase.regionserver.HStoreFile; class GaussianFileListGenerator extends StoreFileListGenerator { - GaussianFileListGenerator() { - super(GaussianFileListGenerator.class); - } - @Override public Iterator> iterator() { return new Iterator>() { - private GaussianRandomGenerator gen = - new GaussianRandomGenerator(new MersenneTwister(random.nextInt())); + private GaussianRandomGenerator gen = new GaussianRandomGenerator(new MersenneTwister()); private int count = 0; @Override @@ -61,4 +55,5 @@ public void remove() { } }; } + } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/MockStoreFileGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/MockStoreFileGenerator.java index cb6c48601235..d6d84ddbf9c8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/MockStoreFileGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/MockStoreFileGenerator.java @@ -23,8 +23,7 @@ import java.util.LinkedList; import java.util.List; -import java.util.Random; - +import java.util.concurrent.ThreadLocalRandom; import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.regionserver.HStoreFile; @@ -39,12 +38,6 @@ class MockStoreFileGenerator { /** How many chars long the store file name will be. */ private static final int FILENAME_LENGTH = 10; - /** The random number generator. */ - protected Random random; - - MockStoreFileGenerator(Class klass) { - random = new Random(klass.getSimpleName().hashCode()); - } protected List createStoreFileList(final int[] fs) { List storeFiles = new LinkedList<>(); @@ -66,10 +59,10 @@ protected HStoreFile createMockStoreFile(final long sizeInBytes, final long seqI HStoreFile mockSf = mock(HStoreFile.class); StoreFileReader reader = mock(StoreFileReader.class); String stringPath = "/hbase/testTable/regionA/" + - RandomStringUtils.random(FILENAME_LENGTH, 0, 0, true, true, null, random); + RandomStringUtils.random(FILENAME_LENGTH, 0, 0, true, true, null, + ThreadLocalRandom.current()); Path path = new Path(stringPath); - when(reader.getSequenceID()).thenReturn(seqId); when(reader.getTotalUncompressedBytes()).thenReturn(sizeInBytes); when(reader.length()).thenReturn(sizeInBytes); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicies.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicies.java index 31b95ee14d6e..73204475b42a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicies.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicies.java @@ -125,7 +125,6 @@ public PerfTestCompactionPolicies( final int inMin, final float inRatio) throws IllegalAccessException, InstantiationException, NoSuchMethodException, InvocationTargetException { - super(PerfTestCompactionPolicies.class); this.fileGenClass = fileGenClass; this.max = inMmax; this.min = inMin; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SemiConstantSizeFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SemiConstantSizeFileListGenerator.java index c89f136c86fe..8c3db2002ecb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SemiConstantSizeFileListGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SemiConstantSizeFileListGenerator.java @@ -20,13 +20,10 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; - +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.regionserver.HStoreFile; class SemiConstantSizeFileListGenerator extends StoreFileListGenerator { - SemiConstantSizeFileListGenerator() { - super(SemiConstantSizeFileListGenerator.class); - } @Override public Iterator> iterator() { @@ -43,7 +40,7 @@ public List next() { count += 1; ArrayList files = new ArrayList<>(NUM_FILES_GEN); for (int i = 0; i < NUM_FILES_GEN; i++) { - files.add(createMockStoreFile(random.nextInt(5) + 30)); + files.add(createMockStoreFile(ThreadLocalRandom.current().nextInt(5) + 30)); } return files; } @@ -54,4 +51,5 @@ public void remove() { } }; } + } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SinusoidalFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SinusoidalFileListGenerator.java index d270da74d27d..75779c182440 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SinusoidalFileListGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SinusoidalFileListGenerator.java @@ -24,9 +24,6 @@ import org.apache.hadoop.hbase.regionserver.HStoreFile; class SinusoidalFileListGenerator extends StoreFileListGenerator { - SinusoidalFileListGenerator() { - super(SinusoidalFileListGenerator.class); - } @Override public Iterator> iterator() { @@ -53,4 +50,5 @@ public void remove() { } }; } + } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SpikyFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SpikyFileListGenerator.java index bed1342a03a6..98c1469d512f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SpikyFileListGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SpikyFileListGenerator.java @@ -20,15 +20,12 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; - +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.regionserver.HStoreFile; class SpikyFileListGenerator extends StoreFileListGenerator { - SpikyFileListGenerator() { - super(SpikyFileListGenerator.class); - } - @Override public Iterator> iterator() { return new Iterator>() { @@ -43,10 +40,11 @@ public boolean hasNext() { public List next() { count += 1; ArrayList files = new ArrayList<>(NUM_FILES_GEN); + Random rand = ThreadLocalRandom.current(); for (int x = 0; x < NUM_FILES_GEN; x++) { - int fileSize = random.nextInt(5) + 1; + int fileSize = rand.nextInt(5) + 1; if ( x % 10 == 0) { - fileSize = random.nextInt(5) + 50; + fileSize = rand.nextInt(5) + 50; } files.add(createMockStoreFile(fileSize)); } @@ -58,4 +56,5 @@ public void remove() { } }; } + } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/StoreFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/StoreFileListGenerator.java index 88546766fa72..7828a4a97b95 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/StoreFileListGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/StoreFileListGenerator.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.regionserver.compactions; import java.util.List; - import org.apache.hadoop.hbase.regionserver.HStoreFile; public abstract class StoreFileListGenerator extends MockStoreFileGenerator @@ -27,7 +26,4 @@ public abstract class StoreFileListGenerator extends MockStoreFileGenerator public static final int MAX_FILE_GEN_ITERS = 10; public static final int NUM_FILES_GEN = 1000; - StoreFileListGenerator(Class klass) { - super(klass); - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java index 9e8b9b47911b..552d3f360654 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java @@ -21,8 +21,6 @@ import java.io.IOException; import java.util.List; -import java.util.concurrent.ThreadLocalRandom; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -107,7 +105,7 @@ private HStore prepareData() throws IOException { for (int i = 0; i < 10; i++) { for (int j = 0; j < 10; j++) { byte[] value = new byte[128 * 1024]; - ThreadLocalRandom.current().nextBytes(value); + Bytes.random(value); table.put(new Put(Bytes.toBytes(i * 10 + j)).addColumn(family, qualifier, value)); } admin.flush(tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestCompactionWithThroughputController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestCompactionWithThroughputController.java index 811cf1bd02df..17cd05a81296 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestCompactionWithThroughputController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestCompactionWithThroughputController.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.util.List; -import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; @@ -97,7 +96,7 @@ private HStore prepareData() throws IOException { for (int i = 0; i < 10; i++) { for (int j = 0; j < 10; j++) { byte[] value = new byte[128 * 1024]; - ThreadLocalRandom.current().nextBytes(value); + Bytes.random(value); table.put(new Put(Bytes.toBytes(i * 10 + j)).addColumn(family, qualifier, value)); } admin.flush(tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java index aee6c9f6e022..b1de5fdfa396 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/throttle/TestFlushWithThroughputController.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.util.List; -import java.util.Random; import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -115,13 +114,12 @@ private Pair generateAndFlushData(Table table) throws IOException // Internally, throughput is controlled after every cell write, so keep value size less for // better control. final int NUM_FLUSHES = 3, NUM_PUTS = 50, VALUE_SIZE = 200 * 1024; - Random rand = new Random(); long duration = 0; for (int i = 0; i < NUM_FLUSHES; i++) { // Write about 10M (10 times of throughput rate) per iteration. for (int j = 0; j < NUM_PUTS; j++) { byte[] value = new byte[VALUE_SIZE]; - rand.nextBytes(value); + Bytes.random(value); table.put(new Put(Bytes.toBytes(i * 10 + j)).addColumn(family, qualifier, value)); } long startTime = System.nanoTime(); @@ -190,11 +188,10 @@ public void testFlushThroughputTuning() throws Exception { assertTrue(regionServer.getFlushPressure() < pressure); Thread.sleep(5000); Table table = conn.getTable(tableName); - Random rand = new Random(); for (int i = 0; i < 10; i++) { for (int j = 0; j < 10; j++) { byte[] value = new byte[256 * 1024]; - rand.nextBytes(value); + Bytes.random(value); table.put(new Put(Bytes.toBytes(i * 10 + j)).addColumn(family, qualifier, value)); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java index 502c6f242564..d614b79d0742 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java @@ -26,8 +26,9 @@ import java.util.Arrays; import java.util.List; import java.util.Optional; -import java.util.Random; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -582,12 +583,12 @@ public void tearDown() throws IOException { @SuppressWarnings("resource") private void startMiniClusters(int numClusters) throws Exception { - Random random = new Random(); utilities = new HBaseTestingUtil[numClusters]; configurations = new Configuration[numClusters]; for (int i = 0; i < numClusters; i++) { Configuration conf = new Configuration(baseConfiguration); - conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/" + i + random.nextInt()); + conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/" + + i + ThreadLocalRandom.current().nextInt()); HBaseTestingUtil utility = new HBaseTestingUtil(conf); if (i == 0) { utility.startMiniZKCluster(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java index c964417147a7..10a5affcbce2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java @@ -19,7 +19,6 @@ import static org.junit.Assert.assertEquals; -import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -29,6 +28,7 @@ import java.util.Map; import java.util.Random; import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -357,10 +357,10 @@ public void testReplicateEntriesForHFiles() throws Exception { final String hfilePrefix = "hfile-"; // 1. Generate 25 hfile ranges - Random rng = new SecureRandom(); + Random rand = ThreadLocalRandom.current(); Set numbers = new HashSet<>(); while (numbers.size() < 50) { - numbers.add(rng.nextInt(1000)); + numbers.add(rand.nextInt(1000)); } List numberList = new ArrayList<>(numbers); Collections.sort(numberList); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin2.java index 7bd2cd12bd0e..16c27af94cc4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsAdmin2.java @@ -31,6 +31,7 @@ import java.util.Map; import java.util.Random; import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; import org.apache.hadoop.hbase.ClusterMetrics.Option; @@ -612,7 +613,7 @@ public void testFailedMoveServersAndRepair() throws Exception { // create table // randomly set a region state to SPLITTING to make move abort Pair gotPair = - createTableWithRegionSplitting(newGroup, new Random().nextInt(8) + 4); + createTableWithRegionSplitting(newGroup, ThreadLocalRandom.current().nextInt(8) + 4); RegionStateNode rsn = gotPair.getSecond(); ServerName srcServer = rsn.getRegionLocation(); @@ -651,8 +652,9 @@ public void testFailedMoveServersTablesAndRepair() throws Exception { final byte[] familyNameBytes = Bytes.toBytes("f"); TableName table1 = TableName.valueOf(tableName.getNameAsString() + "_1"); TableName table2 = TableName.valueOf(tableName.getNameAsString() + "_2"); - TEST_UTIL.createMultiRegionTable(table1, familyNameBytes, new Random().nextInt(12) + 4); - TEST_UTIL.createMultiRegionTable(table2, familyNameBytes, new Random().nextInt(12) + 4); + Random rand = ThreadLocalRandom.current(); + TEST_UTIL.createMultiRegionTable(table1, familyNameBytes, rand.nextInt(12) + 4); + TEST_UTIL.createMultiRegionTable(table2, familyNameBytes, rand.nextInt(12) + 4); // randomly set a region state to SPLITTING to make move abort Pair gotPair = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestDataGeneratorWithTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestDataGeneratorWithTags.java index 2ef38212707f..81117b21f40f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestDataGeneratorWithTags.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestDataGeneratorWithTags.java @@ -20,6 +20,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.Cell; @@ -38,7 +39,6 @@ public class LoadTestDataGeneratorWithTags extends DefaultDataGenerator { private int minNumTags, maxNumTags; private int minTagLength, maxTagLength; - private Random random = new Random(); public LoadTestDataGeneratorWithTags(int minValueSize, int maxValueSize, int minColumnsPerKey, int maxColumnsPerKey, byte[]... columnFamilies) { @@ -66,17 +66,18 @@ public void initialize(String[] args) { public Mutation beforeMutate(long rowkeyBase, Mutation m) throws IOException { if (m instanceof Put) { List updatedCells = new ArrayList<>(); + Random rand = ThreadLocalRandom.current(); int numTags; if (minNumTags == maxNumTags) { numTags = minNumTags; } else { - numTags = minNumTags + random.nextInt(maxNumTags - minNumTags); + numTags = minNumTags + rand.nextInt(maxNumTags - minNumTags); } List tags; for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) { Cell cell = cellScanner.current(); - byte[] tag = LoadTestDataGenerator.generateData(random, - minTagLength + random.nextInt(maxTagLength - minTagLength)); + byte[] tag = LoadTestDataGenerator.generateData(rand, + minTagLength + rand.nextInt(maxTagLength - minTagLength)); tags = new ArrayList<>(); for (int n = 0; n < numTags; n++) { tags.add(new ArrayBackedTag((byte) 127, tag)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java index 19461434e894..2e1483d4df13 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedAction.java @@ -25,8 +25,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.conf.Configuration; @@ -86,7 +86,6 @@ public static class DefaultDataGenerator extends LoadTestDataGenerator { private byte[][] columnFamilies = null; private int minColumnsPerKey; private int maxColumnsPerKey; - private final Random random = new Random(); public DefaultDataGenerator(int minValueSize, int maxValueSize, int minColumnsPerKey, int maxColumnsPerKey, byte[]... columnFamilies) { @@ -113,7 +112,8 @@ public byte[][] getColumnFamilies() { @Override public byte[][] generateColumnsForCf(byte[] rowKey, byte[] cf) { - int numColumns = minColumnsPerKey + random.nextInt(maxColumnsPerKey - minColumnsPerKey + 1); + int numColumns = minColumnsPerKey + + ThreadLocalRandom.current().nextInt(maxColumnsPerKey - minColumnsPerKey + 1); byte[][] columns = new byte[numColumns][]; for (int i = 0; i < numColumns; ++i) { columns[i] = Bytes.toBytes(Integer.toString(i)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java index 58b046f48daf..4c046c82870e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReader.java @@ -19,9 +19,10 @@ import java.io.IOException; import java.util.Arrays; import java.util.HashSet; +import java.util.Random; import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.lang3.RandomUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.TableName; @@ -263,18 +264,19 @@ protected long getNextKeyToRead() { // later. Set a flag to make sure that we don't count this key towards // the set of unique keys we have verified. readingRandomKey = true; - return startKey + Math.abs(RandomUtils.nextLong()) + return startKey + Math.abs(ThreadLocalRandom.current().nextLong()) % (maxKeyToRead - startKey + 1); } private Get[] readKey(long[] keysToRead) { + Random rand = ThreadLocalRandom.current(); Get [] gets = new Get[keysToRead.length]; int i = 0; for (long keyToRead : keysToRead) { try { gets[i] = createGet(keyToRead); if (keysToRead.length == 1) { - queryKey(gets[i], RandomUtils.nextInt(0, 100) < verifyPercent, keyToRead); + queryKey(gets[i], rand.nextInt(100) < verifyPercent, keyToRead); } i++; } catch (IOException e) { @@ -290,7 +292,7 @@ private Get[] readKey(long[] keysToRead) { } if (keysToRead.length > 1) { try { - queryKey(gets, RandomUtils.nextInt(0, 100) < verifyPercent, keysToRead); + queryKey(gets, rand.nextInt(100) < verifyPercent, keysToRead); } catch (IOException e) { numReadFailures.addAndGet(gets.length); for (long keyToRead : keysToRead) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdater.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdater.java index 30050a489c00..78c698975a7f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdater.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdater.java @@ -27,9 +27,10 @@ import java.util.Arrays; import java.util.HashSet; import java.util.Map; +import java.util.Random; import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; -import org.apache.commons.lang3.RandomUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -134,11 +135,12 @@ protected Table createTable() throws IOException { @Override public void run() { try { + Random rand = ThreadLocalRandom.current(); long rowKeyBase; StringBuilder buf = new StringBuilder(); byte[][] columnFamilies = dataGenerator.getColumnFamilies(); while ((rowKeyBase = getNextKeyToUpdate()) < endKey) { - if (RandomUtils.nextInt(0, 100) < updatePercent) { + if (rand.nextInt(100) < updatePercent) { byte[] rowKey = dataGenerator.getDeterministicUniqueKey(rowKeyBase); Increment inc = new Increment(rowKey); Append app = new Append(rowKey); @@ -187,8 +189,8 @@ public void run() { if (Bytes.equals(column, INCREMENT) || Bytes.equals(column, MUTATE_INFO)) { continue; } - MutationType mt = MutationType - .valueOf(RandomUtils.nextInt(0, MutationType.values().length)); + MutationType mt = + MutationType.values()[rand.nextInt(MutationType.values().length)]; long columnHash = Arrays.hashCode(column); long hashCode = cfHash + columnHash; byte[] hashCodeBytes = Bytes.toBytes(hashCode); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java index f4a4df53895d..ecdb4f675368 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java @@ -321,7 +321,7 @@ public void testPermMask() throws Exception { assertEquals(new FsPermission("700"), filePerm); // then that the correct file is created - Path p = new Path("target" + File.separator + htu.getRandomUUID().toString()); + Path p = new Path("target" + File.separator + HBaseTestingUtil.getRandomUUID().toString()); try { FSDataOutputStream out = FSUtils.create(conf, fs, p, filePerm, null); out.close(); @@ -340,7 +340,7 @@ public void testDeleteAndExists() throws Exception { conf.setBoolean(HConstants.ENABLE_DATA_FILE_UMASK, true); FsPermission perms = CommonFSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY); // then that the correct file is created - String file = htu.getRandomUUID().toString(); + String file = HBaseTestingUtil.getRandomUUID().toString(); Path p = new Path(htu.getDataTestDir(), "temptarget" + File.separator + file); Path p1 = new Path(htu.getDataTestDir(), "temppath" + File.separator + file); try { @@ -381,7 +381,7 @@ public void testRenameAndSetModifyTime() throws Exception { FileSystem fs = FileSystem.get(conf); Path testDir = htu.getDataTestDirOnTestFS("testArchiveFile"); - String file = htu.getRandomUUID().toString(); + String file = HBaseTestingUtil.getRandomUUID().toString(); Path p = new Path(testDir, file); FSDataOutputStream out = fs.create(p); @@ -395,7 +395,7 @@ public void testRenameAndSetModifyTime() throws Exception { mockEnv.setValue(expect); EnvironmentEdgeManager.injectEdge(mockEnv); try { - String dstFile = htu.getRandomUUID().toString(); + String dstFile = HBaseTestingUtil.getRandomUUID().toString(); Path dst = new Path(testDir , dstFile); assertTrue(CommonFSUtils.renameAndSetModifyTime(fs, p, dst)); @@ -477,7 +477,7 @@ private void verifyFileInDirWithStoragePolicy(final String policy) throws Except conf.get(HConstants.WAL_STORAGE_POLICY, HConstants.DEFAULT_WAL_STORAGE_POLICY); CommonFSUtils.setStoragePolicy(fs, testDir, storagePolicy); - String file =htu.getRandomUUID().toString(); + String file = HBaseTestingUtil.getRandomUUID().toString(); Path p = new Path(testDir, file); WriteDataToHDFS(fs, p, 4096); HFileSystem hfs = new HFileSystem(fs); @@ -492,7 +492,7 @@ private void verifyFileInDirWithStoragePolicy(final String policy) throws Except } else { Assert.assertEquals(policy, policySet); } - // will assert existance before deleting. + // will assert existence before deleting. cleanupFile(fs, testDir); } finally { cluster.shutdown(); @@ -562,11 +562,12 @@ public void testCopyFilesParallel() throws Exception { // Below is taken from TestPread over in HDFS. static final int blockSize = 4096; static final long seed = 0xDEADBEEFL; + private Random rand = new Random(); // This test depends on Random#setSeed private void pReadFile(FileSystem fileSys, Path name) throws IOException { FSDataInputStream stm = fileSys.open(name); byte[] expected = new byte[12 * blockSize]; - Random rand = new Random(seed); + rand.setSeed(seed); rand.nextBytes(expected); // do a sanity check. Read first 4K bytes byte[] actual = new byte[4096]; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java index 90fe71eb5029..817e1f07193e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java @@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue; import java.security.Key; -import java.security.SecureRandom; import java.util.ArrayList; import java.util.List; import javax.crypto.spec.SecretKeySpec; @@ -80,9 +79,8 @@ public void setUp() throws Exception { conf.set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase"); // Create the test encryption key - SecureRandom rng = new SecureRandom(); byte[] keyBytes = new byte[AES.KEY_LENGTH]; - rng.nextBytes(keyBytes); + Bytes.secureRandom(keyBytes); String algorithm = conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES); cfKey = new SecretKeySpec(keyBytes,algorithm); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdLock.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdLock.java index 0ea742281601..15871745b8b1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdLock.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdLock.java @@ -27,6 +27,7 @@ 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 org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -66,7 +67,7 @@ public IdLockTestThread(String clientId) { @Override public Boolean call() throws Exception { Thread.currentThread().setName(clientId); - Random rand = new Random(); + Random rand = ThreadLocalRandom.current(); long endTime = EnvironmentEdgeManager.currentTime() + NUM_SECONDS * 1000; while (EnvironmentEdgeManager.currentTime() < endTime) { long id = rand.nextInt(NUM_IDS); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdReadWriteLockWithObjectPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdReadWriteLockWithObjectPool.java index 86ed427cbfe0..47d6dfd627b7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdReadWriteLockWithObjectPool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdReadWriteLockWithObjectPool.java @@ -29,6 +29,7 @@ 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.locks.Lock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -83,7 +84,7 @@ public IdLockTestThread(String clientId) { @Override public Boolean call() throws Exception { Thread.currentThread().setName(clientId); - Random rand = new Random(); + Random rand = ThreadLocalRandom.current(); long endTime = EnvironmentEdgeManager.currentTime() + NUM_SECONDS * 1000; while (EnvironmentEdgeManager.currentTime() < endTime) { long id = rand.nextInt(NUM_IDS); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java index 84e952abe040..f52cfb4b84c6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java @@ -35,6 +35,7 @@ import java.util.Random; import java.util.Set; import java.util.TreeMap; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.stream.IntStream; import org.apache.hadoop.conf.Configuration; @@ -144,7 +145,6 @@ class WALPutBenchmark implements Runnable { public void run() { byte[] key = new byte[keySize]; byte[] value = new byte[valueSize]; - Random rand = new Random(Thread.currentThread().getId()); WAL wal = region.getWAL(); Span threadSpan = TraceUtil.getGlobalTracer() .spanBuilder("WALPerfEval." + Thread.currentThread().getName()).startSpan(); @@ -155,7 +155,7 @@ public void run() { Span loopSpan = TraceUtil.getGlobalTracer().spanBuilder("runLoopIter" + i).startSpan(); try (Scope loopScope = loopSpan.makeCurrent()) { long now = System.nanoTime(); - Put put = setupPut(rand, key, value, numFamilies); + Put put = setupPut(ThreadLocalRandom.current(), key, value, numFamilies); WALEdit walEdit = new WALEdit(); walEdit.add(put.getFamilyCellMap()); RegionInfo hri = region.getRegionInfo(); diff --git a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java index 7421f3eeff49..1a5864959df2 100644 --- a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java +++ b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java @@ -23,7 +23,6 @@ import java.util.Arrays; import java.util.HashSet; import java.util.List; -import java.util.Random; import java.util.Set; import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; @@ -272,10 +271,7 @@ public long waitFor(long timeout, long interval, return Waiter.waitFor(this.conf, timeout, interval, failIfTimeout, predicate); } - // Support for Random Port Generation. - static Random random = new Random(); - - private static final PortAllocator portAllocator = new PortAllocator(random); + private static final PortAllocator portAllocator = new PortAllocator(); public static int randomFreePort() { return portAllocator.randomFreePort(); @@ -288,11 +284,9 @@ static class PortAllocator { /** A set of ports that have been claimed using {@link #randomFreePort()}. */ private final Set takenRandomPorts = new HashSet<>(); - private final Random random; private final AvailablePortChecker portChecker; - public PortAllocator(Random random) { - this.random = random; + public PortAllocator() { this.portChecker = new AvailablePortChecker() { @Override public boolean available(int port) { @@ -307,8 +301,7 @@ public boolean available(int port) { }; } - public PortAllocator(Random random, AvailablePortChecker portChecker) { - this.random = random; + public PortAllocator(AvailablePortChecker portChecker) { this.portChecker = portChecker; } @@ -339,7 +332,7 @@ public int randomFreePort() { */ private int randomPort() { return MIN_RANDOM_PORT - + random.nextInt(MAX_RANDOM_PORT - MIN_RANDOM_PORT); + + ThreadLocalRandom.current().nextInt(MAX_RANDOM_PORT - MIN_RANDOM_PORT); } interface AvailablePortChecker { diff --git a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 8e054e8b09a5..345c2d6d4053 100644 --- a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -44,6 +44,7 @@ import java.util.Random; import java.util.Set; import java.util.TreeSet; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BooleanSupplier; @@ -2261,10 +2262,9 @@ public void loadNumericRows(final Table t, final byte[] f, int startRow, int end public void loadRandomRows(final Table t, final byte[] f, int rowSize, int totalRows) throws IOException { - Random r = new Random(); byte[] row = new byte[rowSize]; for (int i = 0; i < totalRows; i++) { - r.nextBytes(row); + Bytes.random(row); Put put = new Put(row); put.addColumn(f, new byte[]{0}, new byte[]{0}); t.put(put); @@ -3129,7 +3129,7 @@ public HRegion getSplittableRegion(TableName tableName, int maxAttempts) { // There are chances that before we get the region for the table from an RS the region may // be going for CLOSE. This may be because online schema change is enabled if (regCount > 0) { - idx = random.nextInt(regCount); + idx = ThreadLocalRandom.current().nextInt(regCount); // if we have just tried this region, there is no need to try again if (attempted.contains(idx)) { continue; @@ -3712,7 +3712,6 @@ public Table createRandomTable(TableName tableName, numRowsPerFlush + " rows per flush, maxVersions=" + maxVersions + "\n"); - final Random rand = new Random(tableName.hashCode() * 17L + 12938197137L); final int numCF = families.size(); final byte[][] cfBytes = new byte[numCF][]; { @@ -3740,6 +3739,7 @@ public Table createRandomTable(TableName tableName, BufferedMutator mutator = getConnection().getBufferedMutator(tableName); + final Random rand = ThreadLocalRandom.current(); for (int iFlush = 0; iFlush < numFlushes; ++iFlush) { for (int iRow = 0; iRow < numRowsPerFlush; ++iRow) { final byte[] row = Bytes.toBytes(String.format(keyFormat, @@ -3786,7 +3786,7 @@ public static int randomFreePort() { return HBaseCommonTestingUtility.randomFreePort(); } public static String randomMultiCastAddress() { - return "226.1.1." + random.nextInt(254); + return "226.1.1." + ThreadLocalRandom.current().nextInt(254); } public static void waitForHostPort(String host, int port) diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java index 22e480d748a5..62b27de6bd80 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java @@ -29,7 +29,8 @@ import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.List; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.net.Address; @@ -143,7 +144,7 @@ private int selectClientPort(int seedPort) { if (defaultClientPort > 0) { returnClientPort = defaultClientPort; } else { - returnClientPort = 0xc000 + new Random().nextInt(0x3f00); + returnClientPort = 0xc000 + ThreadLocalRandom.current().nextInt(0x3f00); } } // Make sure that the port is unused. diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.java index 108fdf887f42..686bca31bf72 100644 --- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.java @@ -43,7 +43,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Exchanger; import java.util.concurrent.ExecutionException; -import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseZKTestingUtil; @@ -51,6 +50,7 @@ import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ZKTests; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; @@ -87,7 +87,7 @@ public static void setUp() throws Exception { ZooKeeper zk = ZooKeeperHelper.getConnectedZooKeeper(hostPort, 10000); DATA = new byte[10]; - ThreadLocalRandom.current().nextBytes(DATA); + Bytes.random(DATA); zk.create(PATH, DATA, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); for (int i = 0; i < CHILDREN; i++) { zk.create(PATH + "/c" + i, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);