From notifications-return-44801-archive-asf-public=cust-asf.ponee.io@accumulo.apache.org Thu Sep 6 16:49:50 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id C41E2180668 for ; Thu, 6 Sep 2018 16:49:47 +0200 (CEST) Received: (qmail 26442 invoked by uid 500); 6 Sep 2018 14:49:46 -0000 Mailing-List: contact notifications-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: jira@apache.org Delivered-To: mailing list notifications@accumulo.apache.org Received: (qmail 26431 invoked by uid 99); 6 Sep 2018 14:49:46 -0000 Received: from ec2-52-202-80-70.compute-1.amazonaws.com (HELO gitbox.apache.org) (52.202.80.70) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 06 Sep 2018 14:49:46 +0000 From: GitBox To: notifications@accumulo.apache.org Subject: [GitHub] milleruntime closed pull request #628: Add sec-bugs plugin. Closes #618 Message-ID: <153624538581.20016.7571617180568911539.gitbox@gitbox.apache.org> Date: Thu, 06 Sep 2018 14:49:45 -0000 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 8bit milleruntime closed pull request #628: Add sec-bugs plugin. Closes #618 URL: https://github.com/apache/accumulo/pull/628 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java index 3aef5feb0c..c5997b0e18 100644 --- a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java +++ b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.net.InetAddress; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -646,7 +647,7 @@ public float getProgress() throws IOException { log.setLevel(logLevel); validateOptions(job); - Random random = new Random(); + Random random = new SecureRandom(); LinkedList splits = new LinkedList<>(); Map tableConfigs = getInputTableConfigs(job); for (Map.Entry tableConfigEntry : tableConfigs.entrySet()) { diff --git a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java index 5366c616ae..084c3f3d49 100644 --- a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java +++ b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.net.InetAddress; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -671,7 +672,7 @@ public V getCurrentValue() throws IOException, InterruptedException { Level logLevel = getLogLevel(context); log.setLevel(logLevel); validateOptions(context); - Random random = new Random(); + Random random = new SecureRandom(); LinkedList splits = new LinkedList<>(); Map tableConfigs = getInputTableConfigs(context); for (Map.Entry tableConfigEntry : tableConfigs.entrySet()) { diff --git a/client/mapreduce/src/main/spotbugs/exclude-filter.xml b/client/mapreduce/src/main/spotbugs/exclude-filter.xml index 108e26ef88..24ce67931b 100644 --- a/client/mapreduce/src/main/spotbugs/exclude-filter.xml +++ b/client/mapreduce/src/main/spotbugs/exclude-filter.xml @@ -26,4 +26,10 @@ + + + + + + diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java index a75b538d09..d392ded2ad 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java @@ -29,6 +29,7 @@ import java.io.IOException; import java.io.InputStreamReader; import java.nio.ByteBuffer; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -1077,7 +1078,7 @@ public void setLocalityGroups(String tableName, Map> groups) if (maxSplits == 1) return Collections.singleton(range); - Random random = new Random(); + Random random = new SecureRandom(); Map>> binnedRanges = new HashMap<>(); Table.ID tableId = Tables.getTableId(context, tableName); TabletLocator tl = TabletLocator.getLocator(context, tableId); @@ -1511,7 +1512,7 @@ public void importTable(String tableName, String importDir) && !entry.getValue().contains(Constants.CORE_PACKAGE_NAME)) { LoggerFactory.getLogger(this.getClass()).info( "Imported table sets '{}' to '{}'. Ensure this class is on Accumulo classpath.", - entry.getKey(), entry.getValue()); + sanitize(entry.getKey()), sanitize(entry.getValue())); } } @@ -1536,6 +1537,14 @@ public void importTable(String tableName, String importDir) } + /** + * Prevent potential CRLF injection into logs from read in user data See + * https://find-sec-bugs.github.io/bugs.htm#CRLF_INJECTION_LOGS + */ + private String sanitize(String msg) { + return msg.replaceAll("[\r\n]", ""); + } + @Override public void exportTable(String tableName, String exportDir) throws TableNotFoundException, AccumuloException, AccumuloSecurityException { diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java index 5c54bd39a2..0a57a34f92 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java @@ -17,12 +17,14 @@ package org.apache.accumulo.core.client.impl; import java.io.IOException; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collections; import java.util.EnumMap; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.Set; import java.util.SortedMap; import java.util.SortedSet; @@ -76,6 +78,7 @@ public static final Map> serversWaitedForWrites = new EnumMap<>( TabletType.class); + private static Random secureRandom = new SecureRandom(); static { for (TabletType ttype : TabletType.values()) { @@ -225,7 +228,7 @@ public ScanState(ClientContext context, Table.ID tableId, Authorizations authori static long pause(long millis, long maxSleep) throws InterruptedException { Thread.sleep(millis); // wait 2 * last time, with +-10% random jitter - return (long) (Math.min(millis * 2, maxSleep) * (.9 + Math.random() / 5)); + return (long) (Math.min(millis * 2, maxSleep) * (.9 + secureRandom.nextDouble() / 5)); } public static List scan(ClientContext context, ScanState scanState, int timeOut) diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftTransportPool.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftTransportPool.java index 9ab7dce740..8b340c2718 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftTransportPool.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftTransportPool.java @@ -16,6 +16,7 @@ */ package org.apache.accumulo.core.client.impl; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -25,7 +26,6 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.Random; import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; @@ -45,7 +45,7 @@ public class ThriftTransportPool { - private static final Random random = new Random(); + private static final SecureRandom random = new SecureRandom(); private long killTime = 1000 * 3; private static class CachedConnections { diff --git a/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java b/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java index 8ca966d92f..1e1d3fe638 100644 --- a/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java +++ b/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java @@ -24,12 +24,12 @@ import java.io.DataOutputStream; import java.io.IOException; import java.io.PrintStream; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.Map; -import java.util.Random; import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutorService; @@ -264,10 +264,10 @@ public long getLength() throws IOException { bloomFilter = null; } catch (ClassNotFoundException e) { - LOG.error("Failed to find KeyFunctor in config: " + ClassName, e); + LOG.error("Failed to find KeyFunctor in config: " + sanitize(ClassName), e); bloomFilter = null; } catch (InstantiationException e) { - LOG.error("Could not instantiate KeyFunctor: " + ClassName, e); + LOG.error("Could not instantiate KeyFunctor: " + sanitize(ClassName), e); bloomFilter = null; } catch (IllegalAccessException e) { LOG.error("Illegal acess exception", e); @@ -292,6 +292,14 @@ public long getLength() throws IOException { } + /** + * Prevent potential CRLF injection into logs from read in user data See + * https://find-sec-bugs.github.io/bugs.htm#CRLF_INJECTION_LOGS + */ + private String sanitize(String msg) { + return msg.replaceAll("[\r\n]", ""); + } + private synchronized void initiateLoad(int maxLoadThreads) { // ensure only one thread initiates loading of bloom filter by // only taking action when loadTask != null @@ -445,7 +453,7 @@ public FileSKVIterator getSample(SamplerConfigurationImpl sampleConfig) { public static void main(String[] args) throws IOException { PrintStream out = System.out; - Random r = new Random(); + SecureRandom r = new SecureRandom(); HashSet valsSet = new HashSet<>(); diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/VisMetricsGatherer.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/VisMetricsGatherer.java index 5f5b4e833f..ff876ab07b 100644 --- a/core/src/main/java/org/apache/accumulo/core/file/rfile/VisMetricsGatherer.java +++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/VisMetricsGatherer.java @@ -26,6 +26,7 @@ import java.util.Map; import java.util.Map.Entry; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.data.ArrayByteSequence; import org.apache.accumulo.core.data.ByteSequence; import org.apache.accumulo.core.data.Key; @@ -131,15 +132,16 @@ public void printMetrics(boolean hash, String metricWord, PrintStream out) { + "Percent of blocks"); for (Entry entry : metric.get(lGName).asMap().entrySet()) { if (hash) { - String md5String = ""; + String encodedKey = ""; try { - byte[] md5Bytes = MessageDigest.getInstance("MD5") + byte[] encodedBytes = MessageDigest.getInstance(Constants.PW_HASH_ALGORITHM) .digest(entry.getKey().getBytes(UTF_8)); - md5String = new String(md5Bytes, UTF_8); + encodedKey = new String(encodedBytes, UTF_8); } catch (NoSuchAlgorithmException e) { - out.println("Failed to convert key to MD5 hash: " + e.getMessage()); + out.println("Failed to convert key to " + Constants.PW_HASH_ALGORITHM + " hash: " + + e.getMessage()); } - out.printf("%-20s", md5String.substring(0, 8)); + out.printf("%-20s", encodedKey.substring(0, 8)); } else out.printf("%-20s", entry.getKey()); out.print("\t\t" + entry.getValue() + "\t\t\t"); diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java b/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java index e6bd20dc5e..e404a704fb 100644 --- a/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.net.InetAddress; import java.security.KeyStore; +import java.security.SecureRandom; import java.util.HashMap; import java.util.Map; import java.util.Random; @@ -64,7 +65,7 @@ public static final String GSSAPI = "GSSAPI", DIGEST_MD5 = "DIGEST-MD5"; - private static final Random SASL_BACKOFF_RAND = new Random(); + private static final Random SASL_BACKOFF_RAND = new SecureRandom(); private static final int RELOGIN_MAX_BACKOFF = 5000; /** diff --git a/core/src/main/java/org/apache/accumulo/core/security/crypto/CryptoUtils.java b/core/src/main/java/org/apache/accumulo/core/security/crypto/CryptoUtils.java index 0e69d6c68a..8cd6567b1a 100644 --- a/core/src/main/java/org/apache/accumulo/core/security/crypto/CryptoUtils.java +++ b/core/src/main/java/org/apache/accumulo/core/security/crypto/CryptoUtils.java @@ -24,10 +24,6 @@ import java.security.SecureRandom; import java.util.Objects; -import javax.crypto.Cipher; -import javax.crypto.NoSuchPaddingException; -import javax.crypto.NullCipher; - import org.apache.accumulo.core.spi.crypto.CryptoService.CryptoException; import org.apache.commons.io.IOUtils; import org.slf4j.Logger; @@ -55,38 +51,6 @@ private static SecureRandom newSecureRandom(String secureRNG, String secureRNGPr return secureRandom; } - public static Cipher getCipher(String cipherSuite, String securityProvider) { - Cipher cipher = null; - - if (cipherSuite.equals("NullCipher")) { - cipher = new NullCipher(); - } else { - try { - if (securityProvider == null || securityProvider.equals("")) { - cipher = Cipher.getInstance(cipherSuite); - } else { - cipher = Cipher.getInstance(cipherSuite, securityProvider); - } - } catch (NoSuchAlgorithmException e) { - log.error(String.format("Accumulo configuration file contained a cipher" - + " suite \"%s\" that was not recognized by any providers", cipherSuite)); - throw new CryptoException(e); - } catch (NoSuchPaddingException e) { - log.error(String.format( - "Accumulo configuration file contained a" - + " cipher, \"%s\" with a padding that was not recognized by any" + " providers", - cipherSuite)); - throw new CryptoException(e); - } catch (NoSuchProviderException e) { - log.error(String.format( - "Accumulo configuration file contained a provider, \"%s\" an unrecognized provider", - securityProvider)); - throw new CryptoException(e); - } - } - return cipher; - } - /** * Read the decryption parameters from the DataInputStream */ diff --git a/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java b/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java index 5f897269f1..c12374bdc4 100644 --- a/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java +++ b/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java @@ -18,6 +18,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; +import java.security.SecureRandom; import java.util.Collections; import java.util.ConcurrentModificationException; import java.util.HashMap; @@ -59,6 +60,7 @@ private final HashMap> childrenCache; private final ZooReader zReader; + private final SecureRandom secureRandom = new SecureRandom(); public static class ZcStat { private long ephemeralOwner; @@ -287,7 +289,7 @@ public T retry() { } LockSupport.parkNanos(sleepTime); if (sleepTime < 10_000) { - sleepTime = (int) (sleepTime + sleepTime * Math.random()); + sleepTime = (int) (sleepTime + sleepTime * secureRandom.nextDouble()); } } } diff --git a/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooSession.java b/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooSession.java index 584a7fc2ef..a0ad36fe02 100644 --- a/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooSession.java +++ b/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooSession.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.net.UnknownHostException; +import java.security.SecureRandom; import java.util.HashMap; import java.util.Map; @@ -53,6 +54,8 @@ public ZooSessionInfo(ZooKeeper zooKeeper, ZooWatcher watcher) { private static Map sessions = new HashMap<>(); + private static final SecureRandom secureRandom = new SecureRandom(); + private static String sessionKey(String keepers, int timeout, String scheme, byte[] auth) { return keepers + ":" + timeout + ":" + (scheme == null ? "" : scheme) + ":" + (auth == null ? "" : new String(auth, UTF_8)); @@ -138,7 +141,7 @@ public static ZooKeeper connect(String host, int timeout, String scheme, byte[] } UtilWaitThread.sleep(sleepTime); if (sleepTime < 10000) - sleepTime = sleepTime + (long) (sleepTime * Math.random()); + sleepTime = sleepTime + (long) (sleepTime * secureRandom.nextDouble()); } } diff --git a/core/src/main/spotbugs/exclude-filter.xml b/core/src/main/spotbugs/exclude-filter.xml index 46e176cb45..39025c9d12 100644 --- a/core/src/main/spotbugs/exclude-filter.xml +++ b/core/src/main/spotbugs/exclude-filter.xml @@ -15,6 +15,107 @@ limitations under the License. --> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @@ -30,11 +131,12 @@ + + - + + - - @@ -87,7 +189,6 @@ - diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/ConcurrentKeyExtentCacheTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/ConcurrentKeyExtentCacheTest.java index 45d9456675..4b80665a49 100644 --- a/core/src/test/java/org/apache/accumulo/core/client/impl/ConcurrentKeyExtentCacheTest.java +++ b/core/src/test/java/org/apache/accumulo/core/client/impl/ConcurrentKeyExtentCacheTest.java @@ -17,6 +17,7 @@ package org.apache.accumulo.core.client.impl; import java.io.IOException; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.HashSet; import java.util.List; @@ -99,7 +100,7 @@ private void testLookup(TestCache tc, Text lookupRow) { @Test public void testExactEndRows() { - Random rand = new Random(42); + Random rand = new SecureRandom(); TestCache tc = new TestCache(); rand.ints(10000, 0, 256).mapToObj(i -> new Text(String.format("%02x", i))).sequential() .forEach(lookupRow -> testLookup(tc, lookupRow)); @@ -116,7 +117,7 @@ public void testExactEndRows() { public void testRandom() throws Exception { TestCache tc = new TestCache(); - Random rand = new Random(42); + Random rand = new SecureRandom(); rand.ints(10000).mapToObj(i -> new Text(String.format("%08x", i))).sequential() .forEach(lookupRow -> testLookup(tc, lookupRow)); Assert.assertEquals(256, tc.updates.get()); diff --git a/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileTest.java b/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileTest.java index 292a071c2b..6ac89bea4e 100644 --- a/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileTest.java +++ b/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileTest.java @@ -24,6 +24,7 @@ import java.io.File; import java.io.IOException; +import java.security.SecureRandom; import java.util.AbstractMap; import java.util.ArrayList; import java.util.Arrays; @@ -514,7 +515,7 @@ public void testCache() throws Exception { Scanner scanner = RFile.newScanner().from(testFile).withFileSystem(localFs) .withIndexCache(1000000).withDataCache(10000000).build(); - Random rand = new Random(5); + Random rand = new SecureRandom(); for (int i = 0; i < 100; i++) { int r = rand.nextInt(10000); diff --git a/core/src/test/java/org/apache/accumulo/core/file/BloomFilterLayerLookupTest.java b/core/src/test/java/org/apache/accumulo/core/file/BloomFilterLayerLookupTest.java index 799992dd7d..83bdb2bf34 100644 --- a/core/src/test/java/org/apache/accumulo/core/file/BloomFilterLayerLookupTest.java +++ b/core/src/test/java/org/apache/accumulo/core/file/BloomFilterLayerLookupTest.java @@ -20,6 +20,7 @@ import java.io.File; import java.io.IOException; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; @@ -48,7 +49,7 @@ public class BloomFilterLayerLookupTest { private static final Logger log = LoggerFactory.getLogger(BloomFilterLayerLookupTest.class); - private static Random random = new Random(); + private static Random random = new SecureRandom(); @Rule public TestName testName = new TestName(); diff --git a/core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/TestLruBlockCache.java b/core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/TestLruBlockCache.java index 056a2af287..150c87830e 100644 --- a/core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/TestLruBlockCache.java +++ b/core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/TestLruBlockCache.java @@ -17,6 +17,7 @@ */ package org.apache.accumulo.core.file.blockfile.cache; +import java.security.SecureRandom; import java.util.Arrays; import java.util.Random; @@ -492,7 +493,7 @@ public void testScanResistance() throws Exception { private Block[] generateRandomBlocks(int numBlocks, long maxSize) { Block[] blocks = new Block[numBlocks]; - Random r = new Random(); + Random r = new SecureRandom(); for (int i = 0; i < numBlocks; i++) { blocks[i] = new Block("block" + i, r.nextInt((int) maxSize) + 1); } diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java index a53a78aa16..54e61d733d 100644 --- a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java +++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java @@ -18,6 +18,7 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.security.SecureRandom; import java.util.Random; import org.apache.accumulo.core.conf.AccumuloConfiguration; @@ -108,7 +109,7 @@ private void runTest(int maxBlockSize, int num) throws IOException { liter = reader.lookup(new Key(String.format("%05d000", num + 1))); assertFalse(liter.hasNext()); - Random rand = new Random(); + Random rand = new SecureRandom(); for (int i = 0; i < 100; i++) { int k = rand.nextInt(num * 1000); int expected; diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java index e78b1b1f8b..1906555871 100644 --- a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java +++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.io.PrintWriter; import java.io.StringWriter; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -287,7 +288,7 @@ public void testMultipleReaders() throws IOException { } private void validate(TestRFile trf) throws IOException { - Random random = new Random(); + Random random = new SecureRandom(); for (int iteration = 0; iteration < 10; iteration++) { int part = random.nextInt(4); diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java index 52895b4325..9f69c8d477 100644 --- a/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java +++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java @@ -31,6 +31,7 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; +import java.security.SecureRandom; import java.util.AbstractMap; import java.util.ArrayList; import java.util.Arrays; @@ -537,7 +538,7 @@ public void test3() throws IOException { // test seeking to random location and reading all data from that point // there was an off by one bug with this in the transient index - Random rand = new Random(); + Random rand = new SecureRandom(); for (int i = 0; i < 12; i++) { index = rand.nextInt(expectedKeys.size()); trf.seek(expectedKeys.get(index)); @@ -1659,7 +1660,7 @@ public void testReseekUnconsumed() throws Exception { Set cfs = Collections.emptySet(); - Random rand = new Random(); + Random rand = new SecureRandom(); for (int count = 0; count < 100; count++) { @@ -1998,9 +1999,9 @@ private void checkSample(SortedKeyValueIterator sample, sample.seek(new Range(), columnFamilies, inclusive); Assert.assertEquals(sampleData, toList(sample)); - Random rand = new Random(); + Random rand = new SecureRandom(); long seed = rand.nextLong(); - rand = new Random(seed); + rand.setSeed(seed); // randomly seek sample iterator and verify for (int i = 0; i < 33; i++) { diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/RolllingStatsTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/RolllingStatsTest.java index afbb71d1f9..95915a6361 100644 --- a/core/src/test/java/org/apache/accumulo/core/file/rfile/RolllingStatsTest.java +++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/RolllingStatsTest.java @@ -17,6 +17,7 @@ package org.apache.accumulo.core.file.rfile; +import java.security.SecureRandom; import java.util.Random; import java.util.function.IntSupplier; @@ -59,7 +60,7 @@ private static void checkAgreement(DescriptiveStatistics ds, RollingStats rs) { private static class StatTester { - Random rand = new Random(42); + Random rand = new SecureRandom(); private DescriptiveStatistics ds; private RollingStats rs; private RollingStats rsp; @@ -92,7 +93,7 @@ void check() { public void testFewSizes() { StatTester st = new StatTester(1019); int[] keySizes = {103, 113, 123, 2345}; - Random rand = new Random(42); + Random rand = new SecureRandom(); for (int i = 0; i < 10000; i++) { st.addValue(keySizes[rand.nextInt(keySizes.length)]); } @@ -118,7 +119,7 @@ public void testUniformIncreasing() { StatTester st = new StatTester(windowSize); - Random rand = new Random(); + Random rand = new SecureRandom(); for (int i = 0; i < 1000; i++) { int v = 200 + rand.nextInt(50); @@ -173,7 +174,7 @@ public void testNormal() { @Test public void testSpikes() { - Random rand = new Random(); + Random rand = new SecureRandom(); StatTester st = new StatTester(3017); diff --git a/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedInputStreamTest.java b/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedInputStreamTest.java index 44891d4db3..de913cefa4 100644 --- a/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedInputStreamTest.java +++ b/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedInputStreamTest.java @@ -18,6 +18,7 @@ import java.io.IOException; import java.io.InputStream; +import java.security.SecureRandom; import java.util.Random; import org.apache.hadoop.fs.Seekable; @@ -28,7 +29,7 @@ @Test public void permitsAreProperlyAcquired() throws Exception { - Random randGen = new Random(); + Random randGen = new SecureRandom(); MockRateLimiter rateLimiter = new MockRateLimiter(); long bytesRetrieved = 0; try (InputStream is = new RateLimitedInputStream(new RandomInputStream(), rateLimiter)) { @@ -43,7 +44,7 @@ public void permitsAreProperlyAcquired() throws Exception { } private static class RandomInputStream extends InputStream implements Seekable { - private final Random r = new Random(); + private final Random r = new SecureRandom(); @Override public int read() throws IOException { diff --git a/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedOutputStreamTest.java b/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedOutputStreamTest.java index fbcc1640cb..9601681c86 100644 --- a/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedOutputStreamTest.java +++ b/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedOutputStreamTest.java @@ -17,6 +17,7 @@ package org.apache.accumulo.core.file.streams; import java.io.IOException; +import java.security.SecureRandom; import java.util.Random; import org.apache.hadoop.fs.FSDataOutputStream; @@ -30,7 +31,7 @@ @Test public void permitsAreProperlyAcquired() throws Exception { - Random randGen = new Random(); + Random randGen = new SecureRandom(); MockRateLimiter rateLimiter = new MockRateLimiter(); long bytesWritten = 0; try (RateLimitedOutputStream os = new RateLimitedOutputStream(new NullOutputStream(), diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/IndexedDocIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/IndexedDocIteratorTest.java index fdac86a017..b43297687d 100644 --- a/core/src/test/java/org/apache/accumulo/core/iterators/user/IndexedDocIteratorTest.java +++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/IndexedDocIteratorTest.java @@ -17,6 +17,7 @@ package org.apache.accumulo.core.iterators.user; import java.io.IOException; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; @@ -70,7 +71,7 @@ Text[] columnFamilies, Text[] otherColumnFamilies, HashSet docs, Text[] negatedColumns) { StringBuilder sb = new StringBuilder(); - Random r = new Random(); + Random r = new SecureRandom(); Value v = new Value(new byte[0]); TreeMap map = new TreeMap<>(); boolean[] negateMask = new boolean[columnFamilies.length]; diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java index 5c9f0b500b..135b9df980 100644 --- a/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java +++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; @@ -59,7 +60,7 @@ private TreeMap createSortedMap(float hitRatio, int numRows, int numDocsPerRow, Text[] columnFamilies, Text[] otherColumnFamilies, HashSet docs, Text[] negatedColumns) { - Random r = new Random(); + Random r = new SecureRandom(); Value v = new Value(new byte[0]); TreeMap map = new TreeMap<>(); boolean[] negateMask = new boolean[columnFamilies.length]; diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/TestCfCqSlice.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/TestCfCqSlice.java index df1800517f..55b45f4b72 100644 --- a/core/src/test/java/org/apache/accumulo/core/iterators/user/TestCfCqSlice.java +++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/TestCfCqSlice.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.security.SecureRandom; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -376,7 +377,7 @@ private void loadKvs(SortedKeyValueIterator parent, boolean[][][] fou skvi.init(parent, options, null); skvi.seek(range, EMPTY_CF_SET, false); - Random random = new Random(); + Random random = new SecureRandom(); while (skvi.hasTop()) { Key k = skvi.getTopKey(); diff --git a/core/src/test/java/org/apache/accumulo/core/security/AuthenticationTokenTest.java b/core/src/test/java/org/apache/accumulo/core/security/AuthenticationTokenTest.java index f27d27f96c..57abe0b0b2 100644 --- a/core/src/test/java/org/apache/accumulo/core/security/AuthenticationTokenTest.java +++ b/core/src/test/java/org/apache/accumulo/core/security/AuthenticationTokenTest.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertFalse; import java.io.IOException; +import java.security.SecureRandom; import java.util.Random; import org.apache.accumulo.core.client.AccumuloSecurityException; @@ -33,7 +34,7 @@ public class AuthenticationTokenTest { @Test public void testSerializeDeserializeToken() throws AccumuloSecurityException, IOException { - Random random = new Random(); + Random random = new SecureRandom(); byte[] randomBytes = new byte[12]; random.nextBytes(randomBytes); boolean allZero = true; diff --git a/core/src/test/java/org/apache/accumulo/core/security/crypto/BlockedIOStreamTest.java b/core/src/test/java/org/apache/accumulo/core/security/crypto/BlockedIOStreamTest.java index ef1af9f32d..fb2133f705 100644 --- a/core/src/test/java/org/apache/accumulo/core/security/crypto/BlockedIOStreamTest.java +++ b/core/src/test/java/org/apache/accumulo/core/security/crypto/BlockedIOStreamTest.java @@ -24,6 +24,7 @@ import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.IOException; +import java.security.SecureRandom; import java.util.Arrays; import java.util.Random; @@ -82,7 +83,7 @@ public void testSpillingOverOutputStream() throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); // buffer will be size 12 BlockedOutputStream blockOut = new BlockedOutputStream(baos, 16, 16); - Random r = new Random(22); + Random r = new SecureRandom(); byte[] undersized = new byte[11]; byte[] perfectSized = new byte[12]; @@ -126,7 +127,7 @@ public void testGiantWrite() throws IOException { int blockSize = 16; // buffer will be size 12 BlockedOutputStream blockOut = new BlockedOutputStream(baos, blockSize, blockSize); - Random r = new Random(22); + Random r = new SecureRandom(); int size = 1024 * 1024 * 128; byte[] giant = new byte[size]; diff --git a/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/IsolatedDeepCopiesTestCase.java b/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/IsolatedDeepCopiesTestCase.java index 66eb9374a4..69fd588127 100644 --- a/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/IsolatedDeepCopiesTestCase.java +++ b/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/IsolatedDeepCopiesTestCase.java @@ -17,6 +17,7 @@ package org.apache.accumulo.iteratortest.testcases; import java.io.IOException; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -41,7 +42,7 @@ public class IsolatedDeepCopiesTestCase extends OutputVerifyingTestCase { private static final Logger log = LoggerFactory.getLogger(IsolatedDeepCopiesTestCase.class); - private final Random random = new Random(); + private final Random random = new SecureRandom(); @Override public IteratorTestOutput test(IteratorTestInput testInput) { diff --git a/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/MultipleHasTopCalls.java b/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/MultipleHasTopCalls.java index 8974ce3377..583f6ce25c 100644 --- a/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/MultipleHasTopCalls.java +++ b/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/MultipleHasTopCalls.java @@ -17,6 +17,7 @@ package org.apache.accumulo.iteratortest.testcases; import java.io.IOException; +import java.security.SecureRandom; import java.util.Random; import java.util.TreeMap; @@ -41,7 +42,7 @@ private final Random random; public MultipleHasTopCalls() { - this.random = new Random(); + this.random = new SecureRandom(); } @Override diff --git a/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/ReSeekTestCase.java b/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/ReSeekTestCase.java index ad2d8b6b94..61e4e6bb1d 100644 --- a/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/ReSeekTestCase.java +++ b/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/ReSeekTestCase.java @@ -17,6 +17,7 @@ package org.apache.accumulo.iteratortest.testcases; import java.io.IOException; +import java.security.SecureRandom; import java.util.Collection; import java.util.Random; import java.util.TreeMap; @@ -48,7 +49,7 @@ private final Random random; public ReSeekTestCase() { - this.random = new Random(); + this.random = new SecureRandom(); } @Override diff --git a/maven-plugin/src/main/spotbugs/exclude-filter.xml b/maven-plugin/src/main/spotbugs/exclude-filter.xml new file mode 100644 index 0000000000..e4122bbce7 --- /dev/null +++ b/maven-plugin/src/main/spotbugs/exclude-filter.xml @@ -0,0 +1,29 @@ + + + + + + + + + + + + + + diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java index 162c6a3468..74e35cefe6 100644 --- a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java +++ b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java @@ -117,10 +117,18 @@ public int exec(Class clz, String[] args) throws IOException { for (String arg : args) { cmd.add("'" + arg + "'"); } - log.info("Running: '{}' on {}", StringUtils.join(cmd, " "), master); + log.info("Running: '{}' on {}", sanitize(StringUtils.join(cmd, " ")), sanitize(master)); return exec(master, cmd.toArray(new String[cmd.size()])); } + /** + * Prevent potential CRLF injection into logs from read in user data See + * https://find-sec-bugs.github.io/bugs.htm#CRLF_INJECTION_LOGS + */ + private String sanitize(String msg) { + return msg.replaceAll("[\r\n]", ""); + } + public Entry execMapreduceWithStdout(Class clz, String[] args) throws IOException { String host = "localhost"; @@ -280,7 +288,8 @@ public void signal(ServerType server, String hostname, String signal) throws IOE String pid = getPid(server, accumuloHome, hostname); if (pid.trim().isEmpty()) { - log.debug("Found no processes for {} on {}", server, hostname); + log.debug("Found no processes for {} on {}", sanitize(server.prettyPrint()), + sanitize(hostname)); return; } diff --git a/minicluster/src/main/spotbugs/exclude-filter.xml b/minicluster/src/main/spotbugs/exclude-filter.xml index bf850ec170..baf15bce71 100644 --- a/minicluster/src/main/spotbugs/exclude-filter.xml +++ b/minicluster/src/main/spotbugs/exclude-filter.xml @@ -15,6 +15,11 @@ limitations under the License. --> + + + + + @@ -27,4 +32,31 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/pom.xml b/pom.xml index 54ab1f0efd..ca3118ff12 100644 --- a/pom.xml +++ b/pom.xml @@ -800,7 +800,7 @@ true 16 -Dcom.overstock.findbugs.ignore=com.google.common.util.concurrent.RateLimiter,com.google.common.hash.Hasher,com.google.common.hash.HashCode,com.google.common.hash.HashFunction,com.google.common.hash.Hashing,com.google.common.cache.Cache,com.google.common.io.CountingOutputStream,com.google.common.io.ByteStreams,com.google.common.cache.LoadingCache,com.google.common.base.Stopwatch,com.google.common.cache.RemovalNotification,com.google.common.util.concurrent.Uninterruptibles,com.google.common.reflect.ClassPath,com.google.common.reflect.ClassPath$ClassInfo,com.google.common.base.Throwables,com.google.common.collect.Iterators - + com.overstock.findbugs library-detectors @@ -1739,5 +1739,25 @@ + + sec-bugs + + + + com.github.spotbugs + spotbugs-maven-plugin + + + + com.h3xstream.findsecbugs + findsecbugs-plugin + 1.8.0 + + + + + + + diff --git a/proxy/src/main/java/org/apache/accumulo/proxy/Util.java b/proxy/src/main/java/org/apache/accumulo/proxy/Util.java index e5910d1fd0..6774d7fd76 100644 --- a/proxy/src/main/java/org/apache/accumulo/proxy/Util.java +++ b/proxy/src/main/java/org/apache/accumulo/proxy/Util.java @@ -20,6 +20,7 @@ import java.math.BigInteger; import java.nio.ByteBuffer; +import java.security.SecureRandom; import java.util.Random; import org.apache.accumulo.proxy.thrift.IteratorSetting; @@ -27,7 +28,7 @@ public class Util { - private static Random random = new Random(0); + private static Random random = new SecureRandom(); public static String randString(int numbytes) { return new BigInteger(numbytes * 5, random).toString(32); diff --git a/proxy/src/main/spotbugs/exclude-filter.xml b/proxy/src/main/spotbugs/exclude-filter.xml index 432b1b4499..18b778085b 100644 --- a/proxy/src/main/spotbugs/exclude-filter.xml +++ b/proxy/src/main/spotbugs/exclude-filter.xml @@ -24,4 +24,9 @@ + + + + + diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/RandomVolumeChooser.java b/server/base/src/main/java/org/apache/accumulo/server/fs/RandomVolumeChooser.java index e575645c83..d3daecd07f 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/fs/RandomVolumeChooser.java +++ b/server/base/src/main/java/org/apache/accumulo/server/fs/RandomVolumeChooser.java @@ -16,10 +16,11 @@ */ package org.apache.accumulo.server.fs; +import java.security.SecureRandom; import java.util.Random; public class RandomVolumeChooser implements VolumeChooser { - private final Random random = new Random(); + private final Random random = new SecureRandom(); @Override public String choose(VolumeChooserEnvironment env, String[] options) diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java index 936590ba0b..fe47737ff5 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java +++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java @@ -373,7 +373,7 @@ static boolean same(FileSystem fs1, Path dir, FileSystem fs2, Path newDir) private static String hash(FileSystem fs, Path dir, String name) throws IOException { try (FSDataInputStream in = fs.open(new Path(dir, name))) { - return DigestUtils.shaHex(in); + return DigestUtils.sha256Hex(in); } } diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java index 30695679b5..2b53d6f0f6 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java +++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java @@ -16,6 +16,7 @@ */ package org.apache.accumulo.server.master.balancer; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -61,7 +62,7 @@ public ChaoticLoadBalancer(String tableName) { this.tableName = tableName; } - Random r = new Random(); + Random r = new SecureRandom(); @Override public void getAssignments(SortedMap current, diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java index 675e9b1932..4f74517061 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java +++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java @@ -18,6 +18,7 @@ import java.net.InetAddress; import java.net.UnknownHostException; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -377,7 +378,7 @@ public long balance(SortedMap current, if (null == outOfBoundsTablets) { continue; } - Random random = new Random(); + Random random = new SecureRandom(); for (TabletStats ts : outOfBoundsTablets) { KeyExtent ke = new KeyExtent(ts.getExtent()); if (migrations.contains(ke)) { diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java index 38c2a06060..9187335a32 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java +++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java @@ -73,7 +73,7 @@ public static boolean checkPass(byte[] password, byte[] zkData) { log.error("Count not create hashed password", e); return false; } - return java.util.Arrays.equals(passwordToCheck, zkData); + return MessageDigest.isEqual(passwordToCheck, zkData); } public static byte[] createPass(byte[] password) throws AccumuloException { diff --git a/server/base/src/main/java/org/apache/accumulo/server/tablets/UniqueNameAllocator.java b/server/base/src/main/java/org/apache/accumulo/server/tablets/UniqueNameAllocator.java index 1cc8177c15..df14335dc0 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/tablets/UniqueNameAllocator.java +++ b/server/base/src/main/java/org/apache/accumulo/server/tablets/UniqueNameAllocator.java @@ -18,6 +18,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; +import java.security.SecureRandom; import java.util.Random; import org.apache.accumulo.core.Constants; @@ -43,7 +44,7 @@ public UniqueNameAllocator(ServerContext context) { this.context = context; nextNamePath = Constants.ZROOT + "/" + context.getInstanceID() + Constants.ZNEXT_FILE; - rand = new Random(); + rand = new SecureRandom(); } public synchronized String getNextName() { diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java index 8e4d845a7a..21fc52f2f2 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java @@ -18,12 +18,12 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; @@ -83,7 +83,7 @@ private static Path createTmpDir(AccumuloConfiguration acuConf, VolumeManager fs Path result = null; while (result == null) { result = new Path(tabletDirectory + Path.SEPARATOR + "tmp/idxReduce_" - + String.format("%09d", new Random().nextInt(Integer.MAX_VALUE))); + + String.format("%09d", new SecureRandom().nextInt(Integer.MAX_VALUE))); try { fs.getFileStatus(result); result = null; diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/RandomWriter.java b/server/base/src/main/java/org/apache/accumulo/server/util/RandomWriter.java index bbeaac9637..cc0c567b3c 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/RandomWriter.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/RandomWriter.java @@ -16,6 +16,7 @@ */ package org.apache.accumulo.server.util; +import java.security.SecureRandom; import java.util.Iterator; import java.util.Random; @@ -41,7 +42,7 @@ public static class RandomMutationGenerator implements Iterable, Iterator { private long max_mutations; private int mutations_so_far = 0; - private Random r = new Random(); + private Random r = new SecureRandom(); private static final Logger log = LoggerFactory.getLogger(RandomMutationGenerator.class); public RandomMutationGenerator(long num_mutations) { diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java index 5a74023b8a..8db7124b66 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java +++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java @@ -18,6 +18,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -65,7 +66,7 @@ private void lookForWork(final Processor processor, List children) { if (numTask.get() >= threadPool.getCorePoolSize()) return; - Random random = new Random(); + Random random = new SecureRandom(); Collections.shuffle(children, random); try { for (final String child : children) { @@ -160,7 +161,7 @@ public void run() { public DistributedWorkQueue(String path, AccumuloConfiguration config) { // Preserve the old delay and period - this(path, config, new Random().nextInt(60 * 1000), 60 * 1000); + this(path, config, new SecureRandom().nextInt(60 * 1000), 60 * 1000); } public DistributedWorkQueue(String path, AccumuloConfiguration config, long timerInitialDelay, diff --git a/server/base/src/main/spotbugs/exclude-filter.xml b/server/base/src/main/spotbugs/exclude-filter.xml index 26aa1192ca..27dfbfa053 100644 --- a/server/base/src/main/spotbugs/exclude-filter.xml +++ b/server/base/src/main/spotbugs/exclude-filter.xml @@ -35,4 +35,34 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/GroupBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/GroupBalancerTest.java index 82e6a1ee0b..e07e796f8b 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/GroupBalancerTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/GroupBalancerTest.java @@ -17,6 +17,7 @@ package org.apache.accumulo.server.master.balancer; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -327,7 +328,7 @@ public void testMaxMigrations() { @Test public void bigTest() { TabletServers tservers = new TabletServers(); - Random rand = new Random(42); + Random rand = new SecureRandom(); for (int g = 1; g <= 60; g++) { for (int t = 1; t <= 241; t++) { @@ -346,7 +347,7 @@ public void bigTest() { @Test public void bigTest2() { TabletServers tservers = new TabletServers(); - Random rand = new Random(42); + Random rand = new SecureRandom(); for (int g = 1; g <= 60; g++) { for (int t = 1; t <= rand.nextInt(1000); t++) { diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/MasterReplicationCoordinator.java b/server/master/src/main/java/org/apache/accumulo/master/replication/MasterReplicationCoordinator.java index b9d0712d2a..f343b60fbf 100644 --- a/server/master/src/main/java/org/apache/accumulo/master/replication/MasterReplicationCoordinator.java +++ b/server/master/src/main/java/org/apache/accumulo/master/replication/MasterReplicationCoordinator.java @@ -19,6 +19,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static java.nio.charset.StandardCharsets.UTF_8; +import java.security.SecureRandom; import java.util.Iterator; import java.util.Random; import java.util.Set; @@ -56,7 +57,8 @@ public MasterReplicationCoordinator(Master master) { protected MasterReplicationCoordinator(Master master, ZooReader reader) { this.master = master; - this.rand = new Random(358923462L); + this.rand = new SecureRandom(); + this.rand.setSeed(358923462L); this.reader = reader; this.security = SecurityOperation.getInstance(master.getContext(), false); } diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/LoadFiles.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/LoadFiles.java index 6c710d3cba..6341fa39da 100644 --- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/LoadFiles.java +++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/LoadFiles.java @@ -21,6 +21,7 @@ import java.io.BufferedWriter; import java.io.OutputStreamWriter; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -136,7 +137,7 @@ private static synchronized ExecutorService getThreadPool(Master master) { // Use the threadpool to assign files one-at-a-time to the server final List loaded = Collections.synchronizedList(new ArrayList<>()); - final Random random = new Random(); + final Random random = new SecureRandom(); final TServerInstance[] servers; String prop = conf.get(Property.MASTER_BULK_TSERVER_REGEX); if (null == prop || "".equals(prop)) { diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java index ac7fd3f5e7..acb32541fd 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java @@ -81,7 +81,6 @@ import org.glassfish.jersey.jackson.JacksonFeature; import org.glassfish.jersey.logging.LoggingFeature; import org.glassfish.jersey.server.ResourceConfig; -import org.glassfish.jersey.server.ServerProperties; import org.glassfish.jersey.server.mvc.MvcFeature; import org.glassfish.jersey.server.mvc.freemarker.FreemarkerMvcFeature; import org.glassfish.jersey.servlet.ServletContainer; diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemsResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemsResource.java index 8bc0495fe9..970df4f4b1 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemsResource.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemsResource.java @@ -105,7 +105,7 @@ public void clearTableProblems( ProblemReports.getInstance(Monitor.getContext()).deleteProblemReports(Table.ID.of(tableID)); } catch (Exception e) { log.error("Failed to delete problem reports for table " - + (StringUtils.isEmpty(tableID) ? StringUtils.EMPTY : tableID), e); + + (StringUtils.isEmpty(tableID) ? StringUtils.EMPTY : sanitize(tableID)), e); } } @@ -165,10 +165,18 @@ public void clearDetailsProblems( ProblemType.valueOf(ptype), resource); } catch (Exception e) { log.error("Failed to delete problem reports for table " - + (StringUtils.isBlank(tableID) ? "" : tableID), e); + + (StringUtils.isBlank(tableID) ? "" : sanitize(tableID)), e); } } + /** + * Prevent potential CRLF injection into logs from read in user data. See + * https://find-sec-bugs.github.io/bugs.htm#CRLF_INJECTION_LOGS + */ + private String sanitize(String msg) { + return msg.replaceAll("[\r\n]", ""); + } + @GET @Path("exception") public Exception getException() { diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java index e246e71bef..e35c7e8280 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java @@ -16,7 +16,7 @@ */ package org.apache.accumulo.monitor.rest.tservers; -import static org.apache.accumulo.monitor.util.ParameterValidator.SERVER_REGEX; +import static org.apache.accumulo.monitor.util.ParameterValidator.HOSTNAME_PORT_REGEX; import java.lang.management.ManagementFactory; import java.security.MessageDigest; @@ -101,7 +101,7 @@ public TabletServers getTserverSummary() { @POST @Consumes(MediaType.TEXT_PLAIN) public void clearDeadServer( - @QueryParam("server") @NotNull @Pattern(regexp = SERVER_REGEX) String server) { + @QueryParam("server") @NotNull @Pattern(regexp = HOSTNAME_PORT_REGEX) String server) { DeadServerList obit = new DeadServerList(Monitor.getContext(), Monitor.getContext().getZooKeeperRoot() + Constants.ZDEADTSERVERS); obit.delete(server); @@ -148,7 +148,7 @@ public TabletServersRecovery getTserverRecovery() { @Path("{address}") @GET public TabletServerSummary getTserverDetails( - @PathParam("address") @NotNull @Pattern(regexp = SERVER_REGEX) String tserverAddress) + @PathParam("address") @NotNull @Pattern(regexp = HOSTNAME_PORT_REGEX) String tserverAddress) throws Exception { boolean tserverExists = false; @@ -316,7 +316,7 @@ private CurrentTabletResults doCurrentTabletResults(double currentMinorAvg, KeyExtent extent = new KeyExtent(info.extent); Table.ID tableId = extent.getTableId(); - MessageDigest digester = MessageDigest.getInstance("MD5"); + MessageDigest digester = MessageDigest.getInstance(Constants.PW_HASH_ALGORITHM); if (extent.getEndRow() != null && extent.getEndRow().getLength() > 0) { digester.update(extent.getEndRow().getBytes(), 0, extent.getEndRow().getLength()); } diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/util/ParameterValidator.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/util/ParameterValidator.java index c9b29d05d8..1726cbf8f8 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/util/ParameterValidator.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/util/ParameterValidator.java @@ -26,15 +26,8 @@ String ALPHA_NUM_REGEX_TABLE_ID = "[!+]?\\w+"; String ALPHA_NUM_REGEX_BLANK_OK = "\\w*"; - String RESOURCE_REGEX = "(\\w|:)+"; - - // asterisk or - or blank or a namespace name - String NAMESPACE_REGEX = "[*-]?|\\w+"; - - // asterisk or blank or a comma-separated list of - or namespace names (optional trailing comma) - String NAMESPACE_LIST_REGEX = "[*]?|([-]|\\w+)(,([-]|\\w+))*,?"; + String RESOURCE_REGEX = "(?:\\w|:)+"; // host name and port - String SERVER_REGEX = "(\\w+[.-]*\\w*)+(:[0-9]{1,5})*"; - String SERVER_REGEX_BLANK_OK = "(" + SERVER_REGEX + ")*"; + String HOSTNAME_PORT_REGEX = "[a-zA-Z0-9.-]+:[0-9]{2,5}"; } diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/view/WebViews.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/view/WebViews.java index b068e2e993..913ff7141d 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/view/WebViews.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/view/WebViews.java @@ -19,7 +19,7 @@ import static org.apache.accumulo.monitor.util.ParameterValidator.ALPHA_NUM_REGEX; import static org.apache.accumulo.monitor.util.ParameterValidator.ALPHA_NUM_REGEX_BLANK_OK; import static org.apache.accumulo.monitor.util.ParameterValidator.ALPHA_NUM_REGEX_TABLE_ID; -import static org.apache.accumulo.monitor.util.ParameterValidator.SERVER_REGEX_BLANK_OK; +import static org.apache.accumulo.monitor.util.ParameterValidator.HOSTNAME_PORT_REGEX; import static org.apache.commons.lang.StringUtils.isBlank; import static org.apache.commons.lang.StringUtils.isEmpty; import static org.apache.commons.lang.StringUtils.isNotBlank; @@ -151,7 +151,7 @@ private void addExternalResources(Map model) { @Path("tservers") @Template(name = "/default.ftl") public Map getTabletServers( - @QueryParam("s") @Pattern(regexp = SERVER_REGEX_BLANK_OK) String server) { + @QueryParam("s") @Pattern(regexp = HOSTNAME_PORT_REGEX) String server) { Map model = getModel(); model.put("title", "Tablet Server Status"); diff --git a/server/monitor/src/test/java/org/apache/accumulo/monitor/util/ParameterValidatorTest.java b/server/monitor/src/test/java/org/apache/accumulo/monitor/util/ParameterValidatorTest.java index d3d417f87e..891de3ec06 100644 --- a/server/monitor/src/test/java/org/apache/accumulo/monitor/util/ParameterValidatorTest.java +++ b/server/monitor/src/test/java/org/apache/accumulo/monitor/util/ParameterValidatorTest.java @@ -41,17 +41,17 @@ public void testAlphaNumRegex() { @Test public void testServerRegex() throws Exception { - Pattern p = Pattern.compile(ParameterValidator.SERVER_REGEX); + Pattern p = Pattern.compile(ParameterValidator.HOSTNAME_PORT_REGEX); Assert.assertTrue("Did not match hostname with dots", - p.matcher("ab3cd.12d34.3xyz.net").matches()); + p.matcher("ab3cd.12d34.3xyz.net:12").matches()); Assert.assertTrue("Did not match hostname with dash", - p.matcher("abcd.123.server-foo.com").matches()); + p.matcher("abcd.123.server-foo.com:56789").matches()); Assert.assertTrue("Did not match hostname and port", p.matcher("abcd.123.server-foo.com:1234").matches()); - Assert.assertTrue("Did not match all numeric", p.matcher("127.0.0.1").matches()); Assert.assertTrue("Did not match all numeric and port", p.matcher("127.0.0.1:9999").matches()); Assert.assertTrue("Did not match all numeric and port", p.matcher("ServerName:9999").matches()); + Assert.assertFalse("Port number required", p.matcher("127.0.0.1").matches()); Assert.assertFalse(p.matcher("abcd.1234.*.xyz").matches()); Assert.assertFalse(p.matcher("abcd.1234.;xyz").matches()); Assert.assertFalse(p.matcher("abcd.12{3}4.xyz").matches()); @@ -59,13 +59,6 @@ public void testServerRegex() throws Exception { Assert.assertFalse(p.matcher("abcd=4.xyz").matches()); Assert.assertFalse(p.matcher("abcd=\"4.xyz\"").matches()); Assert.assertFalse(p.matcher("abcd\"4.xyz\"").matches()); - - Pattern q = Pattern.compile(ParameterValidator.SERVER_REGEX_BLANK_OK); - Assert.assertTrue(q.matcher("abcd:9997").matches()); - Assert.assertTrue(q.matcher("abcd.123:9997").matches()); - Assert.assertTrue(q.matcher("abcd.123-xyz:9997").matches()); - Assert.assertTrue(q.matcher("abcd.123-xyz").matches()); - Assert.assertTrue(q.matcher("").matches()); } } diff --git a/server/tracer/src/main/java/org/apache/accumulo/tracer/ZooTraceClient.java b/server/tracer/src/main/java/org/apache/accumulo/tracer/ZooTraceClient.java index e340cfbbb0..bd6654934f 100644 --- a/server/tracer/src/main/java/org/apache/accumulo/tracer/ZooTraceClient.java +++ b/server/tracer/src/main/java/org/apache/accumulo/tracer/ZooTraceClient.java @@ -18,6 +18,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -50,7 +51,7 @@ ZooReader zoo = null; String path; boolean pathExists = false; - final Random random = new Random(); + final Random random = new SecureRandom(); final List hosts = new ArrayList<>(); long retryPause = 5000L; diff --git a/server/tracer/src/main/spotbugs/exclude-filter.xml b/server/tracer/src/main/spotbugs/exclude-filter.xml index f6a7fe4f27..80ee4d25bf 100644 --- a/server/tracer/src/main/spotbugs/exclude-filter.xml +++ b/server/tracer/src/main/spotbugs/exclude-filter.xml @@ -15,8 +15,21 @@ limitations under the License. --> + + + + + + + + + + + + + diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java index fa93c08ef1..03f404af31 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java @@ -27,6 +27,7 @@ import java.net.UnknownHostException; import java.nio.ByteBuffer; import java.security.PrivilegedExceptionAction; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -460,7 +461,7 @@ public String getVersion() { } private static long jitter(long ms) { - Random r = new Random(); + Random r = new SecureRandom(); // add a random 10% wait return (long) ((1. + (r.nextDouble() / 10)) * ms); } diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java index c41430c0c4..b0b7a312d5 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java @@ -19,6 +19,7 @@ import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly; import java.io.IOException; +import java.security.SecureRandom; import java.util.Collections; import java.util.Map; import java.util.Random; @@ -139,7 +140,7 @@ public CompactionStats call() { throw new IllegalStateException(e); } - Random random = new Random(); + Random random = new SecureRandom(); int sleep = sleepTime + random.nextInt(sleepTime); log.debug("MinC failed sleeping {} ms before retrying", sleep); diff --git a/server/tserver/src/main/spotbugs/exclude-filter.xml b/server/tserver/src/main/spotbugs/exclude-filter.xml index eb6386a24b..23af6c0b01 100644 --- a/server/tserver/src/main/spotbugs/exclude-filter.xml +++ b/server/tserver/src/main/spotbugs/exclude-filter.xml @@ -27,4 +27,20 @@ + + + + + + + + + + + + + + + + diff --git a/shell/src/main/java/org/apache/accumulo/shell/Shell.java b/shell/src/main/java/org/apache/accumulo/shell/Shell.java index bdab449b75..dc5709ca5b 100644 --- a/shell/src/main/java/org/apache/accumulo/shell/Shell.java +++ b/shell/src/main/java/org/apache/accumulo/shell/Shell.java @@ -656,9 +656,17 @@ public String getDefaultPrompt() { + (getTableName().isEmpty() ? "" : " ") + getTableName() + "> "; } + /** + * Prevent potential CRLF injection into logs from read in user data See + * https://find-sec-bugs.github.io/bugs.htm#CRLF_INJECTION_LOGS + */ + private String sanitize(String msg) { + return msg.replaceAll("[\r\n]", ""); + } + public void execCommand(String input, boolean ignoreAuthTimeout, boolean echoPrompt) throws IOException { - audit.log(Level.INFO, getDefaultPrompt() + input); + audit.log(Level.INFO, sanitize(getDefaultPrompt() + input)); if (echoPrompt) { reader.print(getDefaultPrompt()); reader.println(input); diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/GetSplitsCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/GetSplitsCommand.java index 292d600a6b..18e069f8ec 100644 --- a/shell/src/main/java/org/apache/accumulo/shell/commands/GetSplitsCommand.java +++ b/shell/src/main/java/org/apache/accumulo/shell/commands/GetSplitsCommand.java @@ -23,6 +23,7 @@ import java.util.Iterator; import java.util.Map.Entry; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.Scanner; @@ -112,7 +113,7 @@ private static String encode(final boolean encode, final Text text) { private static String obscuredTabletName(final KeyExtent extent) { MessageDigest digester; try { - digester = MessageDigest.getInstance("MD5"); + digester = MessageDigest.getInstance(Constants.PW_HASH_ALGORITHM); } catch (NoSuchAlgorithmException e) { throw new RuntimeException(e); } diff --git a/shell/src/main/spotbugs/exclude-filter.xml b/shell/src/main/spotbugs/exclude-filter.xml index e21ccff9d5..1be3c690a0 100644 --- a/shell/src/main/spotbugs/exclude-filter.xml +++ b/shell/src/main/spotbugs/exclude-filter.xml @@ -20,4 +20,30 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/shell/src/test/java/org/apache/accumulo/shell/PasswordConverterTest.java b/shell/src/test/java/org/apache/accumulo/shell/PasswordConverterTest.java index 1db0f02b26..0dbc183254 100644 --- a/shell/src/test/java/org/apache/accumulo/shell/PasswordConverterTest.java +++ b/shell/src/test/java/org/apache/accumulo/shell/PasswordConverterTest.java @@ -25,6 +25,7 @@ import java.io.OutputStreamWriter; import java.io.PipedInputStream; import java.io.PipedOutputStream; +import java.security.SecureRandom; import java.util.Scanner; import org.apache.accumulo.shell.ShellOptionsJC.PasswordConverter; @@ -74,7 +75,7 @@ public void teardown() { @Test public void testPass() { - String expected = String.valueOf(Math.random()); + String expected = String.valueOf(new SecureRandom().nextDouble()); argv[1] = "pass:" + expected; new JCommander(password).parse(argv); assertEquals(expected, password.password); diff --git a/start/src/main/spotbugs/exclude-filter.xml b/start/src/main/spotbugs/exclude-filter.xml new file mode 100644 index 0000000000..9159e5d472 --- /dev/null +++ b/start/src/main/spotbugs/exclude-filter.xml @@ -0,0 +1,36 @@ + + + + + + + + + + + + + + + + + + + + + diff --git a/test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java b/test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java index 5c65e3d766..0f931cacf7 100644 --- a/test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java +++ b/test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java @@ -20,7 +20,7 @@ import java.io.File; import java.io.IOException; -import java.util.Random; +import java.security.SecureRandom; import org.apache.accumulo.cluster.ClusterUser; import org.apache.accumulo.cluster.ClusterUsers; @@ -105,7 +105,7 @@ public static void startMiniClusterWithConfig( } cluster = harness.create(SharedMiniClusterBase.class.getName(), - System.currentTimeMillis() + "_" + new Random().nextInt(Short.MAX_VALUE), token, + System.currentTimeMillis() + "_" + new SecureRandom().nextInt(Short.MAX_VALUE), token, miniClusterCallback, krb); cluster.start(); diff --git a/test/src/main/java/org/apache/accumulo/test/CompactionRateLimitingIT.java b/test/src/main/java/org/apache/accumulo/test/CompactionRateLimitingIT.java index a810176988..efbf2328d4 100644 --- a/test/src/main/java/org/apache/accumulo/test/CompactionRateLimitingIT.java +++ b/test/src/main/java/org/apache/accumulo/test/CompactionRateLimitingIT.java @@ -16,6 +16,7 @@ */ package org.apache.accumulo.test; +import java.security.SecureRandom; import java.util.Random; import org.apache.accumulo.core.client.BatchWriter; @@ -48,7 +49,7 @@ public void majorCompactionsAreRateLimited() throws Exception { Connector conn = getCluster().getConnector("root", new PasswordToken(ROOT_PASSWORD)); conn.tableOperations().create(tableName); try (BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig())) { - Random r = new Random(); + Random r = new SecureRandom(); while (bytesWritten < BYTES_TO_WRITE) { byte[] rowKey = new byte[32]; r.nextBytes(rowKey); diff --git a/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java b/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java index 4d18a20e25..b2b82f6f77 100644 --- a/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -859,7 +860,7 @@ public void testBigBatch() throws Exception { ArrayList rows = new ArrayList<>(num); ArrayList cml = new ArrayList<>(num); - Random r = new Random(); + Random r = new SecureRandom(); byte[] e = new byte[0]; for (int i = 0; i < num; i++) { @@ -934,7 +935,7 @@ public void testBatchErrors() throws Exception { ColumnVisibility cvaob = new ColumnVisibility("A|B"); ColumnVisibility cvaab = new ColumnVisibility("A&B"); - switch ((new Random()).nextInt(3)) { + switch ((new SecureRandom()).nextInt(3)) { case 1: conn.tableOperations().addSplits(tableName, nss("6")); break; @@ -1151,7 +1152,7 @@ public MutatorTask(String tableName, Connector conn, ArrayList row public void run() { try (Scanner scanner = new IsolatedScanner( conn.createScanner(tableName, Authorizations.EMPTY))) { - Random rand = new Random(); + Random rand = new SecureRandom(); for (int i = 0; i < 20; i++) { int numRows = rand.nextInt(10) + 1; @@ -1197,7 +1198,7 @@ public void testThreads() throws Exception { conn.tableOperations().create(tableName); - Random rand = new Random(); + Random rand = new SecureRandom(); switch (rand.nextInt(3)) { case 1: diff --git a/test/src/main/java/org/apache/accumulo/test/CreateRandomRFile.java b/test/src/main/java/org/apache/accumulo/test/CreateRandomRFile.java index 6cca4e1569..7431eb8c9c 100644 --- a/test/src/main/java/org/apache/accumulo/test/CreateRandomRFile.java +++ b/test/src/main/java/org/apache/accumulo/test/CreateRandomRFile.java @@ -17,6 +17,7 @@ package org.apache.accumulo.test; import java.io.IOException; +import java.security.SecureRandom; import java.util.Arrays; import java.util.Random; @@ -35,7 +36,8 @@ private static String file; public static byte[] createValue(long rowid, int dataSize) { - Random r = new Random(rowid); + Random r = new SecureRandom(); + r.setSeed(rowid); byte value[] = new byte[dataSize]; r.nextBytes(value); @@ -57,7 +59,7 @@ public static void main(String[] args) { num = Integer.parseInt(args[1]); long rands[] = new long[num]; - Random r = new Random(); + Random r = new SecureRandom(); for (int i = 0; i < rands.length; i++) { rands[i] = (r.nextLong() & 0x7fffffffffffffffL) % 10000000000L; diff --git a/test/src/main/java/org/apache/accumulo/test/FaultyConditionalWriter.java b/test/src/main/java/org/apache/accumulo/test/FaultyConditionalWriter.java index 36f36172ec..968ced4467 100644 --- a/test/src/main/java/org/apache/accumulo/test/FaultyConditionalWriter.java +++ b/test/src/main/java/org/apache/accumulo/test/FaultyConditionalWriter.java @@ -16,6 +16,7 @@ */ package org.apache.accumulo.test; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; @@ -40,7 +41,7 @@ public FaultyConditionalWriter(ConditionalWriter cw, double unknownProbability, this.cw = cw; this.up = unknownProbability; this.wp = writeProbability; - this.rand = new Random(); + this.rand = new SecureRandom(); } diff --git a/test/src/main/java/org/apache/accumulo/test/IMMLGBenchmark.java b/test/src/main/java/org/apache/accumulo/test/IMMLGBenchmark.java index 320b9c23d2..6c5501b6b0 100644 --- a/test/src/main/java/org/apache/accumulo/test/IMMLGBenchmark.java +++ b/test/src/main/java/org/apache/accumulo/test/IMMLGBenchmark.java @@ -18,6 +18,7 @@ import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -134,7 +135,7 @@ private static long scan(Connector conn, ArrayList cfset, String table, private static long write(Connector conn, ArrayList cfset, String table) throws TableNotFoundException, MutationsRejectedException { - Random rand = new Random(); + Random rand = new SecureRandom(); byte val[] = new byte[50]; diff --git a/test/src/main/java/org/apache/accumulo/test/MetaGetsReadersIT.java b/test/src/main/java/org/apache/accumulo/test/MetaGetsReadersIT.java index 1d50513952..e9c1f81c77 100644 --- a/test/src/main/java/org/apache/accumulo/test/MetaGetsReadersIT.java +++ b/test/src/main/java/org/apache/accumulo/test/MetaGetsReadersIT.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.security.SecureRandom; import java.util.Iterator; import java.util.Map.Entry; import java.util.Random; @@ -81,7 +82,7 @@ public void test() throws Exception { final String tableName = getUniqueNames(1)[0]; final Connector c = getConnector(); c.tableOperations().create(tableName); - Random random = new Random(); + Random random = new SecureRandom(); BatchWriter bw = c.createBatchWriter(tableName, null); for (int i = 0; i < 50000; i++) { byte[] row = new byte[100]; diff --git a/test/src/main/java/org/apache/accumulo/test/MultiTableRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/MultiTableRecoveryIT.java index 166b2ec497..fe4cec6731 100644 --- a/test/src/main/java/org/apache/accumulo/test/MultiTableRecoveryIT.java +++ b/test/src/main/java/org/apache/accumulo/test/MultiTableRecoveryIT.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.security.SecureRandom; import java.util.Map.Entry; import java.util.Random; import java.util.concurrent.TimeUnit; @@ -73,7 +74,7 @@ public void testRecoveryOverMultipleTables() throws Exception { final Thread agitator = agitator(stop); agitator.start(); System.out.println("writing"); - final Random random = new Random(); + final Random random = new SecureRandom(); for (i = 0; i < 1_000_000; i++) { // make non-negative avoiding Math.abs, because that can still be negative long randomRow = random.nextLong() & Long.MAX_VALUE; diff --git a/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java b/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java index d21eb7346b..ae084b2260 100644 --- a/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java +++ b/test/src/main/java/org/apache/accumulo/test/QueryMetadataTable.java @@ -16,6 +16,7 @@ */ package org.apache.accumulo.test; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.HashSet; import java.util.Map.Entry; @@ -129,7 +130,7 @@ public static void main(String[] args) ArrayList rows = new ArrayList<>(rowSet); - Random r = new Random(); + Random r = new SecureRandom(); ExecutorService tp = Executors.newFixedThreadPool(opts.numThreads); diff --git a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java index 6ba505b54e..3a98ad4d7c 100644 --- a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java @@ -31,6 +31,7 @@ import java.io.OutputStream; import java.io.PrintWriter; import java.lang.reflect.Constructor; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -953,7 +954,7 @@ public void testCompactionSelection() throws Exception { assertEquals(2, countFiles(cloneId)); // create two large files - Random rand = new Random(); + Random rand = new SecureRandom(); StringBuilder sb = new StringBuilder("insert b v q "); for (int i = 0; i < 10000; i++) { sb.append('a' + rand.nextInt(26)); diff --git a/test/src/main/java/org/apache/accumulo/test/TableConfigurationUpdateIT.java b/test/src/main/java/org/apache/accumulo/test/TableConfigurationUpdateIT.java index 06c8be05be..3ab625562f 100644 --- a/test/src/main/java/org/apache/accumulo/test/TableConfigurationUpdateIT.java +++ b/test/src/main/java/org/apache/accumulo/test/TableConfigurationUpdateIT.java @@ -16,6 +16,7 @@ */ package org.apache.accumulo.test; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Random; import java.util.concurrent.Callable; @@ -107,7 +108,7 @@ public TableConfRunner(int randMax, int iterations, AccumuloConfiguration tableC @Override public Exception call() { - Random r = new Random(); + Random r = new SecureRandom(); countDown.countDown(); try { countDown.await(); diff --git a/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java b/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java index 3b78473f7f..461efda8ff 100644 --- a/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java +++ b/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java @@ -18,6 +18,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; +import java.security.SecureRandom; import java.util.Iterator; import java.util.Map.Entry; import java.util.Random; @@ -160,7 +161,7 @@ public static void runTest(Connector connector, Opts opts, BatchWriterOpts bwOpt } else if (opts.mode.equals("randomLookups")) { int numLookups = 1000; - Random r = new Random(); + Random r = new SecureRandom(); long t1 = System.currentTimeMillis(); diff --git a/test/src/main/java/org/apache/accumulo/test/TestIngest.java b/test/src/main/java/org/apache/accumulo/test/TestIngest.java index 7926e13141..a47efe4bd2 100644 --- a/test/src/main/java/org/apache/accumulo/test/TestIngest.java +++ b/test/src/main/java/org/apache/accumulo/test/TestIngest.java @@ -19,6 +19,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import java.io.IOException; +import java.security.SecureRandom; import java.util.Map.Entry; import java.util.Random; import java.util.Set; @@ -214,7 +215,7 @@ public static void ingest(Connector connector, FileSystem fs, Opts opts, BatchWr byte[][] bytevals = generateValues(opts.dataSize); byte randomValue[] = new byte[opts.dataSize]; - Random random = new Random(); + Random random = new SecureRandom(); long bytesWritten = 0; diff --git a/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java b/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java index f7285f2238..492f7a0c83 100644 --- a/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java +++ b/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java @@ -19,6 +19,7 @@ import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly; import static org.junit.Assert.assertTrue; +import java.security.SecureRandom; import java.util.Random; import java.util.concurrent.TimeUnit; @@ -54,7 +55,7 @@ public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) @Test(timeout = 4 * 60 * 1000) public void test() throws Exception { - Random random = new Random(); + Random random = new SecureRandom(); Connector c = getConnector(); c.instanceOperations().setProperty(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX.getKey(), "" + SMALL_QUEUE_SIZE); diff --git a/test/src/main/java/org/apache/accumulo/test/UserCompactionStrategyIT.java b/test/src/main/java/org/apache/accumulo/test/UserCompactionStrategyIT.java index 4ffdffc93b..1486d180dd 100644 --- a/test/src/main/java/org/apache/accumulo/test/UserCompactionStrategyIT.java +++ b/test/src/main/java/org/apache/accumulo/test/UserCompactionStrategyIT.java @@ -19,6 +19,7 @@ import java.io.File; import java.io.IOException; +import java.security.SecureRandom; import java.util.Arrays; import java.util.HashSet; import java.util.Map; @@ -297,7 +298,7 @@ public void testConcurrent() throws Exception { } void writeRandomValue(Connector c, String tableName, int size) throws Exception { - Random rand = new Random(); + Random rand = new SecureRandom(); byte data1[] = new byte[size]; rand.nextBytes(data1); diff --git a/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java b/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java index c025226930..f713fefad4 100644 --- a/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java +++ b/test/src/main/java/org/apache/accumulo/test/VerifyIngest.java @@ -16,6 +16,7 @@ */ package org.apache.accumulo.test; +import java.security.SecureRandom; import java.util.Arrays; import java.util.Iterator; import java.util.Map.Entry; @@ -92,7 +93,7 @@ public static void verifyIngest(Connector connector, Opts opts, ScannerOpts scan long t1 = System.currentTimeMillis(); byte randomValue[] = new byte[opts.dataSize]; - Random random = new Random(); + Random random = new SecureRandom(); Key endKey = new Key(new Text("row_" + String.format("%010d", opts.rows + opts.startRow))); diff --git a/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java index dfe61f8dad..51a7690ae0 100644 --- a/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java +++ b/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.security.SecureRandom; import java.util.Random; import java.util.SortedSet; import java.util.TreeSet; @@ -47,7 +48,7 @@ private final static byte[] HEXCHARS = {0x30, 0x31, 0x32, 0x33, 0x34, 0x35, 0x36, 0x37, 0x38, 0x39, 0x61, 0x62, 0x63, 0x64, 0x65, 0x66}; - private final static Random random = new Random(); + private final static Random random = new SecureRandom(); public static byte[] randomHex(int n) { byte[] binary = new byte[n]; diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java index 2b26d683d8..7f4d44326f 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java @@ -19,6 +19,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -89,7 +90,7 @@ public void test() throws Exception { System.out.println("splits : " + splits); - Random random = new Random(19011230); + Random random = new SecureRandom(); HashMap expected = new HashMap<>(); ArrayList ranges = new ArrayList<>(); for (int i = 0; i < 100; i++) { diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java index 771ab8e90f..e573142c5f 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java @@ -19,6 +19,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; @@ -107,7 +108,7 @@ private void runFlushTest(String tableName) throws AccumuloException, AccumuloSe TableNotFoundException, MutationsRejectedException, Exception { BatchWriter bw = getConnector().createBatchWriter(tableName, new BatchWriterConfig()); try (Scanner scanner = getConnector().createScanner(tableName, Authorizations.EMPTY)) { - Random r = new Random(); + Random r = new SecureRandom(); for (int i = 0; i < 4; i++) { for (int j = 0; j < NUM_TO_FLUSH; j++) { diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterIT.java index 350c862939..2ea9b2fd1f 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterIT.java @@ -16,6 +16,7 @@ */ package org.apache.accumulo.test.functional; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Arrays; import java.util.HashSet; @@ -176,7 +177,7 @@ private void timeCheck(long t1, long t2) throws Exception { private long query(Connector c, String table, int depth, long start, long end, int num, int step) throws Exception { - Random r = new Random(42); + Random r = new SecureRandom(); HashSet expected = new HashSet<>(); List ranges = new ArrayList<>(num); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java index 6438275bb1..43ac5f6b46 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java @@ -23,6 +23,7 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.ObjectInputStream; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Map; import java.util.Random; @@ -61,7 +62,7 @@ public static void main(String[] args) throws Exception { ArrayList children = new ArrayList<>(); - Random r = new Random(); + Random r = new SecureRandom(); while (count++ < numVerifications) { diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ConcurrentDeleteTableIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ConcurrentDeleteTableIT.java index 3f1d85f876..6cb9790ebe 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/ConcurrentDeleteTableIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/ConcurrentDeleteTableIT.java @@ -17,6 +17,7 @@ package org.apache.accumulo.test.functional; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -258,7 +259,7 @@ protected void doTableOp() throws Exception { private void writeData(Connector c, String table) throws TableNotFoundException, MutationsRejectedException { try (BatchWriter bw = c.createBatchWriter(table, new BatchWriterConfig())) { - Random rand = new Random(); + Random rand = new SecureRandom(); for (int i = 0; i < 1000; i++) { Mutation m = new Mutation(String.format("%09x", rand.nextInt(100000 * 1000))); m.put("m", "order", "" + i); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableCompactionIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableCompactionIT.java index 8e4ee1203b..a54ca53f47 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableCompactionIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableCompactionIT.java @@ -20,6 +20,7 @@ import java.io.File; import java.io.IOException; +import java.security.SecureRandom; import java.util.Arrays; import java.util.Collections; import java.util.Map; @@ -141,7 +142,7 @@ private void writeFlush(Connector conn, String tablename, String row) throws Exc conn.tableOperations().flush(tablename, null, null, true); } - final static Random r = new Random(); + final static Random r = new SecureRandom(); private void makeFile(Connector conn, String tablename) throws Exception { BatchWriter bw = conn.createBatchWriter(tablename, new BatchWriterConfig()); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java b/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java index 9e9f35bf21..e5deec8187 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java @@ -16,6 +16,7 @@ */ package org.apache.accumulo.test.functional; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.List; import java.util.Random; @@ -58,7 +59,7 @@ public void run() throws Exception { c.tableOperations().flush(tableName, null, null, true); List splits = new ArrayList<>(TestIngest.getSplitPoints(0, 100000, 67)); - Random rand = new Random(); + Random rand = new SecureRandom(); for (int i = 0; i < 100; i++) { int idx1 = rand.nextInt(splits.size() - 1); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/LargeRowIT.java b/test/src/main/java/org/apache/accumulo/test/functional/LargeRowIT.java index 7fba087193..fb98c0be0d 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/LargeRowIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/LargeRowIT.java @@ -19,6 +19,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly; +import java.security.SecureRandom; import java.util.Map; import java.util.Map.Entry; import java.util.Random; @@ -108,7 +109,7 @@ public void resetMajcDelay() throws Exception { @Test public void run() throws Exception { - Random r = new Random(); + Random r = new SecureRandom(); byte rowData[] = new byte[ROW_SIZE]; r.setSeed(SEED + 1); TreeSet splitPoints = new TreeSet<>(); @@ -149,7 +150,7 @@ private void test2(Connector c) throws Exception { private void basicTest(Connector c, String table, int expectedSplits) throws Exception { BatchWriter bw = c.createBatchWriter(table, new BatchWriterConfig()); - Random r = new Random(); + Random r = new SecureRandom(); byte rowData[] = new byte[ROW_SIZE]; r.setSeed(SEED); @@ -188,7 +189,7 @@ private void basicTest(Connector c, String table, int expectedSplits) throws Exc } private void verify(Connector c, String table) throws Exception { - Random r = new Random(); + Random r = new SecureRandom(); byte rowData[] = new byte[ROW_SIZE]; r.setSeed(SEED); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java index 95dd90b889..d9b0b849fa 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java @@ -16,6 +16,7 @@ */ package org.apache.accumulo.test.functional; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -139,7 +140,7 @@ private long batchScan(Connector c, String tableName, List ranges, int th long t1 = System.currentTimeMillis(); byte rval[] = new byte[50]; - Random random = new Random(); + Random random = new SecureRandom(); for (Entry entry : bs) { count++; diff --git a/test/src/main/java/org/apache/accumulo/test/functional/NativeMapIT.java b/test/src/main/java/org/apache/accumulo/test/functional/NativeMapIT.java index 32e33ee150..e3f90ee15c 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/NativeMapIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/NativeMapIT.java @@ -26,6 +26,7 @@ import java.io.File; import java.io.IOException; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; @@ -471,7 +472,7 @@ public void test11() { // insert things with varying field sizes and value sizes // generate random data - Random r = new Random(75); + Random r = new SecureRandom(); ArrayList> testData = new ArrayList<>(); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ScanIdIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ScanIdIT.java index 7ec2e2e861..4487e5b953 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/ScanIdIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/ScanIdIT.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.security.SecureRandom; import java.util.EnumSet; import java.util.HashSet; import java.util.List; @@ -87,7 +88,7 @@ private static final int NUM_DATA_ROWS = 100; - private static final Random random = new Random(); + private static final Random random = new SecureRandom(); private static final ExecutorService pool = Executors.newFixedThreadPool(NUM_SCANNERS); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java index 0b9a7aba99..426dab5129 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java @@ -31,6 +31,7 @@ import static org.apache.accumulo.test.functional.BasicSummarizer.TOTAL_STAT; import java.io.IOException; +import java.security.SecureRandom; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -831,7 +832,7 @@ public void testManyFiles() throws Exception { ntc.enableSummarization(SummarizerConfiguration.builder(FamilySummarizer.class).build()); c.tableOperations().create(table, ntc); - Random rand = new Random(42); + Random rand = new SecureRandom(); int q = 0; SortedSet partitionKeys = new TreeSet<>(); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java index 89ce65ede0..353eaf53be 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java @@ -29,6 +29,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -167,7 +168,7 @@ private void verifySomeData(Connector c, String tableName, int expected) throws } private void writeSomeData(Connector conn, String tableName, int row, int col) throws Exception { - Random rand = new Random(); + Random rand = new SecureRandom(); BatchWriter bw = conn.createBatchWriter(tableName, null); byte[] rowData = new byte[10]; byte[] cq = new byte[10]; diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java index ebb8f3190c..5e65a793bb 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java @@ -19,6 +19,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly; +import java.security.SecureRandom; import java.util.HashMap; import java.util.Random; import java.util.concurrent.TimeUnit; @@ -98,7 +99,8 @@ synchronized public void halt(TInfo tinfo, TCredentials credentials, String lock private static final Logger log = LoggerFactory.getLogger(ZombieTServer.class); public static void main(String[] args) throws Exception { - Random random = new Random(System.currentTimeMillis() % 1000); + Random random = new SecureRandom(); + random.setSeed(System.currentTimeMillis() % 1000); int port = random.nextInt(30000) + 2000; ServerContext context = new ServerContext(new SiteConfiguration()); TransactionWatcher watcher = new TransactionWatcher(context); diff --git a/test/src/main/java/org/apache/accumulo/test/master/SuspendedTabletsIT.java b/test/src/main/java/org/apache/accumulo/test/master/SuspendedTabletsIT.java index 5f4530cf3a..061ff72825 100644 --- a/test/src/main/java/org/apache/accumulo/test/master/SuspendedTabletsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/master/SuspendedTabletsIT.java @@ -20,6 +20,7 @@ import static java.util.concurrent.TimeUnit.NANOSECONDS; import static java.util.concurrent.TimeUnit.SECONDS; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -68,7 +69,7 @@ public class SuspendedTabletsIT extends ConfigurableMacBase { private static final Logger log = LoggerFactory.getLogger(SuspendedTabletsIT.class); - private static final Random RANDOM = new Random(); + private static final Random RANDOM = new SecureRandom(); private static ExecutorService THREAD_POOL; public static final int TSERVERS = 5; diff --git a/test/src/main/java/org/apache/accumulo/test/performance/ContinuousIngest.java b/test/src/main/java/org/apache/accumulo/test/performance/ContinuousIngest.java index ba356f4a51..aa5fb544b5 100644 --- a/test/src/main/java/org/apache/accumulo/test/performance/ContinuousIngest.java +++ b/test/src/main/java/org/apache/accumulo/test/performance/ContinuousIngest.java @@ -20,6 +20,7 @@ import java.io.BufferedReader; import java.io.InputStreamReader; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -99,7 +100,7 @@ public static void main(String[] args) throws Exception { bwOpts.getBatchWriterConfig()); bw = Trace.wrapAll(bw, new CountSampler(1024)); - Random r = new Random(); + Random r = new SecureRandom(); byte[] ingestInstanceId = UUID.randomUUID().toString().getBytes(UTF_8); diff --git a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java index d0474309b1..f8527d6196 100644 --- a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java +++ b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java @@ -18,6 +18,7 @@ import java.io.IOException; import java.net.InetAddress; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -381,7 +382,7 @@ private static void runTest(String desc, List tests, int numThreads, private static List selectRandomTablets(int numThreads, List candidates) { List tabletsToTest = new ArrayList<>(); - Random rand = new Random(); + Random rand = new SecureRandom(); for (int i = 0; i < numThreads; i++) { int rindex = rand.nextInt(candidates.size()); tabletsToTest.add(candidates.get(rindex)); diff --git a/test/src/main/java/org/apache/accumulo/test/util/metadata/MetadataBatchScan.java b/test/src/main/java/org/apache/accumulo/test/util/metadata/MetadataBatchScan.java index 9118d3dd1b..f2bac6400c 100644 --- a/test/src/main/java/org/apache/accumulo/test/util/metadata/MetadataBatchScan.java +++ b/test/src/main/java/org/apache/accumulo/test/util/metadata/MetadataBatchScan.java @@ -18,6 +18,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.HashSet; import java.util.List; @@ -65,7 +66,7 @@ public static void main(String[] args) throws Exception { final Connector connector = opts.getConnector(); TreeSet splits = new TreeSet<>(); - Random r = new Random(42); + Random r = new SecureRandom(); while (splits.size() < 99999) { splits.add((r.nextLong() & 0x7fffffffffffffffL) % 1000000000000L); diff --git a/test/src/main/java/org/apache/accumulo/test/util/nativemap/NativeMapPerformance.java b/test/src/main/java/org/apache/accumulo/test/util/nativemap/NativeMapPerformance.java index 2673aee314..c9b05af8e7 100644 --- a/test/src/main/java/org/apache/accumulo/test/util/nativemap/NativeMapPerformance.java +++ b/test/src/main/java/org/apache/accumulo/test/util/nativemap/NativeMapPerformance.java @@ -19,6 +19,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly; +import java.security.SecureRandom; import java.util.Collections; import java.util.Iterator; import java.util.Map.Entry; @@ -69,7 +70,7 @@ else if (mapType.equals("NATIVE_MAP")) else throw new IllegalArgumentException(" map type must be SKIP_LIST, TREE_MAP, or NATIVE_MAP"); - Random rand = new Random(19); + Random rand = new SecureRandom(); // puts long tps = System.currentTimeMillis(); @@ -115,7 +116,7 @@ else if (mapType.equals("NATIVE_MAP")) long tie = System.currentTimeMillis(); - rand = new Random(19); + rand = new SecureRandom(); int rowsToLookup[] = new int[numLookups]; int colsToLookup[] = new int[numLookups]; for (int i = 0; i < Math.min(numLookups, numRows); i++) { diff --git a/test/src/main/java/org/apache/accumulo/test/util/nativemap/NativeMapStress.java b/test/src/main/java/org/apache/accumulo/test/util/nativemap/NativeMapStress.java index 3854cd34b4..15c1c5af22 100644 --- a/test/src/main/java/org/apache/accumulo/test/util/nativemap/NativeMapStress.java +++ b/test/src/main/java/org/apache/accumulo/test/util/nativemap/NativeMapStress.java @@ -18,6 +18,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; @@ -69,7 +70,7 @@ private static void testLotsOfGetsAndScans() { public void run() { NativeMap nm = new NativeMap(); - Random r = new Random(); + Random r = new SecureRandom(); OpTimer timer = null; AtomicLong nextOpid = new AtomicLong(); @@ -258,7 +259,7 @@ private static void testLotsOfOverwrites() { Runnable r = new Runnable() { @Override public void run() { - Random r = new Random(); + Random r = new SecureRandom(); int inserts = 0; for (int i = 0; i < insertsPerThread / 100.0; i++) { diff --git a/test/src/main/spotbugs/exclude-filter.xml b/test/src/main/spotbugs/exclude-filter.xml index 346aac57ba..dec316caf9 100644 --- a/test/src/main/spotbugs/exclude-filter.xml +++ b/test/src/main/spotbugs/exclude-filter.xml @@ -15,6 +15,10 @@ limitations under the License. --> + + + + ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: users@infra.apache.org With regards, Apache Git Services