You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2019/04/10 07:46:32 UTC

[lucene-solr] branch branch_8x updated: LUCENE-8754: Prevent ConcurrentModificationException in SegmentInfo (#637)

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

simonw pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new addb397  LUCENE-8754: Prevent ConcurrentModificationException in SegmentInfo (#637)
addb397 is described below

commit addb3977fd735bd852aad2c737c27d8888f70dca
Author: Simon Willnauer <si...@apache.org>
AuthorDate: Wed Apr 10 09:29:22 2019 +0200

    LUCENE-8754: Prevent ConcurrentModificationException in SegmentInfo (#637)
    
    In order to prevent ConcurrentModificationException this change makes
    an unmodifiable copy on write for all maps in SegmentInfo. MergePolicies
    can access these maps without synchronization and cause exceptions if
    it's modified in the merge thread.
---
 lucene/CHANGES.txt                                  |  3 +++
 .../simpletext/SimpleTextSegmentInfoFormat.java     |  4 +---
 .../codecs/lucene50/Lucene50PostingsFormat.java     |  4 ++++
 .../lucene/index/DocumentsWriterPerThread.java      |  3 +--
 .../java/org/apache/lucene/index/IndexWriter.java   | 11 +++++++----
 .../java/org/apache/lucene/index/SegmentInfo.java   | 21 ++++++++++++++++-----
 .../lucene/index/TestDemoParallelLeafReader.java    |  5 ++++-
 .../java/org/apache/lucene/index/IndexSplitter.java |  4 ++--
 .../codecs/cheapbastard/CheapBastardCodec.java      |  3 +--
 .../lucene/index/BaseCompoundFormatTestCase.java    |  3 +--
 .../lucene/index/BaseFieldInfoFormatTestCase.java   |  3 +--
 .../lucene/index/BaseIndexFileFormatTestCase.java   |  2 +-
 .../lucene/index/BaseSegmentInfoFormatTestCase.java | 20 ++++++++++----------
 .../java/org/apache/lucene/index/RandomCodec.java   |  5 +----
 .../src/java/org/apache/lucene/util/TestUtil.java   |  4 ++--
 15 files changed, 55 insertions(+), 40 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index c32bb35..4c57617 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -47,6 +47,9 @@ Bug fixes
   Scorer instead of Scorable, leading to ClassCastExceptions (Markus Jelsma,
   Alan Woodward)
 
+* LUCENE-8754: Fix ConcurrentModificationException in SegmentInfo if
+  attributes are accessed in MergePolicy while the merge is running (Simon Willnauer)
+
 Improvements
 
 * LUCENE-8673: Use radix partitioning when merging dimensional points instead
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
index 6d5bfe4..5f22f62 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.text.ParseException;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -305,8 +304,7 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
       SimpleTextUtil.checkFooter(input);
 
       SegmentInfo info = new SegmentInfo(directory, version, minVersion, segmentName, docCount,
-                                         isCompoundFile, null, Collections.unmodifiableMap(diagnostics),
-                                         id, Collections.unmodifiableMap(attributes), indexSort);
+                                         isCompoundFile, null, diagnostics, id, attributes, indexSort);
       info.setFiles(files);
       return info;
     }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java
index 32f89eb..2839233 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java
@@ -456,6 +456,10 @@ public final class Lucene50PostingsFormat extends PostingsFormat {
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
     PostingsWriterBase postingsWriter = new Lucene50PostingsWriter(state);
+    if (state.segmentInfo.getAttribute(MODE_KEY) != null && fstLoadMode.name().equals(state.segmentInfo.getAttribute(MODE_KEY)) == false) {
+      throw new IllegalStateException("found existing value for " + MODE_KEY + " for segment: " + state.segmentInfo.name +
+          " old=" + state.segmentInfo.getAttribute(MODE_KEY) + ", new=" + fstLoadMode.name());
+    }
     state.segmentInfo.putAttribute(MODE_KEY, fstLoadMode.name());
     boolean success = false;
     try {
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
index 22b8e3a..2f520e0 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
@@ -19,7 +19,6 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.text.NumberFormat;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Locale;
 import java.util.Set;
@@ -190,7 +189,7 @@ final class DocumentsWriterPerThread {
     assert numDocsInRAM == 0 : "num docs " + numDocsInRAM;
     deleteSlice = deleteQueue.newSlice();
    
-    segmentInfo = new SegmentInfo(directoryOrig, Version.LATEST, Version.LATEST, segmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), indexWriterConfig.getIndexSort());
+    segmentInfo = new SegmentInfo(directoryOrig, Version.LATEST, Version.LATEST, segmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), Collections.emptyMap(), indexWriterConfig.getIndexSort());
     assert numDocsInRAM == 0;
     if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
       infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segmentName + " delQueue=" + deleteQueue);  
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index f9aaf34..74405af 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -1971,7 +1971,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
       for(final MergePolicy.OneMerge merge  : pendingMerges) {
         merge.maxNumSegments = maxNumSegments;
         if (merge.info != null) {
-          // TODO: explain why this is sometimes still null
+          // this can be null since we register the merge under lock before we then do the actual merge and
+          // set the merge.info in _mergeInit
           segmentsToMerge.put(merge.info, Boolean.TRUE);
         }
       }
@@ -1979,7 +1980,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
       for (final MergePolicy.OneMerge merge: runningMerges) {
         merge.maxNumSegments = maxNumSegments;
         if (merge.info != null) {
-          // TODO: explain why this is sometimes still null
+          // this can be null since we put the merge on runningMerges before we do the actual merge and
+          // set the merge.info in _mergeInit
           segmentsToMerge.put(merge.info, Boolean.TRUE);
         }
       }
@@ -2973,7 +2975,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
 
       // We set the min version to null for now, it will be set later by SegmentMerger
       SegmentInfo info = new SegmentInfo(directoryOrig, Version.LATEST, null, mergedName, -1,
-                                         false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), config.getIndexSort());
+                                         false, codec, Collections.emptyMap(), StringHelper.randomId(), Collections.emptyMap(), config.getIndexSort());
 
       SegmentMerger merger = new SegmentMerger(Arrays.asList(readers), info, infoStream, trackingDir,
                                                globalFieldNumberMap, 
@@ -4248,7 +4250,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
     // names.
     final String mergeSegmentName = newSegmentName();
     // We set the min version to null for now, it will be set later by SegmentMerger
-    SegmentInfo si = new SegmentInfo(directoryOrig, Version.LATEST, null, mergeSegmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), config.getIndexSort());
+    SegmentInfo si = new SegmentInfo(directoryOrig, Version.LATEST, null, mergeSegmentName, -1, false, codec,
+        Collections.emptyMap(), StringHelper.randomId(), Collections.emptyMap(), config.getIndexSort());
     Map<String,String> details = new HashMap<>();
     details.put("mergeMaxNumSegments", "" + merge.maxNumSegments);
     details.put("mergeFactor", Integer.toString(merge.segments.size()));
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java
index 7c91eeb..4c64e40 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Locale;
 import java.util.Map;
