You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by xv...@apache.org on 2019/08/24 22:24:56 UTC

[incubator-druid] branch master updated: Fix ConcurrentModificationException in JDK11 (#8391)

This is an automated email from the ASF dual-hosted git repository.

xvrl pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git


The following commit(s) were added to refs/heads/master by this push:
     new 20f7db5  Fix ConcurrentModificationException in JDK11 (#8391)
20f7db5 is described below

commit 20f7db5d22fe746472731b5c9cea9e5a232fe1d3
Author: Xavier Léauté <xv...@apache.org>
AuthorDate: Sat Aug 24 18:24:50 2019 -0400

    Fix ConcurrentModificationException in JDK11 (#8391)
    
    When building column/dimension selectors, calling computeIfAbsent can
    cause the applied function to modify the same cache through virtual
    column references. The JDK11 map implementation detects this change and
    will throw an exception.
    
    This fix – while not as elegant – breaks the single call into two
    steps to avoid this problem.
---
 .../QueryableIndexColumnSelectorFactory.java       | 85 +++++++++++++---------
 .../incremental/OnheapIncrementalIndex.java        | 10 ++-
 2 files changed, 59 insertions(+), 36 deletions(-)

diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexColumnSelectorFactory.java
index 0794b06..49175a0 100644
--- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexColumnSelectorFactory.java
+++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexColumnSelectorFactory.java
@@ -32,6 +32,7 @@ import org.apache.druid.segment.data.ReadableOffset;
 import javax.annotation.Nullable;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.function.Function;
 
 /**
  * The basic implementation of {@link ColumnSelectorFactory} over a historical segment (i. e. {@link QueryableIndex}).
@@ -76,21 +77,29 @@ class QueryableIndexColumnSelectorFactory implements ColumnSelectorFactory
   @Override
   public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec)
   {
-    return dimensionSelectorCache.computeIfAbsent(
-        dimensionSpec,
-        spec -> {
-          if (virtualColumns.exists(spec.getDimension())) {
-            DimensionSelector dimensionSelector = virtualColumns.makeDimensionSelector(dimensionSpec, index, offset);
-            if (dimensionSelector == null) {
-              return virtualColumns.makeDimensionSelector(dimensionSpec, this);
-            } else {
-              return dimensionSelector;
-            }
-          }
-
-          return spec.decorate(makeDimensionSelectorUndecorated(spec));
+    Function<DimensionSpec, DimensionSelector> mappingFunction = spec -> {
+      if (virtualColumns.exists(spec.getDimension())) {
+        DimensionSelector dimensionSelector = virtualColumns.makeDimensionSelector(dimensionSpec, index, offset);
+        if (dimensionSelector == null) {
+          return virtualColumns.makeDimensionSelector(dimensionSpec, this);
+        } else {
+          return dimensionSelector;
         }
-    );
+      }
+
+      return spec.decorate(makeDimensionSelectorUndecorated(spec));
+    };
+
+    // We cannot use dimensionSelectorCache.computeIfAbsent() here since the function being
+    // applied may modify the dimensionSelectorCache itself through virtual column references,
+    // triggering a ConcurrentModificationException in JDK 9 and above.
+    DimensionSelector dimensionSelector = dimensionSelectorCache.get(dimensionSpec);
+    if (dimensionSelector == null) {
+      dimensionSelector = mappingFunction.apply(dimensionSpec);
+      dimensionSelectorCache.put(dimensionSpec, dimensionSelector);
+    }
+
+    return dimensionSelector;
   }
 
   private DimensionSelector makeDimensionSelectorUndecorated(DimensionSpec dimensionSpec)
@@ -124,27 +133,35 @@ class QueryableIndexColumnSelectorFactory implements ColumnSelectorFactory
   @Override
   public ColumnValueSelector<?> makeColumnValueSelector(String columnName)
   {
-    return valueSelectorCache.computeIfAbsent(
-        columnName,
-        name -> {
-          if (virtualColumns.exists(columnName)) {
-            ColumnValueSelector<?> selector = virtualColumns.makeColumnValueSelector(columnName, index, offset);
-            if (selector == null) {
-              return virtualColumns.makeColumnValueSelector(columnName, this);
-            } else {
-              return selector;
-            }
-          }
-
-          BaseColumn column = getCachedColumn(columnName, BaseColumn.class);
-
-          if (column != null) {
-            return column.makeColumnValueSelector(offset);
-          } else {
-            return NilColumnValueSelector.instance();
-          }
+    Function<String, ColumnValueSelector<?>> mappingFunction = name -> {
+      if (virtualColumns.exists(columnName)) {
+        ColumnValueSelector<?> selector = virtualColumns.makeColumnValueSelector(columnName, index, offset);
+        if (selector == null) {
+          return virtualColumns.makeColumnValueSelector(columnName, this);
+        } else {
+          return selector;
         }
-    );
+      }
+
+      BaseColumn column = getCachedColumn(columnName, BaseColumn.class);
+
+      if (column != null) {
+        return column.makeColumnValueSelector(offset);
+      } else {
+        return NilColumnValueSelector.instance();
+      }
+    };
+
+    // We cannot use valueSelectorCache.computeIfAbsent() here since the function being
+    // applied may modify the valueSelectorCache itself through virtual column references,
+    // triggering a ConcurrentModificationException in JDK 9 and above.
+    ColumnValueSelector<?> columnValueSelector = valueSelectorCache.get(columnName);
+    if (columnValueSelector == null) {
+      columnValueSelector = mappingFunction.apply(columnName);
+      valueSelectorCache.put(columnName, columnValueSelector);
+    }
+
+    return columnValueSelector;
   }
 
   @Nullable
diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java
index 8e5c55b..2d64346 100644
--- a/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java
+++ b/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java
@@ -414,11 +414,17 @@ public class OnheapIncrementalIndex extends IncrementalIndex<Aggregator>
     @Override
     public ColumnValueSelector<?> makeColumnValueSelector(String columnName)
     {
-      final ColumnValueSelector existing = columnSelectorMap.get(columnName);
+      ColumnValueSelector existing = columnSelectorMap.get(columnName);
       if (existing != null) {
         return existing;
       }
-      return columnSelectorMap.computeIfAbsent(columnName, delegate::makeColumnValueSelector);
+
+      // We cannot use columnSelectorMap.computeIfAbsent(columnName, delegate::makeColumnValueSelector)
+      // here since makeColumnValueSelector may modify the columnSelectorMap itself through
+      // virtual column references, triggering a ConcurrentModificationException in JDK 9 and above.
+      ColumnValueSelector<?> columnValueSelector = delegate.makeColumnValueSelector(columnName);
+      existing = columnSelectorMap.putIfAbsent(columnName, columnValueSelector);
+      return existing != null ? existing : columnValueSelector;
     }
 
     @Nullable


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