mirror of https://github.com/apache/druid.git
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:
parent
0799640299
commit
78159d7ca4
|
@ -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()
|
||||||
|
|
|
@ -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();
|
||||||
|
|
|
@ -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();
|
||||||
|
|
|
@ -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()
|
||||||
|
|
Loading…
Reference in New Issue