You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2022/11/24 10:20:11 UTC

[lucene] branch main updated: Ensure collections are properly sized on creation (#11942)

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

jpountz pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/lucene.git


The following commit(s) were added to refs/heads/main by this push:
     new 135f3fab412 Ensure collections are properly sized on creation (#11942)
135f3fab412 is described below

commit 135f3fab4124657b288ca546f04d27596c04d96d
Author: Simon Cooper <si...@elastic.co>
AuthorDate: Thu Nov 24 10:20:04 2022 +0000

    Ensure collections are properly sized on creation (#11942)
    
    A few other optimisations along the way
---
 .../analysis/charfilter/NormalizeCharMap.java      |  3 +--
 .../compound/hyphenation/HyphenationTree.java      |  8 +++----
 .../lucene/analysis/custom/CustomAnalyzer.java     | 12 +++++-----
 .../analysis/phonetic/PhoneticFilterFactory.java   | 22 ++++++++----------
 .../blocktree/Lucene40BlockTreeTermsReader.java    |  4 ++--
 .../lucene50/Lucene50CompoundReader.java           |  4 ++--
 .../lucene80/TestDocValuesCompression.java         |  3 ++-
 .../simpletext/SimpleTextSegmentInfoFormat.java    |  9 ++++----
 .../sharedterms/STUniformSplitTermsWriter.java     | 14 ++++++------
 .../codecs/lucene90/Lucene90CompoundReader.java    |  4 ++--
 .../blocktree/Lucene90BlockTreeTermsReader.java    |  4 ++--
 .../codecs/perfield/PerFieldPostingsFormat.java    |  3 ++-
 .../document/DocumentStoredFieldVisitor.java       |  6 ++---
 .../java/org/apache/lucene/index/SegmentInfos.java |  3 ++-
 .../lucene/index/StandardDirectoryReader.java      |  6 ++---
 .../org/apache/lucene/util/CollectionUtil.java     | 26 +++++++++++++++++++---
 .../src/java/org/apache/lucene/util/MapOfSets.java | 16 ++-----------
 .../lucene90/TestLucene90DocValuesFormat.java      |  3 ++-
 .../apache/lucene/search/TestCollectorManager.java | 24 +++++++++-----------
 .../lucene/search/TestMultiCollectorManager.java   | 25 ++++++++++-----------
 .../org/apache/lucene/util/TestSentinelIntSet.java |  2 +-
 .../apache/lucene/util/bkd/TestDocIdsWriter.java   |  4 ++--
 .../grouping/FirstPassGroupingCollector.java       |  3 ++-
 .../lucene/search/highlight/QueryScorer.java       | 17 +++++++-------
 .../search/uhighlight/UnifiedHighlighter.java      |  3 ++-
 .../org/apache/lucene/monitor/QueryDecomposer.java |  3 ++-
 .../lucene/queryparser/flexible/messages/NLS.java  | 17 +++++++-------
 .../sandbox/search/QueryProfilerBreakdown.java     |  4 ++--
 .../spatial/prefix/tree/S2PrefixTreeCell.java      |  7 +++---
 .../lucene/spatial3d/geom/GeoConcavePolygon.java   |  8 +++----
 .../lucene/spatial3d/geom/GeoConvexPolygon.java    |  8 +++----
 .../suggest/document/CompletionFieldsProducer.java |  4 ++--
 .../search/suggest/document/ContextQuery.java      |  3 ++-
 .../suggest/document/ContextSuggestField.java      |  9 ++++----
 .../analyzing/TestSuggestStopFilterFactory.java    |  4 ++--
 .../tests/index/BaseDocValuesFormatTestCase.java   |  2 +-
 .../tests/index/BaseTermVectorsFormatTestCase.java | 21 +++++------------
 37 files changed, 156 insertions(+), 162 deletions(-)

diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/NormalizeCharMap.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/NormalizeCharMap.java
index b3dba24718b..2d6ccd0ebb8 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/NormalizeCharMap.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/NormalizeCharMap.java
@@ -94,10 +94,9 @@ public class NormalizeCharMap {
       if (match.length() == 0) {
         throw new IllegalArgumentException("cannot match the empty string");
       }
-      if (pendingPairs.containsKey(match)) {
+      if (pendingPairs.putIfAbsent(match, replacement) != null) {
         throw new IllegalArgumentException("match \"" + match + "\" was already added");
       }
-      pendingPairs.put(match, replacement);
     }
 
     /** Builds the NormalizeCharMap; call this once you are done calling {@link #add}. */
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationTree.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationTree.java
index acc095539e2..4b38fb5d3bc 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationTree.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/hyphenation/HyphenationTree.java
@@ -313,17 +313,17 @@ public class HyphenationTree extends TernaryTree implements PatternConsumer {
 
     // check exception list first
     String sw = new String(word, 1, len);
-    if (stoplist.containsKey(sw)) {
+    ArrayList<Object> hw;
+    if ((hw = stoplist.get(sw)) != null) {
       // assume only simple hyphens (Hyphen.pre="-", Hyphen.post = Hyphen.no =
       // null)
-      ArrayList<Object> hw = stoplist.get(sw);
       int j = 0;
       for (i = 0; i < hw.size(); i++) {
         Object o = hw.get(i);
         // j = index(sw) = letterindex(word)?
         // result[k] = corresponding index(w)
-        if (o instanceof String) {
-          j += ((String) o).length();
+        if (o instanceof String s) {
+          j += s.length();
           if (j >= remainCharCount && j < (len - pushCharCount)) {
             result[k++] = j + iIgnoreAtBeginning;
           }
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
index 1663f619842..2c8afef5f2b 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/custom/CustomAnalyzer.java
@@ -24,7 +24,6 @@ import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
@@ -42,6 +41,7 @@ import org.apache.lucene.analysis.miscellaneous.ConditionalTokenFilterFactory;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.util.FilesystemResourceLoader;
 import org.apache.lucene.util.ClasspathResourceLoader;
+import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.ResourceLoader;
 import org.apache.lucene.util.ResourceLoaderAware;
 import org.apache.lucene.util.SetOnce;
@@ -594,11 +594,9 @@ public final class CustomAnalyzer extends Analyzer {
     }
 
     private Map<String, String> applyDefaultParams(Map<String, String> map) {
-      if (defaultMatchVersion.get() != null
-          && !map.containsKey(AbstractAnalysisFactory.LUCENE_MATCH_VERSION_PARAM)) {
-        map.put(
-            AbstractAnalysisFactory.LUCENE_MATCH_VERSION_PARAM,
-            defaultMatchVersion.get().toString());
+      Version v = defaultMatchVersion.get();
+      if (v != null) {
+        map.putIfAbsent(AbstractAnalysisFactory.LUCENE_MATCH_VERSION_PARAM, v.toString());
       }
       return map;
     }
@@ -608,7 +606,7 @@ public final class CustomAnalyzer extends Analyzer {
         throw new IllegalArgumentException(
             "Key-value pairs expected, so the number of params must be even.");
       }
-      final Map<String, String> map = new HashMap<>();
+      final Map<String, String> map = CollectionUtil.newHashMap(params.length);
       for (int i = 0; i < params.length; i += 2) {
         Objects.requireNonNull(params[i], "Key of param may not be null.");
         map.put(params[i], params[i + 1]);
diff --git a/lucene/analysis/phonetic/src/java/org/apache/lucene/analysis/phonetic/PhoneticFilterFactory.java b/lucene/analysis/phonetic/src/java/org/apache/lucene/analysis/phonetic/PhoneticFilterFactory.java
index 37eab53ca71..15a454202da 100644
--- a/lucene/analysis/phonetic/src/java/org/apache/lucene/analysis/phonetic/PhoneticFilterFactory.java
+++ b/lucene/analysis/phonetic/src/java/org/apache/lucene/analysis/phonetic/PhoneticFilterFactory.java
@@ -19,7 +19,6 @@ package org.apache.lucene.analysis.phonetic;
 import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
-import java.util.HashMap;
 import java.util.Locale;
 import java.util.Map;
 import org.apache.commons.codec.Encoder;
@@ -83,18 +82,15 @@ public class PhoneticFilterFactory extends TokenFilterFactory implements Resourc
 
   private static final String PACKAGE_CONTAINING_ENCODERS = "org.apache.commons.codec.language.";
 
-  // Effectively constants; uppercase keys
-  private static final Map<String, Class<? extends Encoder>> registry = new HashMap<>(6);
-
-  static {
-    registry.put("DoubleMetaphone".toUpperCase(Locale.ROOT), DoubleMetaphone.class);
-    registry.put("Metaphone".toUpperCase(Locale.ROOT), Metaphone.class);
-    registry.put("Soundex".toUpperCase(Locale.ROOT), Soundex.class);
-    registry.put("RefinedSoundex".toUpperCase(Locale.ROOT), RefinedSoundex.class);
-    registry.put("Caverphone".toUpperCase(Locale.ROOT), Caverphone2.class);
-    registry.put("ColognePhonetic".toUpperCase(Locale.ROOT), ColognePhonetic.class);
-    registry.put("Nysiis".toUpperCase(Locale.ROOT), Nysiis.class);
-  }
+  private static final Map<String, Class<? extends Encoder>> registry =
+      Map.of(
+          "DoubleMetaphone".toUpperCase(Locale.ROOT), DoubleMetaphone.class,
+          "Metaphone".toUpperCase(Locale.ROOT), Metaphone.class,
+          "Soundex".toUpperCase(Locale.ROOT), Soundex.class,
+          "RefinedSoundex".toUpperCase(Locale.ROOT), RefinedSoundex.class,
+          "Caverphone".toUpperCase(Locale.ROOT), Caverphone2.class,
+          "ColognePhonetic".toUpperCase(Locale.ROOT), ColognePhonetic.class,
+          "Nysiis".toUpperCase(Locale.ROOT), Nysiis.class);
 
   final boolean inject; // accessed by the test
   private final String name;
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/Lucene40BlockTreeTermsReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/Lucene40BlockTreeTermsReader.java
index 45b3cc04995..6ea2533574c 100644
--- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/Lucene40BlockTreeTermsReader.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/Lucene40BlockTreeTermsReader.java
@@ -19,7 +19,6 @@ package org.apache.lucene.backward_codecs.lucene40.blocktree;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -36,6 +35,7 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.fst.ByteSequenceOutputs;
 import org.apache.lucene.util.fst.Outputs;
@@ -198,7 +198,7 @@ public final class Lucene40BlockTreeTermsReader extends FieldsProducer {
           if (numFields < 0) {
             throw new CorruptIndexException("invalid numFields: " + numFields, termsMetaIn);
           }
-          fieldMap = new HashMap<>((int) (numFields / 0.75f) + 1);
+          fieldMap = CollectionUtil.newHashMap(numFields);
           for (int i = 0; i < numFields; ++i) {
             final int field = termsMetaIn.readVInt();
             final long numTerms = termsMetaIn.readVLong();
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene50/Lucene50CompoundReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene50/Lucene50CompoundReader.java
index dbb2892d012..d833ba7b342 100644
--- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene50/Lucene50CompoundReader.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene50/Lucene50CompoundReader.java
@@ -19,7 +19,6 @@ package org.apache.lucene.backward_codecs.lucene50;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 import org.apache.lucene.backward_codecs.store.EndiannessReverserUtil;
@@ -32,6 +31,7 @@ import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.IOUtils;
 
 /**
@@ -132,7 +132,7 @@ final class Lucene50CompoundReader extends CompoundDirectory {
 
   private Map<String, FileEntry> readMapping(IndexInput entriesStream) throws IOException {
     final int numEntries = entriesStream.readVInt();
-    Map<String, FileEntry> mapping = new HashMap<>(numEntries);
+    Map<String, FileEntry> mapping = CollectionUtil.newHashMap(numEntries);
     for (int i = 0; i < numEntries; i++) {
       final FileEntry fileEntry = new FileEntry();
       final String id = entriesStream.readString();
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestDocValuesCompression.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestDocValuesCompression.java
index 42e9ec28be6..4aae092e2e1 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestDocValuesCompression.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene80/TestDocValuesCompression.java
@@ -41,6 +41,7 @@ import org.apache.lucene.tests.util.LuceneTestCase;
 import org.apache.lucene.tests.util.LuceneTestCase.Nightly;
 import org.apache.lucene.tests.util.TestUtil;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CollectionUtil;
 
 @Nightly // N-2 formats are only tested on nightly runs
 public class TestDocValuesCompression extends LuceneTestCase {
@@ -89,7 +90,7 @@ public class TestDocValuesCompression extends LuceneTestCase {
 
   public void testReseekAfterSkipDecompression() throws IOException {
     final int CARDINALITY = (Lucene80DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SIZE << 1) + 11;
-    Set<String> valueSet = new HashSet<>(CARDINALITY);
+    Set<String> valueSet = CollectionUtil.newHashSet(CARDINALITY);
     for (int i = 0; i < CARDINALITY; i++) {
       valueSet.add(TestUtil.randomSimpleString(random(), 64));
     }
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 ae0d86d6769..e7981f72e37 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
@@ -20,8 +20,6 @@ import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.text.ParseException;
 import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import org.apache.lucene.codecs.SegmentInfoFormat;
@@ -41,6 +39,7 @@ import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.Version;
 
@@ -117,7 +116,7 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
       SimpleTextUtil.readLine(input, scratch);
       assert StringHelper.startsWith(scratch.get(), SI_NUM_DIAG);
       int numDiag = Integer.parseInt(readString(SI_NUM_DIAG.length, scratch));
-      Map<String, String> diagnostics = new HashMap<>();
+      Map<String, String> diagnostics = CollectionUtil.newHashMap(numDiag);
 
       for (int i = 0; i < numDiag; i++) {
         SimpleTextUtil.readLine(input, scratch);
@@ -133,7 +132,7 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
       SimpleTextUtil.readLine(input, scratch);
       assert StringHelper.startsWith(scratch.get(), SI_NUM_ATT);
       int numAtt = Integer.parseInt(readString(SI_NUM_ATT.length, scratch));
-      Map<String, String> attributes = new HashMap<>(numAtt);
+      Map<String, String> attributes = CollectionUtil.newHashMap(numAtt);
 
       for (int i = 0; i < numAtt; i++) {
         SimpleTextUtil.readLine(input, scratch);
@@ -149,7 +148,7 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
       SimpleTextUtil.readLine(input, scratch);
       assert StringHelper.startsWith(scratch.get(), SI_NUM_FILES);
       int numFiles = Integer.parseInt(readString(SI_NUM_FILES.length, scratch));
-      Set<String> files = new HashSet<>();
+      Set<String> files = CollectionUtil.newHashSet(numFiles);
 
       for (int i = 0; i < numFiles; i++) {
         SimpleTextUtil.readLine(input, scratch);
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STUniformSplitTermsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STUniformSplitTermsWriter.java
index 1803c8cb420..25d4a81a592 100755
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STUniformSplitTermsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STUniformSplitTermsWriter.java
@@ -50,6 +50,7 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.ByteBuffersDataOutput;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.PriorityQueue;
 
 /**
@@ -332,7 +333,7 @@ public class STUniformSplitTermsWriter extends UniformSplitTermsWriter {
     TermIteratorQueue<SegmentTerms> segmentTermsQueue = createSegmentTermsQueue(segmentTermsList);
     List<TermIterator<SegmentTerms>> groupedSegmentTerms = new ArrayList<>(segmentTermsList.size());
     Map<String, List<SegmentPostings>> fieldPostingsMap =
-        new HashMap<>(mergeState.fieldInfos.length);
+        CollectionUtil.newHashMap(mergeState.fieldInfos.length);
     List<MergingFieldTerms> groupedFieldTerms = new ArrayList<>(mergeState.fieldInfos.length);
     List<FieldMetadataTermState> termStates = new ArrayList<>(mergeState.fieldInfos.length);
 
@@ -351,7 +352,8 @@ public class STUniformSplitTermsWriter extends UniformSplitTermsWriter {
 
   private Map<String, MergingFieldTerms> createMergingFieldTermsMap(
       List<FieldMetadata> fieldMetadataList, int numSegments) {
-    Map<String, MergingFieldTerms> fieldTermsMap = new HashMap<>(fieldMetadataList.size() * 2);
+    Map<String, MergingFieldTerms> fieldTermsMap =
+        CollectionUtil.newHashMap(fieldMetadataList.size());
     for (FieldMetadata fieldMetadata : fieldMetadataList) {
       FieldInfo fieldInfo = fieldMetadata.getFieldInfo();
       fieldTermsMap.put(
@@ -382,11 +384,9 @@ public class STUniformSplitTermsWriter extends UniformSplitTermsWriter {
       SegmentTerms segmentTerms = (SegmentTerms) segmentTermIterator;
       for (Map.Entry<String, BlockTermState> fieldTermState :
           segmentTerms.fieldTermStatesMap.entrySet()) {
-        List<SegmentPostings> segmentPostingsList = fieldPostingsMap.get(fieldTermState.getKey());
-        if (segmentPostingsList == null) {
-          segmentPostingsList = new ArrayList<>(groupedSegmentTerms.size());
-          fieldPostingsMap.put(fieldTermState.getKey(), segmentPostingsList);
-        }
+        List<SegmentPostings> segmentPostingsList =
+            fieldPostingsMap.computeIfAbsent(
+                fieldTermState.getKey(), k -> new ArrayList<>(groupedSegmentTerms.size()));
         segmentPostingsList.add(
             new SegmentPostings(
                 segmentTerms.segmentIndex,
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90CompoundReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90CompoundReader.java
index 9686375de4a..a6719a119fa 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90CompoundReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90CompoundReader.java
@@ -19,7 +19,6 @@ package org.apache.lucene.codecs.lucene90;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 import org.apache.lucene.codecs.CodecUtil;
@@ -31,6 +30,7 @@ import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.IOUtils;
 
 /**
@@ -132,7 +132,7 @@ final class Lucene90CompoundReader extends CompoundDirectory {
 
   private Map<String, FileEntry> readMapping(IndexInput entriesStream) throws IOException {
     final int numEntries = entriesStream.readVInt();
-    Map<String, FileEntry> mapping = new HashMap<>(numEntries);
+    Map<String, FileEntry> mapping = CollectionUtil.newHashMap(numEntries);
     for (int i = 0; i < numEntries; i++) {
       final FileEntry fileEntry = new FileEntry();
       final String id = entriesStream.readString();
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsReader.java
index 659f7618ec9..cbc80b0cf01 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/Lucene90BlockTreeTermsReader.java
@@ -19,7 +19,6 @@ package org.apache.lucene.codecs.lucene90.blocktree;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -36,6 +35,7 @@ import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.fst.ByteSequenceOutputs;
 import org.apache.lucene.util.fst.Outputs;
@@ -166,7 +166,7 @@ public final class Lucene90BlockTreeTermsReader extends FieldsProducer {
           if (numFields < 0) {
             throw new CorruptIndexException("invalid numFields: " + numFields, metaIn);
           }
-          fieldMap = new HashMap<>((int) (numFields / 0.75f) + 1);
+          fieldMap = CollectionUtil.newHashMap(numFields);
           for (int i = 0; i < numFields; ++i) {
             final int field = metaIn.readVInt();
             final long numTerms = metaIn.readVLong();
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
index 64688972f4d..515e68aa29c 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
@@ -43,6 +43,7 @@ import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
+import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.MergedIterator;
 
@@ -266,7 +267,7 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
       }
 
       Map<PostingsFormat, FieldsGroup> formatToGroups =
-          new HashMap<>((int) (formatToGroupBuilders.size() / 0.75f) + 1);
+          CollectionUtil.newHashMap(formatToGroupBuilders.size());
       formatToGroupBuilders.forEach(
           (postingsFormat, builder) -> formatToGroups.put(postingsFormat, builder.build()));
       return formatToGroups;
diff --git a/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java b/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java
index 39aa5e98db9..838a0494fbf 100644
--- a/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java
+++ b/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java
@@ -17,6 +17,7 @@
 package org.apache.lucene.document;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Objects;
 import java.util.Set;
@@ -49,10 +50,7 @@ public class DocumentStoredFieldVisitor extends StoredFieldVisitor {
 
   /** Load only fields named in the provided fields. */
   public DocumentStoredFieldVisitor(String... fields) {
-    fieldsToAdd = new HashSet<>(fields.length);
-    for (String field : fields) {
-      fieldsToAdd.add(field);
-    }
+    fieldsToAdd = new HashSet<>(Arrays.asList(fields));
   }
 
   /** Load all stored fields. */
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
index c1cc07c0ccb..5ef4f5c3b53 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
@@ -43,6 +43,7 @@ import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.Version;
@@ -439,7 +440,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
       if (numDVFields == 0) {
         dvUpdateFiles = Collections.emptyMap();
       } else {
-        Map<Integer, Set<String>> map = new HashMap<>(numDVFields);
+        Map<Integer, Set<String>> map = CollectionUtil.newHashMap(numDVFields);
         for (int i = 0; i < numDVFields; i++) {
           map.put(CodecUtil.readBEInt(input), input.readSetOfStrings());
         }
diff --git a/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java b/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
index ab7b3e1442f..614c6c306e0 100644
--- a/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
@@ -23,7 +23,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -32,6 +31,7 @@ import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.IOFunction;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.Version;
@@ -184,10 +184,10 @@ public final class StandardDirectoryReader extends DirectoryReader {
 
     // we put the old SegmentReaders in a map, that allows us
     // to lookup a reader using its segment name
-    final Map<String, Integer> segmentReaders =
-        (oldReaders == null ? Collections.emptyMap() : new HashMap<>(oldReaders.size()));
+    Map<String, Integer> segmentReaders = Collections.emptyMap();
 
     if (oldReaders != null) {
+      segmentReaders = CollectionUtil.newHashMap(oldReaders.size());
       // create a Map SegmentName->SegmentReader
       for (int i = 0, c = oldReaders.size(); i < c; i++) {
         final SegmentReader sr = (SegmentReader) oldReaders.get(i);
diff --git a/lucene/core/src/java/org/apache/lucene/util/CollectionUtil.java b/lucene/core/src/java/org/apache/lucene/util/CollectionUtil.java
index bc1eab3e67b..8e66694543a 100644
--- a/lucene/core/src/java/org/apache/lucene/util/CollectionUtil.java
+++ b/lucene/core/src/java/org/apache/lucene/util/CollectionUtil.java
@@ -18,13 +18,15 @@ package org.apache.lucene.util;
 
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.RandomAccess;
 
 /**
- * Methods for manipulating (sorting) collections. Sort methods work directly on the supplied lists
- * and don't copy to/from arrays before/after. For medium size collections as used in the Lucene
- * indexer that is much more efficient.
+ * Methods for manipulating (sorting) and creating collections. Sort methods work directly on the
+ * supplied lists and don't copy to/from arrays before/after. For medium size collections as used in
+ * the Lucene indexer that is much more efficient.
  *
  * @lucene.internal
  */
@@ -32,6 +34,24 @@ public final class CollectionUtil {
 
   private CollectionUtil() {} // no instance
 
+  /**
+   * Returns a new {@link HashMap} sized to contain {@code size} items without resizing the internal
+   * array.
+   */
+  public static <K, V> HashMap<K, V> newHashMap(int size) {
+    // This should be replaced with HashMap.newHashMap when lucene moves to jdk19 minimum version
+    return new HashMap<>((int) (size / 0.75f) + 1);
+  }
+
+  /**
+   * Returns a new {@link HashSet} sized to contain {@code size} items without resizing the internal
+   * array.
+   */
+  public static <E> HashSet<E> newHashSet(int size) {
+    // This should be replaced with HashSet.newHashSet when lucene moves to jdk19 minimum version
+    return new HashSet<>((int) (size / 0.75f) + 1);
+  }
+
   private static final class ListIntroSorter<T> extends IntroSorter {
 
     T pivot;
diff --git a/lucene/core/src/java/org/apache/lucene/util/MapOfSets.java b/lucene/core/src/java/org/apache/lucene/util/MapOfSets.java
index eef74b7f69f..bb4df20e384 100644
--- a/lucene/core/src/java/org/apache/lucene/util/MapOfSets.java
+++ b/lucene/core/src/java/org/apache/lucene/util/MapOfSets.java
@@ -52,13 +52,7 @@ public class MapOfSets<K, V> {
    * @return the size of the Set associated with key once val is added to it.
    */
   public int put(K key, V val) {
-    final Set<V> theSet;
-    if (theMap.containsKey(key)) {
-      theSet = theMap.get(key);
-    } else {
-      theSet = new HashSet<>(23);
-      theMap.put(key, theSet);
-    }
+    final Set<V> theSet = theMap.computeIfAbsent(key, k -> new HashSet<>(23));
     theSet.add(val);
     return theSet.size();
   }
@@ -69,13 +63,7 @@ public class MapOfSets<K, V> {
    * @return the size of the Set associated with key once val is added to it.
    */
   public int putAll(K key, Collection<? extends V> vals) {
-    final Set<V> theSet;
-    if (theMap.containsKey(key)) {
-      theSet = theMap.get(key);
-    } else {
-      theSet = new HashSet<>(23);
-      theMap.put(key, theSet);
-    }
+    final Set<V> theSet = theMap.computeIfAbsent(key, k -> new HashSet<>(23));
     theSet.addAll(vals);
     return theSet.size();
   }
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormat.java
index 20764441cea..ec5add25531 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormat.java
@@ -69,6 +69,7 @@ import org.apache.lucene.tests.index.RandomIndexWriter;
 import org.apache.lucene.tests.util.TestUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.CollectionUtil;
 
 /** Tests Lucene90DocValuesFormat */
 public class TestLucene90DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
@@ -778,7 +779,7 @@ public class TestLucene90DocValuesFormat extends BaseCompressingDocValuesFormatT
 
   public void testReseekAfterSkipDecompression() throws IOException {
     final int CARDINALITY = (Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SIZE << 1) + 11;
-    Set<String> valueSet = new HashSet<>(CARDINALITY);
+    Set<String> valueSet = CollectionUtil.newHashSet(CARDINALITY);
     for (int i = 0; i < CARDINALITY; i++) {
       valueSet.add(TestUtil.randomSimpleString(random(), 64));
     }
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestCollectorManager.java b/lucene/core/src/test/org/apache/lucene/search/TestCollectorManager.java
index 7c1adf543b9..ad21aa1de37 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestCollectorManager.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestCollectorManager.java
@@ -24,12 +24,11 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Random;
-import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
 import java.util.function.Predicate;
-import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.DirectoryReader;
@@ -57,7 +56,7 @@ public class TestCollectorManager extends LuceneTestCase {
 
     for (int iter = 0; iter < 100; iter++) {
       int docs = RandomNumbers.randomIntBetween(random(), 1000, 10000);
-      List<Integer> expected = generateDocIds(docs, random());
+      Collection<Integer> expected = generateDocIds(docs, random());
       IntStream expectedEven = expected.stream().filter(evenPredicate).mapToInt(i -> i);
       IntStream expectedOdd = expected.stream().filter(oddPredicate).mapToInt(i -> i);
 
@@ -91,7 +90,7 @@ public class TestCollectorManager extends LuceneTestCase {
   }
 
   private static <C extends Collector> Object collectAll(
-      LeafReaderContext ctx, List<Integer> values, CollectorManager<C, ?> collectorManager)
+      LeafReaderContext ctx, Collection<Integer> values, CollectorManager<C, ?> collectorManager)
       throws IOException {
     List<C> collectors = new ArrayList<>();
     C collector = collectorManager.newCollector();
@@ -109,18 +108,17 @@ public class TestCollectorManager extends LuceneTestCase {
   }
 
   /**
-   * Generate test doc ids. This will de-dupe and create a sorted list to be more realistic with
-   * real-world use-cases. Note that it's possible this will generate fewer than 'count' entries
-   * because of de-duping, but that should be quite rare and probably isn't worth worrying about for
-   * these testing purposes.
+   * Generate test doc ids. This will de-dupe and create a sorted collection to be more realistic
+   * with real-world use-cases. Note that it's possible this will generate fewer than 'count'
+   * entries because of de-duping, but that should be quite rare and probably isn't worth worrying
+   * about for these testing purposes.
    */
-  private List<Integer> generateDocIds(int count, Random random) {
-    Set<Integer> generated = new HashSet<>(count);
+  private static SortedSet<Integer> generateDocIds(int count, Random random) {
+    SortedSet<Integer> generated = new TreeSet<>();
     for (int i = 0; i < count; i++) {
       generated.add(random.nextInt());
     }
-
-    return generated.stream().sorted().collect(Collectors.toList());
+    return generated;
   }
 
   private static final class CompositeCollectorManager
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestMultiCollectorManager.java b/lucene/core/src/test/org/apache/lucene/search/TestMultiCollectorManager.java
index 22017a863cc..fcb6dce2f79 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestMultiCollectorManager.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestMultiCollectorManager.java
@@ -20,10 +20,10 @@ import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Random;
-import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import org.apache.lucene.document.Document;
@@ -52,7 +52,7 @@ public class TestMultiCollectorManager extends LuceneTestCase {
 
     for (int iter = 0; iter < 100; iter++) {
       int docs = RandomNumbers.randomIntBetween(random(), 1000, 10000);
-      List<Integer> expected = generateDocIds(docs, random());
+      SortedSet<Integer> expected = generateDocIds(docs, random());
       List<Integer> expectedEven =
           expected.stream().filter(evenPredicate).collect(Collectors.toList());
       List<Integer> expectedOdd =
@@ -157,7 +157,7 @@ public class TestMultiCollectorManager extends LuceneTestCase {
     LeafReaderContext ctx = reader.leaves().get(0);
 
     int docs = RandomNumbers.randomIntBetween(random(), 1000, 10000);
-    List<Integer> expected = generateDocIds(docs, random());
+    Collection<Integer> expected = generateDocIds(docs, random());
 
     // The first collector manager should collect all docs even though the second throws
     // CollectionTerminatedException immediately:
@@ -192,7 +192,7 @@ public class TestMultiCollectorManager extends LuceneTestCase {
   }
 
   private static <C extends Collector> Object collectAll(
-      LeafReaderContext ctx, List<Integer> values, CollectorManager<C, ?> collectorManager)
+      LeafReaderContext ctx, Collection<Integer> values, CollectorManager<C, ?> collectorManager)
       throws IOException {
     List<C> collectors = new ArrayList<>();
     C collector = collectorManager.newCollector();
@@ -210,18 +210,17 @@ public class TestMultiCollectorManager extends LuceneTestCase {
   }
 
   /**
-   * Generate test doc ids. This will de-dupe and create a sorted list to be more realistic with
-   * real-world use-cases. Note that it's possible this will generate fewer than 'count' entries
-   * because of de-duping, but that should be quite rare and probably isn't worth worrying about for
-   * these testing purposes.
+   * Generate test doc ids. This will de-dupe and create a sorted collection to be more realistic
+   * with real-world use-cases. Note that it's possible this will generate fewer than 'count'
+   * entries because of de-duping, but that should be quite rare and probably isn't worth worrying
+   * about for these testing purposes.
    */
-  private List<Integer> generateDocIds(int count, Random random) {
-    Set<Integer> generated = new HashSet<>(count);
+  private static SortedSet<Integer> generateDocIds(int count, Random random) {
+    SortedSet<Integer> generated = new TreeSet<>();
     for (int i = 0; i < count; i++) {
       generated.add(random.nextInt());
     }
-
-    return generated.stream().sorted().collect(Collectors.toList());
+    return generated;
   }
 
   private static final class SimpleCollectorManager
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestSentinelIntSet.java b/lucene/core/src/test/org/apache/lucene/util/TestSentinelIntSet.java
index 17424a276a3..bf9785b5838 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestSentinelIntSet.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestSentinelIntSet.java
@@ -51,7 +51,7 @@ public class TestSentinelIntSet extends LuceneTestCase {
       int maxVal =
           (random().nextBoolean() ? random().nextInt(50) : random().nextInt(Integer.MAX_VALUE)) + 1;
 
-      HashSet<Integer> a = new HashSet<>(initSz);
+      HashSet<Integer> a = CollectionUtil.newHashSet(initSz);
       SentinelIntSet b = new SentinelIntSet(initSz, -1);
 
       for (int j = 0; j < num; j++) {
diff --git a/lucene/core/src/test/org/apache/lucene/util/bkd/TestDocIdsWriter.java b/lucene/core/src/test/org/apache/lucene/util/bkd/TestDocIdsWriter.java
index 7409c71821c..6610bdb620d 100644
--- a/lucene/core/src/test/org/apache/lucene/util/bkd/TestDocIdsWriter.java
+++ b/lucene/core/src/test/org/apache/lucene/util/bkd/TestDocIdsWriter.java
@@ -18,7 +18,6 @@ package org.apache.lucene.util.bkd;
 
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.HashSet;
 import java.util.Set;
 import org.apache.lucene.index.PointValues.IntersectVisitor;
 import org.apache.lucene.index.PointValues.Relation;
@@ -28,6 +27,7 @@ import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.tests.util.LuceneTestCase;
 import org.apache.lucene.tests.util.TestUtil;
+import org.apache.lucene.util.CollectionUtil;
 
 public class TestDocIdsWriter extends LuceneTestCase {
 
@@ -80,7 +80,7 @@ public class TestDocIdsWriter extends LuceneTestCase {
     try (Directory dir = newDirectory()) {
       for (int iter = 0; iter < numIters; ++iter) {
         int size = 1 + random().nextInt(5000);
-        Set<Integer> set = new HashSet<>(size);
+        Set<Integer> set = CollectionUtil.newHashSet(size);
         int small = random().nextInt(1000);
         while (set.size() < size) {
           set.add(small + random().nextInt(size * 16));
diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java
index 1f34f67a92e..a03563c25f9 100644
--- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java
+++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java
@@ -30,6 +30,7 @@ import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.SimpleCollector;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
+import org.apache.lucene.util.CollectionUtil;
 
 /**
  * FirstPassGroupingCollector is the first of two passes necessary to collect grouped hits. This
@@ -97,7 +98,7 @@ public class FirstPassGroupingCollector<T> extends SimpleCollector {
     }
 
     spareSlot = topNGroups;
-    groupMap = new HashMap<>(topNGroups);
+    groupMap = CollectionUtil.newHashMap(topNGroups);
   }
 
   @Override
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/QueryScorer.java b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/QueryScorer.java
index 3a46ff94733..b10a64230aa 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/QueryScorer.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/QueryScorer.java
@@ -17,7 +17,6 @@
 package org.apache.lucene.search.highlight;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
@@ -29,6 +28,7 @@ import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.memory.MemoryIndex;
 import org.apache.lucene.queries.spans.SpanQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.util.CollectionUtil;
 
 /**
  * {@link Scorer} implementation which scores text fragments by the number of unique query terms
@@ -99,16 +99,16 @@ public class QueryScorer implements Scorer {
    * @param weightedTerms an array of pre-created {@link WeightedSpanTerm}s
    */
   public QueryScorer(WeightedSpanTerm[] weightedTerms) {
-    this.fieldWeightedSpanTerms = new HashMap<>(weightedTerms.length);
+    this.fieldWeightedSpanTerms = CollectionUtil.newHashMap(weightedTerms.length);
 
-    for (int i = 0; i < weightedTerms.length; i++) {
-      WeightedSpanTerm existingTerm = fieldWeightedSpanTerms.get(weightedTerms[i].term);
+    for (WeightedSpanTerm weightedTerm : weightedTerms) {
+      WeightedSpanTerm existingTerm = fieldWeightedSpanTerms.get(weightedTerm.term);
 
-      if ((existingTerm == null) || (existingTerm.weight < weightedTerms[i].weight)) {
+      if ((existingTerm == null) || (existingTerm.weight < weightedTerm.weight)) {
         // if a term is defined more than once, always use the highest
         // scoring weight
-        fieldWeightedSpanTerms.put(weightedTerms[i].term, weightedTerms[i]);
-        maxTermWeight = Math.max(maxTermWeight, weightedTerms[i].getWeight());
+        fieldWeightedSpanTerms.put(weightedTerm.term, weightedTerm);
+        maxTermWeight = Math.max(maxTermWeight, weightedTerm.getWeight());
       }
     }
     skipInitExtractor = true;
@@ -156,9 +156,8 @@ public class QueryScorer implements Scorer {
     float score = weightedSpanTerm.getWeight();
 
     // found a query term - is it unique in this doc?
-    if (!foundTerms.contains(termText)) {
+    if (foundTerms.add(termText)) {
       totalScore += score;
-      foundTerms.add(termText);
     }
 
     return score;
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UnifiedHighlighter.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UnifiedHighlighter.java
index efb6e939bd0..44385b9085a 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UnifiedHighlighter.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UnifiedHighlighter.java
@@ -58,6 +58,7 @@ import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.InPlaceMergeSorter;
 
@@ -953,7 +954,7 @@ public class UnifiedHighlighter {
     //    caller simply iterates it to build another structure.
 
     // field -> object highlights parallel to docIdsIn
-    Map<String, Object[]> resultMap = new HashMap<>(fields.length);
+    Map<String, Object[]> resultMap = CollectionUtil.newHashMap(fields.length);
     for (int f = 0; f < fields.length; f++) {
       resultMap.put(fields[f], highlightDocsInByField[f]);
     }
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/QueryDecomposer.java b/lucene/monitor/src/java/org/apache/lucene/monitor/QueryDecomposer.java
index ef1ed6a9836..81679f8bb55 100644
--- a/lucene/monitor/src/java/org/apache/lucene/monitor/QueryDecomposer.java
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/QueryDecomposer.java
@@ -25,6 +25,7 @@ import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BoostQuery;
 import org.apache.lucene.search.DisjunctionMaxQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.util.CollectionUtil;
 
 /**
  * Split a disjunction query into its consituent parts, so that they can be indexed and run
@@ -103,7 +104,7 @@ public class QueryDecomposer {
 
     // If there are exclusions, then we need to add them to all the decomposed
     // queries
-    Set<Query> rewrittenSubqueries = new HashSet<>(subqueries.size());
+    Set<Query> rewrittenSubqueries = CollectionUtil.newHashSet(subqueries.size());
     for (Query subquery : subqueries) {
       BooleanQuery.Builder bq = new BooleanQuery.Builder();
       bq.add(subquery, BooleanClause.Occur.MUST);
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/messages/NLS.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/messages/NLS.java
index 67d030b3579..da72b80e19f 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/messages/NLS.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/messages/NLS.java
@@ -20,11 +20,11 @@ import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
 import java.text.MessageFormat;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.Locale;
 import java.util.Map;
 import java.util.MissingResourceException;
 import java.util.ResourceBundle;
+import org.apache.lucene.util.CollectionUtil;
 
 /**
  * MessageBundles classes extend this class, to implement a bundle.
@@ -41,7 +41,7 @@ import java.util.ResourceBundle;
  */
 public class NLS {
 
-  private static Map<String, Class<? extends NLS>> bundles = new HashMap<>(0);
+  private static final Map<String, Class<? extends NLS>> bundles = new HashMap<>(0);
 
   protected NLS() {
     // Do not instantiate
@@ -83,7 +83,7 @@ public class NLS {
   protected static void initializeMessages(String bundleName, Class<? extends NLS> clazz) {
     try {
       load(clazz);
-      if (!bundles.containsKey(bundleName)) bundles.put(bundleName, clazz);
+      bundles.putIfAbsent(bundleName, clazz);
     } catch (
         @SuppressWarnings("unused")
         Throwable e) {
@@ -96,8 +96,7 @@ public class NLS {
 
     // slow resource checking
     // need to loop thru all registered resource bundles
-    for (Iterator<String> it = bundles.keySet().iterator(); it.hasNext(); ) {
-      Class<? extends NLS> clazz = bundles.get(it.next());
+    for (Class<? extends NLS> clazz : bundles.values()) {
       ResourceBundle resourceBundle = ResourceBundle.getBundle(clazz.getName(), locale);
       if (resourceBundle != null) {
         try {
@@ -119,10 +118,10 @@ public class NLS {
 
     // build a map of field names to Field objects
     final int len = fieldArray.length;
-    Map<String, Field> fields = new HashMap<>(len * 2);
-    for (int i = 0; i < len; i++) {
-      fields.put(fieldArray[i].getName(), fieldArray[i]);
-      loadfieldValue(fieldArray[i], clazz);
+    Map<String, Field> fields = CollectionUtil.newHashMap(len);
+    for (Field field : fieldArray) {
+      fields.put(field.getName(), field);
+      loadfieldValue(field, clazz);
     }
   }
 
diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerBreakdown.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerBreakdown.java
index 895ac00eaed..03a1880d48e 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerBreakdown.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerBreakdown.java
@@ -18,8 +18,8 @@
 package org.apache.lucene.sandbox.search;
 
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Map;
+import org.apache.lucene.util.CollectionUtil;
 
 /**
  * A record of timings for the various operations that may happen during query execution. A node's
@@ -44,7 +44,7 @@ class QueryProfilerBreakdown {
 
   /** Build a timing count breakdown. */
   public final Map<String, Long> toBreakdownMap() {
-    Map<String, Long> map = new HashMap<>(timers.length * 2);
+    Map<String, Long> map = CollectionUtil.newHashMap(timers.length * 2);
     for (QueryProfilerTimingType type : QueryProfilerTimingType.values()) {
       map.put(type.toString(), timers[type.ordinal()].getApproximateTiming());
       map.put(type.toString() + "_count", timers[type.ordinal()].getCount());
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/S2PrefixTreeCell.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/S2PrefixTreeCell.java
index ee283141921..4b8bdd81a13 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/S2PrefixTreeCell.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/S2PrefixTreeCell.java
@@ -35,7 +35,7 @@ import org.locationtech.spatial4j.shape.SpatialRelation;
 class S2PrefixTreeCell implements CellCanPrune {
 
   // Faces of S2 Geometry
-  private static S2CellId[] FACES = new S2CellId[6];
+  private static final S2CellId[] FACES = new S2CellId[6];
 
   static {
     FACES[0] = S2CellId.fromFacePosLevel(0, 0, 0);
@@ -59,10 +59,11 @@ class S2PrefixTreeCell implements CellCanPrune {
   private static final Map<Byte, Integer> PIXELS;
 
   static {
-    PIXELS = new HashMap<>(TOKENS.length);
+    Map<Byte, Integer> pixels = new HashMap<>();
     for (int i = 0; i < TOKENS.length; i++) {
-      PIXELS.put(TOKENS[i], i);
+      pixels.put(TOKENS[i], i);
     }
+    PIXELS = Map.copyOf(pixels);
   }
 
   S2CellId cellId;
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConcavePolygon.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConcavePolygon.java
index 203d6bd0fa1..f3f1a04d2fb 100644
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConcavePolygon.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConcavePolygon.java
@@ -21,9 +21,9 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.BitSet;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import org.apache.lucene.util.CollectionUtil;
 
 /**
  * GeoConcavePolygon objects are generic building blocks of more complex structures. The only
@@ -245,9 +245,9 @@ class GeoConcavePolygon extends GeoBasePolygon {
     }
 
     // For each edge, create a bounds object.
-    eitherBounds = new HashMap<>(edges.length);
-    prevBrotherMap = new HashMap<>(edges.length);
-    nextBrotherMap = new HashMap<>(edges.length);
+    eitherBounds = CollectionUtil.newHashMap(edges.length);
+    prevBrotherMap = CollectionUtil.newHashMap(edges.length);
+    nextBrotherMap = CollectionUtil.newHashMap(edges.length);
     for (int edgeIndex = 0; edgeIndex < edges.length; edgeIndex++) {
       final SidedPlane edge = edges[edgeIndex];
       final SidedPlane invertedEdge = invertedEdges[edgeIndex];
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConvexPolygon.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConvexPolygon.java
index 9d2550b8180..5d8363d5fad 100755
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConvexPolygon.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/geom/GeoConvexPolygon.java
@@ -21,9 +21,9 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.BitSet;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import org.apache.lucene.util.CollectionUtil;
 
 /**
  * GeoConvexPolygon objects are generic building blocks of more complex structures. The only
@@ -239,9 +239,9 @@ class GeoConvexPolygon extends GeoBasePolygon {
     }
 
     // For each edge, create a bounds object.
-    eitherBounds = new HashMap<>(edges.length);
-    prevBrotherMap = new HashMap<>(edges.length);
-    nextBrotherMap = new HashMap<>(edges.length);
+    eitherBounds = CollectionUtil.newHashMap(edges.length);
+    prevBrotherMap = CollectionUtil.newHashMap(edges.length);
+    nextBrotherMap = CollectionUtil.newHashMap(edges.length);
     for (int edgeIndex = 0; edgeIndex < edges.length; edgeIndex++) {
       final SidedPlane edge = edges[edgeIndex];
       int bound1Index = legalIndex(edgeIndex + 1);
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java
index e93351af45f..c556a92256b 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java
@@ -25,7 +25,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -41,6 +40,7 @@ import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
+import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.IOUtils;
 
 /**
@@ -104,7 +104,7 @@ final class CompletionFieldsProducer extends FieldsProducer implements Accountab
 
       // read suggest field numbers and their offsets in the terms file from index
       int numFields = index.readVInt();
-      readers = new HashMap<>(numFields);
+      readers = CollectionUtil.newHashMap(numFields);
       for (int i = 0; i < numFields; i++) {
         int fieldNumber = index.readVInt();
         long offset = index.readVLong();
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java
index d8d17af7ca0..f495ef7802c 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java
@@ -29,6 +29,7 @@ import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.RamUsageEstimator;
@@ -197,7 +198,7 @@ public class ContextQuery extends CompletionQuery implements Accountable {
     contextsAutomaton =
         Operations.determinize(contextsAutomaton, Operations.DEFAULT_DETERMINIZE_WORK_LIMIT);
 
-    final Map<IntsRef, Float> contextMap = new HashMap<>(contexts.size());
+    final Map<IntsRef, Float> contextMap = CollectionUtil.newHashMap(contexts.size());
     final TreeSet<Integer> contextLengths = new TreeSet<>();
     for (Map.Entry<IntsRef, ContextMetaData> entry : contexts.entrySet()) {
       ContextMetaData contextMetaData = entry.getValue();
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java
index 1b43e192069..bd25d3ef3af 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java
@@ -17,6 +17,8 @@
 package org.apache.lucene.search.suggest.document;
 
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -63,10 +65,9 @@ public class ContextSuggestField extends SuggestField {
   public ContextSuggestField(String name, String value, int weight, CharSequence... contexts) {
     super(name, value, weight);
     validate(value);
-    this.contexts = new HashSet<>((contexts != null) ? contexts.length : 0);
-    if (contexts != null) {
-      Collections.addAll(this.contexts, contexts);
-    }
+    Collection<CharSequence> contextColl =
+        contexts != null ? Arrays.asList(contexts) : Collections.emptyList();
+    this.contexts = new HashSet<>(contextColl);
   }
 
   /** Expert: Sub-classes can inject contexts at index-time */
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestSuggestStopFilterFactory.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestSuggestStopFilterFactory.java
index 5acbcfee415..1cc77b6cf01 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestSuggestStopFilterFactory.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestSuggestStopFilterFactory.java
@@ -17,12 +17,12 @@
 package org.apache.lucene.search.suggest.analyzing;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.Map;
 import org.apache.lucene.analysis.CharArraySet;
 import org.apache.lucene.analysis.en.EnglishAnalyzer;
 import org.apache.lucene.tests.analysis.BaseTokenStreamTestCase;
 import org.apache.lucene.util.ClasspathResourceLoader;
+import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.ResourceLoader;
 import org.apache.lucene.util.Version;
 
@@ -108,7 +108,7 @@ public class TestSuggestStopFilterFactory extends BaseTokenStreamTestCase {
     if (params.length % 2 != 0) {
       throw new IllegalArgumentException("invalid keysAndValues map");
     }
-    Map<String, String> args = new HashMap<>(params.length / 2);
+    Map<String, String> args = CollectionUtil.newHashMap(params.length / 2);
     for (int i = 0; i < params.length; i += 2) {
       String previous = args.put(params[i], params[i + 1]);
       assertNull("duplicate values for key: " + params[i], previous);
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseDocValuesFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseDocValuesFormatTestCase.java
index e5ca317bc02..5402a6c0f1c 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseDocValuesFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseDocValuesFormatTestCase.java
@@ -282,7 +282,7 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
     RandomIndexWriter iwriter = new RandomIndexWriter(random(), directory);
     int numDocs = 1 + random().nextInt(100);
 
-    HashMap<Integer, BytesRef> writtenValues = new HashMap<>(numDocs);
+    HashMap<Integer, BytesRef> writtenValues = new HashMap<>();
 
     // Small vocabulary ranges will be highly compressible
     int vocabRange = 1 + random().nextInt(Byte.MAX_VALUE - 1);
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseTermVectorsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseTermVectorsFormatTestCase.java
index dd8500de714..a349e4e918d 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseTermVectorsFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseTermVectorsFormatTestCase.java
@@ -74,6 +74,7 @@ import org.apache.lucene.tests.util.TestUtil;
 import org.apache.lucene.util.AttributeImpl;
 import org.apache.lucene.util.AttributeReflector;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.IOUtils;
 
 /**
@@ -250,26 +251,16 @@ public abstract class BaseTermVectorsFormatTestCase extends BaseIndexFileFormatT
         }
       }
 
-      positionToTerms = new HashMap<>(len);
-      startOffsetToTerms = new HashMap<>(len);
+      positionToTerms = CollectionUtil.newHashMap(len);
+      startOffsetToTerms = CollectionUtil.newHashMap(len);
       for (int i = 0; i < len; ++i) {
-        if (!positionToTerms.containsKey(positions[i])) {
-          positionToTerms.put(positions[i], new HashSet<Integer>(1));
-        }
-        positionToTerms.get(positions[i]).add(i);
-        if (!startOffsetToTerms.containsKey(startOffsets[i])) {
-          startOffsetToTerms.put(startOffsets[i], new HashSet<Integer>(1));
-        }
-        startOffsetToTerms.get(startOffsets[i]).add(i);
+        positionToTerms.computeIfAbsent(positions[i], k -> new HashSet<>(1)).add(i);
+        startOffsetToTerms.computeIfAbsent(startOffsets[i], k -> new HashSet<>(1)).add(i);
       }
 
       freqs = new HashMap<>();
       for (String term : terms) {
-        if (freqs.containsKey(term)) {
-          freqs.put(term, freqs.get(term) + 1);
-        } else {
-          freqs.put(term, 1);
-        }
+        freqs.merge(term, 1, Integer::sum);
       }
 
       addAttributeImpl(new PermissiveOffsetAttributeImpl());