Move off-heap QTL global cache delete lock outside of subclass lock (#3597)

* Move off-heap QTL global cache delete lock outside of subclass lock

* Make `delete` thread safe
This commit is contained in:
Charles Allen 2016-10-27 09:53:53 -07:00 committed by Nishant
parent 0799640299
commit 78159d7ca4
4 changed files with 104 additions and 81 deletions

View File

@ -37,6 +37,7 @@ import io.druid.java.util.common.logger.Logger;
import io.druid.query.lookup.namespace.ExtractionNamespace; import io.druid.query.lookup.namespace.ExtractionNamespace;
import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory; import io.druid.query.lookup.namespace.ExtractionNamespaceCacheFactory;
import javax.annotation.concurrent.GuardedBy;
import java.util.Collection; import java.util.Collection;
import java.util.Map; import java.util.Map;
import java.util.UUID; import java.util.UUID;
@ -49,6 +50,7 @@ import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
/** /**
* *
@ -71,8 +73,10 @@ public abstract class NamespaceExtractionCacheManager
final ListenableFuture<?> future; final ListenableFuture<?> future;
final ExtractionNamespace namespace; final ExtractionNamespace namespace;
final String name; final String name;
final Object changeLock = new Object();
final AtomicBoolean enabled = new AtomicBoolean(false); final AtomicBoolean enabled = new AtomicBoolean(false);
final CountDownLatch firstRun = new CountDownLatch(1); final CountDownLatch firstRun = new CountDownLatch(1);
final AtomicReference<String> latestVersion = new AtomicReference<>(null);
} }
private static final Logger log = new Logger(NamespaceExtractionCacheManager.class); private static final Logger log = new Logger(NamespaceExtractionCacheManager.class);
@ -80,7 +84,6 @@ public abstract class NamespaceExtractionCacheManager
protected final ConcurrentMap<String, NamespaceImplData> implData = new ConcurrentHashMap<>(); protected final ConcurrentMap<String, NamespaceImplData> implData = new ConcurrentHashMap<>();
protected final AtomicLong tasksStarted = new AtomicLong(0); protected final AtomicLong tasksStarted = new AtomicLong(0);
protected final ServiceEmitter serviceEmitter; protected final ServiceEmitter serviceEmitter;
private final ConcurrentHashMap<String, String> lastVersion = new ConcurrentHashMap<>();
private final Map<Class<? extends ExtractionNamespace>, ExtractionNamespaceCacheFactory<?>> namespaceFunctionFactoryMap; private final Map<Class<? extends ExtractionNamespace>, ExtractionNamespaceCacheFactory<?>> namespaceFunctionFactoryMap;
public NamespaceExtractionCacheManager( public NamespaceExtractionCacheManager(
@ -148,10 +151,8 @@ public abstract class NamespaceExtractionCacheManager
} }
protected <T extends ExtractionNamespace> Runnable getPostRunnable( protected Runnable getPostRunnable(
final String id, final String id,
final T namespace,
final ExtractionNamespaceCacheFactory<T> factory,
final String cacheId final String cacheId
) )
{ {
@ -165,17 +166,20 @@ public abstract class NamespaceExtractionCacheManager
// was removed // was removed
return; return;
} }
synchronized (namespaceDatum.enabled) { try {
try { if (!namespaceDatum.enabled.get()) {
// skip because it was disabled
return;
}
synchronized (namespaceDatum.enabled) {
if (!namespaceDatum.enabled.get()) { if (!namespaceDatum.enabled.get()) {
// skip because it was disabled
return; return;
} }
swapAndClearCache(id, cacheId); swapAndClearCache(id, cacheId);
} }
finally { }
namespaceDatum.firstRun.countDown(); finally {
} namespaceDatum.firstRun.countDown();
} }
} }
}; };
@ -221,7 +225,10 @@ public abstract class NamespaceExtractionCacheManager
if (log.isDebugEnabled()) { if (log.isDebugEnabled()) {
log.debug("Namespace [%s] needs updated to [%s]", implDatum.namespace, namespace); log.debug("Namespace [%s] needs updated to [%s]", implDatum.namespace, namespace);
} }
removeNamespaceLocalMetadata(implDatum); // Ensure it is not changing state right now.
synchronized (implDatum.changeLock) {
removeNamespaceLocalMetadata(implDatum);
}
schedule(id, namespace); schedule(id, namespace);
return true; return true;
} }
@ -257,59 +264,59 @@ public abstract class NamespaceExtractionCacheManager
return success; return success;
} }
@GuardedBy("implDatum.changeLock")
private void cancelFuture(final NamespaceImplData implDatum) private void cancelFuture(final NamespaceImplData implDatum)
{ {
synchronized (implDatum.enabled) { final CountDownLatch latch = new CountDownLatch(1);
final CountDownLatch latch = new CountDownLatch(1); final ListenableFuture<?> future = implDatum.future;
final ListenableFuture<?> future = implDatum.future; Futures.addCallback(
Futures.addCallback( future, new FutureCallback<Object>()
future, new FutureCallback<Object>() {
@Override
public void onSuccess(Object result)
{ {
@Override latch.countDown();
public void onSuccess(Object result) }
{
latch.countDown();
}
@Override @Override
public void onFailure(Throwable t) public void onFailure(Throwable t)
{ {
// Expect CancellationException // Expect CancellationException
latch.countDown(); latch.countDown();
if (!(t instanceof CancellationException)) { if (!(t instanceof CancellationException)) {
log.error(t, "Error in namespace [%s]", implDatum.name); log.error(t, "Error in namespace [%s]", implDatum.name);
}
} }
} }
); }
if (!future.isDone() );
&& !future.cancel(true)) { // Interrupt to make sure we don't pollute stuff after we've already cleaned up if (!future.isDone()
throw new ISE("Future for namespace [%s] was not able to be canceled", implDatum.name); && !future.cancel(true)) { // Interrupt to make sure we don't pollute stuff after we've already cleaned up
} throw new ISE("Future for namespace [%s] was not able to be canceled", implDatum.name);
try { }
latch.await(); try {
} latch.await();
catch (InterruptedException e) { }
Thread.currentThread().interrupt(); catch (InterruptedException e) {
throw Throwables.propagate(e); Thread.currentThread().interrupt();
} throw Throwables.propagate(e);
} }
} }
// Not thread safe
@GuardedBy("implDatum.changeLock")
private boolean removeNamespaceLocalMetadata(final NamespaceImplData implDatum) private boolean removeNamespaceLocalMetadata(final NamespaceImplData implDatum)
{ {
if (implDatum == null) { if (implDatum == null) {
return false; return false;
} }
synchronized (implDatum.enabled) { // "Leader" election for doing the deletion
if (!implDatum.enabled.compareAndSet(true, false)) { if (!implDatum.enabled.compareAndSet(true, false)) {
return false; return false;
}
if (!implDatum.future.isDone()) {
cancelFuture(implDatum);
}
return implData.remove(implDatum.name, implDatum);
} }
if (!implDatum.future.isDone()) {
cancelFuture(implDatum);
}
return implData.remove(implDatum.name, implDatum);
} }
// Optimistic scheduling of updates to a namespace. // Optimistic scheduling of updates to a namespace.
@ -321,7 +328,7 @@ public abstract class NamespaceExtractionCacheManager
throw new ISE("Cannot find factory for namespace [%s]", namespace); throw new ISE("Cannot find factory for namespace [%s]", namespace);
} }
final String cacheId = String.format("namespace-cache-%s-%s", id, UUID.randomUUID().toString()); final String cacheId = String.format("namespace-cache-%s-%s", id, UUID.randomUUID().toString());
return schedule(id, namespace, factory, getPostRunnable(id, namespace, factory, cacheId), cacheId); return schedule(id, namespace, factory, getPostRunnable(id, cacheId), cacheId);
} }
// For testing purposes this is protected // For testing purposes this is protected
@ -336,7 +343,7 @@ public abstract class NamespaceExtractionCacheManager
log.debug("Trying to update namespace [%s]", id); log.debug("Trying to update namespace [%s]", id);
final NamespaceImplData implDatum = implData.get(id); final NamespaceImplData implDatum = implData.get(id);
if (implDatum != null) { if (implDatum != null) {
synchronized (implDatum.enabled) { synchronized (implDatum.changeLock) {
if (implDatum.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());
@ -345,6 +352,8 @@ public abstract class NamespaceExtractionCacheManager
} }
final long updateMs = namespace.getPollMs(); final long updateMs = namespace.getPollMs();
final CountDownLatch startLatch = new CountDownLatch(1); final CountDownLatch startLatch = new CountDownLatch(1);
// Must be set before leader election occurs or else runnable will fail
final AtomicReference<NamespaceImplData> implDataAtomicReference = new AtomicReference<>(null);
final Runnable command = new Runnable() final Runnable command = new Runnable()
{ {
@ -354,8 +363,13 @@ public abstract class NamespaceExtractionCacheManager
try { try {
startLatch.await(); // wait for "election" to leadership or cancellation startLatch.await(); // wait for "election" to leadership or cancellation
if (!Thread.currentThread().isInterrupted()) { if (!Thread.currentThread().isInterrupted()) {
final NamespaceImplData implData = implDataAtomicReference.get();
if (implData == null) {
// should never happen
throw new NullPointerException(String.format("No data for namespace [%s]", id));
}
final Map<String, String> cache = getCacheMap(cacheId); final Map<String, String> cache = getCacheMap(cacheId);
final String preVersion = lastVersion.get(id); final String preVersion = implData.latestVersion.get();
final Callable<String> runnable = factory.getCachePopulator(id, namespace, preVersion, cache); final Callable<String> runnable = factory.getCachePopulator(id, namespace, preVersion, cache);
tasksStarted.incrementAndGet(); tasksStarted.incrementAndGet();
@ -364,7 +378,9 @@ public abstract class NamespaceExtractionCacheManager
throw new CancellationException(String.format("Version `%s` already exists", preVersion)); throw new CancellationException(String.format("Version `%s` already exists", preVersion));
} }
if (newVersion != null) { if (newVersion != null) {
lastVersion.put(id, newVersion); if (!implData.latestVersion.compareAndSet(preVersion, newVersion)) {
log.wtf("Somehow multiple threads are updating the same implData for [%s]", id);
}
} }
postRunnable.run(); postRunnable.run();
log.debug("Namespace [%s] successfully updated", id); log.debug("Namespace [%s] successfully updated", id);
@ -392,7 +408,9 @@ public abstract class NamespaceExtractionCacheManager
future = listeningScheduledExecutorService.schedule(command, 0, TimeUnit.MILLISECONDS); future = listeningScheduledExecutorService.schedule(command, 0, TimeUnit.MILLISECONDS);
} }
// Do not need to synchronize here as we haven't set enabled to true yet, and haven't released startLatch
final NamespaceImplData me = new NamespaceImplData(future, namespace, id); final NamespaceImplData me = new NamespaceImplData(future, namespace, id);
implDataAtomicReference.set(me);
final NamespaceImplData other = implData.putIfAbsent(id, me); final NamespaceImplData other = implData.putIfAbsent(id, me);
if (other != null) { if (other != null) {
if (!future.isDone() && !future.cancel(true)) { if (!future.isDone() && !future.cancel(true)) {
@ -433,8 +451,6 @@ public abstract class NamespaceExtractionCacheManager
/** /**
* Clears out resources used by the namespace such as threads. Implementations may override this and call super.delete(...) if they have resources of their own which need cleared. * Clears out resources used by the namespace such as threads. Implementations may override this and call super.delete(...) if they have resources of their own which need cleared.
* <p/>
* This particular method is NOT thread safe, and any impl which is intended to be thread safe should safe-guard calls to this method.
* *
* @param ns The namespace to be deleted * @param ns The namespace to be deleted
* *
@ -445,25 +461,31 @@ public abstract class NamespaceExtractionCacheManager
public boolean delete(final String ns) public boolean delete(final String ns)
{ {
final NamespaceImplData implDatum = implData.get(ns); final NamespaceImplData implDatum = implData.get(ns);
final boolean deleted = removeNamespaceLocalMetadata(implDatum); if (implDatum == null) {
// At this point we have won leader election on canceling this implDatum log.debug("Found no running cache for [%s]", ns);
if (deleted) {
log.info("Deleting namespace [%s]", ns);
lastVersion.remove(implDatum.name);
return true;
} else {
log.debug("Did not delete namespace [%s]", ns);
return false; return false;
} }
synchronized (implDatum.changeLock) {
if (removeNamespaceLocalMetadata(implDatum)) {
log.info("Deleted namespace [%s]", ns);
return true;
} else {
log.debug("Did not delete namespace [%s]", ns);
return false;
}
}
} }
public String getVersion(String namespace) public String getVersion(String namespace)
{ {
if (namespace == null) { if (namespace == null) {
return null; return null;
} else {
return lastVersion.get(namespace);
} }
final NamespaceImplData implDatum = implData.get(namespace);
if (implDatum == null) {
return null;
}
return implDatum.latestVersion.get();
} }
public Collection<String> getKnownIDs() public Collection<String> getKnownIDs()

View File

@ -134,22 +134,21 @@ public class OffHeapNamespaceExtractionCacheManager extends NamespaceExtractionC
@Override @Override
public boolean delete(final String namespaceKey) public boolean delete(final String namespaceKey)
{ {
// `super.delete` has a synchronization in it, don't call it in the lock.
if (!super.delete(namespaceKey)) {
return false;
}
final Lock lock = nsLocks.get(namespaceKey); final Lock lock = nsLocks.get(namespaceKey);
lock.lock(); lock.lock();
try { try {
if (super.delete(namespaceKey)) { final String mmapDBkey = currentNamespaceCache.remove(namespaceKey);
final String mmapDBkey = currentNamespaceCache.remove(namespaceKey); if (mmapDBkey == null) {
if (mmapDBkey != null) {
final long pre = tmpFile.length();
mmapDB.delete(mmapDBkey);
log.debug("MapDB file size: pre %d post %d", pre, tmpFile.length());
return true;
} else {
return false;
}
} else {
return false; return false;
} }
final long pre = tmpFile.length();
mmapDB.delete(mmapDBkey);
log.debug("MapDB file size: pre %d post %d", pre, tmpFile.length());
return true;
} }
finally { finally {
lock.unlock(); lock.unlock();

View File

@ -93,10 +93,14 @@ public class OnHeapNamespaceExtractionCacheManager extends NamespaceExtractionCa
@Override @Override
public boolean delete(final String namespaceKey) public boolean delete(final String namespaceKey)
{ {
// `super.delete` has a synchronization in it, don't call it in the lock.
if (!super.delete(namespaceKey)) {
return false;
}
final Lock lock = nsLocks.get(namespaceKey); final Lock lock = nsLocks.get(namespaceKey);
lock.lock(); lock.lock();
try { try {
return super.delete(namespaceKey) && mapMap.remove(namespaceKey) != null; return mapMap.remove(namespaceKey) != null;
} }
finally { finally {
lock.unlock(); lock.unlock();

View File

@ -123,14 +123,12 @@ public class NamespaceExtractionCacheManagerExecutorsTest
) )
{ {
@Override @Override
protected <T extends ExtractionNamespace> Runnable getPostRunnable( protected Runnable getPostRunnable(
final String id, final String id,
final T namespace,
final ExtractionNamespaceCacheFactory<T> factory,
final String cacheId final String cacheId
) )
{ {
final Runnable runnable = super.getPostRunnable(id, namespace, factory, cacheId); final Runnable runnable = super.getPostRunnable(id, cacheId);
cacheUpdateAlerts.putIfAbsent(id, new Object()); cacheUpdateAlerts.putIfAbsent(id, new Object());
final Object cacheUpdateAlerter = cacheUpdateAlerts.get(id); final Object cacheUpdateAlerter = cacheUpdateAlerts.get(id);
return new Runnable() return new Runnable()