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:49:51 UTC
[lucene] 01/02: 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 branch_9x
in repository https://gitbox.apache.org/repos/asf/lucene.git
commit e0b8c88ecc7d453608111da2ea1abc01d3631c13
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 | 4 ++--
.../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, 154 insertions(+), 160 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..24f5e1dd426 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,10 +313,10 @@ 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);
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 676f1483e40..0bf4024ffb9 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
@@ -40,6 +40,7 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CollectionUtil;
public class TestDocValuesCompression extends LuceneTestCase {
private final Codec bestSpeed =
@@ -87,7 +88,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 8650d96867d..b98c061b811 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;
@@ -450,7 +451,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 4acad8330d9..6111ba87800 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 {
@@ -780,7 +781,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 2d76099ae1c..41d496a8c01 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
@@ -59,6 +59,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;
@@ -966,7 +967,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 7414d98f310..973aabb462e 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());