mirror of https://github.com/apache/druid.git
Merge pull request #1694 from metamx/namespaceExtractionCacheManagerTestImprovements
Better timing and locking in NamespaceExtractionCacheManagerExecutorsTest
This commit is contained in:
commit
07266d682a
|
@ -148,9 +148,9 @@ public abstract class NamespaceExtractionCacheManager
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void waitForServiceToEnd(long time, TimeUnit unit) throws InterruptedException
|
protected boolean waitForServiceToEnd(long time, TimeUnit unit) throws InterruptedException
|
||||||
{
|
{
|
||||||
listeningScheduledExecutorService.awaitTermination(time, unit);
|
return listeningScheduledExecutorService.awaitTermination(time, unit);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -23,15 +23,14 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.util.concurrent.Futures;
|
|
||||||
import com.google.common.util.concurrent.ListenableFuture;
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||||
import com.google.common.util.concurrent.ListeningScheduledExecutorService;
|
import com.google.common.util.concurrent.ListeningScheduledExecutorService;
|
||||||
import com.google.common.util.concurrent.MoreExecutors;
|
import com.google.common.util.concurrent.MoreExecutors;
|
||||||
|
import com.metamx.common.IAE;
|
||||||
import com.metamx.common.lifecycle.Lifecycle;
|
import com.metamx.common.lifecycle.Lifecycle;
|
||||||
import com.metamx.common.logger.Logger;
|
import io.druid.concurrent.Execs;
|
||||||
import io.druid.data.SearchableVersionedDataFinder;
|
import io.druid.data.SearchableVersionedDataFinder;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
|
||||||
import io.druid.query.extraction.namespace.ExtractionNamespace;
|
import io.druid.query.extraction.namespace.ExtractionNamespace;
|
||||||
import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory;
|
import io.druid.query.extraction.namespace.ExtractionNamespaceFunctionFactory;
|
||||||
import io.druid.query.extraction.namespace.URIExtractionNamespace;
|
import io.druid.query.extraction.namespace.URIExtractionNamespace;
|
||||||
|
@ -39,13 +38,13 @@ import io.druid.query.extraction.namespace.URIExtractionNamespaceTest;
|
||||||
import io.druid.segment.loading.LocalFileTimestampVersionFinder;
|
import io.druid.segment.loading.LocalFileTimestampVersionFinder;
|
||||||
import io.druid.server.metrics.NoopServiceEmitter;
|
import io.druid.server.metrics.NoopServiceEmitter;
|
||||||
import io.druid.server.namespace.URIExtractionNamespaceFunctionFactory;
|
import io.druid.server.namespace.URIExtractionNamespaceFunctionFactory;
|
||||||
import org.apache.commons.io.FileUtils;
|
|
||||||
import org.joda.time.Period;
|
import org.joda.time.Period;
|
||||||
import org.junit.AfterClass;
|
import org.junit.After;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.TemporaryFolder;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.FileOutputStream;
|
import java.io.FileOutputStream;
|
||||||
|
@ -58,13 +57,14 @@ import java.nio.file.Path;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.UUID;
|
|
||||||
import java.util.concurrent.Callable;
|
import java.util.concurrent.Callable;
|
||||||
|
import java.util.concurrent.CancellationException;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
|
import java.util.concurrent.ConcurrentMap;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.Executors;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.concurrent.TimeoutException;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -72,73 +72,125 @@ import java.util.concurrent.atomic.AtomicLong;
|
||||||
*/
|
*/
|
||||||
public class NamespaceExtractionCacheManagerExecutorsTest
|
public class NamespaceExtractionCacheManagerExecutorsTest
|
||||||
{
|
{
|
||||||
private static final Logger log = new Logger(NamespaceExtractionCacheManagerExecutorsTest.class);
|
private static final String KEY = "foo";
|
||||||
private static Path tmpDir;
|
private static final String VALUE = "bar";
|
||||||
|
@Rule
|
||||||
|
public final TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||||
private Lifecycle lifecycle;
|
private Lifecycle lifecycle;
|
||||||
private NamespaceExtractionCacheManager manager;
|
private NamespaceExtractionCacheManager manager;
|
||||||
private File tmpFile;
|
private File tmpFile;
|
||||||
private URIExtractionNamespaceFunctionFactory factory;
|
private final ConcurrentMap<String, Function<String, String>> fnCache = new ConcurrentHashMap<>();
|
||||||
private final ConcurrentHashMap<String, Function<String, String>> fnCache = new ConcurrentHashMap<String, Function<String, String>>();
|
private final ConcurrentMap<String, Object> cacheUpdateAlerts = new ConcurrentHashMap<>();
|
||||||
|
|
||||||
@BeforeClass
|
private final AtomicLong numRuns = new AtomicLong(0L);
|
||||||
public static void setUpStatic() throws IOException
|
|
||||||
{
|
|
||||||
tmpDir = Files.createTempDirectory("TestNamespaceExtractionCacheManagerExecutors");
|
|
||||||
}
|
|
||||||
|
|
||||||
@AfterClass
|
|
||||||
public static void tearDownStatic() throws IOException
|
|
||||||
{
|
|
||||||
FileUtils.deleteDirectory(tmpDir.toFile());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setUp() throws IOException
|
public void setUp() throws IOException
|
||||||
{
|
{
|
||||||
|
final Path tmpDir = temporaryFolder.newFolder().toPath();
|
||||||
lifecycle = new Lifecycle();
|
lifecycle = new Lifecycle();
|
||||||
manager = new OnHeapNamespaceExtractionCacheManager(
|
final URIExtractionNamespaceFunctionFactory factory = new URIExtractionNamespaceFunctionFactory(
|
||||||
lifecycle, fnCache, new NoopServiceEmitter(),
|
|
||||||
ImmutableMap.<Class<? extends ExtractionNamespace>, ExtractionNamespaceFunctionFactory<?>>of(
|
|
||||||
URIExtractionNamespace.class,
|
|
||||||
new URIExtractionNamespaceFunctionFactory(
|
|
||||||
ImmutableMap.<String, SearchableVersionedDataFinder>of("file", new LocalFileTimestampVersionFinder())
|
|
||||||
)
|
|
||||||
)
|
|
||||||
);
|
|
||||||
tmpFile = Files.createTempFile(tmpDir, "druidTestURIExtractionNS", ".dat").toFile();
|
|
||||||
tmpFile.deleteOnExit();
|
|
||||||
final ObjectMapper mapper = new DefaultObjectMapper();
|
|
||||||
try (OutputStream ostream = new FileOutputStream(tmpFile)) {
|
|
||||||
try (OutputStreamWriter out = new OutputStreamWriter(ostream)) {
|
|
||||||
out.write(mapper.writeValueAsString(ImmutableMap.<String, String>of("foo", "bar")));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
factory = new URIExtractionNamespaceFunctionFactory(
|
|
||||||
ImmutableMap.<String, SearchableVersionedDataFinder>of("file", new LocalFileTimestampVersionFinder())
|
ImmutableMap.<String, SearchableVersionedDataFinder>of("file", new LocalFileTimestampVersionFinder())
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
@Override
|
||||||
public Callable<String> getCachePopulator(
|
public Callable<String> getCachePopulator(
|
||||||
final URIExtractionNamespace extractionNamespace,
|
final URIExtractionNamespace extractionNamespace,
|
||||||
final String lastVersion,
|
final String lastVersion,
|
||||||
final Map<String, String> cache
|
final Map<String, String> cache
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final Callable<String> superCallable = super.getCachePopulator(extractionNamespace, lastVersion, cache);
|
|
||||||
return new Callable<String>()
|
return new Callable<String>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public String call() throws Exception
|
public String call() throws Exception
|
||||||
{
|
{
|
||||||
superCallable.call();
|
// Don't actually read off disk because TravisCI doesn't like that
|
||||||
return String.format("%d", System.currentTimeMillis());
|
cache.put(KEY, VALUE);
|
||||||
|
Thread.sleep(2);// To make absolutely sure there is a unique currentTimeMillis
|
||||||
|
return Long.toString(System.currentTimeMillis());
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
manager = new OnHeapNamespaceExtractionCacheManager(
|
||||||
|
lifecycle, fnCache, new NoopServiceEmitter(),
|
||||||
|
ImmutableMap.<Class<? extends ExtractionNamespace>, ExtractionNamespaceFunctionFactory<?>>of(
|
||||||
|
URIExtractionNamespace.class,
|
||||||
|
factory
|
||||||
|
)
|
||||||
|
)
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
protected <T extends ExtractionNamespace> Runnable getPostRunnable(
|
||||||
|
final T namespace,
|
||||||
|
final ExtractionNamespaceFunctionFactory<T> factory,
|
||||||
|
final String cacheId
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final Runnable runnable = super.getPostRunnable(namespace, factory, cacheId);
|
||||||
|
cacheUpdateAlerts.putIfAbsent(namespace.getNamespace(), new Object());
|
||||||
|
final Object cacheUpdateAlerter = cacheUpdateAlerts.get(namespace.getNamespace());
|
||||||
|
return new Runnable()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public void run()
|
||||||
|
{
|
||||||
|
synchronized (cacheUpdateAlerter) {
|
||||||
|
try {
|
||||||
|
runnable.run();
|
||||||
|
numRuns.incrementAndGet();
|
||||||
|
}
|
||||||
|
finally {
|
||||||
|
cacheUpdateAlerter.notifyAll();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
tmpFile = Files.createTempFile(tmpDir, "druidTestURIExtractionNS", ".dat").toFile();
|
||||||
|
try (OutputStream ostream = new FileOutputStream(tmpFile)) {
|
||||||
|
try (OutputStreamWriter out = new OutputStreamWriter(ostream)) {
|
||||||
|
// Since Travis sucks with disk related stuff, we override the disk reading part above.
|
||||||
|
// This is safe and should shake out any problem areas that accidentally read the file.
|
||||||
|
out.write("SHOULDN'T TRY TO PARSE");
|
||||||
|
out.flush();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void tearDown()
|
||||||
|
{
|
||||||
|
lifecycle.stop();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = IAE.class)
|
||||||
|
public void testDoubleSubmission()
|
||||||
|
{
|
||||||
|
URIExtractionNamespace namespace = new URIExtractionNamespace(
|
||||||
|
"ns",
|
||||||
|
tmpFile.toURI(),
|
||||||
|
new URIExtractionNamespace.ObjectMapperFlatDataParser(
|
||||||
|
URIExtractionNamespaceTest.registerTypes(new ObjectMapper())
|
||||||
|
),
|
||||||
|
new Period(0),
|
||||||
|
null
|
||||||
|
);
|
||||||
|
final ListenableFuture<?> future = manager.schedule(namespace);
|
||||||
|
Assert.assertFalse(future.isDone());
|
||||||
|
Assert.assertFalse(future.isCancelled());
|
||||||
|
try {
|
||||||
|
manager.schedule(namespace).cancel(true);
|
||||||
|
}
|
||||||
|
finally {
|
||||||
|
future.cancel(true);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test(timeout = 50_000)
|
@Test(timeout = 60_000)
|
||||||
public void testSimpleSubmission() throws ExecutionException, InterruptedException
|
public void testSimpleSubmission() throws ExecutionException, InterruptedException
|
||||||
{
|
{
|
||||||
URIExtractionNamespace namespace = new URIExtractionNamespace(
|
URIExtractionNamespace namespace = new URIExtractionNamespace(
|
||||||
|
@ -150,23 +202,16 @@ public class NamespaceExtractionCacheManagerExecutorsTest
|
||||||
new Period(0),
|
new Period(0),
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
try {
|
NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(namespace));
|
||||||
NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(namespace));
|
|
||||||
}
|
|
||||||
finally {
|
|
||||||
lifecycle.stop();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 50_000)
|
@Test(timeout = 60_000)
|
||||||
public void testRepeatSubmission() throws ExecutionException, InterruptedException
|
public void testRepeatSubmission() throws ExecutionException, InterruptedException
|
||||||
{
|
{
|
||||||
final int repeatCount = 5;
|
final int repeatCount = 5;
|
||||||
final long delay = 5;
|
final long delay = 5;
|
||||||
final AtomicLong ranCount = new AtomicLong(0l);
|
|
||||||
final long totalRunCount;
|
final long totalRunCount;
|
||||||
final long start;
|
final long start;
|
||||||
final CountDownLatch latch = new CountDownLatch(repeatCount);
|
|
||||||
try {
|
try {
|
||||||
final URIExtractionNamespace namespace = new URIExtractionNamespace(
|
final URIExtractionNamespace namespace = new URIExtractionNamespace(
|
||||||
"ns",
|
"ns",
|
||||||
|
@ -177,27 +222,28 @@ public class NamespaceExtractionCacheManagerExecutorsTest
|
||||||
new Period(delay),
|
new Period(delay),
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
|
cacheUpdateAlerts.putIfAbsent(namespace.getNamespace(), new Object());
|
||||||
start = System.currentTimeMillis();
|
start = System.currentTimeMillis();
|
||||||
final String cacheId = UUID.randomUUID().toString();
|
ListenableFuture<?> future = manager.schedule(namespace);
|
||||||
ListenableFuture<?> future = manager.schedule(
|
|
||||||
namespace, factory, new Runnable()
|
Assert.assertFalse(future.isDone());
|
||||||
{
|
Assert.assertFalse(future.isCancelled());
|
||||||
@Override
|
|
||||||
public void run()
|
final long preRunCount;
|
||||||
{
|
final Object cacheUpdateAlerter = cacheUpdateAlerts.get(namespace.getNamespace());
|
||||||
try {
|
synchronized (cacheUpdateAlerter) {
|
||||||
manager.getPostRunnable(namespace, factory, cacheId).run();
|
preRunCount = numRuns.get();
|
||||||
ranCount.incrementAndGet();
|
}
|
||||||
}
|
for (; ; ) {
|
||||||
finally {
|
synchronized (cacheUpdateAlerter) {
|
||||||
latch.countDown();
|
if (numRuns.get() - preRunCount >= repeatCount) {
|
||||||
}
|
break;
|
||||||
}
|
} else {
|
||||||
},
|
cacheUpdateAlerter.wait();
|
||||||
cacheId
|
}
|
||||||
);
|
}
|
||||||
latch.await();
|
}
|
||||||
|
|
||||||
long minEnd = start + ((repeatCount - 1) * delay);
|
long minEnd = start + ((repeatCount - 1) * delay);
|
||||||
long end = System.currentTimeMillis();
|
long end = System.currentTimeMillis();
|
||||||
Assert.assertTrue(
|
Assert.assertTrue(
|
||||||
|
@ -205,148 +251,175 @@ public class NamespaceExtractionCacheManagerExecutorsTest
|
||||||
"Didn't wait long enough between runs. Expected more than %d was %d",
|
"Didn't wait long enough between runs. Expected more than %d was %d",
|
||||||
minEnd - start,
|
minEnd - start,
|
||||||
end - start
|
end - start
|
||||||
), minEnd < end
|
), minEnd <= end
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
lifecycle.stop();
|
lifecycle.stop();
|
||||||
}
|
}
|
||||||
totalRunCount = ranCount.get();
|
|
||||||
Thread.sleep(50);
|
totalRunCount = numRuns.get();
|
||||||
Assert.assertEquals(totalRunCount, ranCount.get(), 1);
|
Thread.sleep(delay * 10);
|
||||||
|
Assert.assertEquals(totalRunCount, numRuns.get(), 1);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test(timeout = 50_000)
|
@Test(timeout = 600_000) // This is very fast when run locally. Speed on Travis completely depends on noisy neighbors.
|
||||||
public void testConcurrentDelete() throws ExecutionException, InterruptedException
|
public void testConcurrentAddDelete() throws ExecutionException, InterruptedException, TimeoutException
|
||||||
{
|
{
|
||||||
final int threads = 5;
|
final int threads = 10;
|
||||||
ListeningExecutorService executorService = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(threads));
|
final int deletesPerThread = 5;
|
||||||
|
ListeningExecutorService executorService = MoreExecutors.listeningDecorator(
|
||||||
|
Execs.multiThreaded(
|
||||||
|
threads,
|
||||||
|
"concurrentTestingPool-%s"
|
||||||
|
)
|
||||||
|
);
|
||||||
final CountDownLatch latch = new CountDownLatch(threads);
|
final CountDownLatch latch = new CountDownLatch(threads);
|
||||||
Collection<ListenableFuture<?>> futures = new ArrayList<>();
|
Collection<ListenableFuture<?>> futures = new ArrayList<>();
|
||||||
for (int i = 0; i < threads; ++i) {
|
for (int i = 0; i < threads; ++i) {
|
||||||
final int loopNum = i;
|
final int ii = i;
|
||||||
ListenableFuture<?> future = executorService.submit(
|
futures.add(
|
||||||
new Runnable()
|
executorService.submit(
|
||||||
{
|
new Runnable()
|
||||||
@Override
|
{
|
||||||
public void run()
|
@Override
|
||||||
{
|
public void run()
|
||||||
try {
|
{
|
||||||
latch.countDown();
|
try {
|
||||||
latch.await();
|
latch.countDown();
|
||||||
for (int j = 0; j < 10; ++j) {
|
if (!latch.await(5, TimeUnit.SECONDS)) {
|
||||||
testDelete(String.format("ns-%d", loopNum));
|
throw new RuntimeException(new TimeoutException("Took too long to wait for more tasks"));
|
||||||
|
}
|
||||||
|
for (int j = 0; j < deletesPerThread; ++j) {
|
||||||
|
testDelete(String.format("ns-%d-%d", ii, j));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (InterruptedException e) {
|
||||||
|
Thread.currentThread().interrupt();
|
||||||
|
throw Throwables.propagate(e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
)
|
||||||
throw Throwables.propagate(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
Futures.allAsList(futures).get();
|
// Create an all-encompassing exception if any of them failed
|
||||||
executorService.shutdown();
|
final Collection<Exception> exceptions = new ArrayList<>();
|
||||||
}
|
|
||||||
|
|
||||||
@Test(timeout = 50_000)
|
|
||||||
public void testDelete()
|
|
||||||
throws NoSuchFieldException, IllegalAccessException, InterruptedException, ExecutionException
|
|
||||||
{
|
|
||||||
try {
|
try {
|
||||||
testDelete("ns");
|
for (ListenableFuture<?> future : futures) {
|
||||||
|
try {
|
||||||
|
future.get();
|
||||||
|
}
|
||||||
|
catch (InterruptedException e) {
|
||||||
|
Thread.currentThread().interrupt();
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
exceptions.add(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (!exceptions.isEmpty()) {
|
||||||
|
final RuntimeException e = new RuntimeException("Futures failed");
|
||||||
|
for (Exception ex : exceptions) {
|
||||||
|
e.addSuppressed(ex);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
lifecycle.stop();
|
executorService.shutdownNow();
|
||||||
}
|
}
|
||||||
|
checkNoMoreRunning();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout = 60_000L)
|
||||||
|
public void testSimpleDelete() throws InterruptedException
|
||||||
|
{
|
||||||
|
testDelete("someNamespace");
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testDelete(final String ns)
|
public void testDelete(final String ns)
|
||||||
throws InterruptedException
|
throws InterruptedException
|
||||||
{
|
{
|
||||||
final CountDownLatch latch = new CountDownLatch(5);
|
cacheUpdateAlerts.putIfAbsent(ns, new Object());
|
||||||
final CountDownLatch latchMore = new CountDownLatch(10);
|
final Object cacheUpdateAlerter = cacheUpdateAlerts.get(ns);
|
||||||
|
|
||||||
final AtomicLong runs = new AtomicLong(0);
|
final long period = 1_000L;// Give it some time between attempts to update
|
||||||
long prior = 0;
|
|
||||||
final URIExtractionNamespace namespace = new URIExtractionNamespace(
|
final URIExtractionNamespace namespace = new URIExtractionNamespace(
|
||||||
ns,
|
ns,
|
||||||
tmpFile.toURI(),
|
tmpFile.toURI(),
|
||||||
new URIExtractionNamespace.ObjectMapperFlatDataParser(
|
new URIExtractionNamespace.ObjectMapperFlatDataParser(
|
||||||
URIExtractionNamespaceTest.registerTypes(new ObjectMapper())
|
URIExtractionNamespaceTest.registerTypes(new ObjectMapper())
|
||||||
),
|
),
|
||||||
new Period(1l),
|
new Period(period),
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
final String cacheId = UUID.randomUUID().toString();
|
final ListenableFuture<?> future = manager.schedule(namespace);
|
||||||
final CountDownLatch latchBeforeMore = new CountDownLatch(1);
|
|
||||||
ListenableFuture<?> future =
|
|
||||||
manager.schedule(
|
|
||||||
namespace, factory, new Runnable()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public void run()
|
|
||||||
{
|
|
||||||
try {
|
|
||||||
if (!Thread.interrupted()) {
|
|
||||||
manager.getPostRunnable(namespace, factory, cacheId).run();
|
|
||||||
} else {
|
|
||||||
Thread.currentThread().interrupt();
|
|
||||||
}
|
|
||||||
if (!Thread.interrupted()) {
|
|
||||||
runs.incrementAndGet();
|
|
||||||
} else {
|
|
||||||
Thread.currentThread().interrupt();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
finally {
|
|
||||||
latch.countDown();
|
|
||||||
try {
|
|
||||||
if (latch.getCount() == 0) {
|
|
||||||
latchBeforeMore.await();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
catch (InterruptedException e) {
|
|
||||||
log.debug("Interrupted");
|
|
||||||
Thread.currentThread().interrupt();
|
|
||||||
}
|
|
||||||
finally {
|
|
||||||
latchMore.countDown();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
},
|
|
||||||
cacheId
|
|
||||||
);
|
|
||||||
latch.await();
|
|
||||||
prior = runs.get();
|
|
||||||
latchBeforeMore.countDown();
|
|
||||||
Assert.assertFalse(future.isCancelled());
|
Assert.assertFalse(future.isCancelled());
|
||||||
Assert.assertFalse(future.isDone());
|
Assert.assertFalse(future.isDone());
|
||||||
Assert.assertTrue(fnCache.containsKey(ns));
|
|
||||||
latchMore.await();
|
|
||||||
Assert.assertTrue(runs.get() > prior);
|
|
||||||
|
|
||||||
|
long start = 0L;
|
||||||
|
|
||||||
|
final long timeout = 45_000L;
|
||||||
|
do {
|
||||||
|
synchronized (cacheUpdateAlerter) {
|
||||||
|
if (!fnCache.containsKey(ns)) {
|
||||||
|
cacheUpdateAlerter.wait(10_000);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (future.isDone()) {
|
||||||
|
try {
|
||||||
|
// Bubble up the exception
|
||||||
|
Assert.assertNull(future.get());
|
||||||
|
Assert.fail("Task finished");
|
||||||
|
}
|
||||||
|
catch (ExecutionException e) {
|
||||||
|
throw Throwables.propagate(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (!fnCache.containsKey(ns) && System.currentTimeMillis() - start > timeout) {
|
||||||
|
throw new RuntimeException(
|
||||||
|
new TimeoutException(
|
||||||
|
String.format(
|
||||||
|
"Namespace took too long to appear in cache for %s",
|
||||||
|
namespace
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
} while (!fnCache.containsKey(ns));
|
||||||
|
|
||||||
|
Assert.assertEquals(VALUE, manager.getCacheMap(ns).get(KEY));
|
||||||
|
|
||||||
|
Assert.assertTrue(fnCache.containsKey(ns));
|
||||||
Assert.assertTrue(manager.implData.containsKey(ns));
|
Assert.assertTrue(manager.implData.containsKey(ns));
|
||||||
|
|
||||||
manager.delete("ns");
|
Assert.assertTrue(manager.delete(ns));
|
||||||
|
|
||||||
|
try {
|
||||||
|
Assert.assertNull(future.get());
|
||||||
|
}
|
||||||
|
catch (CancellationException e) {
|
||||||
|
// Ignore
|
||||||
|
}
|
||||||
|
catch (ExecutionException e) {
|
||||||
|
if (!future.isCancelled()) {
|
||||||
|
throw Throwables.propagate(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
Assert.assertFalse(manager.implData.containsKey(ns));
|
Assert.assertFalse(manager.implData.containsKey(ns));
|
||||||
Assert.assertFalse(fnCache.containsKey(ns));
|
Assert.assertFalse(fnCache.containsKey(ns));
|
||||||
Assert.assertTrue(future.isCancelled());
|
Assert.assertTrue(future.isCancelled());
|
||||||
Assert.assertTrue(future.isDone());
|
Assert.assertTrue(future.isDone());
|
||||||
prior = runs.get();
|
|
||||||
Thread.sleep(20);
|
|
||||||
Assert.assertEquals(prior, runs.get());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 50_000)
|
@Test(timeout = 60_000)
|
||||||
public void testShutdown()
|
public void testShutdown()
|
||||||
throws NoSuchFieldException, IllegalAccessException, InterruptedException, ExecutionException
|
throws NoSuchFieldException, IllegalAccessException, InterruptedException, ExecutionException
|
||||||
{
|
{
|
||||||
final CountDownLatch latch = new CountDownLatch(1);
|
final long period = 5L;
|
||||||
final ListenableFuture future;
|
final ListenableFuture future;
|
||||||
final AtomicLong runs = new AtomicLong(0);
|
|
||||||
long prior = 0;
|
long prior = 0;
|
||||||
try {
|
try {
|
||||||
|
|
||||||
|
@ -356,43 +429,34 @@ public class NamespaceExtractionCacheManagerExecutorsTest
|
||||||
new URIExtractionNamespace.ObjectMapperFlatDataParser(
|
new URIExtractionNamespace.ObjectMapperFlatDataParser(
|
||||||
URIExtractionNamespaceTest.registerTypes(new ObjectMapper())
|
URIExtractionNamespaceTest.registerTypes(new ObjectMapper())
|
||||||
),
|
),
|
||||||
new Period(1l),
|
new Period(period),
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
final String cacheId = UUID.randomUUID().toString();
|
cacheUpdateAlerts.putIfAbsent(namespace.getNamespace(), new Object());
|
||||||
final Runnable runnable = manager.getPostRunnable(namespace, factory, cacheId);
|
|
||||||
future =
|
future = manager.schedule(namespace);
|
||||||
manager.schedule(
|
|
||||||
namespace, factory, new Runnable()
|
final Object cacheUpdateAlerter = cacheUpdateAlerts.get(namespace.getNamespace());
|
||||||
{
|
synchronized (cacheUpdateAlerter) {
|
||||||
@Override
|
cacheUpdateAlerter.wait();
|
||||||
public void run()
|
}
|
||||||
{
|
|
||||||
runnable.run();
|
|
||||||
latch.countDown();
|
|
||||||
runs.incrementAndGet();
|
|
||||||
}
|
|
||||||
},
|
|
||||||
cacheId
|
|
||||||
);
|
|
||||||
|
|
||||||
latch.await();
|
|
||||||
Assert.assertFalse(future.isCancelled());
|
Assert.assertFalse(future.isCancelled());
|
||||||
Assert.assertFalse(future.isDone());
|
Assert.assertFalse(future.isDone());
|
||||||
prior = runs.get();
|
|
||||||
while (runs.get() <= prior) {
|
synchronized (cacheUpdateAlerter) {
|
||||||
Thread.sleep(50);
|
prior = numRuns.get();
|
||||||
|
cacheUpdateAlerter.wait();
|
||||||
}
|
}
|
||||||
Assert.assertTrue(runs.get() > prior);
|
Assert.assertTrue(numRuns.get() > prior);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
lifecycle.stop();
|
lifecycle.stop();
|
||||||
}
|
}
|
||||||
manager.waitForServiceToEnd(1_000, TimeUnit.MILLISECONDS);
|
while (!manager.waitForServiceToEnd(1_000, TimeUnit.MILLISECONDS)) {
|
||||||
|
}
|
||||||
|
|
||||||
prior = runs.get();
|
checkNoMoreRunning();
|
||||||
Thread.sleep(50);
|
|
||||||
Assert.assertEquals(prior, runs.get());
|
|
||||||
|
|
||||||
Field execField = NamespaceExtractionCacheManager.class.getDeclaredField("listeningScheduledExecutorService");
|
Field execField = NamespaceExtractionCacheManager.class.getDeclaredField("listeningScheduledExecutorService");
|
||||||
execField.setAccessible(true);
|
execField.setAccessible(true);
|
||||||
|
@ -400,62 +464,48 @@ public class NamespaceExtractionCacheManagerExecutorsTest
|
||||||
Assert.assertTrue(((ListeningScheduledExecutorService) execField.get(manager)).isTerminated());
|
Assert.assertTrue(((ListeningScheduledExecutorService) execField.get(manager)).isTerminated());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 50_000)
|
@Test(timeout = 60_000)
|
||||||
public void testRunCount()
|
public void testRunCount()
|
||||||
throws InterruptedException, ExecutionException
|
throws InterruptedException, ExecutionException
|
||||||
{
|
{
|
||||||
final Lifecycle lifecycle = new Lifecycle();
|
final long numWaits = 5;
|
||||||
final NamespaceExtractionCacheManager onHeap;
|
final ListenableFuture<?> future;
|
||||||
final AtomicLong runCount = new AtomicLong(0);
|
|
||||||
final CountDownLatch latch = new CountDownLatch(1);
|
|
||||||
try {
|
try {
|
||||||
onHeap = new OnHeapNamespaceExtractionCacheManager(
|
|
||||||
lifecycle,
|
|
||||||
new ConcurrentHashMap<String, Function<String, String>>(),
|
|
||||||
new NoopServiceEmitter(),
|
|
||||||
ImmutableMap.<Class<? extends ExtractionNamespace>, ExtractionNamespaceFunctionFactory<?>>of(
|
|
||||||
URIExtractionNamespace.class,
|
|
||||||
new URIExtractionNamespaceFunctionFactory(
|
|
||||||
ImmutableMap.<String, SearchableVersionedDataFinder>of(
|
|
||||||
"file",
|
|
||||||
new LocalFileTimestampVersionFinder()
|
|
||||||
)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
);
|
|
||||||
|
|
||||||
|
|
||||||
final URIExtractionNamespace namespace = new URIExtractionNamespace(
|
final URIExtractionNamespace namespace = new URIExtractionNamespace(
|
||||||
"ns",
|
"ns",
|
||||||
tmpFile.toURI(),
|
tmpFile.toURI(),
|
||||||
new URIExtractionNamespace.ObjectMapperFlatDataParser(
|
new URIExtractionNamespace.ObjectMapperFlatDataParser(
|
||||||
URIExtractionNamespaceTest.registerTypes(new ObjectMapper())
|
URIExtractionNamespaceTest.registerTypes(new ObjectMapper())
|
||||||
),
|
),
|
||||||
new Period(1l),
|
new Period(5l),
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
final String cacheId = UUID.randomUUID().toString();
|
|
||||||
ListenableFuture<?> future =
|
cacheUpdateAlerts.putIfAbsent(namespace.getNamespace(), new Object());
|
||||||
onHeap.schedule(
|
future = manager.schedule(namespace);
|
||||||
namespace, factory, new Runnable()
|
Assert.assertFalse(future.isDone());
|
||||||
{
|
|
||||||
@Override
|
final Object cacheUpdateAlerter = cacheUpdateAlerts.get(namespace.getNamespace());
|
||||||
public void run()
|
for (int i = 0; i < numWaits; ++i) {
|
||||||
{
|
synchronized (cacheUpdateAlerter) {
|
||||||
manager.getPostRunnable(namespace, factory, cacheId).run();
|
cacheUpdateAlerter.wait();
|
||||||
latch.countDown();
|
}
|
||||||
runCount.incrementAndGet();
|
}
|
||||||
}
|
Assert.assertFalse(future.isDone());
|
||||||
},
|
|
||||||
cacheId
|
|
||||||
);
|
|
||||||
latch.await();
|
|
||||||
Thread.sleep(20);
|
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
lifecycle.stop();
|
lifecycle.stop();
|
||||||
}
|
}
|
||||||
onHeap.waitForServiceToEnd(1_000, TimeUnit.MILLISECONDS);
|
while (!manager.waitForServiceToEnd(1_000, TimeUnit.MILLISECONDS)) {
|
||||||
Assert.assertTrue(runCount.get() > 5);
|
}
|
||||||
|
Assert.assertTrue(numRuns.get() >= numWaits);
|
||||||
|
checkNoMoreRunning();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void checkNoMoreRunning() throws InterruptedException
|
||||||
|
{
|
||||||
|
final long pre = numRuns.get();
|
||||||
|
Thread.sleep(100L);
|
||||||
|
Assert.assertEquals(pre, numRuns.get(), 1); // since we don't synchronize here we might have an extra increment
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue