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:
Furkan KAMACI 2018-12-07 19:10:29 +03:00 committed by Roman Leventov
parent d525e5b18e
commit bbb283fa34
8 changed files with 135 additions and 52 deletions

View File

@ -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>

View File

@ -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

View File

@ -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>

View File

@ -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>

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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