@@ -68,7 +69,7 @@ public final class SegmentInfo {
 
   private Map<String,String> diagnostics;
   
-  private final Map<String,String> attributes;
+  private Map<String,String> attributes;
 
   private final Sort indexSort;
 
@@ -86,7 +87,7 @@ public final class SegmentInfo {
   Version minVersion;
 
   void setDiagnostics(Map<String, String> diagnostics) {
-    this.diagnostics = Objects.requireNonNull(diagnostics);
+    this.diagnostics = Collections.unmodifiableMap(new HashMap<>(Objects.requireNonNull(diagnostics)));
   }
 
   /** Returns diagnostics saved into the segment when it was
@@ -111,12 +112,12 @@ public final class SegmentInfo {
     this.maxDoc = maxDoc;
     this.isCompoundFile = isCompoundFile;
     this.codec = codec;
-    this.diagnostics = Objects.requireNonNull(diagnostics);
+    this.diagnostics = Collections.unmodifiableMap(new HashMap<>(Objects.requireNonNull(diagnostics)));
     this.id = id;
     if (id.length != StringHelper.ID_LENGTH) {
       throw new IllegalArgumentException("invalid id: " + Arrays.toString(id));
     }
-    this.attributes = Objects.requireNonNull(attributes);
+    this.attributes = Collections.unmodifiableMap(new HashMap<>(Objects.requireNonNull(attributes)));
     this.indexSort = indexSort;
   }
 
@@ -324,9 +325,19 @@ public final class SegmentInfo {
    * <p>
    * If a value already exists for the field, it will be replaced with the new
    * value.
+   * This method make a copy on write for every attribute change.
    */
   public String putAttribute(String key, String value) {
-    return attributes.put(key, value);
+    HashMap<String, String> newMap = new HashMap<>(attributes);
+    String oldValue = newMap.put(key, value);
+    // we make a full copy of this to prevent concurrent modifications to this in the toString method
+    // this method is only called when a segment is written but the SegmentInfo might be exposed
+    // in running merges which can cause ConcurrentModificationExceptions if we modify / share
+    // the same instance. Technically that's an unsafe publication but IW design would require
+    // significant changes to prevent this. On the other hand, since we expose the map in getAttributes()
+    // it's a good design to make it unmodifiable anyway.
+    attributes = Collections.unmodifiableMap(newMap);
+    return oldValue;
   }
   
   /**
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java b/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
index 8f5e05b..87dbdef 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
@@ -26,6 +26,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -557,7 +558,9 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
         @Override
         public void setMergeInfo(SegmentCommitInfo info) {
           // Record that this merged segment is current as of this schemaGen:
-          info.info.getDiagnostics().put(SCHEMA_GEN_KEY, Long.toString(schemaGen));
+          Map<String, String> copy = new HashMap<>(info.info.getDiagnostics());
+          copy.put(SCHEMA_GEN_KEY, Long.toString(schemaGen));
+          info.info.setDiagnostics(copy);
           super.setMergeInfo(info);
         }
 
diff --git a/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java b/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java
index a586f83..afe45cc 100644
--- a/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java
+++ b/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java
@@ -24,7 +24,7 @@ import java.text.DecimalFormat;
 import java.text.DecimalFormatSymbols;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashMap;
+import java.util.Collections;
 import java.util.List;
 import java.util.Locale;
 
@@ -140,7 +140,7 @@ public class IndexSplitter {
       SegmentInfo info = infoPerCommit.info;
       // Same info just changing the dir:
       SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.getMinVersion(), info.name, info.maxDoc(),
-                                            info.getUseCompoundFile(), info.getCodec(), info.getDiagnostics(), info.getId(), new HashMap<>(), null);
+                                            info.getUseCompoundFile(), info.getCodec(), info.getDiagnostics(), info.getId(), Collections.emptyMap(), null);
       destInfos.add(new SegmentCommitInfo(newInfo, infoPerCommit.getDelCount(), infoPerCommit.getSoftDelCount(),
           infoPerCommit.getDelGen(), infoPerCommit.getFieldInfosGen(),
           infoPerCommit.getDocValuesGen()));
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardCodec.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardCodec.java
index a2a84d2..f34aa82 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardCodec.java
@@ -18,7 +18,6 @@ package org.apache.lucene.codecs.cheapbastard;
 
 import org.apache.lucene.codecs.FilterCodec;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat;
 import org.apache.lucene.util.TestUtil;
 
 /** Codec that tries to use as little ram as possible because he spent all his money on beer */
@@ -27,7 +26,7 @@ import org.apache.lucene.util.TestUtil;
 public class CheapBastardCodec extends FilterCodec {
   
   // TODO: would be better to have no terms index at all and bsearch a terms dict
-  private final PostingsFormat postings = TestUtil.getDefaultPostingsFormat(100, 200, Lucene50PostingsFormat.FSTLoadMode.OFF_HEAP);
+  private final PostingsFormat postings = TestUtil.getDefaultPostingsFormat(100, 200);
 
   public CheapBastardCodec() {
     super("CheapBastard", TestUtil.getDefaultCodec());
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java
index 07a3f06..1d28046 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java
@@ -20,7 +20,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Random;
 
@@ -639,7 +638,7 @@ public abstract class BaseCompoundFormatTestCase extends BaseIndexFileFormatTest
   /** Returns a new fake segment */
   protected static SegmentInfo newSegmentInfo(Directory dir, String name) {
     Version minVersion = random().nextBoolean() ? null : Version.LATEST;
-    return new SegmentInfo(dir, Version.LATEST, minVersion,  name, 10000, false, Codec.getDefault(), Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
+    return new SegmentInfo(dir, Version.LATEST, minVersion,  name, 10000, false, Codec.getDefault(), Collections.emptyMap(), StringHelper.randomId(), Collections.emptyMap(), null);
   }
   
   /** Creates a file of the specified size with random data. */
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java
index 3515b9a..3cd7fcd 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Random;
@@ -348,7 +347,7 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes
   /** Returns a new fake segment */
   protected static SegmentInfo newSegmentInfo(Directory dir, String name) {
     Version minVersion = random().nextBoolean() ? null : Version.LATEST;
-    return new SegmentInfo(dir, Version.LATEST, minVersion, name, 10000, false, Codec.getDefault(), Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
+    return new SegmentInfo(dir, Version.LATEST, minVersion, name, 10000, false, Codec.getDefault(), Collections.emptyMap(), StringHelper.randomId(), Collections.emptyMap(), null);
   }
   
   @Override
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
index 395b920..5865f27 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
@@ -343,7 +343,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
     Directory dir = newFSDirectory(createTempDir("justSoYouGetSomeChannelErrors"));
     Codec codec = getCodec();
     
-    SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "_0", 1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
+    SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "_0", 1, false, codec, Collections.emptyMap(), StringHelper.randomId(), Collections.emptyMap(), null);
     FieldInfo proto = oneDocReader.getFieldInfos().fieldInfo("field");
     FieldInfo field = new FieldInfo(proto.name, proto.number, proto.hasVectors(), proto.omitsNorms(), proto.hasPayloads(), 
                                     proto.getIndexOptions(), proto.getDocValuesType(), proto.getDocValuesGen(), new HashMap<>(),
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java
index 814fb4b..91eb971 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java
@@ -60,7 +60,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
     Codec codec = getCodec();
     byte id[] = StringHelper.randomId();
     SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, 
-                                       Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
+                                       Collections.emptyMap(), id, Collections.emptyMap(), null);
     info.setFiles(Collections.<String>emptySet());
     codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
     SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
@@ -74,7 +74,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
     Codec codec = getCodec();
     byte id[] = StringHelper.randomId();
     SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, 
-                                       Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
+                                       Collections.emptyMap(), id, Collections.emptyMap(), null);
     Set<String> originalFiles = Collections.singleton("_123.a");
     info.setFiles(originalFiles);
     codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
@@ -103,7 +103,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
     diagnostics.put("key1", "value1");
     diagnostics.put("key2", "value2");
     SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, 
-                                       diagnostics, id, new HashMap<>(), null);
+                                       diagnostics, id, Collections.emptyMap(), null);
     info.setFiles(Collections.<String>emptySet());
     codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
     SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
@@ -146,7 +146,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
     Directory dir = newDirectory();
     byte id[] = StringHelper.randomId();
     SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, 
-                                       Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
+                                       Collections.<String,String>emptyMap(), id, Collections.emptyMap(), null);
     info.setFiles(Collections.<String>emptySet());
     codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
     SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
@@ -162,7 +162,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
         Directory dir = newDirectory();
         byte id[] = StringHelper.randomId();
         SegmentInfo info = new SegmentInfo(dir, v, minV, "_123", 1, false, codec, 
-                                           Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
+                                           Collections.<String,String>emptyMap(), id, Collections.emptyMap(), null);
         info.setFiles(Collections.<String>emptySet());
         codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
         SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
@@ -275,7 +275,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
       Codec codec = getCodec();
       byte id[] = StringHelper.randomId();
       SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, 
-          Collections.<String,String>emptyMap(), id, new HashMap<>(), sort);
+          Collections.<String,String>emptyMap(), id, Collections.emptyMap(), sort);
       info.setFiles(Collections.<String>emptySet());
       codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
       SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
