Skip to content

Commit 300f9b9

Browse files
committed
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 <[email protected]>
1 parent 98836fb commit 300f9b9

File tree

179 files changed

+666
-789
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

179 files changed

+666
-789
lines changed

hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutput.java

Lines changed: 9 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,6 @@
3434
import java.util.Random;
3535
import java.util.concurrent.CompletableFuture;
3636
import java.util.concurrent.ExecutionException;
37-
import java.util.concurrent.ThreadLocalRandom;
3837
import org.apache.hadoop.fs.FSDataInputStream;
3938
import org.apache.hadoop.fs.FileSystem;
4039
import org.apache.hadoop.fs.Path;
@@ -44,6 +43,7 @@
4443
import org.apache.hadoop.hbase.io.asyncfs.monitor.StreamSlowMonitor;
4544
import org.apache.hadoop.hbase.testclassification.MediumTests;
4645
import org.apache.hadoop.hbase.testclassification.MiscTests;
46+
import org.apache.hadoop.hbase.util.Bytes;
4747
import org.apache.hadoop.hdfs.DistributedFileSystem;
4848
import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
4949
import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -57,7 +57,6 @@
5757
import org.junit.rules.TestName;
5858
import org.slf4j.Logger;
5959
import org.slf4j.LoggerFactory;
60-
6160
import org.apache.hbase.thirdparty.io.netty.channel.Channel;
6261
import org.apache.hbase.thirdparty.io.netty.channel.EventLoop;
6362
import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
@@ -72,13 +71,9 @@ public class TestFanOutOneBlockAsyncDFSOutput extends AsyncFSTestBase {
7271
HBaseClassTestRule.forClass(TestFanOutOneBlockAsyncDFSOutput.class);
7372

7473
private static final Logger LOG = LoggerFactory.getLogger(TestFanOutOneBlockAsyncDFSOutput.class);
75-
7674
private static DistributedFileSystem FS;
77-
7875
private static EventLoopGroup EVENT_LOOP_GROUP;
79-
8076
private static Class<? extends Channel> CHANNEL_CLASS;
81-
8277
private static int READ_TIMEOUT_MS = 2000;
8378

8479
private static StreamSlowMonitor MONITOR;
@@ -104,14 +99,16 @@ public static void tearDown() throws IOException, InterruptedException {
10499
shutdownMiniDFSCluster();
105100
}
106101

102+
private static final Random RNG = new Random(); // This test depends on Random#setSeed
103+
107104
static void writeAndVerify(FileSystem fs, Path f, AsyncFSOutput out)
108105
throws IOException, InterruptedException, ExecutionException {
109106
List<CompletableFuture<Long>> futures = new ArrayList<>();
110107
byte[] b = new byte[10];
111-
Random rand = new Random(12345);
112108
// test pipelined flush
109+
RNG.setSeed(12345);
113110
for (int i = 0; i < 10; i++) {
114-
rand.nextBytes(b);
111+
RNG.nextBytes(b);
115112
out.write(b);
116113
futures.add(out.flush(false));
117114
futures.add(out.flush(false));
@@ -123,11 +120,11 @@ static void writeAndVerify(FileSystem fs, Path f, AsyncFSOutput out)
123120
out.close();
124121
assertEquals(b.length * 10, fs.getFileStatus(f).getLen());
125122
byte[] actual = new byte[b.length];
126-
rand.setSeed(12345);
123+
RNG.setSeed(12345);
127124
try (FSDataInputStream in = fs.open(f)) {
128125
for (int i = 0; i < 10; i++) {
129126
in.readFully(actual);
130-
rand.nextBytes(b);
127+
RNG.nextBytes(b);
131128
assertArrayEquals(b, actual);
132129
}
133130
assertEquals(-1, in.read());
@@ -150,7 +147,7 @@ public void testRecover() throws IOException, InterruptedException, ExecutionExc
150147
FanOutOneBlockAsyncDFSOutput out = FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS, f, true,
151148
false, (short) 3, FS.getDefaultBlockSize(), eventLoop, CHANNEL_CLASS, MONITOR);
152149
byte[] b = new byte[10];
153-
ThreadLocalRandom.current().nextBytes(b);
150+
Bytes.random(b);
154151
out.write(b, 0, b.length);
155152
out.flush(false).get();
156153
// restart one datanode which causes one connection broken
@@ -260,7 +257,7 @@ public void testWriteLargeChunk() throws IOException, InterruptedException, Exec
260257
FanOutOneBlockAsyncDFSOutput out = FanOutOneBlockAsyncDFSOutputHelper.createOutput(FS, f, true,
261258
false, (short) 3, 1024 * 1024 * 1024, eventLoop, CHANNEL_CLASS, MONITOR);
262259
byte[] b = new byte[50 * 1024 * 1024];
263-
ThreadLocalRandom.current().nextBytes(b);
260+
Bytes.random(b);
264261
out.write(b);
265262
out.flush(false);
266263
assertEquals(b.length, out.flush(false).get().longValue());

hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestFanOutOneBlockAsyncDFSOutputHang.java

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,6 @@
1717
*/
1818
package org.apache.hadoop.hbase.io.asyncfs;
1919

20-
2120
import static org.junit.Assert.assertTrue;
2221
import static org.junit.Assert.fail;
2322

@@ -29,12 +28,12 @@
2928
import java.util.concurrent.CompletableFuture;
3029
import java.util.concurrent.CyclicBarrier;
3130
import java.util.concurrent.ExecutionException;
32-
import java.util.concurrent.ThreadLocalRandom;
3331
import org.apache.hadoop.fs.Path;
3432
import org.apache.hadoop.hbase.HBaseClassTestRule;
3533
import org.apache.hadoop.hbase.io.asyncfs.monitor.StreamSlowMonitor;
3634
import org.apache.hadoop.hbase.testclassification.MediumTests;
3735
import org.apache.hadoop.hbase.testclassification.MiscTests;
36+
import org.apache.hadoop.hbase.util.Bytes;
3837
import org.apache.hadoop.hdfs.DistributedFileSystem;
3938
import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
4039
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -57,7 +56,6 @@
5756
import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
5857
import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
5958

60-
6159
/**
6260
* Testcase for HBASE-26679, here we introduce a separate test class and not put the testcase in
6361
* {@link TestFanOutOneBlockAsyncDFSOutput} because we will send heartbeat to DN when there is no
@@ -191,7 +189,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception
191189
});
192190

193191
byte[] b = new byte[10];
194-
ThreadLocalRandom.current().nextBytes(b);
192+
Bytes.random(b);
195193
OUT.write(b, 0, b.length);
196194
CompletableFuture<Long> future = OUT.flush(false);
197195
/**

hbase-client/src/main/java/org/apache/hadoop/hbase/client/PerClientRandomNonceGenerator.java

Lines changed: 4 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -19,8 +19,7 @@
1919
package org.apache.hadoop.hbase.client;
2020

2121
import java.util.Arrays;
22-
import java.util.Random;
23-
22+
import java.util.concurrent.ThreadLocalRandom;
2423
import org.apache.hadoop.hbase.HConstants;
2524
import org.apache.yetus.audience.InterfaceAudience;
2625

@@ -33,12 +32,12 @@ public final class PerClientRandomNonceGenerator implements NonceGenerator {
3332

3433
private static final PerClientRandomNonceGenerator INST = new PerClientRandomNonceGenerator();
3534

36-
private final Random rdm = new Random();
3735
private final long clientId;
3836

3937
private PerClientRandomNonceGenerator() {
4038
byte[] clientIdBase = ClientIdGenerator.generateClientId();
41-
this.clientId = (((long) Arrays.hashCode(clientIdBase)) << 32) + rdm.nextInt();
39+
this.clientId = (((long) Arrays.hashCode(clientIdBase)) << 32) +
40+
ThreadLocalRandom.current().nextInt();
4241
}
4342

4443
@Override
@@ -50,7 +49,7 @@ public long getNonceGroup() {
5049
public long newNonce() {
5150
long result = HConstants.NO_NONCE;
5251
do {
53-
result = rdm.nextLong();
52+
result = ThreadLocalRandom.current().nextLong();
5453
} while (result == HConstants.NO_NONCE);
5554
return result;
5655
}

hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@
2020
package org.apache.hadoop.hbase.filter;
2121

2222
import java.util.Objects;
23-
import java.util.Random;
23+
import java.util.concurrent.ThreadLocalRandom;
2424

2525
import org.apache.hadoop.hbase.Cell;
2626
import org.apache.yetus.audience.InterfaceAudience;
@@ -35,7 +35,6 @@
3535
*/
3636
@InterfaceAudience.Public
3737
public class RandomRowFilter extends FilterBase {
38-
protected static final Random random = new Random();
3938

4039
protected float chance;
4140
protected boolean filterOutRow;
@@ -104,7 +103,7 @@ public boolean filterRowKey(Cell firstRowCell) {
104103
filterOutRow = false;
105104
} else {
106105
// roll the dice
107-
filterOutRow = !(random.nextFloat() < chance);
106+
filterOutRow = !(ThreadLocalRandom.current().nextFloat() < chance);
108107
}
109108
return filterOutRow;
110109
}

hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java

Lines changed: 2 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,6 @@
2222
import java.io.IOException;
2323
import java.security.Key;
2424
import java.security.KeyException;
25-
import java.security.SecureRandom;
2625
import java.util.Properties;
2726
import javax.crypto.spec.SecretKeySpec;
2827
import org.apache.commons.crypto.cipher.CryptoCipherFactory;
@@ -37,7 +36,6 @@
3736
import org.apache.yetus.audience.InterfaceStability;
3837
import org.slf4j.Logger;
3938
import org.slf4j.LoggerFactory;
40-
4139
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
4240
import org.apache.hadoop.hbase.shaded.protobuf.generated.EncryptionProtos;
4341
import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos;
@@ -50,8 +48,6 @@
5048
public final class EncryptionUtil {
5149
static private final Logger LOG = LoggerFactory.getLogger(EncryptionUtil.class);
5250

53-
static private final SecureRandom RNG = new SecureRandom();
54-
5551
/**
5652
* Private constructor to keep this class from being instantiated.
5753
*/
@@ -96,7 +92,7 @@ public static byte[] wrapKey(Configuration conf, String subject, Key key)
9692
byte[] iv = null;
9793
if (cipher.getIvLength() > 0) {
9894
iv = new byte[cipher.getIvLength()];
99-
RNG.nextBytes(iv);
95+
Bytes.secureRandom(iv);
10096
builder.setIv(UnsafeByteOperations.unsafeWrap(iv));
10197
}
10298
byte[] keyBytes = key.getEncoded();
@@ -286,7 +282,7 @@ public static Key unwrapKey(Configuration conf, byte[] keyBytes) throws IOExcept
286282
* @throws IOException if create CryptoAES failed
287283
*/
288284
public static CryptoAES createCryptoAES(RPCProtos.CryptoCipherMeta cryptoCipherMeta,
289-
Configuration conf) throws IOException {
285+
Configuration conf) throws IOException {
290286
Properties properties = new Properties();
291287
// the property for cipher class
292288
properties.setProperty(CryptoCipherFactory.CLASSES_KEY,

hbase-client/src/main/java/org/apache/hadoop/hbase/slowlog/SlowLogTableAccessor.java

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,8 @@
2222
import java.io.IOException;
2323
import java.util.ArrayList;
2424
import java.util.List;
25-
import java.util.Random;
25+
import java.util.concurrent.ThreadLocalRandom;
26+
2627
import org.apache.hadoop.conf.Configuration;
2728
import org.apache.hadoop.hbase.HConstants;
2829
import org.apache.hadoop.hbase.NamespaceDescriptor;
@@ -49,8 +50,6 @@ public class SlowLogTableAccessor {
4950

5051
private static final Logger LOG = LoggerFactory.getLogger(SlowLogTableAccessor.class);
5152

52-
private static final Random RANDOM = new Random();
53-
5453
private static Connection connection;
5554

5655
/**
@@ -139,7 +138,7 @@ private static byte[] getRowKey(final TooSlowLog.SlowLogPayload slowLogPayload)
139138
String lastFiveDig =
140139
hashcode.substring((hashcode.length() > 5) ? (hashcode.length() - 5) : 0);
141140
if (lastFiveDig.startsWith("-")) {
142-
lastFiveDig = String.valueOf(RANDOM.nextInt(99999));
141+
lastFiveDig = String.valueOf(ThreadLocalRandom.current().nextInt(99999));
143142
}
144143
final long currentTime = EnvironmentEdgeManager.currentTime();
145144
final String timeAndHashcode = currentTime + lastFiveDig;

hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestEncryptionUtil.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@
2323

2424
import java.security.Key;
2525
import java.security.KeyException;
26-
import java.security.SecureRandom;
26+
2727
import javax.crypto.spec.SecretKeySpec;
2828
import org.apache.hadoop.conf.Configuration;
2929
import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -110,7 +110,7 @@ public void testWALKeyWrappingWithIncorrectKey() throws Exception {
110110

111111
// generate a test key
112112
byte[] keyBytes = new byte[AES.KEY_LENGTH];
113-
new SecureRandom().nextBytes(keyBytes);
113+
Bytes.secureRandom(keyBytes);
114114
String algorithm = conf.get(HConstants.CRYPTO_WAL_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES);
115115
Key key = new SecretKeySpec(keyBytes, algorithm);
116116

@@ -152,7 +152,7 @@ private void testKeyWrapping(String hashAlgorithm) throws Exception {
152152

153153
// generate a test key
154154
byte[] keyBytes = new byte[AES.KEY_LENGTH];
155-
new SecureRandom().nextBytes(keyBytes);
155+
Bytes.secureRandom(keyBytes);
156156
String algorithm =
157157
conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES);
158158
Key key = new SecretKeySpec(keyBytes, algorithm);
@@ -189,7 +189,7 @@ private void testWALKeyWrapping(String hashAlgorithm) throws Exception {
189189

190190
// generate a test key
191191
byte[] keyBytes = new byte[AES.KEY_LENGTH];
192-
new SecureRandom().nextBytes(keyBytes);
192+
Bytes.secureRandom(keyBytes);
193193
String algorithm = conf.get(HConstants.CRYPTO_WAL_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES);
194194
Key key = new SecretKeySpec(keyBytes, algorithm);
195195

@@ -214,7 +214,7 @@ private void testKeyWrappingWithMismatchingAlgorithms(Configuration conf) throws
214214

215215
// generate a test key
216216
byte[] keyBytes = new byte[AES.KEY_LENGTH];
217-
new SecureRandom().nextBytes(keyBytes);
217+
Bytes.secureRandom(keyBytes);
218218
String algorithm =
219219
conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES);
220220
Key key = new SecretKeySpec(keyBytes, algorithm);

hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestRoundRobinPoolMap.java

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -25,11 +25,9 @@
2525
import java.util.Collections;
2626
import java.util.Iterator;
2727
import java.util.List;
28-
import java.util.Random;
2928
import java.util.concurrent.CompletableFuture;
3029
import java.util.concurrent.CompletionException;
3130
import java.util.concurrent.ExecutionException;
32-
import java.util.concurrent.ThreadLocalRandom;
3331
import java.util.concurrent.atomic.AtomicInteger;
3432
import org.apache.hadoop.hbase.HBaseClassTestRule;
3533
import org.apache.hadoop.hbase.testclassification.MiscTests;

hbase-client/src/test/java/org/apache/hadoop/hbase/util/TestThreadLocalPoolMap.java

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,11 +20,9 @@
2020
import static org.junit.Assert.assertEquals;
2121

2222
import java.io.IOException;
23-
import java.util.Random;
2423
import java.util.concurrent.CompletableFuture;
2524
import java.util.concurrent.CompletionException;
2625
import java.util.concurrent.ExecutionException;
27-
import java.util.concurrent.ThreadLocalRandom;
2826
import java.util.concurrent.atomic.AtomicInteger;
2927
import org.apache.hadoop.hbase.HBaseClassTestRule;
3028
import org.apache.hadoop.hbase.testclassification.MiscTests;

hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/Encryption.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -317,7 +317,7 @@ public static byte[] generateSecretKey(Configuration conf, String cypherAlg, byt
317317
*/
318318
private static byte[] generateSecretKey(String algorithm, int keyLengthBytes, char[] password) {
319319
byte[] salt = new byte[keyLengthBytes];
320-
Bytes.random(salt);
320+
Bytes.secureRandom(salt);
321321
PBEKeySpec spec = new PBEKeySpec(password, salt, 10000, keyLengthBytes*8);
322322
try {
323323
return SecretKeyFactory.getInstance(algorithm).generateSecret(spec).getEncoded();

0 commit comments

Comments
 (0)