You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2018/12/07 16:10:31 UTC

[GitHub] leventov closed pull request #6662: Double-checked locking bugs

leventov closed pull request #6662: Double-checked locking bugs
URL: https://github.com/apache/incubator-druid/pull/6662
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/aws-common/pom.xml b/aws-common/pom.xml
index 4b941c46c7d..941dc2847b0 100644
--- a/aws-common/pom.xml
+++ b/aws-common/pom.xml
@@ -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>
diff --git a/aws-common/src/main/java/org/apache/druid/common/aws/LazyFileSessionCredentialsProvider.java b/aws-common/src/main/java/org/apache/druid/common/aws/LazyFileSessionCredentialsProvider.java
index 7fc046b3165..483b32a777a 100644
--- a/aws-common/src/main/java/org/apache/druid/common/aws/LazyFileSessionCredentialsProvider.java
+++ b/aws-common/src/main/java/org/apache/druid/common/aws/LazyFileSessionCredentialsProvider.java
@@ -21,27 +21,43 @@
 
 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
diff --git a/pom.xml b/pom.xml
index 4c2938ebad2..46a1355d728 100644
--- a/pom.xml
+++ b/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>
diff --git a/processing/pom.xml b/processing/pom.xml
index 4963e32acbe..c4d817ca926 100644
--- a/processing/pom.xml
+++ b/processing/pom.xml
@@ -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>
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/JavaScriptAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/JavaScriptAggregatorFactory.java
index e747ad68d73..b5e9ed173ee 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/JavaScriptAggregatorFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/JavaScriptAggregatorFactory.java
@@ -33,6 +33,8 @@
 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 @@
   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 JavaScriptAggregatorFactory(
   @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 Aggregator factorize(final ColumnSelectorFactory columnFactory)
   @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 Comparator getComparator()
   @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 void reset(ColumnValueSelector selector)
       @Override
       public void fold(ColumnValueSelector selector)
       {
-        checkAndCompileScript();
+        JavaScriptAggregator.ScriptAggregator compiledScript = getCompiledScript();
         combined = compiledScript.combine(combined, selector.getDouble());
       }
 
@@ -283,19 +292,24 @@ public String toString()
    * 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
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/post/JavaScriptPostAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/post/JavaScriptPostAggregator.java
index 9bd6dc66a45..1ed1318fd88 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/post/JavaScriptPostAggregator.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/post/JavaScriptPostAggregator.java
@@ -28,6 +28,8 @@
 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 double apply(Object[] args)
   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 Comparator getComparator()
   @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 Object compute(Map<String, Object> combinedAggregators)
    * {@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");
-
-      // 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.
+    // 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 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
diff --git a/processing/src/main/java/org/apache/druid/query/extraction/JavaScriptExtractionFn.java b/processing/src/main/java/org/apache/druid/query/extraction/JavaScriptExtractionFn.java
index 06892ecd772..3b95b7507b1 100644
--- a/processing/src/main/java/org/apache/druid/query/extraction/JavaScriptExtractionFn.java
+++ b/processing/src/main/java/org/apache/druid/query/extraction/JavaScriptExtractionFn.java
@@ -27,6 +27,8 @@
 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 String apply(Object input)
   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 boolean isInjective()
   @Nullable
   public String apply(@Nullable Object value)
   {
-    checkAndCompileScript();
+    Function<Object, String> fn = getCompiledScript();
     return NullHandling.emptyToNullIfNeeded(fn.apply(value));
   }
 
@@ -120,19 +130,24 @@ public String apply(@Nullable Object value)
    * {@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
diff --git a/processing/src/main/java/org/apache/druid/query/filter/JavaScriptDimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/JavaScriptDimFilter.java
index 0569aa481d6..b211e38ad78 100644
--- a/processing/src/main/java/org/apache/druid/query/filter/JavaScriptDimFilter.java
+++ b/processing/src/main/java/org/apache/druid/query/filter/JavaScriptDimFilter.java
@@ -30,6 +30,8 @@
 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 @@
   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 DimFilter optimize()
   @Override
   public Filter toFilter()
   {
-    checkAndCreatePredicateFactory();
+    JavaScriptPredicateFactory predicateFactory = getPredicateFactory();
     return new JavaScriptFilter(dimension, predicateFactory);
   }
 
@@ -115,19 +125,24 @@ public Filter toFilter()
    * 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


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org