@@ -305,7 +305,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
     Codec codec = getCodec();
     byte id[] = StringHelper.randomId();
     SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, 
-                                       Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
+                                       Collections.<String,String>emptyMap(), id, Collections.emptyMap(), null);
     info.setFiles(Collections.<String>emptySet());
     
     fail.setDoFail();
@@ -338,7 +338,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
     Codec codec = getCodec();
     byte id[] = StringHelper.randomId();
     SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, 
-                                       Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
+                                       Collections.<String,String>emptyMap(), id, Collections.emptyMap(), null);
     info.setFiles(Collections.<String>emptySet());
     
     fail.setDoFail();
@@ -371,7 +371,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
     Codec codec = getCodec();
     byte id[] = StringHelper.randomId();
     SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, 
-                                       Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
+                                       Collections.<String,String>emptyMap(), id, Collections.emptyMap(), null);
     info.setFiles(Collections.<String>emptySet());
     codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
     
@@ -405,7 +405,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
     Codec codec = getCodec();
     byte id[] = StringHelper.randomId();
     SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, 
-                                       Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
+                                       Collections.<String,String>emptyMap(), id, Collections.emptyMap(), null);
     info.setFiles(Collections.<String>emptySet());
     codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
     
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java b/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
index 87126ca..ec3c323 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
@@ -26,7 +26,6 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 
-import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.PointsFormat;
 import org.apache.lucene.codecs.PointsReader;
