Forbidden Thread.sleep API (#13001)

Co-authored-by: Shubham Chaudhary <cshbha@amazon.com>
This commit is contained in:
Shubham Chaudhary 2024-02-05 21:53:52 +05:30 committed by GitHub
parent 63d4ba938f
commit 4b5917029f
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
44 changed files with 105 additions and 1 deletions

View File

@ -74,3 +74,5 @@ javax.sql.rowset.spi.SyncFactory
@defaultMessage Math.fma is insanely slow (2500x) in many environments (e.g. VMs). Use multiply/add and suffer the extra rounding
java.lang.Math#fma(float,float,float)
java.lang.Math#fma(double,double,double)
java.lang.Thread#sleep(**) @ Thread.sleep makes inefficient use of resources, introduces weird race conditions and slows down the code/tests. Not a scalable and good practice so we should prevent it creeping into lucene code

View File

@ -276,6 +276,8 @@ Other
* GITHUB#13053: Minor AnyQueryNode code cleanup (Dmitry Cherniachenko)
* GITHUB#13001: Forbidden Thread#sleep API. (Shubham Chaudhary)
======================== Lucene 9.9.2 =======================
Bug Fixes

View File

@ -20,6 +20,7 @@ import org.apache.lucene.benchmark.byTask.PerfRunData;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.SuppressForbidden;
/**
* Spawns a BG thread that periodically (defaults to 3.0 seconds, but accepts param in seconds)
@ -40,6 +41,7 @@ public class NearRealtimeReaderTask extends PerfTask {
super(runData);
}
@SuppressForbidden(reason = "Thread sleep")
@Override
public int doLogic() throws Exception {

View File

@ -25,8 +25,10 @@ import org.apache.lucene.benchmark.byTask.PerfRunData;
import org.apache.lucene.benchmark.byTask.feeds.NoMoreDataException;
import org.apache.lucene.benchmark.byTask.stats.TaskStats;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.SuppressForbidden;
/** Sequence of parallel or sequential tasks. */
@SuppressForbidden(reason = "Thread sleep")
public class TaskSequence extends PerfTask {
public static int REPEAT_EXHAUST = -2;
private ArrayList<PerfTask> tasks;

View File

@ -17,6 +17,7 @@
package org.apache.lucene.benchmark.byTask.tasks;
import org.apache.lucene.benchmark.byTask.PerfRunData;
import org.apache.lucene.util.SuppressForbidden;
/**
* Simply waits for the specified (via the parameter) amount of time. For example Wait(30s) waits
@ -58,6 +59,7 @@ public class WaitTask extends PerfTask {
}
}
@SuppressForbidden(reason = "Thread sleep")
@Override
public int doLogic() throws Exception {
Thread.sleep((long) (1000 * waitTimeSec));

View File

@ -64,6 +64,7 @@ import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.NamedThreadFactory;
import org.apache.lucene.util.SuppressForbidden;
import org.junit.Test;
@LuceneTestCase.SuppressSysoutChecks(bugUrl = "none")
@ -79,6 +80,7 @@ public final class Test20NewsgroupsClassification extends LuceneTestCase {
private static boolean index = true;
private static boolean split = true;
@SuppressForbidden(reason = "Thread sleep")
@Test
public void test20Newsgroups() throws Exception {

View File

@ -4264,6 +4264,7 @@ public final class CheckIndex implements Closeable {
* @param opts The options to use for this check
* @return 0 iff the index is clean, 1 otherwise
*/
@SuppressForbidden(reason = "Thread sleep")
public int doCheck(Options opts) throws IOException, InterruptedException {
setLevel(opts.level);
setInfoStream(opts.out, opts.verbose);

View File

@ -19,6 +19,7 @@ package org.apache.lucene.search;
import java.io.IOException;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.ThreadInterruptedException;
/**
@ -262,6 +263,7 @@ public class TimeLimitingCollector implements Collector {
this(DEFAULT_RESOLUTION, counter);
}
@SuppressForbidden(reason = "Thread sleep")
@Override
public void run() {
while (!stop) {

View File

@ -17,6 +17,7 @@
package org.apache.lucene.store;
import java.io.IOException;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.ThreadInterruptedException;
/**
@ -93,6 +94,7 @@ public abstract class RateLimiter {
*
* @return the pause time in nano seconds
*/
@SuppressForbidden(reason = "Thread sleep")
@Override
public long pause(long bytes) {

View File

@ -17,6 +17,7 @@
package org.apache.lucene.store;
import java.io.IOException;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.ThreadInterruptedException;
/**
@ -73,6 +74,7 @@ public final class SleepingLockWrapper extends FilterDirectory {
}
}
@SuppressForbidden(reason = "Thread sleep")
@Override
public Lock obtainLock(String lockName) throws IOException {
LockObtainFailedException failureReason = null;

View File

@ -53,6 +53,7 @@ import org.apache.lucene.tests.store.MockDirectoryWrapper;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.SuppressForbidden;
public class TestAddIndexes extends LuceneTestCase {
@ -1255,6 +1256,7 @@ public class TestAddIndexes extends LuceneTestCase {
}
// LUCENE-1335: test simultaneous addIndexes & close
@SuppressForbidden(reason = "Thread sleep")
public void testAddIndexesWithCloseNoWait() throws Throwable {
final int NUM_COPY = 50;
@ -1279,6 +1281,7 @@ public class TestAddIndexes extends LuceneTestCase {
}
// LUCENE-1335: test simultaneous addIndexes & close
@SuppressForbidden(reason = "Thread sleep")
public void testAddIndexesWithRollback() throws Throwable {
final int NUM_COPY = TEST_NIGHTLY ? 50 : 5;

View File

@ -42,6 +42,7 @@ import org.apache.lucene.tests.store.MockDirectoryWrapper;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.SuppressForbidden;
public class TestConcurrentMergeScheduler extends LuceneTestCase {
@ -283,6 +284,7 @@ public class TestConcurrentMergeScheduler extends LuceneTestCase {
}
// LUCENE-4544
@SuppressForbidden(reason = "Thread sleep")
public void testMaxMergeCount() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc =
@ -301,7 +303,7 @@ public class TestConcurrentMergeScheduler extends LuceneTestCase {
ConcurrentMergeScheduler cms =
new ConcurrentMergeScheduler() {
@SuppressForbidden(reason = "Thread sleep")
@Override
protected void doMerge(MergeSource mergeSource, MergePolicy.OneMerge merge)
throws IOException {
@ -551,6 +553,7 @@ public class TestConcurrentMergeScheduler extends LuceneTestCase {
}
// LUCENE-6094
@SuppressForbidden(reason = "Thread sleep")
public void testHangDuringRollback() throws Throwable {
Directory dir = newMockDirectory();
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));

View File

@ -34,6 +34,7 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.tests.analysis.MockAnalyzer;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.Version;
/*
@ -222,6 +223,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
*/
// TODO: this wall-clock-dependent test doesn't seem to actually test any deletionpolicy logic?
@Nightly
@SuppressForbidden(reason = "Thread sleep")
public void testExpirationTimeDeletionPolicy() throws IOException, InterruptedException {
final double SECONDS = 2.0;

View File

@ -48,6 +48,7 @@ import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.Version;
import org.junit.Assume;
@ -1005,6 +1006,7 @@ public class TestDirectoryReader extends LuceneTestCase {
dir.close();
}
@SuppressForbidden(reason = "Thread sleep")
public void testStressTryIncRef() throws IOException, InterruptedException {
Directory dir = newDirectory();
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));

View File

@ -23,6 +23,7 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.ThreadInterruptedException;
/** Tests for {@link DocumentsWriterStallControl} */
@ -50,6 +51,7 @@ public class TestDocumentsWriterStallControl extends LuceneTestCase {
join(waitThreads);
}
@SuppressForbidden(reason = "Thread sleep")
public void testRandom() throws InterruptedException {
final DocumentsWriterStallControl ctrl = new DocumentsWriterStallControl();
ctrl.updateStalled(false);
@ -166,6 +168,7 @@ public class TestDocumentsWriterStallControl extends LuceneTestCase {
}
}
@SuppressForbidden(reason = "Thread sleep")
private void assertState(
int numReleasers,
int numStallers,
@ -296,6 +299,7 @@ public class TestDocumentsWriterStallControl extends LuceneTestCase {
return true;
}
@SuppressForbidden(reason = "Thread sleep")
public static void start(Thread[] tostart) throws InterruptedException {
for (Thread thread : tostart) {
thread.start();
@ -324,6 +328,7 @@ public class TestDocumentsWriterStallControl extends LuceneTestCase {
}
/** Waits for all incoming threads to be in wait() methods. */
@SuppressForbidden(reason = "Thread sleep")
public static void awaitState(Thread.State state, Thread... threads) throws InterruptedException {
while (true) {
boolean done = true;

View File

@ -41,6 +41,7 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.tests.analysis.MockAnalyzer;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.TestVectorUtil;
/**
@ -67,6 +68,7 @@ public class TestExitableDirectoryReader extends LuceneTestCase {
}
/** Sleep between iterations to timeout things. */
@SuppressForbidden(reason = "Thread sleep")
@Override
public BytesRef next() throws IOException {
try {

View File

@ -113,6 +113,7 @@ import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.SetOnce;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.ThreadInterruptedException;
import org.apache.lucene.util.Version;
import org.apache.lucene.util.automaton.Automata;
@ -1086,6 +1087,7 @@ public class TestIndexWriter extends LuceneTestCase {
}
}
@SuppressForbidden(reason = "Thread sleep")
public void testThreadInterruptDeadlock() throws Exception {
IndexerThreadInterrupt t = new IndexerThreadInterrupt(1);
t.setDaemon(true);
@ -2610,6 +2612,7 @@ public class TestIndexWriter extends LuceneTestCase {
}
/** Make sure that close waits for any still-running commits. */
@SuppressForbidden(reason = "Thread sleep")
public void testCloseDuringCommit() throws Exception {
final CountDownLatch startCommit = new CountDownLatch(1);

View File

@ -72,6 +72,7 @@ public class TestIndexWriterOnJRECrash extends TestNRTThreads {
final int crashTime = TestUtil.nextInt(random(), 3000, 4000);
Thread t =
new Thread() {
@SuppressForbidden(reason = "Thread sleep")
@Override
public void run() {
try {

View File

@ -38,6 +38,7 @@ import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.Version;
@LuceneTestCase.SuppressCodecs({"SimpleText", "Direct"})
@ -250,6 +251,7 @@ public class TestIndexWriterThreadsToSegments extends LuceneTestCase {
IOUtils.close(checker, w, dir);
}
@SuppressForbidden(reason = "Thread sleep")
public void testManyThreadsClose() throws Exception {
Directory dir = newDirectory();
Random r = random();

View File

@ -44,6 +44,7 @@ import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.ThreadInterruptedException;
/** MultiThreaded IndexWriter tests */
@ -65,6 +66,7 @@ public class TestIndexWriterWithThreads extends LuceneTestCase {
this.syncStart = syncStart;
}
@SuppressForbidden(reason = "Thread sleep")
@Override
public void run() {
try {
@ -190,6 +192,7 @@ public class TestIndexWriterWithThreads extends LuceneTestCase {
// threads are trying to add documents. Strictly
// speaking, this isn't valid us of Lucene's APIs, but we
// still want to be robust to this case:
@SuppressForbidden(reason = "Thread sleep")
public void testCloseWithThreads() throws Exception {
int NUM_THREADS = 3;
int numIterations = TEST_NIGHTLY ? 7 : 3;

View File

@ -29,6 +29,7 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.Version;
public class TestMergePolicy extends LuceneTestCase {
@ -82,6 +83,7 @@ public class TestMergePolicy extends LuceneTestCase {
}
}
@SuppressForbidden(reason = "Thread sleep")
public void testTimeoutLargeNumberOfMerges() throws IOException, InterruptedException {
try (Directory dir = newDirectory()) {
MergePolicy.MergeSpecification ms = createRandomMergeSpecification(dir, 10000);

View File

@ -26,12 +26,14 @@ import org.apache.lucene.tests.index.RandomIndexWriter;
import org.apache.lucene.tests.store.BaseDirectoryWrapper;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.SuppressForbidden;
// Make sure if you use NoDeletionPolicy that no file
// referenced by a commit point is ever deleted
public class TestNeverDelete extends LuceneTestCase {
@SuppressForbidden(reason = "Thread sleep")
public void testIndexing() throws Exception {
final Path tmpDir = createTempDir("TestNeverDelete");
final BaseDirectoryWrapper d = newFSDirectory(tmpDir);

View File

@ -29,6 +29,7 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.tests.analysis.MockAnalyzer;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.ThreadInterruptedException;
import org.junit.Test;
@ -103,6 +104,7 @@ public class TestSnapshotDeletionPolicy extends LuceneTestCase {
fsDir.close();
}
@SuppressForbidden(reason = "Thread sleep")
private void runTest(Random random, Directory dir) throws Exception {
final int maxIterations = TEST_NIGHTLY ? 100 : 10;
@ -125,6 +127,7 @@ public class TestSnapshotDeletionPolicy extends LuceneTestCase {
final Thread t =
new Thread() {
@SuppressForbidden(reason = "Thread sleep")
@Override
public void run() {
int iterations = 0;
@ -225,6 +228,7 @@ public class TestSnapshotDeletionPolicy extends LuceneTestCase {
byte[] buffer = new byte[4096];
@SuppressForbidden(reason = "Thread sleep")
private void readFile(Directory dir, String name) throws Exception {
IndexInput input = dir.openInput(name, newIOContext(random()));
try {

View File

@ -73,6 +73,7 @@ import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.SuppressForbidden;
public class TestLRUQueryCache extends LuceneTestCase {
@ -233,6 +234,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
}
}
@SuppressForbidden(reason = "Thread sleep")
public void testLRUEviction() throws Exception {
Directory dir = newDirectory();
final RandomIndexWriter w = new RandomIndexWriter(random(), dir);

View File

@ -47,6 +47,7 @@ import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.tests.util.LuceneTestCase.SuppressCodecs;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.NamedThreadFactory;
import org.apache.lucene.util.SuppressForbidden;
@SuppressCodecs({"SimpleText", "Direct"})
public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
@ -105,11 +106,13 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
lifetimeMGR = new SearcherLifetimeManager();
}
@SuppressForbidden(reason = "Thread sleep")
@Override
protected void doSearching(ExecutorService es, final int maxIterations) throws Exception {
Thread reopenThread =
new Thread() {
@SuppressForbidden(reason = "Thread sleep")
@Override
public void run() {
try {

View File

@ -32,6 +32,7 @@ import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.tests.util.LuceneTestCase.SuppressSysoutChecks;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.ThreadInterruptedException;
/**
@ -377,6 +378,7 @@ public class TestTimeLimitingCollector extends LuceneTestCase {
// scorer is not needed
}
@SuppressForbidden(reason = "Thread sleep")
@Override
public void collect(final int doc) throws IOException {
int docId = doc + docBase;

View File

@ -22,12 +22,14 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.store.RateLimiter.SimpleRateLimiter;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.ThreadInterruptedException;
/** Simple testcase for RateLimiter.SimpleRateLimiter */
public final class TestRateLimiter extends LuceneTestCase {
// LUCENE-6075
@SuppressForbidden(reason = "Thread sleep")
public void testOverflowInt() throws Exception {
Thread t =
new Thread() {

View File

@ -32,6 +32,7 @@ public class TestSetOnce extends LuceneTestCase {
RAND = new Random(random.nextLong());
}
@SuppressForbidden(reason = "Thread sleep")
@Override
public void run() {
try {

View File

@ -27,6 +27,7 @@ import org.apache.lucene.tests.util.LuceneTestCase;
public class TestWeakIdentityMap extends LuceneTestCase {
@SuppressForbidden(reason = "Thread sleep")
public void testSimpleHashMap() {
final WeakIdentityMap<String, String> map = WeakIdentityMap.newHashMap(random().nextBoolean());
// we keep strong references to the keys,
@ -162,6 +163,7 @@ public class TestWeakIdentityMap extends LuceneTestCase {
assertTrue(map.isEmpty());
}
@SuppressForbidden(reason = "Thread sleep")
public void testConcurrentHashMap() throws Exception {
// don't make threadCount and keyCount random, otherwise easily OOMs or fails otherwise:
final int threadCount = TEST_NIGHTLY ? 8 : 2;

View File

@ -25,6 +25,7 @@ import org.apache.lucene.store.FilterIndexOutput;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.ThreadInterruptedException;
/** Test utility - slow directory */
@ -63,6 +64,7 @@ public class SlowDirectory extends FilterDirectory {
return super.openInput(name, context);
}
@SuppressForbidden(reason = "Thread sleep")
void doSleep(Random random, int length) {
int sTime = length < 10 ? sleepMillis : (int) (sleepMillis * Math.log(length));
if (random != null) {

View File

@ -44,6 +44,7 @@ import org.apache.lucene.tests.analysis.MockAnalyzer;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.SuppressForbidden;
@LuceneTestCase.SuppressCodecs("SimpleText")
public class TestSearcherTaxonomyManager extends FacetTestCase {
@ -124,6 +125,7 @@ public class TestSearcherTaxonomyManager extends FacetTestCase {
}
}
@SuppressForbidden(reason = "Thread sleep")
public void testNRT() throws Exception {
Directory dir = newDirectory();
Directory taxoDir = newDirectory();
@ -151,6 +153,7 @@ public class TestSearcherTaxonomyManager extends FacetTestCase {
Thread reopener =
new Thread() {
@SuppressForbidden(reason = "Thread sleep")
@Override
public void run() {
while (!stop.get()) {

View File

@ -29,6 +29,7 @@ import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.tests.util.LuceneTestCase.SuppressCodecs;
import org.apache.lucene.util.SuppressForbidden;
import org.junit.Test;
@SuppressCodecs("SimpleText")
@ -770,6 +771,7 @@ public class TestTaxonomyCombined extends FacetTestCase {
indexDirBase.close();
}
@SuppressForbidden(reason = "Thread sleep")
private void assertConsistentYoungestChild(
final FacetLabel abPath,
final int abOrd,
@ -801,6 +803,7 @@ public class TestTaxonomyCombined extends FacetTestCase {
Thread thread =
new Thread("Child Arrays Verifier") {
@SuppressForbidden(reason = "Thread sleep")
@Override
public void run() {
setPriority(1 + getPriority());

View File

@ -66,6 +66,7 @@ import org.apache.lucene.tests.analysis.MockAnalyzer;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.ThreadInterruptedException;
/** A primary node that uses simple TCP connections to send commands and copy files */
@ -175,6 +176,7 @@ class SimplePrimaryNode extends PrimaryNode {
return writer;
}
@SuppressForbidden(reason = "Thread sleep")
@Override
protected void preCopyMergedSegmentFiles(SegmentCommitInfo info, Map<String, FileMetaData> files)
throws IOException {
@ -510,6 +512,7 @@ class SimplePrimaryNode extends PrimaryNode {
tokenizedWithTermVectors.setStoreTermVectorPositions(true);
}
@SuppressForbidden(reason = "Thread sleep")
private void handleIndexing(
Socket socket,
AtomicBoolean stop,
@ -655,6 +658,7 @@ class SimplePrimaryNode extends PrimaryNode {
static final byte CMD_SET_CLOSE_WAIT_MS = 25;
/** Handles incoming request to the naive TCP server wrapping this node */
@SuppressForbidden(reason = "Thread sleep")
void handleOneConnection(
Random random,
ServerSocket ss,

View File

@ -51,6 +51,7 @@ import org.apache.lucene.store.RateLimitedIndexOutput;
import org.apache.lucene.store.RateLimiter;
import org.apache.lucene.tests.store.MockDirectoryWrapper;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.util.SuppressForbidden;
class SimpleReplicaNode extends ReplicaNode {
final int tcpPort;
@ -170,6 +171,7 @@ class SimpleReplicaNode extends ReplicaNode {
static final byte CMD_PRE_COPY_MERGE = 17;
/** Handles incoming request to the naive TCP server wrapping this node */
@SuppressForbidden(reason = "Thread sleep")
void handleOneConnection(
ServerSocket ss,
AtomicBoolean stop,

View File

@ -677,6 +677,7 @@ public class TestNRTReplication extends LuceneTestCase {
}
@Nightly
@SuppressForbidden(reason = "Thread sleep")
public void testCrashPrimaryWhileCopying() throws Exception {
Path path1 = createTempDir("1");
@ -964,6 +965,7 @@ public class TestNRTReplication extends LuceneTestCase {
}
}
@SuppressForbidden(reason = "Thread sleep")
private void waitForVersionAndHits(NodeProcess node, long expectedVersion, int expectedHitCount)
throws Exception {
try (Connection c = new Connection(node.tcpPort)) {

View File

@ -328,6 +328,7 @@ public class TestSimpleServer extends LuceneTestCase {
Thread t =
new Thread() {
@SuppressForbidden(reason = "Thread sleep")
@Override
public void run() {
long endTime = System.nanoTime() + waitForMS * 1000000L;

View File

@ -188,6 +188,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
final Set<Integer> crashingNodes = Collections.synchronizedSet(new HashSet<>());
@Nightly
@SuppressForbidden(reason = "Thread sleep")
public void test() throws Exception {
Node.globalStartNS = System.nanoTime();
@ -852,6 +853,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
}
// Periodically wakes up and starts up any down nodes:
@SuppressForbidden(reason = "Thread sleep")
private class RestartThread extends Thread {
@Override
public void run() {
@ -978,6 +980,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
}
/** Randomly picks a node and runs a search against it */
@SuppressForbidden(reason = "Thread sleep")
private class SearchThread extends Thread {
@Override
@ -1177,6 +1180,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
}
}
@SuppressForbidden(reason = "Thread sleep")
private class IndexThread extends Thread {
@Override

View File

@ -56,6 +56,7 @@ import org.apache.lucene.tests.index.RandomIndexWriter;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.SuppressForbidden;
/** Basic tests for IDVersionPostingsFormat */
// Cannot extend BasePostingsFormatTestCase because this PF is not
@ -677,6 +678,7 @@ public class TestIDVersionPostingsFormat extends LuceneTestCase {
// Simulates optimistic concurrency in a distributed indexing app and confirms the latest version
// always wins:
@SuppressForbidden(reason = "Thread sleep")
public void testGlobalVersions() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
@ -750,6 +752,7 @@ public class TestIDVersionPostingsFormat extends LuceneTestCase {
}
}
@SuppressForbidden(reason = "Thread sleep")
private void runForReal() throws IOException, InterruptedException {
startingGun.await();
PerThreadVersionPKLookup lookup = null;

View File

@ -39,6 +39,7 @@ import org.apache.lucene.tests.analysis.MockAnalyzer;
import org.apache.lucene.tests.util.English;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.util.NamedThreadFactory;
import org.apache.lucene.util.SuppressForbidden;
/** Spell checker test case */
public class TestSpellChecker extends LuceneTestCase {
@ -430,6 +431,7 @@ public class TestSpellChecker extends LuceneTestCase {
* tests if the internally shared indexsearcher is correctly closed
* when the spellchecker is concurrently accessed and closed.
*/
@SuppressForbidden(reason = "Thread sleep")
public void testConcurrentAccess() throws IOException, InterruptedException {
assertEquals(1, searchers.size());
final IndexReader r = DirectoryReader.open(userindex);

View File

@ -65,6 +65,7 @@ import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.NamedThreadFactory;
import org.apache.lucene.util.PrintStreamInfoStream;
import org.apache.lucene.util.SuppressForbidden;
// TODO
// - mix in forceMerge, addIndexes
@ -146,6 +147,7 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
for (int thread = 0; thread < numThreads; thread++) {
threads[thread] =
new Thread() {
@SuppressForbidden(reason = "Thread sleep")
@Override
public void run() {
// TODO: would be better if this were cross thread, so that we make sure one thread
@ -490,6 +492,7 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
private final Map<Object, Boolean> warmed = Collections.synchronizedMap(new WeakHashMap<>());
@SuppressForbidden(reason = "Thread sleep")
public void runTest(String testName) throws Exception {
failed.set(false);

View File

@ -18,6 +18,7 @@ package org.apache.lucene.tests.store;
import java.io.IOException;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.ThreadInterruptedException;
/**
@ -32,6 +33,7 @@ class SlowClosingMockIndexInputWrapper extends MockIndexInputWrapper {
super(dir, name, delegate, null);
}
@SuppressForbidden(reason = "Thread sleep")
@Override
public void close() throws IOException {
try {

View File

@ -18,6 +18,7 @@ package org.apache.lucene.tests.store;
import java.io.IOException;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.ThreadInterruptedException;
/**
@ -26,6 +27,7 @@ import org.apache.lucene.util.ThreadInterruptedException;
*/
class SlowOpeningMockIndexInputWrapper extends MockIndexInputWrapper {
@SuppressForbidden(reason = "Thread sleep")
public SlowOpeningMockIndexInputWrapper(
MockDirectoryWrapper dir, String name, IndexInput delegate) throws IOException {
super(dir, name, delegate, null);

View File

@ -20,6 +20,7 @@ import java.io.IOException;
import java.util.concurrent.TimeUnit;
import org.apache.lucene.store.FilterIndexOutput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.SuppressForbidden;
import org.apache.lucene.util.ThreadInterruptedException;
/** Intentionally slow IndexOutput for testing. */
@ -115,6 +116,7 @@ public class ThrottledIndexOutput extends FilterIndexOutput {
return 0;
}
@SuppressForbidden(reason = "Thread sleep")
private static void sleep(long ms) {
if (ms <= 0) {
return;

View File

@ -18,6 +18,7 @@ package org.apache.lucene.tests.util;
import com.carrotsearch.randomizedtesting.RandomizedTest;
import com.carrotsearch.randomizedtesting.annotations.Timeout;
import org.apache.lucene.util.SuppressForbidden;
import org.junit.Ignore;
public class TestWorstCaseTestBehavior extends LuceneTestCase {
@ -25,6 +26,7 @@ public class TestWorstCaseTestBehavior extends LuceneTestCase {
public void testThreadLeak() {
Thread t =
new Thread() {
@SuppressForbidden(reason = "Thread sleep")
@Override
public void run() {
try {
@ -83,12 +85,14 @@ public class TestWorstCaseTestBehavior extends LuceneTestCase {
@Ignore
@Timeout(millis = 500)
@SuppressForbidden(reason = "Thread sleep")
public void testTimeout() throws Exception {
Thread.sleep(5000);
}
@Ignore
@Timeout(millis = 1000)
@SuppressForbidden(reason = "Thread sleep")
public void testZombie() throws Exception {
while (true) {
try {