Add missing spaces in concatenated strings (#12967)

This commit is contained in:
sabi0 2023-12-24 02:30:30 +01:00 committed by GitHub
parent dc9f154aa5
commit 02722eeb69
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
28 changed files with 137 additions and 152 deletions

View File

@ -94,8 +94,8 @@ public class Lucene90HnswVectorsFormat extends KnnVectorsFormat {
/**
* The number of candidate neighbors to track while searching the graph for each newly inserted
* node. Defaults to to {@link Lucene90HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link
* HnswGraph} for details.
* node. Defaults to {@link Lucene90HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link HnswGraph}
* for details.
*/
final int beamWidth;

View File

@ -70,7 +70,7 @@ import org.apache.lucene.util.hnsw.HnswGraph;
* <li><b>[int8]</b> if equals to -1, dense all documents have values for a field. If equals to
* 0, sparse some documents missing values.
* <li><b>array[int]</b> for sparse case, the docids of documents having vectors, in order
* <li><b>[int]</b> the maximum number of connections (neigbours) that each node can have
* <li><b>[int]</b> the maximum number of connections (neighbours) that each node can have
* <li><b>[int]</b> number of levels in the graph
* <li>Graph nodes by level. For each level
* <ul>
@ -110,8 +110,8 @@ public class Lucene91HnswVectorsFormat extends KnnVectorsFormat {
/**
* The number of candidate neighbors to track while searching the graph for each newly inserted
* node. Defaults to to {@link Lucene91HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link
* HnswGraph} for details.
* node. Defaults to {@link Lucene91HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link HnswGraph}
* for details.
*/
final int beamWidth;

View File

@ -83,7 +83,7 @@ import org.apache.lucene.util.hnsw.HnswGraph;
* <li>DocIds were encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput, byte)}
* <li>OrdToDoc was encoded by {@link org.apache.lucene.util.packed.DirectMonotonicWriter}, note
* that only in sparse case
* <li><b>[int]</b> the maximum number of connections (neigbours) that each node can have
* <li><b>[int]</b> the maximum number of connections (neighbours) that each node can have
* <li><b>[int]</b> number of levels in the graph
* <li>Graph nodes by level. For each level
* <ul>
@ -123,7 +123,7 @@ public class Lucene92HnswVectorsFormat extends KnnVectorsFormat {
/**
* The number of candidate neighbors to track while searching the graph for each newly inserted
* node. Defaults to to {@link #DEFAULT_BEAM_WIDTH}. See {@link HnswGraph} for details.
* node. Defaults to {@link #DEFAULT_BEAM_WIDTH}. See {@link HnswGraph} for details.
*/
final int beamWidth;

View File

@ -84,7 +84,7 @@ import org.apache.lucene.util.hnsw.HnswGraph;
* <li>DocIds were encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput, byte)}
* <li>OrdToDoc was encoded by {@link org.apache.lucene.util.packed.DirectMonotonicWriter}, note
* that only in sparse case
* <li><b>[int]</b> the maximum number of connections (neigbours) that each node can have
* <li><b>[int]</b> the maximum number of connections (neighbours) that each node can have
* <li><b>[int]</b> number of levels in the graph
* <li>Graph nodes by level. For each level
* <ul>
@ -124,8 +124,8 @@ public class Lucene94HnswVectorsFormat extends KnnVectorsFormat {
/**
* The number of candidate neighbors to track while searching the graph for each newly inserted
* node. Defaults to to {@link Lucene94HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link
* HnswGraph} for details.
* node. Defaults to {@link Lucene94HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link HnswGraph}
* for details.
*/
final int beamWidth;

View File

@ -62,7 +62,7 @@ import org.apache.lucene.util.hnsw.HnswGraph;
* </ul>
* </ul>
* <li>After all levels are encoded memory offsets for each node's neighbor nodes encoded by
* {@link org.apache.lucene.util.packed.DirectMonotonicWriter} are appened to the end of the
* {@link org.apache.lucene.util.packed.DirectMonotonicWriter} are appended to the end of the
* file.
* </ul>
*
@ -84,7 +84,7 @@ import org.apache.lucene.util.hnsw.HnswGraph;
* <li>DocIds were encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput, byte)}
* <li>OrdToDoc was encoded by {@link org.apache.lucene.util.packed.DirectMonotonicWriter}, note
* that only in sparse case
* <li><b>[vint]</b> the maximum number of connections (neigbours) that each node can have
* <li><b>[vint]</b> the maximum number of connections (neighbours) that each node can have
* <li><b>[vint]</b> number of levels in the graph
* <li>Graph nodes by level. For each level
* <ul>
@ -141,8 +141,8 @@ public class Lucene95HnswVectorsFormat extends KnnVectorsFormat {
/**
* The number of candidate neighbors to track while searching the graph for each newly inserted
* node. Defaults to to {@link Lucene95HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link
* HnswGraph} for details.
* node. Defaults to {@link Lucene95HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link HnswGraph}
* for details.
*/
final int beamWidth;
@ -161,14 +161,14 @@ public class Lucene95HnswVectorsFormat extends KnnVectorsFormat {
super("Lucene95HnswVectorsFormat");
if (maxConn <= 0 || maxConn > MAXIMUM_MAX_CONN) {
throw new IllegalArgumentException(
"maxConn must be postive and less than or equal to"
"maxConn must be positive and less than or equal to "
+ MAXIMUM_MAX_CONN
+ "; maxConn="
+ maxConn);
}
if (beamWidth <= 0 || beamWidth > MAXIMUM_BEAM_WIDTH) {
throw new IllegalArgumentException(
"beamWidth must be postive and less than or equal to"
"beamWidth must be positive and less than or equal to "
+ MAXIMUM_BEAM_WIDTH
+ "; beamWidth="
+ beamWidth);

View File

@ -90,7 +90,7 @@ public class TestLegacyPackedInts extends LuceneTestCase {
final long expectedBytesUsed = RamUsageTester.ramUsed(r);
final long computedBytesUsed = r.ramBytesUsed();
assertEquals(
r.getClass() + "expected " + expectedBytesUsed + ", got: " + computedBytesUsed,
r.getClass() + " expected " + expectedBytesUsed + ", got: " + computedBytesUsed,
expectedBytesUsed,
computedBytesUsed);
}

View File

@ -47,7 +47,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
// number of places to multiply out the actual ord, and we
// will overflow int during those multiplies. So to avoid
// having to upgrade each multiple to long in multiple
// places (error prone), we use long here:
// places (error-prone), we use long here:
private final long indexInterval;
private final int packedIntsVersion;
@ -118,7 +118,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
+ packedIndexStart
+ " indexStart: "
+ indexStart
+ "numIndexTerms: "
+ " numIndexTerms: "
+ numIndexTerms,
in);
}
@ -272,7 +272,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
// slurp in the images from disk:
try {
try (clone) {
final long numTermBytes = packedIndexStart - indexStart;
termBytes.copy(clone, numTermBytes);
@ -283,8 +283,6 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
// records offsets into byte[] term data
termOffsets =
MonotonicBlockPackedReader.of(clone, packedIntsVersion, blocksize, 1 + numIndexTerms);
} finally {
clone.close();
}
}

View File

@ -52,8 +52,8 @@ public final class Lucene99HnswScalarQuantizedVectorsFormat extends KnnVectorsFo
/**
* The number of candidate neighbors to track while searching the graph for each newly inserted
* node. Defaults to to {@link Lucene99HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link
* HnswGraph} for details.
* node. Defaults to {@link Lucene99HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link HnswGraph}
* for details.
*/
private final int beamWidth;
@ -99,14 +99,14 @@ public final class Lucene99HnswScalarQuantizedVectorsFormat extends KnnVectorsFo
super("Lucene99HnswScalarQuantizedVectorsFormat");
if (maxConn <= 0 || maxConn > MAXIMUM_MAX_CONN) {
throw new IllegalArgumentException(
"maxConn must be positive and less than or equal to"
"maxConn must be positive and less than or equal to "
+ MAXIMUM_MAX_CONN
+ "; maxConn="
+ maxConn);
}
if (beamWidth <= 0 || beamWidth > MAXIMUM_BEAM_WIDTH) {
throw new IllegalArgumentException(
"beamWidth must be positive and less than or equal to"
"beamWidth must be positive and less than or equal to "
+ MAXIMUM_BEAM_WIDTH
+ "; beamWidth="
+ beamWidth);

View File

@ -51,7 +51,7 @@ import org.apache.lucene.util.hnsw.HnswGraph;
* </ul>
* </ul>
* <li>After all levels are encoded memory offsets for each node's neighbor nodes encoded by
* {@link org.apache.lucene.util.packed.DirectMonotonicWriter} are appened to the end of the
* {@link org.apache.lucene.util.packed.DirectMonotonicWriter} are appended to the end of the
* file.
* </ul>
*
@ -71,7 +71,7 @@ import org.apache.lucene.util.hnsw.HnswGraph;
* <li>DocIds were encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput, byte)}
* <li>OrdToDoc was encoded by {@link org.apache.lucene.util.packed.DirectMonotonicWriter}, note
* that only in sparse case
* <li><b>[vint]</b> the maximum number of connections (neigbours) that each node can have
* <li><b>[vint]</b> the maximum number of connections (neighbours) that each node can have
* <li><b>[vint]</b> number of levels in the graph
* <li>Graph nodes by level. For each level
* <ul>
@ -129,8 +129,8 @@ public final class Lucene99HnswVectorsFormat extends KnnVectorsFormat {
/**
* The number of candidate neighbors to track while searching the graph for each newly inserted
* node. Defaults to to {@link Lucene99HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link
* HnswGraph} for details.
* node. Defaults to {@link Lucene99HnswVectorsFormat#DEFAULT_BEAM_WIDTH}. See {@link HnswGraph}
* for details.
*/
private final int beamWidth;
@ -170,14 +170,14 @@ public final class Lucene99HnswVectorsFormat extends KnnVectorsFormat {
super("Lucene99HnswVectorsFormat");
if (maxConn <= 0 || maxConn > MAXIMUM_MAX_CONN) {
throw new IllegalArgumentException(
"maxConn must be positive and less than or equal to"
"maxConn must be positive and less than or equal to "
+ MAXIMUM_MAX_CONN
+ "; maxConn="
+ maxConn);
}
if (beamWidth <= 0 || beamWidth > MAXIMUM_BEAM_WIDTH) {
throw new IllegalArgumentException(
"beamWidth must be positive and less than or equal to"
"beamWidth must be positive and less than or equal to "
+ MAXIMUM_BEAM_WIDTH
+ "; beamWidth="
+ beamWidth);

View File

@ -45,7 +45,7 @@ abstract class DocValuesUpdate {
final String field;
// used in BufferedDeletes to apply this update only to a slice of docs. It's initialized to
// BufferedUpdates.MAX_INT
// since it's safe and most often used this way we safe object creations.
// since it's safe and most often used this way we save object creations.
final int docIDUpTo;
final boolean hasValue;
@ -57,7 +57,7 @@ abstract class DocValuesUpdate {
*/
protected DocValuesUpdate(
DocValuesType type, Term term, String field, int docIDUpTo, boolean hasValue) {
assert docIDUpTo >= 0 : docIDUpTo + "must be >= 0";
assert docIDUpTo >= 0 : docIDUpTo + " must be >= 0";
this.type = type;
this.term = term;
this.field = field;

View File

@ -58,7 +58,7 @@ import org.apache.lucene.util.InfoStream;
*
* <p>When flush is called by IndexWriter we check out all DWPTs that are associated with the
* current {@link DocumentsWriterDeleteQueue} out of the {@link DocumentsWriterPerThreadPool} and
* write them to disk. The flush process can piggy-back on incoming indexing threads or even block
* write them to disk. The flush process can piggyback on incoming indexing threads or even block
* them from adding documents if flushing can't keep up with new documents being added. Unless the
* stall control kicks in to block indexing threads flushes are happening concurrently to actual
* index requests.
@ -94,7 +94,7 @@ final class DocumentsWriter implements Closeable, Accountable {
volatile DocumentsWriterDeleteQueue deleteQueue;
private final DocumentsWriterFlushQueue ticketQueue = new DocumentsWriterFlushQueue();
/*
* we preserve changes during a full flush since IW might not checkout before
* we preserve changes during a full flush since IW might not check out before
* we release all changes. NRT Readers otherwise suddenly return true from
* isCurrent while there are actually changes currently committed. See also
* #anyChanges() & #flushAllThreads
@ -236,7 +236,7 @@ final class DocumentsWriter implements Closeable, Accountable {
}
}
final boolean flushOneDWPT() throws IOException {
boolean flushOneDWPT() throws IOException {
if (infoStream.isEnabled("DW")) {
infoStream.message("DW", "startFlushOneDWPT");
}
@ -382,7 +382,7 @@ final class DocumentsWriter implements Closeable, Accountable {
while (flushControl.anyStalledThreads()
|| (flushControl.numQueuedFlushes() > 0 && config.checkPendingFlushOnUpdate)) {
// Help out flushing any queued DWPTs so we can un-stall:
// Try pick up pending threads here if possible
// Try pickup pending threads here if possible
// no need to loop over the next pending flushes... doFlush will take care of this
hasEvents |= maybeFlush();
flushControl.waitIfStalled(); // block if stalled
@ -460,7 +460,7 @@ final class DocumentsWriter implements Closeable, Accountable {
|| flushingDWPT.deleteQueue == currentFullFlushDelQueue
: "expected: "
+ currentFullFlushDelQueue
+ "but was: "
+ " but was: "
+ flushingDWPT.deleteQueue
+ " "
+ flushControl.isFullFlush();

View File

@ -172,7 +172,7 @@ final class PendingSoftDeletes extends PendingDeletes {
private boolean assertPendingDeletes() {
assert pendingDeleteCount + info.getSoftDelCount() >= 0
: " illegal pending delete count: " + pendingDeleteCount + info.getSoftDelCount();
: "illegal pending delete count: " + (pendingDeleteCount + info.getSoftDelCount());
assert info.info.maxDoc() >= getDelCount();
return true;
}
@ -199,13 +199,13 @@ final class PendingSoftDeletes extends PendingDeletes {
if (dvGeneration == -2) {
FieldInfos fieldInfos = readFieldInfos();
FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
// we try to only open a reader if it's really necessary ie. indices that are mainly append
// we try to only open a reader if it's really necessary i.e. indices that are mainly append
// only might have
// big segments that don't even have any docs in the soft deletes field. In such a case it's
// simply
// enough to look at the FieldInfo for the field and check if the field has DocValues
if (fieldInfo != null && fieldInfo.getDocValuesType() != DocValuesType.NONE) {
// in order to get accurate numbers we need to have a least one reader see here.
// in order to get accurate numbers we need to have at least one reader see here.
onNewReader(readerIOSupplier.get(), info);
} else {
// we are safe here since we don't have any doc values for the soft-delete field on disk

View File

@ -368,7 +368,7 @@ public final class SortingCodecReader extends FilterCodecReader {
}
if (reader.maxDoc() != docMap.size()) {
throw new IllegalArgumentException(
"reader.maxDoc() should be equal to docMap.size(), got"
"reader.maxDoc() should be equal to docMap.size(), got "
+ reader.maxDoc()
+ " != "
+ docMap.size());
@ -721,8 +721,7 @@ public final class SortingCodecReader extends FilterCodecReader {
private boolean assertCreatedOnlyOnce(String field, boolean norms) {
assert Thread.holdsLock(this);
// this is mainly there to make sure we change anything in the way we merge we realize it early
Integer timesCached =
cacheStats.compute(field + "N:" + norms, (s, i) -> i == null ? 1 : i.intValue() + 1);
int timesCached = cacheStats.compute(field + "N:" + norms, (s, i) -> i == null ? 1 : i + 1);
if (timesCached > 1) {
assert norms == false : "[" + field + "] norms must not be cached twice";
boolean isSortField = false;

View File

@ -93,7 +93,7 @@ public class IndexSearcher {
}
/**
* By default we count hits accurately up to 1000. This makes sure that we don't spend most time
* By default, we count hits accurately up to 1000. This makes sure that we don't spend most time
* on computing hit counts
*/
private static final int TOTAL_HITS_THRESHOLD = 1000;
@ -222,7 +222,7 @@ public class IndexSearcher {
*/
public IndexSearcher(IndexReaderContext context, Executor executor) {
assert context.isTopLevel
: "IndexSearcher's ReaderContext must be topLevel for reader" + context.reader();
: "IndexSearcher's ReaderContext must be topLevel for reader " + context.reader();
reader = context.reader();
this.taskExecutor =
executor == null ? new TaskExecutor(Runnable::run) : new TaskExecutor(executor);
@ -231,7 +231,7 @@ public class IndexSearcher {
Function<List<LeafReaderContext>, LeafSlice[]> slicesProvider =
executor == null
? leaves ->
leaves.size() == 0
leaves.isEmpty()
? new LeafSlice[0]
: new LeafSlice[] {new LeafSlice(new ArrayList<>(leaves))}
: this::slices;
@ -613,7 +613,7 @@ public class IndexSearcher {
if (leafSlices.length == 0) {
// there are no segments, nothing to offload to the executor, but we do need to call reduce to
// create some kind of empty result
assert leafContexts.size() == 0;
assert leafContexts.isEmpty();
return collectorManager.reduce(Collections.singletonList(firstCollector));
} else {
final List<C> collectors = new ArrayList<>(leafSlices.length);
@ -820,7 +820,7 @@ public class IndexSearcher {
}
/**
* Returns this searchers the top-level {@link IndexReaderContext}.
* Returns this searcher's top-level {@link IndexReaderContext}.
*
* @see IndexReader#getContext()
*/
@ -932,7 +932,7 @@ public class IndexSearcher {
/**
* Thrown when a client attempts to execute a Query that has more than {@link
* #getMaxClauseCount()} total clauses cumulatively in all of it's children.
* #getMaxClauseCount()} total clauses cumulatively in all of its children.
*
* @see #rewrite
*/

View File

@ -38,7 +38,7 @@ import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.TrackingDirectoryWrapper;
/**
* On-disk sorting of byte arrays. Each byte array (entry) is a composed of the following fields:
* On-disk sorting of byte arrays. Each byte array (entry) is composed of the following fields:
*
* <ul>
* <li>(two bytes) length of the following byte array,
@ -87,7 +87,7 @@ public class OfflineSorter {
private BufferSize(long bytes) {
if (bytes > Integer.MAX_VALUE) {
throw new IllegalArgumentException(
"Buffer too large for Java (" + (Integer.MAX_VALUE / MB) + "mb max): " + bytes);
"Buffer too large for Java (" + (Integer.MAX_VALUE / MB) + "MB max): " + bytes);
}
if (bytes < ABSOLUTE_MIN_SORT_BUFFER_SIZE) {
@ -120,7 +120,7 @@ public class OfflineSorter {
// by free mem (attempting to not grow the heap for this)
long sortBufferByteSize = free / 2;
final long minBufferSizeBytes = MIN_BUFFER_SIZE_MB * MB;
// lets see if we need/should to grow the heap
// let's see if we need/should to grow the heap
if (sortBufferByteSize < minBufferSizeBytes
|| totalAvailableBytes > 10 * minBufferSizeBytes) {
// there is enough mem for a reasonable buffer
@ -131,7 +131,7 @@ public class OfflineSorter {
sortBufferByteSize = Math.max(ABSOLUTE_MIN_SORT_BUFFER_SIZE, sortBufferByteSize);
}
}
return new BufferSize(Math.min((long) Integer.MAX_VALUE, sortBufferByteSize));
return new BufferSize(Math.min(Integer.MAX_VALUE, sortBufferByteSize));
}
}
@ -183,7 +183,7 @@ public class OfflineSorter {
private final BufferSize ramBufferSize;
SortInfo sortInfo;
private int maxTempFiles;
private final int maxTempFiles;
private final Comparator<BytesRef> comparator;
/** Default comparator: sorts in binary (codepoint) order */
@ -629,7 +629,7 @@ public class OfflineSorter {
public Partition call() throws IOException {
try (IndexOutput tempFile =
dir.createTempOutput(tempFileNamePrefix, "sort", IOContext.DEFAULT);
ByteSequencesWriter out = getWriter(tempFile, part.buffer.size()); ) {
ByteSequencesWriter out = getWriter(tempFile, part.buffer.size())) {
BytesRef spare;
@ -687,7 +687,7 @@ public class OfflineSorter {
}
PriorityQueue<FileAndTop> queue =
new PriorityQueue<FileAndTop>(segmentsToMerge.size()) {
new PriorityQueue<>(segmentsToMerge.size()) {
@Override
protected boolean lessThan(FileAndTop a, FileAndTop b) {
return comparator.compare(a.current, b.current) < 0;

View File

@ -219,7 +219,7 @@ final class DocIdsWriter {
while ((docId = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
docIDs[pos++] = docId;
}
assert pos == count : "pos: " + pos + "count: " + count;
assert pos == count : "pos: " + pos + ", count: " + count;
}
private static void readDelta16(IndexInput in, int count, int[] docIDs) throws IOException {

View File

@ -20,7 +20,6 @@ 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.Locale;
import java.util.Map;
@ -232,9 +231,7 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
// 2> We could have 10 segments and a target count of 2. In that case there could be 5
// segments resulting.
// as long as they're all < 125% max seg size, that's valid.
Iterator<SegmentCommitInfo> iterator = w.cloneSegmentInfos().iterator();
while (iterator.hasNext()) {
SegmentCommitInfo info = iterator.next();
for (SegmentCommitInfo info : w.cloneSegmentInfos()) {
assertTrue(
"No segment should be more than 125% of max segment size ",
max125Pct >= info.sizeInBytes());
@ -294,8 +291,7 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
}
// LUCENE-7976 makes findForceMergeDeletes and findForcedDeletes respect max segment size by
// default,
// so insure that this works.
// default, so ensure that this works.
public void testForcedMergesRespectSegSize() throws Exception {
final Directory dir = newDirectory();
final IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
@ -359,7 +355,7 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
"Should still be deleted docs in the index",
w.getDocStats().numDocs < w.getDocStats().maxDoc);
// This time, forceMerge. By default this should respect max segment size.
// This time, forceMerge. By default, this should respect max segment size.
// Will change for LUCENE-8236
w.forceMerge(Integer.MAX_VALUE);
checkSegmentSizeNotExceeded(w.cloneSegmentInfos(), maxSegBytes);
@ -367,7 +363,7 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
// Now forceMerge down to one segment, there should be exactly remainingDocs in exactly one
// segment.
w.forceMerge(1);
assertEquals("There should be exaclty one segment now", 1, w.getSegmentCount());
assertEquals("There should be exactly one segment now", 1, w.getSegmentCount());
assertEquals(
"maxDoc and numDocs should be identical", w.getDocStats().numDocs, w.getDocStats().maxDoc);
assertEquals(
@ -412,7 +408,7 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
for (int i = 0; i < 50; i++) {
Document doc = new Document();
doc.add(newStringField("id", "" + i + numDocs, Field.Store.NO));
doc.add(newStringField("id", "" + (i + numDocs), Field.Store.NO));
doc.add(newTextField("content", "aaa " + i, Field.Store.NO));
w.addDocument(doc);
}
@ -428,7 +424,7 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
int largeSegDocCount = Math.max(info0.info.maxDoc(), info1.info.maxDoc());
int smallSegDocCount = Math.min(info0.info.maxDoc(), info1.info.maxDoc());
assertEquals("The large segment should have a bunch of docs", largeSegDocCount, remainingDocs);
assertEquals("Small segment shold have fewer docs", smallSegDocCount, 50);
assertEquals("Small segment should have fewer docs", smallSegDocCount, 50);
w.close();
@ -489,17 +485,16 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
}
}
// LUCENE-8688 reports that force merges merged more segments that necessary to respect
// LUCENE-8688 reports that force-merges merged more segments that necessary to respect
// maxSegmentCount as a result
// of LUCENE-7976 so we ensure that it only does the minimum number of merges here.
// of LUCENE-7976, so we ensure that it only does the minimum number of merges here.
public void testForcedMergesUseLeastNumberOfMerges() throws Exception {
TieredMergePolicy tmp = new TieredMergePolicy();
double oneSegmentSizeMB = 1.0D;
double maxMergedSegmentSizeMB = 10 * oneSegmentSizeMB;
tmp.setMaxMergedSegmentMB(maxMergedSegmentSizeMB);
if (VERBOSE) {
System.out.println(
String.format(Locale.ROOT, "TEST: maxMergedSegmentSizeMB=%.2f", maxMergedSegmentSizeMB));
System.out.printf(Locale.ROOT, "TEST: maxMergedSegmentSizeMB=%.2f%n", maxMergedSegmentSizeMB);
}
// create simulated 30 segment index where each segment is 1 MB
@ -581,8 +576,7 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
final MergeSpecification specification =
tmp.findForcedMerges(infos, expectedCount, segmentsToMerge(infos), mergeContext);
// Since we have fewer than 30 (the max merge count) segments more than the final size this
// would have been the final merge
// so we check that it was prevented.
// would have been the final merge, so we check that it was prevented.
assertNull(specification);
}
@ -624,19 +618,15 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
if (twoMayHaveBeenMerged == false || segNamesAfter.size() == segNamesBefore.size()) {
if (segNamesAfter.size() != segNamesBefore.size()) {
fail(
"Segment lists different sizes!: "
+ segNamesBefore.toString()
+ " After list: "
+ segNamesAfter.toString());
fail("Segment lists different sizes!: " + segNamesBefore + " After list: " + segNamesAfter);
}
if (segNamesAfter.containsAll(segNamesBefore) == false) {
fail(
"Segment lists should be identical: "
+ segNamesBefore.toString()
+ segNamesBefore
+ " After list: "
+ segNamesAfter.toString());
+ segNamesAfter);
}
return;
}
@ -646,12 +636,12 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
if (segNamesAfter.size() != segNamesBefore.size() - 1) {
fail(
"forceMerge didn't merge a small and large segment into one segment as expected: "
+ segNamesBefore.toString()
+ segNamesBefore
+ " After list: "
+ segNamesAfter.toString());
+ segNamesAfter);
}
// There shold be exactly two segments in the before not in after and one in after not in
// There should be exactly two segments in the before not in after and one in after not in
// before.
List<String> testBefore = new ArrayList<>(segNamesBefore);
List<String> testAfter = new ArrayList<>(segNamesAfter);
@ -661,10 +651,10 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
if (testBefore.size() != 2 || testAfter.size() != 1) {
fail(
"Segment lists different sizes!: "
+ segNamesBefore.toString()
"Expected two unique 'before' segments and one unique 'after' segment: "
+ segNamesBefore
+ " After list: "
+ segNamesAfter.toString());
+ segNamesAfter);
}
}
@ -816,7 +806,7 @@ public class TestTieredMergePolicy extends BaseMergePolicyTestCase {
IndexWriter w = new IndexWriter(dir, iwc);
for (int i = 0; i < 15000 * RANDOM_MULTIPLIER; i++) {
Document doc = new Document();
// Uncompressible content so that merging 10 segments of size x creates a segment whose size
// Incompressible content so that merging 10 segments of size x creates a segment whose size
// is about 10x
byte[] idBytes = new byte[128];
random().nextBytes(idBytes);

View File

@ -148,7 +148,7 @@ public class TestScorerPerf extends LuceneTestCase {
pos = answer.nextSetBit(pos + 1);
if (pos != doc + docBase) {
throw new RuntimeException("Expected doc " + pos + " but got " + doc + docBase);
throw new RuntimeException("Expected doc " + pos + " but got " + (doc + docBase));
}
super.collect(doc);
}

View File

@ -77,7 +77,7 @@ public class IndexFiles implements AutoCloseable {
String usage =
"java org.apache.lucene.demo.IndexFiles"
+ " [-index INDEX_PATH] [-docs DOCS_PATH] [-update] [-knn_dict DICT_PATH]\n\n"
+ "This indexes the documents in DOCS_PATH, creating a Lucene index"
+ "This indexes the documents in DOCS_PATH, creating a Lucene index "
+ "in INDEX_PATH that can be searched with SearchFiles\n"
+ "IF DICT_PATH contains a KnnVector dictionary, the index will also support KnnVector search";
String indexPath = "index";
@ -140,7 +140,7 @@ public class IndexFiles implements AutoCloseable {
// Optional: for better indexing performance, if you
// are indexing many documents, increase the RAM
// buffer. But if you do this, increase the max heap
// size to the JVM (eg add -Xmx512m or -Xmx1g):
// size to the JVM (e.g. add -Xmx512m or -Xmx1g):
//
// iwc.setRAMBufferSizeMB(256.0);
@ -239,7 +239,7 @@ public class IndexFiles implements AutoCloseable {
// Add the last modified date of the file a field named "modified".
// Use a LongField that is indexed with points and doc values, and is efficient
// for both filtering (LongField#newRangeQuery) and sorting
// (LongField#newSortField). This indexes to milli-second resolution, which
// (LongField#newSortField). This indexes to millisecond resolution, which
// is often too fine. You could instead create a number based on
// year/month/day/hour/minutes/seconds, down the resolution you require.
// For example the long value 2011021714 would mean

View File

@ -153,11 +153,11 @@ final class GlobalOrdinalsWithScoreQuery extends Query implements Accountable {
return "GlobalOrdinalsQuery{"
+ "joinField="
+ joinField
+ "min="
+ ",min="
+ min
+ "max="
+ ",max="
+ max
+ "fromQuery="
+ ",fromQuery="
+ fromQuery
+ '}';
}

View File

@ -90,7 +90,7 @@ public class TestHighFreqTerms extends LuceneTestCase {
for (int i = 0; i < terms.length; i++) {
if (i > 0) {
assertTrue(
"out of order " + terms[i - 1].docFreq + "should be >= " + terms[i].docFreq,
"out of order " + terms[i - 1].docFreq + " should be >= " + terms[i].docFreq,
terms[i - 1].docFreq >= terms[i].docFreq);
}
}
@ -123,7 +123,7 @@ public class TestHighFreqTerms extends LuceneTestCase {
}
} else {
int n = Integer.parseInt(termtext);
assertEquals("doc freq is not as expected", getExpecteddocFreq(n), terms[i].docFreq);
assertEquals("doc freq is not as expected", getExpectedDocFreq(n), terms[i].docFreq);
}
}
}
@ -163,7 +163,7 @@ public class TestHighFreqTerms extends LuceneTestCase {
// order
if (i > 0) {
assertTrue(
"out of order" + terms[i - 1] + " > " + terms[i],
"out of order " + terms[i - 1] + " > " + terms[i],
terms[i - 1].totalTermFreq >= terms[i].totalTermFreq);
}
}
@ -187,9 +187,9 @@ public class TestHighFreqTerms extends LuceneTestCase {
} else {
int n = Integer.parseInt(text);
assertEquals("doc freq is expected", getExpecteddocFreq(n), terms[i].docFreq);
assertEquals("doc freq is expected", getExpectedDocFreq(n), terms[i].docFreq);
assertEquals(
"total term freq is expected", getExpectedtotalTermFreq(n), terms[i].totalTermFreq);
"total term freq is expected", getExpectedTotalTermFreq(n), terms[i].totalTermFreq);
}
}
}
@ -199,7 +199,7 @@ public class TestHighFreqTerms extends LuceneTestCase {
private static void indexDocs(IndexWriter writer) throws Exception {
Random rnd = random();
/**
/*
* Generate 10 documents where term n has a docFreq of n and a totalTermFreq of n*2 (squared).
*/
for (int i = 1; i <= 10; i++) {
@ -269,11 +269,11 @@ public class TestHighFreqTerms extends LuceneTestCase {
return s;
}
private static int getExpectedtotalTermFreq(int i) {
return getExpecteddocFreq(i) * i;
private static int getExpectedTotalTermFreq(int i) {
return getExpectedDocFreq(i) * i;
}
private static int getExpecteddocFreq(int i) {
private static int getExpectedDocFreq(int i) {
return i;
}
}

View File

@ -16,8 +16,6 @@
*/
package org.apache.lucene.queries.mlt;
import static org.hamcrest.core.Is.is;
import java.io.IOException;
import java.io.StringReader;
import java.util.ArrayList;
@ -165,21 +163,21 @@ public class TestMoreLikeThis extends LuceneTestCase {
BooleanQuery query =
(BooleanQuery) mlt.like("one_percent", new StringReader("tenth tenth all"));
Collection<BooleanClause> clauses = query.clauses();
List<BooleanClause> clauses = query.clauses();
assertTrue(clauses.size() == 2);
Term term = ((TermQuery) ((List<BooleanClause>) clauses).get(0).getQuery()).getTerm();
Term term = ((TermQuery) clauses.get(0).getQuery()).getTerm();
assertTrue(term.text().equals("all"));
term = ((TermQuery) ((List<BooleanClause>) clauses).get(1).getQuery()).getTerm();
term = ((TermQuery) clauses.get(1).getQuery()).getTerm();
assertTrue(term.text().equals("tenth"));
query = (BooleanQuery) mlt.like("one_percent", new StringReader("tenth all all"));
clauses = query.clauses();
assertTrue(clauses.size() == 2);
term = ((TermQuery) ((List<BooleanClause>) clauses).get(0).getQuery()).getTerm();
term = ((TermQuery) clauses.get(0).getQuery()).getTerm();
assertTrue(term.text().equals("all"));
term = ((TermQuery) ((List<BooleanClause>) clauses).get(1).getQuery()).getTerm();
term = ((TermQuery) clauses.get(1).getQuery()).getTerm();
assertTrue(term.text().equals("tenth"));
// clean up
@ -287,7 +285,7 @@ public class TestMoreLikeThis extends LuceneTestCase {
assertEquals("Expected 1 clauses only!", 1, clauses.size());
for (BooleanClause clause : clauses) {
Term term = ((TermQuery) clause.getQuery()).getTerm();
assertThat(term, is(new Term(mltField1, "lucene")));
assertEquals(new Term(mltField1, "lucene"), term);
}
analyzer.close();
}
@ -416,7 +414,7 @@ public class TestMoreLikeThis extends LuceneTestCase {
// check best terms are topN of highest idf
Collection<BooleanClause> clauses = query.clauses();
assertEquals("Expected" + topN + "clauses only!", topN, clauses.size());
assertEquals("Expected " + topN + " clauses only!", topN, clauses.size());
Term[] expectedTerms = new Term[topN];
int idx = 0;
@ -517,7 +515,7 @@ public class TestMoreLikeThis extends LuceneTestCase {
BooleanQuery query = (BooleanQuery) mlt.like(inputDocId);
Collection<BooleanClause> clauses = query.clauses();
Collection<BooleanClause> expectedClothesShopClauses = new ArrayList<BooleanClause>();
Collection<BooleanClause> expectedClothesShopClauses = new ArrayList<>();
for (String itemForSale : clothesShopItemForSale) {
BooleanClause booleanClause =
new BooleanClause(
@ -544,7 +542,7 @@ public class TestMoreLikeThis extends LuceneTestCase {
}
}
public void testCustomFrequecy() throws IOException {
public void testCustomFrequency() throws IOException {
// define an analyzer with delimited term frequency, e.g. "foo|2 bar|3"
Analyzer analyzer =
new Analyzer() {
@ -552,8 +550,8 @@ public class TestMoreLikeThis extends LuceneTestCase {
@Override
protected TokenStreamComponents createComponents(String fieldName) {
MockTokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false, 100);
MockTokenFilter filt = new MockTokenFilter(tokenizer, MockTokenFilter.EMPTY_STOPSET);
return new TokenStreamComponents(tokenizer, addCustomTokenFilter(filt));
MockTokenFilter filter = new MockTokenFilter(tokenizer, MockTokenFilter.EMPTY_STOPSET);
return new TokenStreamComponents(tokenizer, addCustomTokenFilter(filter));
}
TokenStream addCustomTokenFilter(TokenStream input) {

View File

@ -98,7 +98,8 @@ public class TestSimpleQueryParser extends LuceneTestCase {
bool.add(new TermQuery(new Term("field", "bar")), Occur.MUST);
assertEquals(
bool.build(), parse("foo~" + LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE + 1 + " bar"));
bool.build(),
parse("foo~" + (LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE + 1) + " bar"));
}
/** test a simple phrase */

View File

@ -145,7 +145,7 @@ public abstract class StrategyTestCase extends SpatialTestCase {
if (!ids.hasNext()) {
fail(
msg
+ " :: Did not get enough results. Expect"
+ " :: Did not get enough results. Expected "
+ q.ids
+ ", got: "
+ got.toDebugString());
@ -165,7 +165,7 @@ public abstract class StrategyTestCase extends SpatialTestCase {
}
for (String s : q.ids) {
if (!found.contains(s)) {
fail("Results are mising id: " + s + " :: " + found);
fail("Results are missing id: " + s + " :: " + found);
}
}
} else {
@ -224,7 +224,7 @@ public abstract class StrategyTestCase extends SpatialTestCase {
for (int i = 0; i < count; i++) {
assertTrue(v.advanceExact(i));
int doc = i + ctx.docBase;
assertEquals("Not equal for doc " + doc, v.doubleValue(), (double) scores[doc], delta);
assertEquals("Not equal for doc " + doc, v.doubleValue(), scores[doc], delta);
}
}
}

View File

@ -650,7 +650,7 @@ public abstract class BaseXYPointTestCase extends LuceneTestCase {
System.out.println("TEST: id=" + id + " docID=" + docID + " should not match but did");
}
System.out.println(" rect=" + rect);
System.out.println(" x=" + xDoc1 + " y=" + yDoc1 + "\n x=" + xDoc2 + " y" + yDoc2);
System.out.println(" x=" + xDoc1 + " y=" + yDoc1 + "\n x=" + xDoc2 + " y=" + yDoc2);
System.out.println(" result1=" + result1 + " result2=" + result2);
fail = true;
}
@ -773,7 +773,7 @@ public abstract class BaseXYPointTestCase extends LuceneTestCase {
protected abstract Query newGeometryQuery(String field, XYGeometry... geometries);
static final boolean rectContainsPoint(XYRectangle rect, double x, double y) {
static boolean rectContainsPoint(XYRectangle rect, double x, double y) {
if (y < rect.minY || y > rect.maxY) {
return false;
}
@ -1365,7 +1365,7 @@ public abstract class BaseXYPointTestCase extends LuceneTestCase {
float docX = storedFields.document(doc).getField("x").numericValue().floatValue();
float docY = storedFields.document(doc).getField("y").numericValue().floatValue();
double distance = cartesianDistance(x, y, docX, docY);
System.out.println("" + doc + ": (" + x + "," + y + "), distance=" + distance);
System.out.println(doc + ": (" + x + "," + y + "), distance=" + distance);
}
throw e;
}

View File

@ -1294,7 +1294,7 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
}
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
String id = "" + i + numDocs;
String id = "" + (i + numDocs);
doc.add(newTextField("id", id, Field.Store.YES));
String string = TestUtil.randomRealisticUnicodeString(random(), 1, maxLength);
BytesRef br = newBytesRef(string);
@ -2272,7 +2272,7 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
Set<String> valueSet = new HashSet<String>();
Set<String> valueSet = new HashSet<>();
for (int i = 0; i < 10000 && valueSet.size() < maxUniqueValues; ++i) {
final int length = TestUtil.nextInt(random(), minLength, maxLength);
valueSet.add(TestUtil.randomSimpleString(random(), length));
@ -3631,9 +3631,9 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes
}
private interface FieldCreator {
public Field next();
Field next();
public DocIdSetIterator iterator(IndexReader r) throws IOException;
DocIdSetIterator iterator(IndexReader r) throws IOException;
}
private void doTestRandomAdvance(FieldCreator fieldCreator) throws IOException {

View File

@ -16,6 +16,8 @@
*/
package org.apache.lucene.tests.index;
import static java.nio.charset.StandardCharsets.UTF_8;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.PrintStream;
@ -105,10 +107,10 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
// metadata or Directory-level objects
private static final Set<Class<?>> EXCLUDED_CLASSES =
Collections.newSetFromMap(new IdentityHashMap<Class<?>, Boolean>());
Collections.newSetFromMap(new IdentityHashMap<>());
static {
// Directory objects, don't take into account eg. the NIO buffers
// Directory objects, don't take into account, e.g. the NIO buffers
EXCLUDED_CLASSES.add(Directory.class);
EXCLUDED_CLASSES.add(IndexInput.class);
@ -247,16 +249,14 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
* that store the same content.
*/
protected Collection<String> excludedExtensionsFromByteCounts() {
return new HashSet<String>(
return new HashSet<>(
Arrays.asList(
new String[] {
// segment infos store various pieces of information that don't solely depend
// on the content of the index in the diagnostics (such as a timestamp) so we
// exclude this file from the bytes counts
"si",
// lock files are 0 bytes (one directory in the test could be RAMDir, the other FSDir)
"lock"
}));
// segment infos store various pieces of information that don't solely depend
// on the content of the index in the diagnostics (such as a timestamp) so we
// exclude this file from the bytes counts
"si",
// lock files are 0 bytes (one directory in the test could be RAMDir, the other FSDir)
"lock"));
}
/**
@ -396,7 +396,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
try (FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(writeState)) {
final Fields fields =
new Fields() {
TreeSet<String> indexedFields =
final TreeSet<String> indexedFields =
new TreeSet<>(FieldInfos.getIndexedFields(oneDocReader));
@Override
@ -622,7 +622,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
// log all exceptions we hit, in case we fail (for debugging)
ByteArrayOutputStream exceptionLog = new ByteArrayOutputStream();
PrintStream exceptionStream = new PrintStream(exceptionLog, true, "UTF-8");
PrintStream exceptionStream = new PrintStream(exceptionLog, true, UTF_8);
// PrintStream exceptionStream = System.out;
Analyzer analyzer = new MockAnalyzer(random());
@ -729,14 +729,14 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
} catch (Throwable t) {
System.out.println("Unexpected exception: dumping fake-exception-log:...");
exceptionStream.flush();
System.out.println(exceptionLog.toString("UTF-8"));
System.out.println(exceptionLog.toString(UTF_8));
System.out.flush();
Rethrow.rethrow(t);
}
if (VERBOSE) {
System.out.println("TEST PASSED: dumping fake-exception-log:...");
System.out.println(exceptionLog.toString("UTF-8"));
System.out.println(exceptionLog.toString(UTF_8));
}
}
@ -772,8 +772,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
/** A directory that tracks created files that haven't been deleted. */
protected static class FileTrackingDirectoryWrapper extends FilterDirectory {
private final Set<String> files =
Collections.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
private final Set<String> files = ConcurrentHashMap.newKeySet();
/** Sole constructor. */
FileTrackingDirectoryWrapper(Directory in) {
@ -932,7 +931,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
}
/**
* This test is a best effort at verifying that checkIntegrity doesn't miss any files. It tests
* This test is the best effort at verifying that checkIntegrity doesn't miss any files. It tests
* that the combination of opening a reader and calling checkIntegrity on it reads all bytes of
* all files.
*/
@ -979,9 +978,9 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
+ unread
+ " of file "
+ name
+ "("
+ " ("
+ unreadBytes.length()
+ "bytes) was not read.");
+ " bytes) was not read.");
}
}
assertTrue(String.join("\n", messages), messages.isEmpty());

View File

@ -115,7 +115,7 @@ public class RandomPostingsTester {
// Test w/ multiple threads
THREADS
};
}
private long totalPostings;
private long totalPayloadBytes;
@ -956,7 +956,7 @@ public class RandomPostingsTester {
assertNotNull("null DocsEnum", postingsEnum);
int initialDocID = postingsEnum.docID();
assertEquals("inital docID should be -1" + postingsEnum, -1, initialDocID);
assertEquals("initial docID should be -1: " + postingsEnum, -1, initialDocID);
if (LuceneTestCase.VERBOSE) {
if (prevPostingsEnum == null) {
@ -1631,7 +1631,7 @@ public class RandomPostingsTester {
}
TermsEnum intersected = fieldsSource.terms(field).intersect(ca, startTerm);
Set<BytesRef> intersectedTerms = new HashSet<BytesRef>();
Set<BytesRef> intersectedTerms = new HashSet<>();
BytesRef term;
while ((term = intersected.next()) != null) {
if (startTerm != null) {