mirror of https://github.com/apache/druid.git
Fix sporadic fail of URIExtractionNamespaceFunctionFactoryTest#testReverseFunction
This commit is contained in:
parent
628643d80e
commit
31b205afcd
|
@ -336,129 +336,43 @@ public abstract class NamespaceExtractionCacheManager
|
||||||
{
|
{
|
||||||
final String namespaceName = namespace.getNamespace();
|
final String namespaceName = namespace.getNamespace();
|
||||||
log.debug("Trying to update namespace [%s]", namespaceName);
|
log.debug("Trying to update namespace [%s]", namespaceName);
|
||||||
final AtomicReference<NamespaceImplData> implDatum = new AtomicReference<>(implData.get(namespaceName));
|
final NamespaceImplData implDatum = implData.get(namespaceName);
|
||||||
if (implDatum.get() != null) {
|
if (implDatum != null) {
|
||||||
synchronized (implDatum.get().enabled) {
|
synchronized (implDatum.enabled) {
|
||||||
if (implDatum.get().enabled.get()) {
|
if (implDatum.enabled.get()) {
|
||||||
// We also check at the end of the function, but fail fast here
|
// We also check at the end of the function, but fail fast here
|
||||||
throw new IAE("Namespace [%s] already exists! Leaving prior running", namespace.toString());
|
throw new IAE("Namespace [%s] already exists! Leaving prior running", namespace.toString());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
final CountDownLatch startLatch = new CountDownLatch(1);
|
|
||||||
try {
|
|
||||||
ListenableFuture<?> future = null;
|
|
||||||
try {
|
|
||||||
if (namespace.getPollMs() > 0) {
|
|
||||||
final long updateMs = namespace.getPollMs();
|
final long updateMs = namespace.getPollMs();
|
||||||
future = listeningScheduledExecutorService.scheduleAtFixedRate(
|
final CountDownLatch startLatch = new CountDownLatch(1);
|
||||||
new Runnable()
|
|
||||||
|
final Runnable command = new Runnable()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public void run()
|
public void run()
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
startLatch.await(); // wait for "election" to leadership or cancellation
|
startLatch.await(); // wait for "election" to leadership or cancellation
|
||||||
if (!Thread.interrupted()) {
|
if (!Thread.currentThread().isInterrupted()) {
|
||||||
final Map<String, String> cache = getCacheMap(cacheId);
|
final Map<String, String> cache = getCacheMap(cacheId);
|
||||||
final String preVersion = lastVersion.get(namespaceName);
|
final String preVersion = lastVersion.get(namespaceName);
|
||||||
final Callable<String> runnable = factory.getCachePopulator(namespace, preVersion, cache);
|
final Callable<String> runnable = factory.getCachePopulator(namespace, preVersion, cache);
|
||||||
|
|
||||||
tasksStarted.incrementAndGet();
|
tasksStarted.incrementAndGet();
|
||||||
final String newVersion = runnable.call();
|
final String newVersion = runnable.call();
|
||||||
|
if (preVersion != null && preVersion.equals(newVersion)) {
|
||||||
|
throw new IllegalStateException("Already exists");
|
||||||
|
}
|
||||||
if (newVersion != null) {
|
if (newVersion != null) {
|
||||||
lastVersion.put(namespaceName, newVersion);
|
lastVersion.put(namespaceName, newVersion);
|
||||||
}
|
}
|
||||||
if (preVersion == null || !preVersion.equals(lastVersion.get(namespaceName))) {
|
|
||||||
postRunnable.run();
|
postRunnable.run();
|
||||||
} else {
|
|
||||||
delete(cacheId);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
Thread.currentThread().interrupt();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
catch (Exception e) {
|
|
||||||
if (e instanceof CancellationException) {
|
|
||||||
log.debug("Thread for namespace[%s] cancelled", namespaceName);
|
|
||||||
} else {
|
|
||||||
log.error(e, "Error in listener for namespace [%s]", namespaceName);
|
|
||||||
}
|
|
||||||
// Don't leave stale cache on error
|
|
||||||
delete(cacheId);
|
|
||||||
throw Throwables.propagate(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
},
|
|
||||||
0,
|
|
||||||
updateMs,
|
|
||||||
TimeUnit.MILLISECONDS
|
|
||||||
);
|
|
||||||
} else {
|
|
||||||
final Map<String, String> cache = getCacheMap(cacheId);
|
|
||||||
final Callable<String> runnable = factory.getCachePopulator(namespace, null, cache);
|
|
||||||
final ListenableFuture<String> futureWithString = listeningScheduledExecutorService.schedule(
|
|
||||||
new Callable<String>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public String call() throws Exception
|
|
||||||
{
|
|
||||||
startLatch.await(); // wait for "election" to leadership or cancellation
|
|
||||||
tasksStarted.incrementAndGet();
|
|
||||||
return runnable.call();
|
|
||||||
}
|
|
||||||
},
|
|
||||||
0,
|
|
||||||
TimeUnit.MILLISECONDS
|
|
||||||
);
|
|
||||||
Futures.addCallback(
|
|
||||||
futureWithString, new FutureCallback<String>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public void onSuccess(String result)
|
|
||||||
{
|
|
||||||
try {
|
|
||||||
postRunnable.run();
|
|
||||||
}
|
|
||||||
catch (RuntimeException e) {
|
|
||||||
delete(cacheId);
|
|
||||||
throw e;
|
|
||||||
}
|
|
||||||
// Must have been set in order to make it here
|
|
||||||
if (implDatum.get().enabled.get()) {
|
|
||||||
lastVersion.put(namespaceName, result);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void onFailure(Throwable t)
|
|
||||||
{
|
|
||||||
// NOOP
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
future = futureWithString;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
catch (Exception e) {
|
|
||||||
if (future != null) {
|
|
||||||
if (!future.isDone() && !future.cancel(true)) {
|
|
||||||
log.warn("Could not cancel future for [%s]", namespaceName);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
throw Throwables.propagate(e);
|
|
||||||
}
|
|
||||||
Futures.addCallback(
|
|
||||||
future, new FutureCallback<Object>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public void onSuccess(@Nullable Object result)
|
|
||||||
{
|
|
||||||
log.debug("Namespace [%s] successfully updated", namespaceName);
|
log.debug("Namespace [%s] successfully updated", namespaceName);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
@Override
|
catch (Throwable t) {
|
||||||
public void onFailure(Throwable t)
|
|
||||||
{
|
|
||||||
delete(cacheId);
|
delete(cacheId);
|
||||||
if (t instanceof CancellationException) {
|
if (t instanceof CancellationException) {
|
||||||
log.debug(t, "Namespace [%s] cancelled", namespaceName);
|
log.debug(t, "Namespace [%s] cancelled", namespaceName);
|
||||||
|
@ -467,19 +381,27 @@ public abstract class NamespaceExtractionCacheManager
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
};
|
||||||
|
|
||||||
|
ListenableFuture<?> future;
|
||||||
|
try {
|
||||||
|
if (updateMs > 0) {
|
||||||
|
future = listeningScheduledExecutorService.scheduleAtFixedRate(command, 0, updateMs, TimeUnit.MILLISECONDS);
|
||||||
|
} else {
|
||||||
|
future = listeningScheduledExecutorService.schedule(command, 0, TimeUnit.MILLISECONDS);
|
||||||
|
}
|
||||||
|
|
||||||
final NamespaceImplData me = new NamespaceImplData(future, namespace, namespaceName);
|
final NamespaceImplData me = new NamespaceImplData(future, namespace, namespaceName);
|
||||||
final NamespaceImplData other = implData.putIfAbsent(namespaceName, me);
|
final NamespaceImplData other = implData.putIfAbsent(namespaceName, me);
|
||||||
if (other != null) {
|
if (other != null) {
|
||||||
if (!future.isDone() && !future.cancel(true)) {
|
if (!future.isDone() && !future.cancel(true)) {
|
||||||
log.warn("Unable to cancle future for namespace[%s] on race loss", namespaceName);
|
log.warn("Unable to cancel future for namespace[%s] on race loss", namespaceName);
|
||||||
}
|
}
|
||||||
throw new IAE("Namespace [%s] already exists! Leaving prior running", namespace);
|
throw new IAE("Namespace [%s] already exists! Leaving prior running", namespace);
|
||||||
} else {
|
} else {
|
||||||
if (!me.enabled.compareAndSet(false, true)) {
|
if (!me.enabled.compareAndSet(false, true)) {
|
||||||
log.wtf("How did someone enable this before ME?");
|
log.wtf("How did someone enable this before ME?");
|
||||||
}
|
}
|
||||||
implDatum.set(me);
|
|
||||||
log.debug("I own namespace [%s]", namespaceName);
|
log.debug("I own namespace [%s]", namespaceName);
|
||||||
return future;
|
return future;
|
||||||
}
|
}
|
||||||
|
|
|
@ -308,10 +308,7 @@ public class URIExtractionNamespaceFunctionFactoryTest
|
||||||
Assert.assertNull(fnCache.get(namespace.getNamespace()));
|
Assert.assertNull(fnCache.get(namespace.getNamespace()));
|
||||||
NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(namespace));
|
NamespaceExtractionCacheManagersTest.waitFor(manager.schedule(namespace));
|
||||||
Function<String, String> fn = fnCache.get(namespace.getNamespace());
|
Function<String, String> fn = fnCache.get(namespace.getNamespace());
|
||||||
while (fn == null) {
|
Assert.assertNotNull(fn);
|
||||||
Thread.sleep(1);
|
|
||||||
fn = fnCache.get(namespace.getNamespace());
|
|
||||||
}
|
|
||||||
Assert.assertEquals("bar", fn.apply("foo"));
|
Assert.assertEquals("bar", fn.apply("foo"));
|
||||||
Assert.assertEquals(null, fn.apply("baz"));
|
Assert.assertEquals(null, fn.apply("baz"));
|
||||||
}
|
}
|
||||||
|
@ -355,10 +352,7 @@ public class URIExtractionNamespaceFunctionFactoryTest
|
||||||
for (int i = 0; i < size; ++i) {
|
for (int i = 0; i < size; ++i) {
|
||||||
URIExtractionNamespace namespace = namespaces.get(i);
|
URIExtractionNamespace namespace = namespaces.get(i);
|
||||||
Function<String, String> fn = fnCache.get(namespace.getNamespace());
|
Function<String, String> fn = fnCache.get(namespace.getNamespace());
|
||||||
while (fn == null) {
|
Assert.assertNotNull(fn);
|
||||||
Thread.sleep(1);
|
|
||||||
fn = fnCache.get(namespace.getNamespace());
|
|
||||||
}
|
|
||||||
Assert.assertEquals("bar", fn.apply("foo"));
|
Assert.assertEquals("bar", fn.apply("foo"));
|
||||||
Assert.assertEquals(null, fn.apply("baz"));
|
Assert.assertEquals(null, fn.apply("baz"));
|
||||||
manager.delete(namespace.getNamespace());
|
manager.delete(namespace.getNamespace());
|
||||||
|
|
|
@ -67,6 +67,7 @@ import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
*
|
||||||
|
|
|
@ -46,6 +46,8 @@ import java.util.List;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
import java.util.concurrent.ExecutionException;
|
||||||
|
import java.util.concurrent.Future;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
*
|
||||||
|
@ -145,31 +147,16 @@ public class NamespaceExtractionCacheManagersTest
|
||||||
Assert.assertArrayEquals(nsList.toArray(), retvalList.toArray());
|
Assert.assertArrayEquals(nsList.toArray(), retvalList.toArray());
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void waitFor(ListenableFuture<?> future) throws InterruptedException
|
public static void waitFor(Future<?> future) throws InterruptedException
|
||||||
{
|
|
||||||
final CountDownLatch latch = new CountDownLatch(1);
|
|
||||||
Futures.addCallback(
|
|
||||||
future, new FutureCallback<Object>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public void onSuccess(Object result)
|
|
||||||
{
|
|
||||||
latch.countDown();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void onFailure(Throwable t)
|
|
||||||
{
|
{
|
||||||
|
while (!future.isDone()) {
|
||||||
try {
|
try {
|
||||||
log.error(t, "Error waiting");
|
future.get();
|
||||||
throw Throwables.propagate(t);
|
|
||||||
}
|
}
|
||||||
finally {
|
catch (ExecutionException e) {
|
||||||
latch.countDown();
|
log.error(e.getCause(), "Error waiting");
|
||||||
|
throw Throwables.propagate(e.getCause());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
|
||||||
latch.await();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue