From 5386325acdbed3909f947170730a5fb4729063a5 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 +-- .../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 ++++++++++++++++-- .../hbase/HBaseCommonTestingUtility.java | 15 ++---- .../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 | 47 ++++++++++--------- .../hadoop/hbase/chaos/actions/Action.java | 13 +++-- .../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 | 16 +++---- .../test/IntegrationTestBigLinkedList.java | 20 ++++---- .../test/IntegrationTestLoadAndVerify.java | 7 +-- ...TimeBoundedRequestsWithRegionReplicas.java | 4 +- ...onTestWithCellVisibilityLoadAndVerify.java | 10 ++-- .../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 | 14 +++--- .../hbase/rest/TestScannerResource.java | 4 +- .../balancer/RSGroupableBalancerTestBase.java | 14 +++--- .../ZkSplitLogWorkerCoordination.java | 5 +- .../hbase/io/hfile/PrefetchExecutor.java | 7 ++- .../master/RegionPlacementMaintainer.java | 4 +- .../AdaptiveMemStoreCompactionStrategy.java | 8 ++-- .../hbase/regionserver/HRegionServer.java | 8 ++-- .../compactions/SortedCompactionPolicy.java | 20 +++----- .../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 ++-- .../hadoop/hbase/HBaseTestingUtility.java | 11 +++-- .../hbase/HFilePerformanceEvaluation.java | 15 ++---- .../hadoop/hbase/TestHBaseTestingUtility.java | 2 +- .../hadoop/hbase/TestMetaTableAccessor.java | 25 ++++++---- .../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 | 15 +++--- .../TestHFileScannerImplReferenceCount.java | 9 ++-- .../hbase/io/hfile/TestHFileWriterV3.java | 17 +++---- .../TestHFileWriterV3WithDataEncoders.java | 11 ++--- .../hfile/TestLazyDataBlockDecompression.java | 10 ++-- .../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 ++-- .../master/balancer/BalancerTestBase.java | 22 ++++----- .../TestBalancerStatusTagInJMXMetrics.java | 4 +- .../TestStochasticBalancerJmxMetrics.java | 4 +- .../master/cleaner/TestCleanerChore.java | 8 ++-- .../master/cleaner/TestHFileCleaner.java | 3 +- .../hbase/master/cleaner/TestLogsCleaner.java | 5 +- ...ocedureSchedulerPerformanceEvaluation.java | 6 +-- .../master/procedure/TestProcedureAdmin.java | 6 +-- .../apache/hadoop/hbase/mob/MobTestUtil.java | 5 +- .../hbase/mob/TestExpiredMobFileCleaner.java | 3 +- .../hbase/mob/TestMobDataBlockEncoding.java | 4 +- .../hadoop/hbase/mob/TestMobFileName.java | 10 ++-- .../quotas/SpaceQuotaHelperForTests.java | 4 +- .../hbase/quotas/TestRegionSizeUse.java | 5 +- .../regionserver/CreateRandomStoreFile.java | 12 +++-- .../regionserver/TestAtomicOperation.java | 2 - .../hbase/regionserver/TestBulkloadBase.java | 6 +-- .../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/TestMobStoreCompaction.java | 3 +- .../regionserver/TestMobStoreScanner.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 +-- .../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 | 6 +-- .../regionserver/TestReplicationSink.java | 6 +-- .../util/LoadTestDataGeneratorWithTags.java | 9 ++-- .../hbase/util/MultiThreadedAction.java | 6 +-- .../hbase/util/MultiThreadedReader.java | 12 ++--- .../hbase/util/MultiThreadedUpdater.java | 10 ++-- .../apache/hadoop/hbase/util/TestFSUtils.java | 15 +++--- .../hbase/util/TestHBaseFsckEncryption.java | 4 +- .../apache/hadoop/hbase/util/TestIdLock.java | 3 +- .../hbase/util/TestIdReadWriteLock.java | 3 +- .../hbase/wal/WALPerformanceEvaluation.java | 4 +- .../hbase/zookeeper/MiniZooKeeperCluster.java | 5 +- .../hbase/zookeeper/TestReadOnlyZKClient.java | 4 +- 178 files changed, 662 insertions(+), 767 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 abdba33b803..d363282921c 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.List; 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.ExcludeDatanodeManager; 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.experimental.categories.Category; 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 class TestFanOutOneBlockAsyncDFSOutput extends AsyncFSTestBase { 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 @@ public class TestFanOutOneBlockAsyncDFSOutput extends AsyncFSTestBase { 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 class TestFanOutOneBlockAsyncDFSOutput extends AsyncFSTestBase { 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 @@ -262,7 +259,7 @@ public class TestFanOutOneBlockAsyncDFSOutput extends AsyncFSTestBase { 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 496dc127fde..8ee838449e1 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.Map; 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.EventLoopGroup; 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 class TestFanOutOneBlockAsyncDFSOutputHang extends AsyncFSTestBase { }); 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-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 c492282e2a7..8aedc4d2205 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 final class PerClientRandomNonceGenerator implements NonceGenerator { 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 2d81878f77a..f25430ce799 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 @@ import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferExce */ @InterfaceAudience.Public public class RandomRowFilter extends FilterBase { - protected static final Random random = new Random(); protected float chance; protected boolean filterOutRow; @@ -104,7 +103,7 @@ public class RandomRowFilter extends FilterBase { 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 8d380dc7fe6..74ad96e2cbd 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.ByteArrayOutputStream; 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.InterfaceAudience; 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 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; 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 final class EncryptionUtil { 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 final class EncryptionUtil { * @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 bf4cd046755..771ee8cffbe 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 @@ package org.apache.hadoop.hbase.slowlog; 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 @@ public class SlowLogTableAccessor { 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 9275dc9e763..cf5939031b0 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 static org.junit.Assert.fail; 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 class TestEncryptionUtil { // 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 @@ public class TestEncryptionUtil { // 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 @@ public class TestEncryptionUtil { // 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 @@ public class TestEncryptionUtil { // 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 ef7cb4e6512..2fd73caea46 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.ArrayList; 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 a1cb610e854..2f497c6fdfb 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 @@ package org.apache.hadoop.hbase.util; 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 807758958d0..6cb0b073e0f 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 final class Encryption { */ 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 8d9e6824fa4..25b946a143f 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.ByteArrayInputStream; 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 class HFileBlockDefaultEncodingContext implements HFileBlockEncodingConte 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 3fe50927e5e..96742f03289 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 @@ -38,6 +38,8 @@ import java.util.Collections; 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.hbase.KeyValue; @@ -2555,11 +2557,16 @@ public class Bytes implements Comparable { 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); @@ -2567,9 +2574,12 @@ public class Bytes implements Comparable { /** * 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"); @@ -2580,6 +2590,33 @@ public class Bytes implements Comparable { 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/HBaseCommonTestingUtility.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java index 487c926a1bc..71704babf78 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java @@ -23,7 +23,6 @@ import java.net.ServerSocket; 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; @@ -269,10 +268,7 @@ public class HBaseCommonTestingUtility { 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(); @@ -285,11 +281,9 @@ public class HBaseCommonTestingUtility { /** 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) { @@ -304,8 +298,7 @@ public class HBaseCommonTestingUtility { }; } - public PortAllocator(Random random, AvailablePortChecker portChecker) { - this.random = random; + public PortAllocator(AvailablePortChecker portChecker) { this.portChecker = portChecker; } @@ -336,7 +329,7 @@ public class HBaseCommonTestingUtility { */ 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-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 8d850a7aa4e..d8057856f62 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 class TestEncryption { @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 class TestEncryption { @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 class TestEncryption { @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 bc8af936164..8452f3ee816 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 static org.junit.Assert.assertTrue; 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 class TestLRUDictionary { @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 ad49e553719..174cd4c37fb 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 0499a04a900..b006f638faf 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.assertEquals; 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 class TestAvlUtil { 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 8384b055cf2..0934581decf 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 static org.junit.Assert.fail; 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 @@ import org.junit.experimental.categories.Category; @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 @@ public class TestByteBufferArray { 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 decb6ce292a..ee37939a227 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.ArrayList; 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; @@ -385,14 +387,11 @@ public class TestBytes { @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'}); @@ -597,10 +596,10 @@ public class TestBytes { 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 a57c935651a..27888db0f6d 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 static org.junit.Assert.assertNotEquals; 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 cddfb1b7920..1e9a2861c9e 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 @@ package org.apache.hadoop.hbase.http; 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 class TestServletFilter extends HttpServerFunctionalTest { 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 2fb12c31240..0f84811eaab 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,8 @@ import java.util.ArrayList; 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 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; @@ -258,7 +256,8 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase { 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); @@ -307,12 +306,12 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase { 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; } } @@ -332,7 +331,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase { 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); @@ -398,8 +397,8 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase { return null; } ArrayList 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; } } @@ -437,8 +436,9 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase { } 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(); @@ -582,8 +582,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase { 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)]; } } @@ -600,7 +599,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase { 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; } @@ -625,7 +624,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase { } private ColumnFamilyDescriptor createFamilyDesc() { - String familyName = String.format("cf-%010d", RandomUtils.nextInt()); + String familyName = String.format("cf-%010d", ThreadLocalRandom.current().nextInt()); return ColumnFamilyDescriptorBuilder.of(familyName); } } @@ -644,7 +643,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase { } 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 + @@ -700,7 +699,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase { // 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); @@ -788,17 +787,18 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase { 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); @@ -842,7 +842,8 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase { 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); @@ -875,7 +876,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase { 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; @@ -884,7 +885,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase { 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 113f1aad2ad..c997ddea27c 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,15 +26,15 @@ import java.util.LinkedList; 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.HBaseCluster; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.IntegrationTestBase; import org.apache.hadoop.hbase.IntegrationTestingUtility; import org.apache.hadoop.hbase.MiniHBaseCluster; @@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.util.Bytes; @@ -286,9 +287,10 @@ public abstract class Action { 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()); - String regionId = HRegionInfo.encodeRegionName(regions.remove(victimIx)); + int victimIx = rand.nextInt(regions.size()); + String regionId = RegionInfo.encodeRegionName(regions.remove(victimIx)); victimRegions.add(Bytes.toBytes(regionId)); } } @@ -296,13 +298,14 @@ public abstract class Action { 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 18ea664eb6b..773f3588d0a 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 class ChangeBloomFilterAction extends Action { @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 8828bc8c1a2..d481d9bfae3 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 @@ 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.compress.Compression.Algorithm; import org.apache.hadoop.io.compress.Compressor; @@ -31,12 +32,10 @@ import org.slf4j.LoggerFactory; */ 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 class ChangeCompressionAction extends Action { // 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 class ChangeCompressionAction extends Action { 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 afa8a250477..61a9bd9cfa0 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 @@ import org.slf4j.LoggerFactory; */ 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 class ChangeEncodingAction extends Action { 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 14a11eedb2f..acfa8692620 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.HBaseTestingUtility; 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 class ChangeSplitPolicyAction extends Action { ConstantSizeRegionSplitPolicy.class.getName(), DisabledRegionSplitPolicy.class.getName() }; - this.random = new Random(); } @Override protected Logger getLogger() { @@ -53,11 +51,11 @@ public class ChangeSplitPolicyAction extends Action { public void perform() throws Exception { HBaseTestingUtility 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 1228db1e495..14a290012f2 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 class ChangeVersionsAction extends Action { @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 889b3fbba3d..cb7b058a65e 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.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -39,8 +39,7 @@ public class CompactMobAction extends Action { 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 class CompactMobAction extends Action { public void perform() throws Exception { HBaseTestingUtility 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 ffc05b837f2..bf5d3dde9bd 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.HBaseTestingUtility; 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 class CompactRandomRegionOfTableAction extends Action { public void perform() throws Exception { HBaseTestingUtility 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 7278d2b67b6..93f277aacf2 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.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -54,7 +54,7 @@ public class CompactTableAction extends Action { public void perform() throws Exception { HBaseTestingUtility 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 3b81f2e617c..616abb2cca1 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 class CorruptDataFilesAction extends Action { 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 0764d008cc3..a384719a51c 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 class DecreaseMaxHFileSizeAction extends Action { // 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 2b869706785..ee4f7deb461 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 class DeleteDataFilesAction extends Action { 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 f4ef7978c97..93bcf6b3a1d 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 @@ package org.apache.hadoop.hbase.chaos.actions; 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 class GracefulRollingRestartRsAction extends RestartActionBaseAction { 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 class GracefulRollingRestartRsAction extends RestartActionBaseAction { } 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 a7e0723cd4f..114880e6aaa 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.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -61,7 +61,7 @@ public class MergeRandomAdjacentRegionsOfTableAction extends Action { 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 a1cab787e39..94b6ae68fe3 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.io.IOException; 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 class MoveRegionsOfTableAction extends Action { 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 a4361f7ce26..ed5cc604433 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 @@ package org.apache.hadoop.hbase.chaos.actions; 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 class RemoveColumnAction extends Action { 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 fcc53339627..04f8d86793a 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 class RestartRsHoldingTableAction extends RestartActionBaseAction { 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 c25a6b31f4b..f54d5c1216c 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.LinkedList; 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 class RollingBatchRestartRsAction extends BatchRestartRsAction { 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 class RollingBatchRestartRsAction extends BatchRestartRsAction { 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 class RollingBatchRestartRsAction extends BatchRestartRsAction { 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 dfae09af255..4c15f66f9c4 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.io.IOException; 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 class RollingBatchSuspendResumeRsAction extends Action { 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 class RollingBatchSuspendResumeRsAction extends Action { 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 ffd841b5bdc..ac620c6df45 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.Logger; 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 class SplitAllRegionOfTableAction extends Action { 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 class SplitAllRegionOfTableAction extends Action { 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 5da10c7b481..9f4361f2bdf 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.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -31,11 +31,9 @@ import org.slf4j.LoggerFactory; 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 class TruncateTableAction extends Action { 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 623d41d5d65..9dc1cf12910 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.ArrayList; 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 class UnbalanceKillAndRebalanceAction extends Action { 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 3d85e85db3e..da7607e48de 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 @@ package org.apache.hadoop.hbase.chaos.actions; 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 class UnbalanceRegionsAction extends Action { 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 e3e2e49ac26..95c5bc68b57 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.Objects; 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 @@ import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFacto */ 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 @@ public class PolicyBasedChaosMonkey extends ChaosMonkey { /** 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 class PolicyBasedChaosMonkey extends ChaosMonkey { 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 class PolicyBasedChaosMonkey extends ChaosMonkey { 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 ae1c65e5527..7c69ee720ed 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 abstract class PeriodicPolicy extends Policy { @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 2dd163305b9..2765c0ed176 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.net.InetSocketAddress; 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 @@ public class IntegrationTestRpcClient { } class Cluster { - Random random = new Random(); ReadWriteLock lock = new ReentrantReadWriteLock(); HashMap rpcServers = new HashMap<>(); List serverList = new ArrayList<>(); @@ -134,7 +132,7 @@ public class IntegrationTestRpcClient { 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 @@ public class IntegrationTestRpcClient { 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 @@ public class IntegrationTestRpcClient { 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 class IntegrationTestRpcClient { @Override public void run() { while (running.get()) { - if (random.nextBoolean()) { + if (ThreadLocalRandom.current().nextBoolean()) { //start a server try { cluster.startServer(); @@ -238,7 +235,6 @@ public class IntegrationTestRpcClient { 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 class IntegrationTestRpcClient { @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 0413f62d3a2..02d4ba571db 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.ArrayList; 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; @@ -155,7 +154,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() {} @@ -379,7 +377,7 @@ public class IntegrationTestBulkLoad extends IntegrationTestBase { 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)}; @@ -397,8 +395,6 @@ public class IntegrationTestBulkLoad extends IntegrationTestBase { public static class LinkedListCreationMapper extends Mapper { - private Random rand = new Random(); - @Override protected void map(LongWritable key, LongWritable value, Context context) throws IOException, InterruptedException { @@ -410,6 +406,7 @@ public class IntegrationTestBulkLoad extends IntegrationTestBase { 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); @@ -419,9 +416,8 @@ public class IntegrationTestBulkLoad extends IntegrationTestBase { // 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); @@ -435,7 +431,7 @@ public class IntegrationTestBulkLoad extends IntegrationTestBase { /** 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 02e6383890b..a4261a7bac6 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.DataOutput; 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 class IntegrationTestBigLinkedList extends IntegrationTestBase { 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 class IntegrationTestBigLinkedList extends IntegrationTestBase { 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 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { 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 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { 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 IntegrationTestBigLinkedList extends IntegrationTestBase { 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 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { if (walkersStop) { throw new InterruptedException(); } - return flushedLoops.get(rand.nextInt(flushedLoops.size())); + return flushedLoops.get(ThreadLocalRandom.current().nextInt(flushedLoops.size())); } } } @@ -1761,7 +1758,6 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { 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; @@ -1776,7 +1772,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { 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 15a227a5d07..8f589042ea1 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 @@ -23,9 +23,9 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Random; import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; import java.util.regex.Matcher; import java.util.regex.Pattern; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -189,10 +189,7 @@ public void cleanUpCluster() throws Exception { 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 @@ public void cleanUpCluster() throws Exception { 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 69ce3e4be9b..3027162cf39 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.List; 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 @@ public class IntegrationTestTimeBoundedRequestsWithRegionReplicas extends Integr @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 c5f614e7023..e4e2263cbe3 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,7 +23,8 @@ import static org.junit.Assert.assertTrue; 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; @@ -57,7 +58,6 @@ import org.apache.hadoop.mapreduce.Job; 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; /** @@ -166,9 +166,10 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT 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); @@ -379,10 +380,9 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT return 0; } - @SuppressWarnings("unchecked") @Override protected void processOptions(CommandLine cmd) { - List args = cmd.getArgList(); + List args = cmd.getArgList(); if (args.size() > 0) { printUsage(); throw new RuntimeException("No args expected."); 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 e9f3aa062e3..35509ecefa0 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.Random; 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; @@ -48,7 +49,6 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; @Category(IntegrationTests.class) @@ -63,7 +63,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 { @@ -227,17 +226,18 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool { 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 073879848e3..417349dcf85 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.ExecutionException; 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; @@ -1516,7 +1517,6 @@ public class PerformanceEvaluation extends Configured implements Tool { 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); @@ -1530,7 +1530,7 @@ public class PerformanceEvaluation extends Configured implements Tool { @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++) { @@ -1938,8 +1938,6 @@ public class PerformanceEvaluation extends Configured implements Tool { 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); @@ -1953,7 +1951,7 @@ public class PerformanceEvaluation extends Configured implements Tool { @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 6e0d6a3bfea..475960dde35 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.LinkedList; 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; @@ -153,11 +155,11 @@ public class TestPerformanceEvaluation { 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 5b26530affd..9b82cbdfc87 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 @@ -41,6 +41,7 @@ import java.util.concurrent.Callable; 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; @@ -96,7 +97,6 @@ import org.apache.hadoop.hbase.regionserver.HRegion; 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; @@ -198,14 +198,13 @@ public class TestHFileOutputFormat2 { 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); @@ -268,14 +267,13 @@ public class TestHFileOutputFormat2 { 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); @@ -556,7 +554,7 @@ public class TestHFileOutputFormat2 { } 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; @@ -568,7 +566,7 @@ public class TestHFileOutputFormat2 { } 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] = @@ -1222,13 +1220,10 @@ public class TestHFileOutputFormat2 { 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 63c1760626f..4d27f9b6e95 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 @@ package org.apache.hadoop.hbase.util; 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; @@ -294,7 +293,7 @@ public class LoadTestTool extends AbstractHBaseTool { } 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 371f767f74c..120f91169c5 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 static org.junit.Assert.assertEquals; 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 class TestFastLongHistogram { // 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 17e4376108f..d88d93e571f 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.fs.Path; 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 @@ public abstract class ProcedureStorePerformanceEvaluation 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 25678e39e0b..e3064c9ab82 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.assertEquals; 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 class TestProcedureStoreTracker { 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 73d30208984..3d46883f2de 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.assertEquals; 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 class TestStressWALProcedureStore { 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 a5a456613d5..51a3408c1e3 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 @@ -59,7 +59,7 @@ public class TestZKReplicationPeerStorage { HBaseClassTestRule.forClass(TestZKReplicationPeerStorage.class); private static final HBaseZKTestingUtility UTIL = new HBaseZKTestingUtility(); - + private static final Random RNG = new Random(); // Seed may be set with Random#setSeed private static ZKReplicationPeerStorage STORAGE; @BeforeClass @@ -96,12 +96,12 @@ public class TestZKReplicationPeerStorage { } private ReplicationPeerConfig getConfig(int seed) { - Random rand = new Random(seed); - return ReplicationPeerConfig.newBuilder().setClusterKey(Long.toHexString(rand.nextLong())) - .setReplicationEndpointImpl(Long.toHexString(rand.nextLong())) - .setNamespaces(randNamespaces(rand)).setExcludeNamespaces(randNamespaces(rand)) - .setTableCFsMap(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())) + .setNamespaces(randNamespaces(RNG)).setExcludeNamespaces(randNamespaces(RNG)) + .setTableCFsMap(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 da09473ced8..b8ff62da72b 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.ArrayList; 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; @@ -94,7 +96,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-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/RSGroupableBalancerTestBase.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/RSGroupableBalancerTestBase.java index 713aab26b94..4644b66cae5 100644 --- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/RSGroupableBalancerTestBase.java +++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/RSGroupableBalancerTestBase.java @@ -23,14 +23,15 @@ import static org.junit.Assert.assertTrue; 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.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; @@ -52,7 +53,6 @@ import org.apache.hadoop.hbase.util.Bytes; 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; @@ -61,7 +61,6 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; */ 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 = @@ -305,10 +304,10 @@ public class RSGroupableBalancerTestBase extends BalancerTestBase{ 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); @@ -351,6 +350,7 @@ public class RSGroupableBalancerTestBase extends BalancerTestBase{ 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); @@ -378,6 +378,7 @@ public class RSGroupableBalancerTestBase extends BalancerTestBase{ groupMap.put(grpName, RSGroupInfo); index++; } + Random rand = ThreadLocalRandom.current(); while (index < servers.size()) { int grpIndex = rand.nextInt(groups.length); groupMap.get(groups[grpIndex]).addServer( @@ -394,6 +395,7 @@ public class RSGroupableBalancerTestBase extends BalancerTestBase{ */ protected static List constructTableDesc(boolean hasBogusTable) { List tds = Lists.newArrayList(); + Random rand = ThreadLocalRandom.current(); int index = rand.nextInt(groups.length); for (int i = 0; i < tables.length; i++) { TableDescriptor htd = TableDescriptorBuilder.newBuilder(tables[i]).build(); 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 e1a49491dc2..07e751716bf 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 static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_MAX_SPLITTER; 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 @@ public class ZkSplitLogWorkerCoordination extends ZKListener implements // 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 62a86d7418d..80de44915f2 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 final class PrefetchExecutor { }); } - 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 final class PrefetchExecutor { 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 a93f7a9e11e..cfae3d84b3c 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 @@ -30,6 +30,8 @@ import java.util.Random; 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; @@ -518,7 +520,7 @@ public class RegionPlacementMaintainer { 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 aff329ff475..3c5c06bfdb0 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 class AdaptiveMemStoreCompactionStrategy extends MemStoreCompactionStrate 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 eb7d9a8573f..860da9f60a4 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 @@ -52,13 +52,13 @@ import java.util.TreeSet; 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.commons.lang3.SystemUtils; import org.apache.hadoop.conf.Configuration; @@ -1964,14 +1964,14 @@ public class HRegionServer extends Thread implements 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); } } } 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 ef9f3ca2e32..db469c420ca 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.hadoop.hbase.regionserver.StoreUtils; 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 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy { 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 abstract class SortedCompactionPolicy extends CompactionPolicy { // deterministic jitter avoids a major compaction storm on restart OptionalInt seed = StoreUtils.getDeterministicRandomSeed(filesToCompact); if (seed.isPresent()) { - // Synchronized to ensure one user of random instance at a time. - double rnd; - synchronized (this) { - this.random.setSeed(seed.getAsInt()); - rnd = this.random.nextDouble(); - } long jitter = Math.round(period * jitterPct); - return period + jitter - Math.round(2L * jitter * rnd); + // Synchronized to ensure one user of random instance at a time. + synchronized (RNG) { + RNG.setSeed(seed.getAsInt()); + return period + jitter - Math.round(2L * jitter * RNG.nextDouble()); + } } 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 743369f2175..42335a52b98 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 static org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.WAL_TRA 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 @@ public abstract class AbstractProtobufLogWriter { 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 469d69266ea..6d2bd61a023 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.ByteArrayOutputStream; 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 @@ public class SecureWALCellCodec extends WALCellCodec { @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 7a93d901ad5..2d72d3f7d4a 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 @@ -15,7 +15,6 @@ import java.io.FileNotFoundException; 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; @@ -26,6 +25,7 @@ import java.util.Map.Entry; 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; @@ -333,7 +333,7 @@ public class HFileReplicator implements Closeable { 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 a8ae781187f..1f791b25b27 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.HashSet; 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.client.ConnectStringParser; 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 @@ public class CanaryTool implements Tool, Canary { 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)); @@ -1832,7 +1829,6 @@ public class CanaryTool implements Tool, Canary { 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); @@ -1849,7 +1845,8 @@ public class CanaryTool implements Tool, Canary { } } 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 0ccf0f4d8b4..cee3b56d6f6 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 class EncryptionTest { 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 d60f2cbc56f..b37dc18dc7e 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.Collection; 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; @@ -164,10 +166,10 @@ public class HBaseFsckRepair { 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 4faf86dd4e6..ae940dc24a1 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 @@ package org.apache.hadoop.hbase; 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 @@ public class AcidGuaranteesTestTool extends AbstractHBaseTool { * 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 class AcidGuaranteesTestTool extends AbstractHBaseTool { @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/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 72e12043c0e..cd0fac2eeee 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -48,6 +48,7 @@ import java.util.Properties; 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; @@ -2419,10 +2420,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility { 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); @@ -3295,7 +3295,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility { // 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; @@ -3894,7 +3894,6 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility { 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][]; { @@ -3922,6 +3921,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility { 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, @@ -3967,8 +3967,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility { 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-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java index 5ab407b9051..a12a83389de 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; @@ -337,7 +336,6 @@ public class HFilePerformanceEvaluation { 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, @@ -354,7 +352,7 @@ public class HFilePerformanceEvaluation { 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)); @@ -376,7 +374,7 @@ public class HFilePerformanceEvaluation { } private byte[] generateValue() { - random.nextBytes(bytes); + Bytes.random(bytes); return bytes; } @@ -447,8 +445,6 @@ public class HFilePerformanceEvaluation { 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); @@ -469,12 +465,11 @@ public class HFilePerformanceEvaluation { } 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) { @@ -507,7 +502,7 @@ public class HFilePerformanceEvaluation { } 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/TestHBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java index 993af218723..3182e62c1bb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java @@ -428,7 +428,7 @@ public class TestHBaseTestingUtility { when(portChecker.available(anyInt())).thenReturn(true); HBaseTestingUtility.PortAllocator portAllocator = - new HBaseTestingUtility.PortAllocator(random, portChecker); + new HBaseTestingUtility.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 e3483b932d7..2409db0d0e7 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 @@ -38,6 +38,7 @@ import java.util.List; import java.util.Map; import java.util.Random; import java.util.concurrent.TimeUnit; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; @@ -89,7 +90,6 @@ public class TestMetaTableAccessor { private static final Logger LOG = LoggerFactory.getLogger(TestMetaTableAccessor.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static Connection connection; - private Random random = new Random(); @Rule public TestName name = new TestName(); @@ -440,9 +440,11 @@ public class TestMetaTableAccessor { @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())) @@ -467,9 +469,9 @@ public class TestMetaTableAccessor { .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, @@ -555,7 +557,8 @@ public class TestMetaTableAccessor { @Test public void testMetaLocationForRegionReplicasIsAddedAtRegionSplit() throws IOException { long regionId = EnvironmentEdgeManager.currentTime(); - ServerName serverName0 = ServerName.valueOf("foo", 60010, random.nextLong()); + ServerName serverName0 = ServerName.valueOf("foo", 60010, + ThreadLocalRandom.current().nextLong()); RegionInfo parent = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName())) .setStartKey(HConstants.EMPTY_START_ROW) .setEndKey(HConstants.EMPTY_END_ROW) @@ -595,7 +598,8 @@ public class TestMetaTableAccessor { @Test public void testMetaLocationForRegionReplicasIsAddedAtRegionMerge() throws IOException { long regionId = EnvironmentEdgeManager.currentTime(); - ServerName serverName0 = ServerName.valueOf("foo", 60010, random.nextLong()); + ServerName serverName0 = ServerName.valueOf("foo", 60010, + ThreadLocalRandom.current().nextLong()); RegionInfo parentA = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName())) .setStartKey(Bytes.toBytes("a")) @@ -882,7 +886,8 @@ public class TestMetaTableAccessor { @Test public void testEmptyMetaDaughterLocationDuringSplit() throws IOException { long regionId = EnvironmentEdgeManager.currentTime(); - ServerName serverName0 = ServerName.valueOf("foo", 60010, random.nextLong()); + ServerName serverName0 = ServerName.valueOf("foo", 60010, + ThreadLocalRandom.current().nextLong()); RegionInfo parent = RegionInfoBuilder.newBuilder(TableName.valueOf("table_foo")) .setStartKey(HConstants.EMPTY_START_ROW) .setEndKey(HConstants.EMPTY_END_ROW) 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 b73aeef4710..5c855f4b47f 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 @@ -30,7 +30,7 @@ import java.util.EnumSet; 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.conf.Configuration; @@ -643,9 +643,7 @@ public class TestAdmin2 extends TestAdminBase { @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 874a01c8c71..21dbc0d47d8 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.Arrays; 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.BeforeClass; 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 class TestAsyncBufferMutator { 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 97b18d324b1..79ab476de7a 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 @@ -197,11 +197,11 @@ public class TestAsyncNonMetaRegionLocator { 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 83845186021..82dbb85e35e 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 static org.junit.Assert.assertTrue; 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 class TestAsyncProcedureAdminApi extends TestAsyncAdminBase { @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 57cfbeca6e2..c4bf8af5c82 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 static org.junit.Assert.assertEquals; 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; @@ -69,7 +68,7 @@ public class TestAsyncTableBatchRetryImmediately { 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 e6f4b1a6e6f..e28ba689f1b 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 @@ -27,11 +27,11 @@ import java.util.ArrayList; 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; @@ -146,7 +146,7 @@ public class TestAsyncTableGetMultiThreaded { 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 9defaf5602f..331d83c8615 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 @@ -92,7 +92,6 @@ public class TestFromClientSide3 { = new HBaseTestingUtility(); 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"); @@ -144,9 +143,10 @@ public class TestFromClientSide3 { 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); @@ -286,7 +286,7 @@ public class TestFromClientSide3 { ClusterConnection connection = (ClusterConnection) TEST_UTIL.getConnection(); // Create 3 store files. - byte[] row = Bytes.toBytes(random.nextInt()); + byte[] row = Bytes.toBytes(ThreadLocalRandom.current().nextInt()); performMultiplePutAndFlush((HBaseAdmin) 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 d8baa8d1cb1..20ab2f92613 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 static junit.framework.TestCase.assertEquals; 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; @@ -153,7 +152,7 @@ public class TestMultiRespectsLimits { // 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 c78bb00c327..5623adcec1e 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 @@ package org.apache.hadoop.hbase.client; 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.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; @@ -33,7 +32,6 @@ import org.junit.BeforeClass; 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 class TestRequestTooBigException { @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 d6a066698f3..032dfb22679 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.Rule; 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 HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @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 2b8c3a927cd..50b6a278a4a 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.times; 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 class TestEntityLocks { 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 33f1599cacf..da0aa459ae5 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.List; 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 class TestMetaTableMetrics { 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 a99074f1c92..759e7e8ce12 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 @@ package org.apache.hadoop.hbase.io.compress; 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 HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); 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 class HFileTestBase { 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 9596ad801c1..8277e43e593 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.Arrays; 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; @@ -252,7 +254,7 @@ public class TestChangingEncoding { @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 390efd20eb3..9d7985d1838 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.ArrayList; 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 class TestDataBlockEncoders { 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 e2e579578e6..483178f7f1b 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.Arrays; 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.HeapSize; 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 class CacheTestUtils { 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 b8c798bf51c..b07dc0177eb 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.Arrays; 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 @@ public class TestHFile { 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 5f2008fcc45..7dc55b88386 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.Executor; 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 HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static final Random RNG = new Random(); // This test depends on Random#setSeed private FileSystem fs; private final boolean includesMemstoreTS; @@ -182,9 +182,9 @@ public class TestHFileBlock { 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 @@ public class TestHFileBlock { 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 @@ public class TestHFileBlock { 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 526072b73ad..c34ac77c8ea 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 @@ -87,7 +87,7 @@ public class TestHFileBlockIndex { } 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 HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @@ -97,7 +97,6 @@ public class TestHFileBlockIndex { private static FileSystem fs; private Path path; - private Random rand; private long rootIndexOffset; private int numRootEntries; private int numLevels; @@ -121,9 +120,9 @@ public class TestHFileBlockIndex { @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); @@ -140,9 +139,9 @@ public class TestHFileBlockIndex { 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); @@ -272,9 +271,8 @@ public class TestHFileBlockIndex { 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(String.valueOf(rand.nextInt(1000)).getBytes()); + hbw.startWriting(BlockType.DATA).write(Bytes.toBytes(String.valueOf(RNG.nextInt(1000)))); long blockOffset = outputStream.getPos(); hbw.writeHeaderAndData(outputStream); @@ -283,7 +281,7 @@ public class TestHFileBlockIndex { 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) { @@ -351,7 +349,7 @@ public class TestHFileBlockIndex { 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); @@ -476,8 +474,8 @@ public class TestHFileBlockIndex { 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 0bd36614f1a..b9724d6c4a4 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 static org.junit.Assert.fail; 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; @@ -66,9 +68,8 @@ public class TestHFileEncryption { private static final Logger LOG = LoggerFactory.getLogger(TestHFileEncryption.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static final SecureRandom RNG = new SecureRandom(); - private static FileSystem fs; + private static FileSystem fs; private static Encryption.Context cryptoContext; @BeforeClass @@ -89,7 +90,7 @@ public class TestHFileEncryption { assertNotNull(aes); cryptoContext.setCipher(aes); byte[] key = new byte[aes.getKeyLength()]; - RNG.nextBytes(key); + Bytes.secureRandom(key); cryptoContext.setKey(key); } @@ -134,8 +135,9 @@ public class TestHFileEncryption { 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 : HBaseTestingUtility.COMPRESSION_ALGORITHMS) { Path path = new Path(TEST_UTIL.getDataTestDir(), "block_v3_" + compression + "_AES"); @@ -269,12 +271,13 @@ public class TestHFileEncryption { // 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 273f674e055..721c4f19084 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.io.IOException; 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 class TestHFileScannerImplReferenceCount { private static final Logger LOG = LoggerFactory.getLogger(TestHFileScannerImplReferenceCount.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + 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 @@ public class TestHFileScannerImplReferenceCount { 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 7e489115604..c94021fb536 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.ArrayList; 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 HBaseTestingUtility TEST_UTIL = - new HBaseTestingUtility(); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static final Random RNG = new Random(9713312); // Just a fixed seed. private Configuration conf; private FileSystem fs; @@ -135,20 +133,17 @@ public class TestHFileWriterV3 { .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 2d9a5bd39c2..bacb0ebc91d 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.Arrays; 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 HBaseTestingUtility TEST_UTIL = - new HBaseTestingUtility(); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static final Random RNG = new Random(9713312); // Just a fixed seed. private Configuration conf; private FileSystem fs; @@ -150,11 +148,8 @@ public class TestHFileWriterV3WithDataEncoders { .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 64d44c8bbce..eec1626dfaf 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 @@ -63,9 +63,9 @@ public class TestLazyDataBlockDecompression { @ClassRule public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestLazyDataBlockDecompression.class); - private static final Logger LOG = LoggerFactory.getLogger(TestLazyDataBlockDecompression.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static final Random RNG = new Random(9713312); // Just a fixed seed. private FileSystem fs; @@ -101,14 +101,12 @@ public class TestLazyDataBlockDecompression { .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 f29d12ac315..a922b308c00 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.nio.ByteBuffer; 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 class TestLruAdaptiveBlockCache { 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 @@ public class TestBucketCache { * 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 5c4eea86f3b..e3a4757dfe3 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.List; 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 @@ class MockRegionServer implements AdminProtos.AdminService.BlockingInterface, } 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 ec845034803..451810199d5 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.HashMap; 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.HBaseClassTestRule; @@ -61,7 +62,6 @@ import org.apache.hadoop.hbase.regionserver.Region; 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; @@ -191,13 +191,13 @@ public class TestRegionPlacement { 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(); @@ -257,10 +257,10 @@ public class TestRegionPlacement { 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(); } } @@ -281,7 +281,7 @@ public class TestRegionPlacement { // 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 85255d29dcf..1449eb6f858 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.io.UncheckedIOException; 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.ExecutionException; 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 abstract class TestAssignmentManagerBase { } 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 @@ public abstract class TestAssignmentManagerBase { 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 abstract class TestAssignmentManagerBase { 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/BalancerTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java index 80df6421ea4..6c779f692e1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerTestBase.java @@ -449,10 +449,9 @@ public class BalancerTestBase { 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); @@ -469,10 +468,9 @@ public class BalancerTestBase { 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()); @@ -480,8 +478,8 @@ public class BalancerTestBase { } 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) @@ -496,15 +494,13 @@ public class BalancerTestBase { 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-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 d23436d3b4c..b19b877ea49 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 @@ package org.apache.hadoop.hbase.master.balancer; 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.HBaseTestingUtility; @@ -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 ad7dcbd9524..7aea24d4ff6 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.List; 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 class TestStochasticBalancerJmxMetrics extends BalancerTestBase { 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 af1e85bba50..52c2c12ac08 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.assertFalse; 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.HBaseTestingUtility; 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; @@ -485,13 +486,12 @@ public class TestCleanerChore { } 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 db311ea78ae..fe4900c9a76 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 static org.junit.Assert.assertTrue; 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; import org.apache.hadoop.fs.FileStatus; @@ -481,7 +482,7 @@ public class TestHFileCleaner { 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 e0414fa2a44..d0e547ef657 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 @@ -31,7 +31,6 @@ import java.util.Iterator; import java.util.List; import java.util.concurrent.ThreadLocalRandom; 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; @@ -55,6 +54,7 @@ import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; 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.zookeeper.RecoverableZooKeeper; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; @@ -329,8 +329,9 @@ public class TestLogsCleaner { 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 0c3303bf904..0bc6ba49b57 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,9 +19,8 @@ package org.apache.hadoop.hbase.master.procedure; import java.io.IOException; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicLong; -import java.util.Random; - import org.apache.commons.lang3.ArrayUtils; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; @@ -201,11 +200,10 @@ public class MasterProcedureSchedulerPerformanceEvaluation extends AbstractHBase 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 b040c71005a..fcd20001609 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.assertFalse; 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.HBaseTestingUtility; @@ -181,11 +182,10 @@ public class TestProcedureAdmin { @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/MobTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobTestUtil.java index f22ec8de5f1..5ee71f66da2 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 @@ package org.apache.hadoop.hbase.mob; 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 9e90145b0ef..429af27c31e 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 @@ package org.apache.hadoop.hbase.mob; 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; @@ -169,7 +168,7 @@ public class TestExpiredMobFileCleaner { 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/TestMobDataBlockEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java index 9ed2ad3add6..a0078faf65b 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.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -55,7 +54,6 @@ public class TestMobDataBlockEncoding { private static Admin admin; private static HColumnDescriptor hcd; private static HTableDescriptor desc; - private static Random random = new Random(); private static long defaultThreshold = 10; @BeforeClass @@ -91,7 +89,7 @@ public class TestMobDataBlockEncoding { */ 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 ae53ff21feb..d9cd9e6b1d4 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.assertNotSame; 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.HBaseTestingUtility; 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 HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private String uuid; private Date date; private String dateStr; @@ -50,11 +49,10 @@ public class TestMobFileName { @Before public void setUp() { - Random random = new Random(); - uuid = TEST_UTIL.getRandomUUID().toString().replaceAll("-", ""); + uuid = HBaseTestingUtility.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/quotas/SpaceQuotaHelperForTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java index b06621584c4..22aaa9558bb 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 @@ -26,7 +26,6 @@ import java.util.Collection; import java.util.List; import java.util.Map.Entry; import java.util.Objects; -import java.util.Random; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; @@ -513,14 +512,13 @@ public class SpaceQuotaHelperForTests { 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 7baec50b0c8..198abb7ff33 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.ArrayList; 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.HBaseTestingUtility; @@ -148,14 +148,13 @@ public class TestRegionSizeUse { 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 67caf8ea209..3b27971f878 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 @@ package org.apache.hadoop.hbase.regionserver; 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 class CreateRandomStoreFile { .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 @@ public class CreateRandomStoreFile { } 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 class CreateRandomStoreFile { } 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 @@ public class CreateRandomStoreFile { } 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 @@ public class CreateRandomStoreFile { } 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 04aa3dfbd86..3cae011f1b9 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.ArrayList; 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; @@ -609,7 +608,6 @@ public class TestAtomicOperation { 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 a2e398ff3a5..0db15e750d6 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 @@ -28,8 +28,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.Random; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; @@ -53,7 +51,6 @@ import org.apache.hadoop.hbase.wal.WALEdit; import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.TypeSafeMatcher; - import org.junit.Before; import org.junit.ClassRule; import org.junit.Rule; @@ -69,7 +66,6 @@ public class TestBulkloadBase { private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); 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"); @@ -80,7 +76,7 @@ public class TestBulkloadBase { @Before public void before() throws IOException { - random.nextBytes(randomBytes); + Bytes.random(randomBytes); } protected Pair withMissingHFileForFamily(byte[] family) { 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 66fd583d0a2..9ef89aa69ed 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.io.IOException; 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.HBaseTestingUtility; import org.apache.hadoop.hbase.TableName; @@ -43,8 +45,6 @@ import org.junit.Rule; 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 HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private final static Random random = new Random(); @Rule public TestName name = new TestName(); @@ -265,9 +263,10 @@ public class TestCompactionState { 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 122cde9aaa1..94baf12a8c2 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; @@ -57,7 +57,6 @@ public class TestDeleteMobTable { private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); 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(); @@ -81,7 +80,7 @@ public class TestDeleteMobTable { */ 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 3fc8e95f010..0d708af780e 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 static org.junit.Assert.assertTrue; 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; @@ -77,13 +77,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 e798c4884b7..1dd6d4e7c93 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.Map; 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.ChoreService; @@ -222,7 +223,7 @@ public class TestEndToEndSplitTransaction { @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 48ca166a9b6..c20a3609a31 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 static org.apache.hadoop.hbase.regionserver.Store.PRIORITY_USER; 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.List; 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 class TestHMobStore { 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 cd7db9f5ad5..05db0d2d8ab 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.ArrayList; 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; @@ -1519,9 +1518,8 @@ public class TestHRegionReplayEvents { 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 3aa35e1cc52..27628f2d51a 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,7 +21,7 @@ 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.hbase.CompareOperator; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -125,16 +125,14 @@ public class TestJoinedScanners { 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 afbbfeabd37..db708c59962 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.nio.ByteBuffer; 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 class TestMemStoreChunkPool { @Test public void testReusingChunks() { - Random rand = new Random(); MemStoreLAB mslab = new MemStoreLABImpl(conf); int expectedOff = 0; ByteBuffer lastBuffer = null; @@ -95,6 +95,7 @@ public class TestMemStoreChunkPool { 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 00569bc37da..032ea4eef60 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.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 org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ByteBufferKeyValue; @@ -87,7 +88,6 @@ public class TestMemStoreLAB { */ @Test public void testLABRandomAllocation() { - Random rand = new Random(); MemStoreLAB mslab = new MemStoreLABImpl(); int expectedOff = 0; ByteBuffer lastBuffer = null; @@ -95,6 +95,7 @@ public class TestMemStoreLAB { // 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 class TestMemStoreLAB { 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 fd2ce0ed9da..1dae13322c3 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.nio.ByteBuffer; 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 class TestMemstoreLABWithoutPool { */ @Test public void testLABRandomAllocation() { - Random rand = new Random(); MemStoreLAB mslab = new MemStoreLABImpl(); int expectedOff = 0; ByteBuffer lastBuffer = null; @@ -77,6 +78,7 @@ public class TestMemstoreLABWithoutPool { // 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/TestMobStoreCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java index 715c63ace95..a8923498739 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java @@ -29,7 +29,6 @@ import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -380,7 +379,7 @@ public class TestMobStoreCompaction { 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/regionserver/TestMobStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java index c430a144911..c0d0f67fe5d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.regionserver; 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 +77,6 @@ public class TestMobStoreScanner { private static Admin admin; private static HColumnDescriptor hcd; private static HTableDescriptor desc; - private static Random random = new Random(); private static long defaultThreshold = 10; private FileSystem fs; private Configuration conf; @@ -122,7 +120,7 @@ public class TestMobStoreScanner { */ 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/TestMultiColumnScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java index 5950b943584..eda7fcc950d 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.Map; 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 abstract class TestMultiColumnScanner { // 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 b08be0f37e7..ca5e83b26ea 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 @@ public class TestMultiVersionConcurrencyControl { this.status = status; } - private Random rnd = new Random(); public boolean failed = false; @Override @@ -60,7 +59,7 @@ public class TestMultiVersionConcurrencyControl { 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 6a79267eb24..d0f71583723 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 @@ -21,7 +21,6 @@ import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1; import static org.junit.Assert.assertEquals; 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; @@ -196,7 +195,6 @@ public class TestParallelPut { 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) { @@ -215,7 +213,7 @@ public class TestParallelPut { // 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<>(); @@ -127,6 +126,7 @@ public class TestSeekOptimizations { private long totalSeekDiligent, totalSeekLazy; private final static HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU(); + private static final Random RNG = new Random(); // This test depends on Random#setSeed @Parameters public static final Collection parameters() { @@ -141,7 +141,7 @@ public class TestSeekOptimizations { @Before public void setUp() { - rand = new Random(91238123L); + RNG.setSeed(91238123L); expectedKVs.clear(); TEST_UTIL.getConfiguration().setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, 10); } @@ -348,7 +348,7 @@ public class TestSeekOptimizations { } 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; @@ -411,7 +411,7 @@ public class TestSeekOptimizations { 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 f20691b95ca..73d20ec591d 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.NavigableSet; import java.util.Random; import java.util.TreeSet; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -185,13 +186,13 @@ public class TestStoreScannerClosure { .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 2e47566a9cb..124d8f10a9f 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 @@ class ConstantSizeFileListGenerator extends StoreFileListGenerator { } }; } + } 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 e14f69618b1..a76a8a510b6 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 @@ package org.apache.hadoop.hbase.regionserver.compactions; import java.util.Iterator; import java.util.List; - import org.apache.hadoop.hbase.regionserver.HStoreFile; /** @@ -27,6 +26,7 @@ import org.apache.hadoop.hbase.regionserver.HStoreFile; * 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 @@ class ExplicitFileListGenerator extends StoreFileListGenerator { } }; } + } 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 fe5f9040988..51930301a8a 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 @@ package org.apache.hadoop.hbase.regionserver.compactions; 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 @@ class GaussianFileListGenerator extends StoreFileListGenerator { } }; } + } 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 cb6c4860123..d6d84ddbf9c 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 static org.mockito.Mockito.when; 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 @@ import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects; 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 @@ class MockStoreFileGenerator { 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 13c7a6bc103..45e9c14ae1f 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 @@ -113,7 +113,6 @@ public class PerfTestCompactionPolicies extends MockStoreFileGenerator { 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 c89f136c86f..8c3db2002ec 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 @@ package org.apache.hadoop.hbase.regionserver.compactions; 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 @@ class SemiConstantSizeFileListGenerator extends StoreFileListGenerator { 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 @@ class SemiConstantSizeFileListGenerator extends StoreFileListGenerator { } }; } + } 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 d270da74d27..75779c18244 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 java.util.List; import org.apache.hadoop.hbase.regionserver.HStoreFile; class SinusoidalFileListGenerator extends StoreFileListGenerator { - SinusoidalFileListGenerator() { - super(SinusoidalFileListGenerator.class); - } @Override public Iterator> iterator() { @@ -53,4 +50,5 @@ class SinusoidalFileListGenerator extends StoreFileListGenerator { } }; } + } 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 bed1342a03a..98c1469d512 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 @@ package org.apache.hadoop.hbase.regionserver.compactions; 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 @@ class SpikyFileListGenerator extends StoreFileListGenerator { 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 @@ class SpikyFileListGenerator extends StoreFileListGenerator { } }; } + } 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 88546766fa7..7828a4a97b9 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 3090572ac30..4148a606d53 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 static org.junit.Assert.assertEquals; 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 @@ public class TestFIFOCompactionPolicy { 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 3157e33c678..87a37697ec5 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 static org.junit.Assert.assertTrue; 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.HBaseTestingUtility; @@ -97,7 +96,7 @@ public class TestCompactionWithThroughputController { 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 98fe132c10e..f69e66bb2f9 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 static org.junit.Assert.assertTrue; 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; @@ -116,13 +115,12 @@ public class TestFlushWithThroughputController { // 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(); @@ -197,11 +195,10 @@ public class TestFlushWithThroughputController { assertEquals(10L * 1024 * 1024, throughputController.getMaxThroughput(), EPSILON); } 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 296808a75fd..f819cd32381 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,8 @@ import java.io.IOException; 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 +582,12 @@ public class TestMasterReplication { @SuppressWarnings("resource") private void startMiniClusters(int numClusters) throws Exception { - Random random = new Random(); utilities = new HBaseTestingUtility[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()); HBaseTestingUtility utility = new HBaseTestingUtility(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 cf7e580015f..02eb3d92d05 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 @@ package org.apache.hadoop.hbase.replication.regionserver; 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.List; 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 class TestReplicationSink { 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/util/LoadTestDataGeneratorWithTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestDataGeneratorWithTags.java index 2ef38212707..81117b21f40 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.io.IOException; 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 class LoadTestDataGeneratorWithTags extends DefaultDataGenerator { 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 4fc0326c9e1..4481a168a9f 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 @@ -26,8 +26,8 @@ import java.util.Arrays; import java.util.Collection; import java.util.HashMap; 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; @@ -84,7 +84,6 @@ public abstract class MultiThreadedAction { 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) { @@ -111,7 +110,8 @@ public abstract class MultiThreadedAction { @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 44b4a2d5eb3..13f45925799 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,15 +19,14 @@ package org.apache.hadoop.hbase.util; 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; import org.apache.hadoop.hbase.client.Get; - import org.apache.hadoop.hbase.client.Consistency; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; @@ -264,18 +263,19 @@ public class MultiThreadedReader extends MultiThreadedAction // 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) { @@ -291,7 +291,7 @@ public class MultiThreadedReader extends MultiThreadedAction } 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 30050a489c0..78c698975a7 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.io.StringWriter; 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 @@ public class MultiThreadedUpdater extends MultiThreadedWriterBase { @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 class MultiThreadedUpdater extends MultiThreadedWriterBase { 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 32d0b74b9ab..7c2fab212cd 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 class TestFSUtils { 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 + HBaseTestingUtility.getRandomUUID().toString()); try { FSDataOutputStream out = FSUtils.create(conf, fs, p, filePerm, null); out.close(); @@ -340,7 +340,7 @@ public class TestFSUtils { 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 = HBaseTestingUtility.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 class TestFSUtils { FileSystem fs = FileSystem.get(conf); Path testDir = htu.getDataTestDirOnTestFS("testArchiveFile"); - String file = htu.getRandomUUID().toString(); + String file = HBaseTestingUtility.getRandomUUID().toString(); Path p = new Path(testDir, file); FSDataOutputStream out = fs.create(p); @@ -395,7 +395,7 @@ public class TestFSUtils { mockEnv.setValue(expect); EnvironmentEdgeManager.injectEdge(mockEnv); try { - String dstFile = htu.getRandomUUID().toString(); + String dstFile = HBaseTestingUtility.getRandomUUID().toString(); Path dst = new Path(testDir , dstFile); assertTrue(CommonFSUtils.renameAndSetModifyTime(fs, p, dst)); @@ -477,7 +477,7 @@ public class TestFSUtils { conf.get(HConstants.WAL_STORAGE_POLICY, HConstants.DEFAULT_WAL_STORAGE_POLICY); CommonFSUtils.setStoragePolicy(fs, testDir, storagePolicy); - String file =htu.getRandomUUID().toString(); + String file = HBaseTestingUtility.getRandomUUID().toString(); Path p = new Path(testDir, file); WriteDataToHDFS(fs, p, 4096); HFileSystem hfs = new HFileSystem(fs); @@ -492,7 +492,7 @@ public class TestFSUtils { } 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 class TestFSUtils { // 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 4b793b64e0e..e60c145451c 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.assertNotNull; 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; @@ -78,9 +77,8 @@ public class TestHBaseFsckEncryption { 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 0ea74228160..15871745b8b 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.ExecutorCompletionService; 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 class TestIdLock { @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/TestIdReadWriteLock.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdReadWriteLock.java index b305743ba00..8d1924d63bc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdReadWriteLock.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestIdReadWriteLock.java @@ -29,6 +29,7 @@ import java.util.concurrent.ExecutorCompletionService; 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; @@ -81,7 +82,7 @@ public class TestIdReadWriteLock { @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 7944e3f8429..43e177b70af 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.NavigableMap; 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 @@ public final class WALPerformanceEvaluation extends Configured implements Tool { 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 final class WALPerformanceEvaluation extends Configured implements Tool { 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-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 eec4ae18a81..5f6addbff1c 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.InetAddress; 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; @@ -141,7 +142,7 @@ public class MiniZooKeeperCluster { 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 a6d9c0f4732..f1846c3721f 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.List; 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.HBaseZKTestingUtility; @@ -51,6 +50,7 @@ import org.apache.hadoop.hbase.HConstants; 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 class TestReadOnlyZKClient { 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);