@@ -41,7 +40,6 @@ import org.apache.lucene.codecs.blockterms.LuceneVarGapDocFreqInterval;
 import org.apache.lucene.codecs.blockterms.LuceneVarGapFixedInterval;
 import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat;
 import org.apache.lucene.codecs.bloom.TestBloomFilteredLucenePostings;
-import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat;
 import org.apache.lucene.codecs.lucene60.Lucene60PointsReader;
 import org.apache.lucene.codecs.lucene60.Lucene60PointsWriter;
 import org.apache.lucene.codecs.memory.DirectDocValuesFormat;
@@ -189,8 +187,7 @@ public class RandomCodec extends AssertingCodec {
     bkdSplitRandomSeed = random.nextInt();
 
     add(avoidCodecs,
-        TestUtil.getDefaultPostingsFormat(minItemsPerBlock, maxItemsPerBlock,
-            RandomPicks.randomFrom(random, Lucene50PostingsFormat.FSTLoadMode.values())),
+        TestUtil.getDefaultPostingsFormat(minItemsPerBlock, maxItemsPerBlock),
         new FSTPostingsFormat(),
         new FSTOrdPostingsFormat(),
         new DirectPostingsFormat(LuceneTestCase.rarely(random) ? 1 : (LuceneTestCase.rarely(random) ? Integer.MAX_VALUE : maxItemsPerBlock),
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
index 280a074..e706c45 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
@@ -911,8 +911,8 @@ public final class TestUtil {
    * Returns the actual default postings format (e.g. LuceneMNPostingsFormat for this version of Lucene.
    * @lucene.internal this may disappear at any time
    */
-  public static PostingsFormat getDefaultPostingsFormat(int minItemsPerBlock, int maxItemsPerBlock, Lucene50PostingsFormat.FSTLoadMode offHeapFST) {
-    return new Lucene50PostingsFormat(minItemsPerBlock, maxItemsPerBlock, offHeapFST);
+  public static PostingsFormat getDefaultPostingsFormat(int minItemsPerBlock, int maxItemsPerBlock) {
+    return new Lucene50PostingsFormat(minItemsPerBlock, maxItemsPerBlock, Lucene50PostingsFormat.FSTLoadMode.AUTO);
   }
   
   /** Returns a random postings format that supports term ordinals */