Improve test to not rely on thread slowness
We have to swap the second latch before we count it down otherwise threads might be faster than the test. This has happend on a recent CI failure: https://elasticsearch-ci.elastic.co/job/elastic+elasticsearch+master+multijob-os-compatibility/os=ubuntu/121/console This commit also adds a synchronized on the close method since it's canceling and modifying a member varialbe that is assigned under lock.
This commit is contained in:
parent
6ede670524
commit
49d868b491
|
@ -780,7 +780,7 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
|
|||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
public synchronized void close() {
|
||||
if (closed.compareAndSet(false, true)) {
|
||||
FutureUtils.cancel(scheduledFuture);
|
||||
scheduledFuture = null;
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.elasticsearch.common.Nullable;
|
|||
import org.elasticsearch.common.compress.CompressedXContent;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.CountDown;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
|
@ -42,6 +43,7 @@ import java.io.IOException;
|
|||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.BiFunction;
|
||||
|
||||
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
|
@ -187,11 +189,18 @@ public class IndexServiceTests extends ESSingleNodeTestCase {
|
|||
return ThreadPool.Names.GENERIC;
|
||||
}
|
||||
};
|
||||
|
||||
BiFunction<AtomicReference<CountDownLatch>, CountDownLatch, CountDownLatch> swapAndReturn = (ref, newLatch) -> {
|
||||
CountDownLatch downLatch = ref.get();
|
||||
ref.set(newLatch);
|
||||
return downLatch;
|
||||
};
|
||||
latch.get().await();
|
||||
latch.set(new CountDownLatch(1));
|
||||
swapAndReturn.apply(latch, new CountDownLatch(1));
|
||||
assertEquals(1, count.get());
|
||||
latch2.get().countDown();
|
||||
latch2.set(new CountDownLatch(1));
|
||||
// here we need to swap first before we let it go otherwise threads might be very fast and run that task twice due to
|
||||
// random exception and the schedule interval is 1ms
|
||||
swapAndReturn.apply(latch2, new CountDownLatch(1)).countDown();
|
||||
|
||||
latch.get().await();
|
||||
assertEquals(2, count.get());
|
||||
|
|
Loading…
Reference in New Issue