mirror of https://github.com/apache/druid.git
Double-checked locking bugs (#6662)
* Double-checked locking bug is fixed. * @Nullable is removed since there is no need to use along with @MonotonicNonNull. * Static import is removed. * Lazy initialization is implemented. * Local variables used instead of volatile ones. * Local variables used instead of volatile ones.
This commit is contained in:
parent
d525e5b18e
commit
bbb283fa34
|
@ -45,6 +45,11 @@
|
|||
<groupId>com.amazonaws</groupId>
|
||||
<artifactId>aws-java-sdk-s3</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.checkerframework</groupId>
|
||||
<artifactId>checker</artifactId>
|
||||
<version>${checkerframework.version}</version>
|
||||
</dependency>
|
||||
|
||||
<!-- Tests -->
|
||||
<dependency>
|
||||
|
|
|
@ -21,27 +21,43 @@ package org.apache.druid.common.aws;
|
|||
|
||||
import com.amazonaws.auth.AWSCredentials;
|
||||
import com.amazonaws.auth.AWSCredentialsProvider;
|
||||
import org.checkerframework.checker.nullness.qual.EnsuresNonNull;
|
||||
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
|
||||
|
||||
public class LazyFileSessionCredentialsProvider implements AWSCredentialsProvider
|
||||
{
|
||||
private AWSCredentialsConfig config;
|
||||
private FileSessionCredentialsProvider provider;
|
||||
private final AWSCredentialsConfig config;
|
||||
|
||||
/**
|
||||
* The field is declared volatile in order to ensure safe publication of the object
|
||||
* in {@link #getUnderlyingProvider()} without worrying about final modifiers
|
||||
* on the fields of the created object
|
||||
*
|
||||
* @see <a href="https://github.com/apache/incubator-druid/pull/6662#discussion_r237013157">
|
||||
* https://github.com/apache/incubator-druid/pull/6662#discussion_r237013157</a>
|
||||
*/
|
||||
@MonotonicNonNull
|
||||
private volatile FileSessionCredentialsProvider provider;
|
||||
|
||||
public LazyFileSessionCredentialsProvider(AWSCredentialsConfig config)
|
||||
{
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
@EnsuresNonNull("provider")
|
||||
private FileSessionCredentialsProvider getUnderlyingProvider()
|
||||
{
|
||||
if (provider == null) {
|
||||
FileSessionCredentialsProvider syncedProvider = provider;
|
||||
if (syncedProvider == null) {
|
||||
synchronized (config) {
|
||||
if (provider == null) {
|
||||
provider = new FileSessionCredentialsProvider(config.getFileSessionCredentials());
|
||||
syncedProvider = provider;
|
||||
if (syncedProvider == null) {
|
||||
syncedProvider = new FileSessionCredentialsProvider(config.getFileSessionCredentials());
|
||||
provider = syncedProvider;
|
||||
}
|
||||
}
|
||||
}
|
||||
return provider;
|
||||
return syncedProvider;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
1
pom.xml
1
pom.xml
|
@ -100,6 +100,7 @@
|
|||
<caffeine.version>2.5.5</caffeine.version>
|
||||
<!-- When upgrading ZK, edit docs and integration tests as well (integration-tests/docker-base/setup.sh) -->
|
||||
<zookeeper.version>3.4.11</zookeeper.version>
|
||||
<checkerframework.version>2.5.7</checkerframework.version>
|
||||
<repoOrgId>apache.snapshots</repoOrgId>
|
||||
<repoOrgName>Apache Snapshot Repository</repoOrgName>
|
||||
<repoOrgUrl>https://repository.apache.org/snapshots</repoOrgUrl>
|
||||
|
|
|
@ -111,6 +111,11 @@
|
|||
<groupId>org.ow2.asm</groupId>
|
||||
<artifactId>asm-commons</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.checkerframework</groupId>
|
||||
<artifactId>checker</artifactId>
|
||||
<version>${checkerframework.version}</version>
|
||||
</dependency>
|
||||
|
||||
<!-- Tests -->
|
||||
<dependency>
|
||||
|
|
|
@ -33,6 +33,8 @@ import org.apache.druid.js.JavaScriptConfig;
|
|||
import org.apache.druid.segment.BaseObjectColumnValueSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.checkerframework.checker.nullness.qual.EnsuresNonNull;
|
||||
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
|
||||
import org.mozilla.javascript.Context;
|
||||
import org.mozilla.javascript.ContextAction;
|
||||
import org.mozilla.javascript.ContextFactory;
|
||||
|
@ -58,8 +60,15 @@ public class JavaScriptAggregatorFactory extends AggregatorFactory
|
|||
private final String fnCombine;
|
||||
private final JavaScriptConfig config;
|
||||
|
||||
// This variable is lazily initialized to avoid unnecessary JavaScript compilation during JSON serde
|
||||
private JavaScriptAggregator.ScriptAggregator compiledScript;
|
||||
/**
|
||||
* The field is declared volatile in order to ensure safe publication of the object
|
||||
* in {@link #compileScript(String, String, String)} without worrying about final modifiers
|
||||
* on the fields of the created object
|
||||
*
|
||||
* @see <a href="https://github.com/apache/incubator-druid/pull/6662#discussion_r237013157">
|
||||
* https://github.com/apache/incubator-druid/pull/6662#discussion_r237013157</a>
|
||||
*/
|
||||
private volatile JavaScriptAggregator.@MonotonicNonNull ScriptAggregator compiledScript;
|
||||
|
||||
@JsonCreator
|
||||
public JavaScriptAggregatorFactory(
|
||||
|
@ -89,7 +98,7 @@ public class JavaScriptAggregatorFactory extends AggregatorFactory
|
|||
@Override
|
||||
public Aggregator factorize(final ColumnSelectorFactory columnFactory)
|
||||
{
|
||||
checkAndCompileScript();
|
||||
JavaScriptAggregator.ScriptAggregator compiledScript = getCompiledScript();
|
||||
return new JavaScriptAggregator(
|
||||
fieldNames.stream().map(columnFactory::makeColumnValueSelector).collect(Collectors.toList()),
|
||||
compiledScript
|
||||
|
@ -99,7 +108,7 @@ public class JavaScriptAggregatorFactory extends AggregatorFactory
|
|||
@Override
|
||||
public BufferAggregator factorizeBuffered(final ColumnSelectorFactory columnSelectorFactory)
|
||||
{
|
||||
checkAndCompileScript();
|
||||
JavaScriptAggregator.ScriptAggregator compiledScript = getCompiledScript();
|
||||
return new JavaScriptBufferAggregator(
|
||||
fieldNames.stream().map(columnSelectorFactory::makeColumnValueSelector).collect(Collectors.toList()),
|
||||
compiledScript
|
||||
|
@ -115,7 +124,7 @@ public class JavaScriptAggregatorFactory extends AggregatorFactory
|
|||
@Override
|
||||
public Object combine(Object lhs, Object rhs)
|
||||
{
|
||||
checkAndCompileScript();
|
||||
JavaScriptAggregator.ScriptAggregator compiledScript = getCompiledScript();
|
||||
return compiledScript.combine(((Number) lhs).doubleValue(), ((Number) rhs).doubleValue());
|
||||
}
|
||||
|
||||
|
@ -135,7 +144,7 @@ public class JavaScriptAggregatorFactory extends AggregatorFactory
|
|||
@Override
|
||||
public void fold(ColumnValueSelector selector)
|
||||
{
|
||||
checkAndCompileScript();
|
||||
JavaScriptAggregator.ScriptAggregator compiledScript = getCompiledScript();
|
||||
combined = compiledScript.combine(combined, selector.getDouble());
|
||||
}
|
||||
|
||||
|
@ -283,19 +292,24 @@ public class JavaScriptAggregatorFactory extends AggregatorFactory
|
|||
* This class can be used by multiple threads, so this function should be thread-safe to avoid extra
|
||||
* script compilation.
|
||||
*/
|
||||
private void checkAndCompileScript()
|
||||
@EnsuresNonNull("compiledScript")
|
||||
private JavaScriptAggregator.ScriptAggregator getCompiledScript()
|
||||
{
|
||||
if (compiledScript == null) {
|
||||
// JavaScript configuration should be checked when it's actually used because someone might still want Druid
|
||||
// nodes to be able to deserialize JavaScript-based objects even though JavaScript is disabled.
|
||||
Preconditions.checkState(config.isEnabled(), "JavaScript is disabled");
|
||||
// JavaScript configuration should be checked when it's actually used because someone might still want Druid
|
||||
// nodes to be able to deserialize JavaScript-based objects even though JavaScript is disabled.
|
||||
Preconditions.checkState(config.isEnabled(), "JavaScript is disabled");
|
||||
|
||||
JavaScriptAggregator.ScriptAggregator syncedCompiledScript = compiledScript;
|
||||
if (syncedCompiledScript == null) {
|
||||
synchronized (config) {
|
||||
if (compiledScript == null) {
|
||||
compiledScript = compileScript(fnAggregate, fnReset, fnCombine);
|
||||
syncedCompiledScript = compiledScript;
|
||||
if (syncedCompiledScript == null) {
|
||||
syncedCompiledScript = compileScript(fnAggregate, fnReset, fnCombine);
|
||||
compiledScript = syncedCompiledScript;
|
||||
}
|
||||
}
|
||||
}
|
||||
return syncedCompiledScript;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
|
|
|
@ -28,6 +28,8 @@ import org.apache.druid.js.JavaScriptConfig;
|
|||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.PostAggregator;
|
||||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||
import org.checkerframework.checker.nullness.qual.EnsuresNonNull;
|
||||
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
|
||||
import org.mozilla.javascript.Context;
|
||||
import org.mozilla.javascript.ContextFactory;
|
||||
import org.mozilla.javascript.ScriptableObject;
|
||||
|
@ -87,8 +89,16 @@ public class JavaScriptPostAggregator implements PostAggregator
|
|||
private final String function;
|
||||
private final JavaScriptConfig config;
|
||||
|
||||
// This variable is lazily initialized to avoid unnecessary JavaScript compilation during JSON serde
|
||||
private Function fn;
|
||||
/**
|
||||
* The field is declared volatile in order to ensure safe publication of the object
|
||||
* in {@link #compile(String)} without worrying about final modifiers
|
||||
* on the fields of the created object
|
||||
*
|
||||
* @see <a href="https://github.com/apache/incubator-druid/pull/6662#discussion_r237013157">
|
||||
* https://github.com/apache/incubator-druid/pull/6662#discussion_r237013157</a>
|
||||
*/
|
||||
@MonotonicNonNull
|
||||
private volatile Function fn;
|
||||
|
||||
@JsonCreator
|
||||
public JavaScriptPostAggregator(
|
||||
|
@ -123,7 +133,7 @@ public class JavaScriptPostAggregator implements PostAggregator
|
|||
@Override
|
||||
public Object compute(Map<String, Object> combinedAggregators)
|
||||
{
|
||||
checkAndCompileScript();
|
||||
Function fn = getCompiledScript();
|
||||
final Object[] args = new Object[fieldNames.size()];
|
||||
int i = 0;
|
||||
for (String field : fieldNames) {
|
||||
|
@ -136,22 +146,24 @@ public class JavaScriptPostAggregator implements PostAggregator
|
|||
* {@link #compute} can be called by multiple threads, so this function should be thread-safe to avoid extra
|
||||
* script compilation.
|
||||
*/
|
||||
private void checkAndCompileScript()
|
||||
@EnsuresNonNull("fn")
|
||||
private Function getCompiledScript()
|
||||
{
|
||||
if (fn == null) {
|
||||
// JavaScript configuration should be checked when it's actually used because someone might still want Druid
|
||||
// nodes to be able to deserialize JavaScript-based objects even though JavaScript is disabled.
|
||||
Preconditions.checkState(config.isEnabled(), "JavaScript is disabled");
|
||||
// JavaScript configuration should be checked when it's actually used because someone might still want Druid
|
||||
// nodes to be able to deserialize JavaScript-based objects even though JavaScript is disabled.
|
||||
Preconditions.checkState(config.isEnabled(), "JavaScript is disabled");
|
||||
|
||||
// Synchronizing here can degrade the performance significantly because this method is called per input row.
|
||||
// However, early compilation of JavaScript functions can occur some memory issues due to unnecessary compilation
|
||||
// involving Java class generation each time, and thus this will be better.
|
||||
Function syncedFn = fn;
|
||||
if (syncedFn == null) {
|
||||
synchronized (config) {
|
||||
if (fn == null) {
|
||||
fn = compile(function);
|
||||
syncedFn = fn;
|
||||
if (syncedFn == null) {
|
||||
syncedFn = compile(function);
|
||||
fn = syncedFn;
|
||||
}
|
||||
}
|
||||
}
|
||||
return syncedFn;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -27,6 +27,8 @@ import com.google.common.base.Preconditions;
|
|||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.js.JavaScriptConfig;
|
||||
import org.checkerframework.checker.nullness.qual.EnsuresNonNull;
|
||||
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
|
||||
import org.mozilla.javascript.Context;
|
||||
import org.mozilla.javascript.ContextFactory;
|
||||
import org.mozilla.javascript.ScriptableObject;
|
||||
|
@ -69,8 +71,16 @@ public class JavaScriptExtractionFn implements ExtractionFn
|
|||
private final boolean injective;
|
||||
private final JavaScriptConfig config;
|
||||
|
||||
// This variable is lazily initialized to avoid unnecessary JavaScript compilation during JSON serde
|
||||
private Function<Object, String> fn;
|
||||
/**
|
||||
* The field is declared volatile in order to ensure safe publication of the object
|
||||
* in {@link #compile(String)} without worrying about final modifiers
|
||||
* on the fields of the created object
|
||||
*
|
||||
* @see <a href="https://github.com/apache/incubator-druid/pull/6662#discussion_r237013157">
|
||||
* https://github.com/apache/incubator-druid/pull/6662#discussion_r237013157</a>
|
||||
*/
|
||||
@MonotonicNonNull
|
||||
private volatile Function<Object, String> fn;
|
||||
|
||||
@JsonCreator
|
||||
public JavaScriptExtractionFn(
|
||||
|
@ -112,7 +122,7 @@ public class JavaScriptExtractionFn implements ExtractionFn
|
|||
@Nullable
|
||||
public String apply(@Nullable Object value)
|
||||
{
|
||||
checkAndCompileScript();
|
||||
Function<Object, String> fn = getCompiledScript();
|
||||
return NullHandling.emptyToNullIfNeeded(fn.apply(value));
|
||||
}
|
||||
|
||||
|
@ -120,19 +130,24 @@ public class JavaScriptExtractionFn implements ExtractionFn
|
|||
* {@link #apply(Object)} can be called by multiple threads, so this function should be thread-safe to avoid extra
|
||||
* script compilation.
|
||||
*/
|
||||
private void checkAndCompileScript()
|
||||
@EnsuresNonNull("fn")
|
||||
private Function<Object, String> getCompiledScript()
|
||||
{
|
||||
if (fn == null) {
|
||||
// JavaScript configuration should be checked when it's actually used because someone might still want Druid
|
||||
// nodes to be able to deserialize JavaScript-based objects even though JavaScript is disabled.
|
||||
Preconditions.checkState(config.isEnabled(), "JavaScript is disabled");
|
||||
// JavaScript configuration should be checked when it's actually used because someone might still want Druid
|
||||
// nodes to be able to deserialize JavaScript-based objects even though JavaScript is disabled.
|
||||
Preconditions.checkState(config.isEnabled(), "JavaScript is disabled");
|
||||
|
||||
Function<Object, String> syncedFn = fn;
|
||||
if (syncedFn == null) {
|
||||
synchronized (config) {
|
||||
if (fn == null) {
|
||||
fn = compile(function);
|
||||
syncedFn = fn;
|
||||
if (syncedFn == null) {
|
||||
syncedFn = compile(function);
|
||||
fn = syncedFn;
|
||||
}
|
||||
}
|
||||
}
|
||||
return syncedFn;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -30,6 +30,8 @@ import org.apache.druid.java.util.common.StringUtils;
|
|||
import org.apache.druid.js.JavaScriptConfig;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
import org.apache.druid.segment.filter.JavaScriptFilter;
|
||||
import org.checkerframework.checker.nullness.qual.EnsuresNonNull;
|
||||
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
|
||||
import org.mozilla.javascript.Context;
|
||||
import org.mozilla.javascript.Function;
|
||||
import org.mozilla.javascript.ScriptableObject;
|
||||
|
@ -44,8 +46,16 @@ public class JavaScriptDimFilter implements DimFilter
|
|||
private final ExtractionFn extractionFn;
|
||||
private final JavaScriptConfig config;
|
||||
|
||||
// This variable is lazily initialized to avoid unnecessary JavaScript compilation during JSON serde
|
||||
private JavaScriptPredicateFactory predicateFactory;
|
||||
/**
|
||||
* The field is declared volatile in order to ensure safe publication of the object
|
||||
* in {@link JavaScriptPredicateFactory(String, ExtractionFn)} without worrying about final modifiers
|
||||
* on the fields of the created object
|
||||
*
|
||||
* @see <a href="https://github.com/apache/incubator-druid/pull/6662#discussion_r237013157">
|
||||
* https://github.com/apache/incubator-druid/pull/6662#discussion_r237013157</a>
|
||||
*/
|
||||
@MonotonicNonNull
|
||||
private volatile JavaScriptPredicateFactory predicateFactory;
|
||||
|
||||
@JsonCreator
|
||||
public JavaScriptDimFilter(
|
||||
|
@ -107,7 +117,7 @@ public class JavaScriptDimFilter implements DimFilter
|
|||
@Override
|
||||
public Filter toFilter()
|
||||
{
|
||||
checkAndCreatePredicateFactory();
|
||||
JavaScriptPredicateFactory predicateFactory = getPredicateFactory();
|
||||
return new JavaScriptFilter(dimension, predicateFactory);
|
||||
}
|
||||
|
||||
|
@ -115,19 +125,24 @@ public class JavaScriptDimFilter implements DimFilter
|
|||
* This class can be used by multiple threads, so this function should be thread-safe to avoid extra
|
||||
* script compilation.
|
||||
*/
|
||||
private void checkAndCreatePredicateFactory()
|
||||
@EnsuresNonNull("predicateFactory")
|
||||
private JavaScriptPredicateFactory getPredicateFactory()
|
||||
{
|
||||
if (predicateFactory == null) {
|
||||
// JavaScript configuration should be checked when it's actually used because someone might still want Druid
|
||||
// nodes to be able to deserialize JavaScript-based objects even though JavaScript is disabled.
|
||||
Preconditions.checkState(config.isEnabled(), "JavaScript is disabled");
|
||||
// JavaScript configuration should be checked when it's actually used because someone might still want Druid
|
||||
// nodes to be able to deserialize JavaScript-based objects even though JavaScript is disabled.
|
||||
Preconditions.checkState(config.isEnabled(), "JavaScript is disabled");
|
||||
|
||||
JavaScriptPredicateFactory syncedFnPredicateFactory = predicateFactory;
|
||||
if (syncedFnPredicateFactory == null) {
|
||||
synchronized (config) {
|
||||
if (predicateFactory == null) {
|
||||
predicateFactory = new JavaScriptPredicateFactory(function, extractionFn);
|
||||
syncedFnPredicateFactory = predicateFactory;
|
||||
if (syncedFnPredicateFactory == null) {
|
||||
syncedFnPredicateFactory = new JavaScriptPredicateFactory(function, extractionFn);
|
||||
predicateFactory = syncedFnPredicateFactory;
|
||||
}
|
||||
}
|
||||
}
|
||||
return syncedFnPredicateFactory;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
Loading…
Reference in New Issue