@@ -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 Lucene99HnswVectorsFormat(
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);
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsReader.java
index 599d1f939ce3..80d249174770 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsReader.java
@@ -149,7 +149,7 @@ static void readVIntBlock(
boolean indexHasFreq,
boolean decodeFreq)
throws IOException {
- GroupVIntReader.readValues(docIn, docBuffer, num);
+ docIn.readGroupVInts(docBuffer, num);
if (indexHasFreq && decodeFreq) {
for (int i = 0; i < num; ++i) {
freqBuffer[i] = docBuffer[i] & 0x01;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsWriter.java
index 2bd562fc3da7..a001bea210c3 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsWriter.java
@@ -92,7 +92,6 @@ public final class Lucene99PostingsWriter extends PushPostingsWriterBase {
private final PForUtil pforUtil;
private final ForDeltaUtil forDeltaUtil;
private final Lucene99SkipWriter skipWriter;
- private final GroupVIntWriter docGroupVIntWriter;
private boolean fieldHasNorms;
private NumericDocValues norms;
@@ -173,7 +172,6 @@ public Lucene99PostingsWriter(SegmentWriteState state) throws IOException {
skipWriter =
new Lucene99SkipWriter(
MAX_SKIP_LEVELS, BLOCK_SIZE, state.segmentInfo.maxDoc(), docOut, posOut, payOut);
- docGroupVIntWriter = new GroupVIntWriter();
}
@Override
@@ -378,7 +376,7 @@ public void finishTerm(BlockTermState _state) throws IOException {
docDeltaBuffer[i] = (docDeltaBuffer[i] << 1) | (freqBuffer[i] == 1 ? 1 : 0);
}
}
- docGroupVIntWriter.writeValues(docOut, docDeltaBuffer, docBufferUpto);
+ docOut.writeGroupVInts(docDeltaBuffer, docBufferUpto);
if (writeFreqs) {
for (int i = 0; i < docBufferUpto; i++) {
final int freq = (int) freqBuffer[i];
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index cd601743f294..f899b331b924 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -2305,31 +2305,26 @@ private static Status.TermIndexStatus checkFields(
}
// Test Terms#intersect
- TermsEnum allTerms = terms.iterator();
// An automaton that should match a good number of terms
- Automaton a =
+ Automaton automaton =
Operations.concatenate(
Arrays.asList(
Automata.makeAnyBinary(),
Automata.makeCharRange('a', 'e'),
Automata.makeAnyBinary()));
- a = Operations.determinize(a, Operations.DEFAULT_DETERMINIZE_WORK_LIMIT);
- CompiledAutomaton ca = new CompiledAutomaton(a);
- ByteRunAutomaton runAutomaton = new ByteRunAutomaton(a);
- TermsEnum filteredTerms = terms.intersect(ca, null);
- for (BytesRef term = allTerms.next(); term != null; term = allTerms.next()) {
- if (runAutomaton.run(term.bytes, term.offset, term.length)) {
- BytesRef filteredTerm = filteredTerms.next();
- if (Objects.equals(term, filteredTerm) == false) {
- throw new CheckIndexException(
- "Expected next filtered term: " + term + ", but got " + filteredTerm);
- }
- }
- }
- BytesRef filteredTerm = filteredTerms.next();
- if (filteredTerm != null) {
- throw new CheckIndexException("Expected exhausted TermsEnum, but got " + filteredTerm);
- }
+ BytesRef startTerm = null;
+ checkTermsIntersect(terms, automaton, startTerm);
+
+ startTerm = new BytesRef();
+ checkTermsIntersect(terms, automaton, startTerm);
+
+ automaton = Automata.makeNonEmptyBinary();
+ startTerm = new BytesRef(new byte[] {'l'});
+ checkTermsIntersect(terms, automaton, startTerm);
+
+ // a term that likely compares greater than every other term in the dictionary
+ startTerm = new BytesRef(new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF});
+ checkTermsIntersect(terms, automaton, startTerm);
}
}
@@ -2370,6 +2365,45 @@ private static Status.TermIndexStatus checkFields(
return status;
}
+ private static void checkTermsIntersect(Terms terms, Automaton automaton, BytesRef startTerm)
+ throws IOException {
+ TermsEnum allTerms = terms.iterator();
+ automaton = Operations.determinize(automaton, Operations.DEFAULT_DETERMINIZE_WORK_LIMIT);
+ CompiledAutomaton compiledAutomaton = new CompiledAutomaton(automaton, false, true, true);
+ ByteRunAutomaton runAutomaton = new ByteRunAutomaton(automaton, true);
+ TermsEnum filteredTerms = terms.intersect(compiledAutomaton, startTerm);
+ BytesRef term;
+ if (startTerm != null) {
+ switch (allTerms.seekCeil(startTerm)) {
+ case FOUND:
+ term = allTerms.next();
+ break;
+ case NOT_FOUND:
+ term = allTerms.term();
+ break;
+ case END:
+ default:
+ term = null;
+ break;
+ }
+ } else {
+ term = allTerms.next();
+ }
+ for (; term != null; term = allTerms.next()) {
+ if (runAutomaton.run(term.bytes, term.offset, term.length)) {
+ BytesRef filteredTerm = filteredTerms.next();
+ if (Objects.equals(term, filteredTerm) == false) {
+ throw new CheckIndexException(
+ "Expected next filtered term: " + term + ", but got " + filteredTerm);
+ }
+ }
+ }
+ BytesRef filteredTerm = filteredTerms.next();
+ if (filteredTerm != null) {
+ throw new CheckIndexException("Expected exhausted TermsEnum, but got " + filteredTerm);
+ }
+ }
+
/**
* For use in tests only.
*
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java b/lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java
index 41e815c14f9e..9be34e997b24 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java
@@ -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;
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
index 2fbb0e8cf12a..37f2539d0f0f 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
@@ -58,7 +58,7 @@
*
* 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 @@ synchronized void abort() throws IOException {
}
}
- final boolean flushOneDWPT() throws IOException {
+ boolean flushOneDWPT() throws IOException {
if (infoStream.isEnabled("DW")) {
infoStream.message("DW", "startFlushOneDWPT");
}
@@ -382,7 +382,7 @@ private boolean preUpdate() throws IOException {
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 @@ private void doFlush(DocumentsWriterPerThread flushingDWPT) throws IOException {
|| flushingDWPT.deleteQueue == currentFullFlushDelQueue
: "expected: "
+ currentFullFlushDelQueue
- + "but was: "
+ + " but was: "
+ flushingDWPT.deleteQueue
+ " "
+ flushControl.isFullFlush();
diff --git a/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java b/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
index a8377ad8f6dc..accdb57d1bae 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
@@ -172,7 +172,7 @@ void onDocValuesUpdate(FieldInfo info, DocValuesFieldUpdates.Iterator iterator)
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 @@ private void ensureInitialized(IOSupplier readerIOSupplier) throws
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
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortingCodecReader.java b/lucene/core/src/java/org/apache/lucene/index/SortingCodecReader.java
index f346313ff990..6a50b06d35a7 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortingCodecReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortingCodecReader.java
@@ -368,7 +368,7 @@ public String toString() {
}
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 @@ private synchronized T getOrCreate(String field, boolean norms, IOSupplier 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;
diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
index 45d436bbf411..d20e80f67291 100644
--- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
+++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
@@ -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 IndexSearcher(IndexReader r, Executor executor) {
*/
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 IndexSearcher(IndexReaderContext context, Executor executor) {
Function, 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 @@ private T search(
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 collectors = new ArrayList<>(leafSlices.length);
@@ -820,7 +820,7 @@ public Weight createWeight(Query query, ScoreMode scoreMode, float boost) throws
}
/**
- * Returns this searchers the top-level {@link IndexReaderContext}.
+ * Returns this searcher's top-level {@link IndexReaderContext}.
*
* @see IndexReader#getContext()
*/
@@ -932,7 +932,7 @@ public int getMaxClauseCount() {
/**
* 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
*/
diff --git a/lucene/core/src/java/org/apache/lucene/store/BufferedIndexInput.java b/lucene/core/src/java/org/apache/lucene/store/BufferedIndexInput.java
index 32ab161e2c4e..33e3a6df119b 100644
--- a/lucene/core/src/java/org/apache/lucene/store/BufferedIndexInput.java
+++ b/lucene/core/src/java/org/apache/lucene/store/BufferedIndexInput.java
@@ -20,6 +20,7 @@
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
+import org.apache.lucene.util.GroupVIntUtil;
/** Base implementation class for buffered {@link IndexInput}. */
public abstract class BufferedIndexInput extends IndexInput implements RandomAccessInput {
@@ -149,6 +150,16 @@ public final int readInt() throws IOException {
}
}
+ @Override
+ protected void readGroupVInt(long[] dst, int offset) throws IOException {
+ final int len =
+ GroupVIntUtil.readGroupVInt(
+ this, buffer.remaining(), p -> buffer.getInt((int) p), buffer.position(), dst, offset);
+ if (len > 0) {
+ buffer.position(buffer.position() + len);
+ }
+ }
+
@Override
public final long readLong() throws IOException {
if (Long.BYTES <= buffer.remaining()) {
diff --git a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataInput.java b/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataInput.java
index 9d0ea74b02b6..7812b2cd1b2a 100644
--- a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataInput.java
+++ b/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataInput.java
@@ -29,6 +29,7 @@
import java.util.Locale;
import java.util.stream.Collectors;
import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.GroupVIntUtil;
import org.apache.lucene.util.RamUsageEstimator;
/**
@@ -212,6 +213,25 @@ public long readLong() throws IOException {
}
}
+ @Override
+ protected void readGroupVInt(long[] dst, int offset) throws IOException {
+ final ByteBuffer block = blocks[blockIndex(pos)];
+ final int blockOffset = blockOffset(pos);
+ // We MUST save the return value to local variable, could not use pos += readGroupVInt(...).
+ // because `pos +=` in java will move current value(not address) of pos to register first,
+ // then call the function, but we will update pos value in function via readByte(), then
+ // `pos +=` will use an old pos value plus return value, thereby missing 1 byte.
+ final int len =
+ GroupVIntUtil.readGroupVInt(
+ this,
+ block.limit() - blockOffset,
+ p -> block.getInt((int) p),
+ blockOffset,
+ dst,
+ offset);
+ pos += len;
+ }
+
@Override
public long length() {
return length;
diff --git a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersIndexInput.java b/lucene/core/src/java/org/apache/lucene/store/ByteBuffersIndexInput.java
index 3a13e0340a06..58101a197449 100644
--- a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersIndexInput.java
+++ b/lucene/core/src/java/org/apache/lucene/store/ByteBuffersIndexInput.java
@@ -205,6 +205,12 @@ public void readLongs(long[] dst, int offset, int length) throws IOException {
in.readLongs(dst, offset, length);
}
+ @Override
+ protected void readGroupVInt(long[] dst, int offset) throws IOException {
+ ensureOpen();
+ in.readGroupVInt(dst, offset);
+ }
+
@Override
public IndexInput clone() {
ensureOpen();
diff --git a/lucene/core/src/java/org/apache/lucene/store/DataInput.java b/lucene/core/src/java/org/apache/lucene/store/DataInput.java
index 24aae0372d1d..781066f02ab5 100644
--- a/lucene/core/src/java/org/apache/lucene/store/DataInput.java
+++ b/lucene/core/src/java/org/apache/lucene/store/DataInput.java
@@ -27,6 +27,7 @@
import java.util.TreeMap;
import java.util.TreeSet;
import org.apache.lucene.util.BitUtil;
+import org.apache.lucene.util.GroupVIntUtil;
/**
* Abstract base class for performing read operations of Lucene's low-level data types.
@@ -98,6 +99,32 @@ public int readInt() throws IOException {
return ((b4 & 0xFF) << 24) | ((b3 & 0xFF) << 16) | ((b2 & 0xFF) << 8) | (b1 & 0xFF);
}
+ /**
+ * Read all the group varints, including the tail vints. we need a long[] because this is what
+ * postings are using, all longs are actually required to be integers.
+ *
+ * @param dst the array to read ints into.
+ * @param limit the number of int values to read.
+ * @lucene.experimental
+ */
+ public final void readGroupVInts(long[] dst, int limit) throws IOException {
+ int i;
+ for (i = 0; i <= limit - 4; i += 4) {
+ readGroupVInt(dst, i);
+ }
+ for (; i < limit; ++i) {
+ dst[i] = readVInt();
+ }
+ }
+
+ /**
+ * Override if you have a efficient implementation. In general this is when the input supports
+ * random access.
+ */
+ protected void readGroupVInt(long[] dst, int offset) throws IOException {
+ GroupVIntUtil.readGroupVInt(this, dst, offset);
+ }
+
/**
* Reads an int stored in variable-length format. Reads between one and five bytes. Smaller values
* take fewer bytes. Negative numbers are supported, but should be avoided.
diff --git a/lucene/core/src/java/org/apache/lucene/store/DataOutput.java b/lucene/core/src/java/org/apache/lucene/store/DataOutput.java
index be5fbf76b4a2..047cfa103b14 100644
--- a/lucene/core/src/java/org/apache/lucene/store/DataOutput.java
+++ b/lucene/core/src/java/org/apache/lucene/store/DataOutput.java
@@ -21,6 +21,7 @@
import java.util.Set;
import org.apache.lucene.util.BitUtil;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
/**
* Abstract base class for performing write operations of Lucene's low-level data types.
@@ -29,6 +30,7 @@
* internal state like file position).
*/
public abstract class DataOutput {
+ private final BytesRefBuilder groupVIntBytes = new BytesRefBuilder();
/**
* Writes a single byte.
@@ -322,4 +324,43 @@ public void writeSetOfStrings(Set set) throws IOException {
writeString(value);
}
}
+
+ /**
+ * Encode integers using group-varint. It uses {@link DataOutput#writeVInt VInt} to encode tail
+ * values that are not enough for a group. we need a long[] because this is what postings are
+ * using, all longs are actually required to be integers.
+ *
+ * @param values the values to write
+ * @param limit the number of values to write.
+ * @lucene.experimental
+ */
+ public void writeGroupVInts(long[] values, int limit) throws IOException {
+ int off = 0;
+
+ // encode each group
+ while ((limit - off) >= 4) {
+ byte flag = 0;
+ groupVIntBytes.setLength(1);
+ flag |= (encodeGroupValue(Math.toIntExact(values[off++])) - 1) << 6;
+ flag |= (encodeGroupValue(Math.toIntExact(values[off++])) - 1) << 4;
+ flag |= (encodeGroupValue(Math.toIntExact(values[off++])) - 1) << 2;
+ flag |= (encodeGroupValue(Math.toIntExact(values[off++])) - 1);
+ groupVIntBytes.setByteAt(0, flag);
+ writeBytes(groupVIntBytes.bytes(), groupVIntBytes.length());
+ }
+
+ // tail vints
+ for (; off < limit; off++) {
+ writeVInt(Math.toIntExact(values[off]));
+ }
+ }
+
+ private int encodeGroupValue(int v) {
+ int lastOff = groupVIntBytes.length();
+ do {
+ groupVIntBytes.append((byte) (v & 0xFF));
+ v >>>= 8;
+ } while (v != 0);
+ return groupVIntBytes.length() - lastOff;
+ }
}
diff --git a/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java b/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java
index 629e289ef4db..336b5b3e6bbb 100644
--- a/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java
+++ b/lucene/core/src/java/org/apache/lucene/util/ArrayUtil.java
@@ -330,15 +330,36 @@ public static int[] growExact(int[] array, int newLength) {
return copy;
}
+ /**
+ * Returns an array whose size is at least {@code minLength}, generally over-allocating
+ * exponentially, but never allocating more than {@code maxLength} elements.
+ */
+ public static int[] growInRange(int[] array, int minLength, int maxLength) {
+ assert minLength >= 0
+ : "length must be positive (got " + minLength + "): likely integer overflow?";
+
+ if (minLength > maxLength) {
+ throw new IllegalArgumentException(
+ "requested minimum array length "
+ + minLength
+ + " is larger than requested maximum array length "
+ + maxLength);
+ }
+
+ if (array.length >= minLength) {
+ return array;
+ }
+
+ int potentialLength = oversize(minLength, Integer.BYTES);
+ return growExact(array, Math.min(maxLength, potentialLength));
+ }
+
/**
* Returns an array whose size is at least {@code minSize}, generally over-allocating
* exponentially
*/
public static int[] grow(int[] array, int minSize) {
- assert minSize >= 0 : "size must be positive (got " + minSize + "): likely integer overflow?";
- if (array.length < minSize) {
- return growExact(array, oversize(minSize, Integer.BYTES));
- } else return array;
+ return growInRange(array, minSize, Integer.MAX_VALUE);
}
/**
diff --git a/lucene/core/src/java/org/apache/lucene/util/GroupVIntUtil.java b/lucene/core/src/java/org/apache/lucene/util/GroupVIntUtil.java
new file mode 100644
index 000000000000..f98b971c0543
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/GroupVIntUtil.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.util;
+
+import java.io.IOException;
+import org.apache.lucene.store.DataInput;
+
+/**
+ * This class contains utility methods and constants for group varint
+ *
+ * @lucene.internal
+ */
+public final class GroupVIntUtil {
+ // the maximum length of a single group-varint is 4 integers + 1 byte flag.
+ public static final int MAX_LENGTH_PER_GROUP = 17;
+ private static final int[] MASKS = new int[] {0xFF, 0xFFFF, 0xFFFFFF, 0xFFFFFFFF};
+
+ /**
+ * Default implementation of read single group, for optimal performance, you should use {@link
+ * DataInput#readGroupVInts(long[], int)} instead.
+ *
+ * @param dst the array to read ints into.
+ * @param offset the offset in the array to start storing ints.
+ */
+ public static void readGroupVInt(DataInput in, long[] dst, int offset) throws IOException {
+ final int flag = in.readByte() & 0xFF;
+
+ final int n1Minus1 = flag >> 6;
+ final int n2Minus1 = (flag >> 4) & 0x03;
+ final int n3Minus1 = (flag >> 2) & 0x03;
+ final int n4Minus1 = flag & 0x03;
+
+ dst[offset] = readLongInGroup(in, n1Minus1);
+ dst[offset + 1] = readLongInGroup(in, n2Minus1);
+ dst[offset + 2] = readLongInGroup(in, n3Minus1);
+ dst[offset + 3] = readLongInGroup(in, n4Minus1);
+ }
+
+ private static long readLongInGroup(DataInput in, int numBytesMinus1) throws IOException {
+ switch (numBytesMinus1) {
+ case 0:
+ return in.readByte() & 0xFFL;
+ case 1:
+ return in.readShort() & 0xFFFFL;
+ case 2:
+ return (in.readShort() & 0xFFFFL) | ((in.readByte() & 0xFFL) << 16);
+ default:
+ return in.readInt() & 0xFFFFFFFFL;
+ }
+ }
+
+ /**
+ * Provides an abstraction for read int values, so that decoding logic can be reused in different
+ * DataInput.
+ */
+ @FunctionalInterface
+ public static interface IntReader {
+ int read(long v);
+ }
+
+ /**
+ * Faster implementation of read single group, It read values from the buffer that would not cross
+ * boundaries.
+ *
+ * @param in the input to use to read data.
+ * @param remaining the number of remaining bytes allowed to read for current block/segment.
+ * @param reader the supplier of read int.
+ * @param pos the start pos to read from the reader.
+ * @param dst the array to read ints into.
+ * @param offset the offset in the array to start storing ints.
+ * @return the number of bytes read excluding the flag. this indicates the number of positions
+ * should to be increased for caller, it is 0 or positive number and less than {@link
+ * #MAX_LENGTH_PER_GROUP}
+ */
+ public static int readGroupVInt(
+ DataInput in, long remaining, IntReader reader, long pos, long[] dst, int offset)
+ throws IOException {
+ if (remaining < MAX_LENGTH_PER_GROUP) {
+ readGroupVInt(in, dst, offset);
+ return 0;
+ }
+ final int flag = in.readByte() & 0xFF;
+ final long posStart = ++pos; // exclude the flag bytes, the position has updated via readByte().
+ final int n1Minus1 = flag >> 6;
+ final int n2Minus1 = (flag >> 4) & 0x03;
+ final int n3Minus1 = (flag >> 2) & 0x03;
+ final int n4Minus1 = flag & 0x03;
+
+ // This code path has fewer conditionals and tends to be significantly faster in benchmarks
+ dst[offset] = reader.read(pos) & MASKS[n1Minus1];
+ pos += 1 + n1Minus1;
+ dst[offset + 1] = reader.read(pos) & MASKS[n2Minus1];
+ pos += 1 + n2Minus1;
+ dst[offset + 2] = reader.read(pos) & MASKS[n3Minus1];
+ pos += 1 + n3Minus1;
+ dst[offset + 3] = reader.read(pos) & MASKS[n4Minus1];
+ pos += 1 + n4Minus1;
+ return (int) (pos - posStart);
+ }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java b/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
index 21fbf76d045a..3800f00af8fb 100644
--- a/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
@@ -38,7 +38,7 @@
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:
*
*
* - (two bytes) length of the following byte array,
@@ -87,7 +87,7 @@ public static final class BufferSize {
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 static BufferSize automatic() {
// 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 static BufferSize automatic() {
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 String toString() {
private final BufferSize ramBufferSize;
SortInfo sortInfo;
- private int maxTempFiles;
+ private final int maxTempFiles;
private final Comparator comparator;
/** Default comparator: sorts in binary (codepoint) order */
@@ -629,7 +629,7 @@ public SortPartitionTask(Directory dir, Partition part) {
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 Partition call() throws IOException {
}
PriorityQueue queue =
- new PriorityQueue(segmentsToMerge.size()) {
+ new PriorityQueue<>(segmentsToMerge.size()) {
@Override
protected boolean lessThan(FileAndTop a, FileAndTop b) {
return comparator.compare(a.current, b.current) < 0;
diff --git a/lucene/core/src/java/org/apache/lucene/util/Version.java b/lucene/core/src/java/org/apache/lucene/util/Version.java
index 8c124888eb7c..56d43c7c51de 100644
--- a/lucene/core/src/java/org/apache/lucene/util/Version.java
+++ b/lucene/core/src/java/org/apache/lucene/util/Version.java
@@ -112,10 +112,17 @@ public final class Version {
/**
* Match settings and bugs in Lucene's 9.9.0 release.
*
- * @deprecated (9.10.0) Use latest
+ * @deprecated (9.9.1) Use latest
*/
@Deprecated public static final Version LUCENE_9_9_0 = new Version(9, 9, 0);
+ /**
+ * Match settings and bugs in Lucene's 9.9.1 release.
+ *
+ * @deprecated (9.10.0) Use latest
+ */
+ @Deprecated public static final Version LUCENE_9_9_1 = new Version(9, 9, 1);
+
/**
* Match settings and bugs in Lucene's 9.10.0 release.
*
diff --git a/lucene/core/src/java/org/apache/lucene/util/automaton/NFARunAutomaton.java b/lucene/core/src/java/org/apache/lucene/util/automaton/NFARunAutomaton.java
index 6ff52baebbc5..02c14ed457de 100644
--- a/lucene/core/src/java/org/apache/lucene/util/automaton/NFARunAutomaton.java
+++ b/lucene/core/src/java/org/apache/lucene/util/automaton/NFARunAutomaton.java
@@ -193,8 +193,12 @@ public void getNextTransition(Transition t) {
// numTransitions times
}
assert dStates[t.source].transitions[t.transitionUpto] != NOT_COMPUTED;
- t.dest = dStates[t.source].transitions[t.transitionUpto];
+ setTransitionAccordingly(t);
+ }
+
+ private void setTransitionAccordingly(Transition t) {
+ t.dest = dStates[t.source].transitions[t.transitionUpto];
t.min = points[t.transitionUpto];
if (t.transitionUpto == points.length - 1) {
t.max = alphabetSize - 1;
@@ -222,12 +226,7 @@ public void getTransition(int state, int index, Transition t) {
}
assert outgoingTransitions == index;
- t.min = points[t.transitionUpto];
- if (t.transitionUpto == points.length - 1) {
- t.max = alphabetSize - 1;
- } else {
- t.max = points[t.transitionUpto + 1] - 1;
- }
+ setTransitionAccordingly(t);
}
private class DState {
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/DocIdsWriter.java b/lucene/core/src/java/org/apache/lucene/util/bkd/DocIdsWriter.java
index 40db4c0069d9..69818b337f21 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/DocIdsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/DocIdsWriter.java
@@ -219,7 +219,7 @@ private static void readBitSet(IndexInput in, int count, int[] docIDs) throws IO
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 {
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/FST.java b/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
index cc161ba3f8db..260c159f2079 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
@@ -1208,7 +1208,7 @@ public abstract static class BytesReader extends DataInput {
}
/**
- * Represent the FST metadata
+ * Represents the FST metadata.
*
* @param the FST output type
*/
@@ -1236,5 +1236,14 @@ public FSTMetadata(
this.version = version;
this.numBytes = numBytes;
}
+
+ /**
+ * Returns the version constant of the binary format this FST was written in. See the {@code
+ * static final int VERSION} constants in FST's javadoc, e.g. {@link
+ * FST#VERSION_CONTINUOUS_ARCS}.
+ */
+ public int getVersion() {
+ return version;
+ }
}
}
diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/IncrementalHnswGraphMerger.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/IncrementalHnswGraphMerger.java
index ddcbfdaf22d0..9c547a8b7037 100644
--- a/lucene/core/src/java/org/apache/lucene/util/hnsw/IncrementalHnswGraphMerger.java
+++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/IncrementalHnswGraphMerger.java
@@ -19,7 +19,6 @@
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
import java.io.IOException;
-import java.util.Map;
import org.apache.lucene.codecs.HnswGraphProvider;
import org.apache.lucene.codecs.KnnVectorsReader;
import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
@@ -30,9 +29,9 @@
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.util.BitSet;
import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.CollectionUtil;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.InfoStream;
+import org.apache.lucene.util.hppc.IntIntHashMap;
/**
* This selects the biggest Hnsw graph from the provided merge state and initializes a new
@@ -162,7 +161,7 @@ protected final int[] getNewOrdMapping(
case FLOAT32 -> initializerIterator = initReader.getFloatVectorValues(fieldInfo.name);
}
- Map newIdToOldOrdinal = CollectionUtil.newHashMap(initGraphSize);
+ IntIntHashMap newIdToOldOrdinal = new IntIntHashMap(initGraphSize);
int oldOrd = 0;
int maxNewDocID = -1;
for (int oldId = initializerIterator.nextDoc();
@@ -182,9 +181,10 @@ protected final int[] getNewOrdMapping(
for (int newDocId = mergedVectorIterator.nextDoc();
newDocId <= maxNewDocID;
newDocId = mergedVectorIterator.nextDoc()) {
- if (newIdToOldOrdinal.containsKey(newDocId)) {
+ int hashDocIndex = newIdToOldOrdinal.indexOf(newDocId);
+ if (newIdToOldOrdinal.indexExists(hashDocIndex)) {
initializedNodes.set(newOrd);
- oldToNewOrdinalMap[newIdToOldOrdinal.get(newDocId)] = newOrd;
+ oldToNewOrdinalMap[newIdToOldOrdinal.indexGet(hashDocIndex)] = newOrd;
}
newOrd++;
}
diff --git a/lucene/core/src/java19/org/apache/lucene/store/MemorySegmentIndexInput.java b/lucene/core/src/java19/org/apache/lucene/store/MemorySegmentIndexInput.java
index 44ecff57cb1e..72c7d69a1f23 100644
--- a/lucene/core/src/java19/org/apache/lucene/store/MemorySegmentIndexInput.java
+++ b/lucene/core/src/java19/org/apache/lucene/store/MemorySegmentIndexInput.java
@@ -25,6 +25,7 @@
import java.util.Arrays;
import java.util.Objects;
import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.GroupVIntUtil;
/**
* Base IndexInput implementation that uses an array of MemorySegments to represent a file.
@@ -305,6 +306,23 @@ public byte readByte(long pos) throws IOException {
}
}
+ @Override
+ protected void readGroupVInt(long[] dst, int offset) throws IOException {
+ try {
+ final int len =
+ GroupVIntUtil.readGroupVInt(
+ this,
+ curSegment.byteSize() - curPosition,
+ p -> curSegment.get(LAYOUT_LE_INT, p),
+ curPosition,
+ dst,
+ offset);
+ curPosition += len;
+ } catch (NullPointerException | IllegalStateException e) {
+ throw alreadyClosed(e);
+ }
+ }
+
@Override
public void readBytes(long pos, byte[] b, int offset, int len) throws IOException {
try {
diff --git a/lucene/core/src/java20/org/apache/lucene/store/MemorySegmentIndexInput.java b/lucene/core/src/java20/org/apache/lucene/store/MemorySegmentIndexInput.java
index 0f2fdccac90c..c8eec75249d6 100644
--- a/lucene/core/src/java20/org/apache/lucene/store/MemorySegmentIndexInput.java
+++ b/lucene/core/src/java20/org/apache/lucene/store/MemorySegmentIndexInput.java
@@ -25,6 +25,7 @@
import java.util.Arrays;
import java.util.Objects;
import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.GroupVIntUtil;
/**
* Base IndexInput implementation that uses an array of MemorySegments to represent a file.
@@ -303,6 +304,23 @@ public byte readByte(long pos) throws IOException {
}
}
+ @Override
+ protected void readGroupVInt(long[] dst, int offset) throws IOException {
+ try {
+ final int len =
+ GroupVIntUtil.readGroupVInt(
+ this,
+ curSegment.byteSize() - curPosition,
+ p -> curSegment.get(LAYOUT_LE_INT, p),
+ curPosition,
+ dst,
+ offset);
+ curPosition += len;
+ } catch (NullPointerException | IllegalStateException e) {
+ throw alreadyClosed(e);
+ }
+ }
+
@Override
public void readBytes(long pos, byte[] b, int offset, int len) throws IOException {
try {
diff --git a/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java b/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java
index 0f2fdccac90c..c8eec75249d6 100644
--- a/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java
+++ b/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java
@@ -25,6 +25,7 @@
import java.util.Arrays;
import java.util.Objects;
import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.GroupVIntUtil;
/**
* Base IndexInput implementation that uses an array of MemorySegments to represent a file.
@@ -303,6 +304,23 @@ public byte readByte(long pos) throws IOException {
}
}
+ @Override
+ protected void readGroupVInt(long[] dst, int offset) throws IOException {
+ try {
+ final int len =
+ GroupVIntUtil.readGroupVInt(
+ this,
+ curSegment.byteSize() - curPosition,
+ p -> curSegment.get(LAYOUT_LE_INT, p),
+ curPosition,
+ dst,
+ offset);
+ curPosition += len;
+ } catch (NullPointerException | IllegalStateException e) {
+ throw alreadyClosed(e);
+ }
+ }
+
@Override
public void readBytes(long pos, byte[] b, int offset, int len) throws IOException {
try {
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestGroupVInt.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestGroupVInt.java
deleted file mode 100644
index 6c819302ac9c..000000000000
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestGroupVInt.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.codecs.lucene99;
-
-import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
-import java.io.IOException;
-import org.apache.lucene.store.ByteArrayDataInput;
-import org.apache.lucene.store.ByteArrayDataOutput;
-import org.apache.lucene.tests.util.LuceneTestCase;
-import org.apache.lucene.tests.util.TestUtil;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.packed.PackedInts;
-
-public class TestGroupVInt extends LuceneTestCase {
-
- public void testEncodeDecode() throws IOException {
- long[] values = new long[ForUtil.BLOCK_SIZE];
- long[] restored = new long[ForUtil.BLOCK_SIZE];
- final int iterations = atLeast(100);
-
- final GroupVIntWriter w = new GroupVIntWriter();
- byte[] encoded = new byte[(int) (Integer.BYTES * ForUtil.BLOCK_SIZE * 1.25)];
-
- for (int i = 0; i < iterations; i++) {
- final int bpv = TestUtil.nextInt(random(), 1, 31);
- final int numValues = TestUtil.nextInt(random(), 1, ForUtil.BLOCK_SIZE);
-
- // encode
- for (int j = 0; j < numValues; j++) {
- values[j] = RandomNumbers.randomIntBetween(random(), 0, (int) PackedInts.maxValue(bpv));
- }
- w.writeValues(new ByteArrayDataOutput(encoded), values, numValues);
-
- // decode
- GroupVIntReader.readValues(new ByteArrayDataInput(encoded), restored, numValues);
- assertArrayEquals(
- ArrayUtil.copyOfSubArray(values, 0, numValues),
- ArrayUtil.copyOfSubArray(restored, 0, numValues));
- }
- }
-}
diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BPoints.java b/lucene/core/src/test/org/apache/lucene/index/Test2BPoints.java
index 1975b6d1188f..ca0318c1a8a3 100644
--- a/lucene/core/src/test/org/apache/lucene/index/Test2BPoints.java
+++ b/lucene/core/src/test/org/apache/lucene/index/Test2BPoints.java
@@ -143,6 +143,6 @@ public void test2D() throws Exception {
}
private static Codec getCodec() {
- return Codec.forName("Lucene84");
+ return Codec.getDefault();
}
}
diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesFixedSorted.java b/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesFixedSorted.java
index 599fa2811b88..826580d8c461 100644
--- a/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesFixedSorted.java
+++ b/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesFixedSorted.java
@@ -82,12 +82,12 @@ public void testFixedSorted() throws Exception {
int expectedValue = 0;
for (LeafReaderContext context : r.leaves()) {
LeafReader reader = context.reader();
- BinaryDocValues dv = DocValues.getBinary(reader, "dv");
+ SortedDocValues dv = DocValues.getSorted(reader, "dv");
for (int i = 0; i < reader.maxDoc(); i++) {
assertEquals(i, dv.nextDoc());
bytes[0] = (byte) (expectedValue >> 8);
bytes[1] = (byte) expectedValue;
- final BytesRef term = dv.binaryValue();
+ final BytesRef term = dv.lookupOrd(dv.ordValue());
assertEquals(data, term);
expectedValue++;
}
diff --git a/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesOrds.java b/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesOrds.java
index 41664442181c..52bcc7a109a1 100644
--- a/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesOrds.java
+++ b/lucene/core/src/test/org/apache/lucene/index/Test2BSortedDocValuesOrds.java
@@ -84,7 +84,7 @@ public void test2BOrds() throws Exception {
int counter = 0;
for (LeafReaderContext context : r.leaves()) {
LeafReader reader = context.reader();
- BinaryDocValues dv = DocValues.getBinary(reader, "dv");
+ SortedDocValues dv = DocValues.getSorted(reader, "dv");
for (int i = 0; i < reader.maxDoc(); i++) {
assertEquals(i, dv.nextDoc());
bytes[0] = (byte) (counter >> 24);
@@ -92,7 +92,7 @@ public void test2BOrds() throws Exception {
bytes[2] = (byte) (counter >> 8);
bytes[3] = (byte) counter;
counter++;
- final BytesRef term = dv.binaryValue();
+ final BytesRef term = dv.lookupOrd(dv.ordValue());
assertEquals(data, term);
}
}
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesCheckIndexHeader.java b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesCheckIndexHeader.java
index 18ca3e856dae..0fe1465fd0ca 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesCheckIndexHeader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesCheckIndexHeader.java
@@ -64,7 +64,7 @@ public void test() throws Exception {
}
if (random().nextInt(15) == 0) {
riw.updateNumericDocValue(
- new Term("docid", Integer.toString(i)), "docid_intDV", Long.valueOf(i));
+ new Term("docid", Integer.toString(i)), "page_views", Long.valueOf(i));
}
}
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java
index e937b8d7704a..b9dd3656da4a 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java
@@ -46,7 +46,7 @@ public void test() throws Exception {
}
if (random().nextInt(15) == 0) {
riw.updateNumericDocValue(
- new Term("docid", Integer.toString(i)), "docid_intDV", Long.valueOf(i));
+ new Term("docid", Integer.toString(i)), "page_views", Long.valueOf(i));
}
}
riw.close();
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
index 9c107599965e..cced891b86ec 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
@@ -49,7 +49,7 @@ public void test() throws Exception {
}
if (random().nextInt(15) == 0) {
riw.updateNumericDocValue(
- new Term("docid", Integer.toString(i)), "docid_intDV", Long.valueOf(i));
+ new Term("docid", Integer.toString(i)), "page_views", Long.valueOf(i));
}
}
riw.close();
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java
index 102d1786e5be..0f348044b605 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java
@@ -39,7 +39,7 @@
import org.apache.lucene.tests.util.TimeUnits;
@SuppressCodecs({"SimpleText", "Direct"})
-@TimeoutSuite(millis = 8 * TimeUnits.HOUR)
+@TimeoutSuite(millis = 24 * TimeUnits.HOUR)
public class TestIndexWriterMaxDocs extends LuceneTestCase {
// The two hour time was achieved on a Linux 3.13 system with these specs:
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestKnnGraph.java b/lucene/core/src/test/org/apache/lucene/index/TestKnnGraph.java
index 6ea8867a36d9..72be0bd929fa 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestKnnGraph.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestKnnGraph.java
@@ -100,22 +100,18 @@ public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
}
};
- if (vectorEncoding == VectorEncoding.FLOAT32) {
- float32Codec = codec;
- } else {
- float32Codec =
- new FilterCodec(TestUtil.getDefaultCodec().getName(), TestUtil.getDefaultCodec()) {
- @Override
- public KnnVectorsFormat knnVectorsFormat() {
- return new PerFieldKnnVectorsFormat() {
- @Override
- public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
- return new Lucene99HnswVectorsFormat(M, HnswGraphBuilder.DEFAULT_BEAM_WIDTH);
- }
- };
- }
- };
- }
+ float32Codec =
+ new FilterCodec(TestUtil.getDefaultCodec().getName(), TestUtil.getDefaultCodec()) {
+ @Override
+ public KnnVectorsFormat knnVectorsFormat() {
+ return new PerFieldKnnVectorsFormat() {
+ @Override
+ public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
+ return new Lucene99HnswVectorsFormat(M, HnswGraphBuilder.DEFAULT_BEAM_WIDTH);
+ }
+ };
+ }
+ };
}
private VectorEncoding randomVectorEncoding() {
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java
index 2df5e5eb630e..5db9b13eb958 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java
@@ -20,7 +20,6 @@
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 void testPartialMerge() throws Exception {
// 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 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 void testForceMergeDeletesMaxSegSize() throws Exception {
}
// 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 void testForcedMergesRespectSegSize() throws Exception {
"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 void testForcedMergesRespectSegSize() throws Exception {
// 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 void testForcedMergesRespectSegSize() throws Exception {
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 void testForcedMergesRespectSegSize() throws Exception {
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 @@ private static void assertMaxMergedSize(
}
}
- // 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 void testForcedMergeWithPending() throws Exception {
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 @@ void checkSegmentsInExpectations(
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 @@ void checkSegmentsInExpectations(
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 testBefore = new ArrayList<>(segNamesBefore);
List testAfter = new ArrayList<>(segNamesAfter);
@@ -661,10 +651,10 @@ void checkSegmentsInExpectations(
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 void testUnbalancedMergeSelection() throws Exception {
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);
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java b/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java
index 52c29d51290c..3f9d86863b48 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java
@@ -148,7 +148,7 @@ public void collect(int doc, float score) {
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);
}
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestArrayUtil.java b/lucene/core/src/test/org/apache/lucene/util/TestArrayUtil.java
index abc41a5bf9fa..972fd0c3c848 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestArrayUtil.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestArrayUtil.java
@@ -18,6 +18,8 @@
import static org.apache.lucene.util.ArrayUtil.copyOfSubArray;
import static org.apache.lucene.util.ArrayUtil.growExact;
+import static org.apache.lucene.util.ArrayUtil.growInRange;
+import static org.apache.lucene.util.ArrayUtil.oversize;
import java.util.Arrays;
import java.util.Collections;
@@ -371,6 +373,36 @@ public void testGrowExact() {
() -> growExact(new String[] {"a", "b", "c"}, random().nextInt(3)));
}
+ public void testGrowInRange() {
+ int[] array = new int[] {1, 2, 3};
+
+ // If minLength is negative, maxLength does not matter
+ expectThrows(AssertionError.class, () -> growInRange(array, -1, 4));
+ expectThrows(AssertionError.class, () -> growInRange(array, -1, 0));
+ expectThrows(AssertionError.class, () -> growInRange(array, -1, -1));
+
+ // If minLength > maxLength, we throw an exception
+ expectThrows(IllegalArgumentException.class, () -> growInRange(array, 1, 0));
+ expectThrows(IllegalArgumentException.class, () -> growInRange(array, 4, 3));
+ expectThrows(IllegalArgumentException.class, () -> growInRange(array, 5, 4));
+
+ // If minLength is sufficient, we return the array
+ assertSame(array, growInRange(array, 1, 4));
+ assertSame(array, growInRange(array, 1, 2));
+ assertSame(array, growInRange(array, 1, 1));
+
+ int minLength = 4;
+ int maxLength = Integer.MAX_VALUE;
+
+ // The array grows normally if maxLength permits
+ assertEquals(
+ oversize(minLength, Integer.BYTES),
+ growInRange(new int[] {1, 2, 3}, minLength, maxLength).length);
+
+ // The array grows to maxLength if maxLength is limiting
+ assertEquals(minLength, growInRange(new int[] {1, 2, 3}, minLength, minLength).length);
+ }
+
public void testCopyOfSubArray() {
short[] shortArray = {1, 2, 3};
assertArrayEquals(new short[] {1}, copyOfSubArray(shortArray, 0, 1));
diff --git a/lucene/core/src/test/org/apache/lucene/util/automaton/TestNFARunAutomaton.java b/lucene/core/src/test/org/apache/lucene/util/automaton/TestNFARunAutomaton.java
index f838a26b180f..3ae55ac46d2f 100644
--- a/lucene/core/src/test/org/apache/lucene/util/automaton/TestNFARunAutomaton.java
+++ b/lucene/core/src/test/org/apache/lucene/util/automaton/TestNFARunAutomaton.java
@@ -73,6 +73,40 @@ public void testWithRandomRegex() {
}
}
+ public void testRandomAccessTransition() {
+ Automaton nfa = new RegExp(AutomatonTestUtil.randomRegexp(random()), RegExp.NONE).toAutomaton();
+ while (nfa.isDeterministic()) {
+ nfa = new RegExp(AutomatonTestUtil.randomRegexp(random()), RegExp.NONE).toAutomaton();
+ }
+ NFARunAutomaton runAutomaton1, runAutomaton2;
+ runAutomaton1 = new NFARunAutomaton(nfa);
+ runAutomaton2 = new NFARunAutomaton(nfa);
+ assertRandomAccessTransition(runAutomaton1, runAutomaton2, 0, new HashSet<>());
+ }
+
+ private void assertRandomAccessTransition(
+ NFARunAutomaton automaton1, NFARunAutomaton automaton2, int state, Set visited) {
+ if (visited.contains(state)) {
+ return;
+ }
+ visited.add(state);
+
+ Transition t1 = new Transition();
+ Transition t2 = new Transition();
+ automaton1.initTransition(state, t1);
+ if (random().nextBoolean()) {
+ // init is not really necessary for t2
+ automaton2.initTransition(state, t2);
+ }
+ int numStates = automaton2.getNumTransitions(state);
+ for (int i = 0; i < numStates; i++) {
+ automaton1.getNextTransition(t1);
+ automaton2.getTransition(state, i, t2);
+ assertEquals(t1.toString(), t2.toString());
+ assertRandomAccessTransition(automaton1, automaton2, t1.dest, visited);
+ }
+ }
+
public void testRandomAutomatonQuery() throws IOException {
final int docNum = 50;
final int automatonNum = 50;
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 6ddcabb03ae7..086eb460e56e 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
@@ -17,8 +17,6 @@
package org.apache.lucene.util.bkd;
import java.io.IOException;
-import java.security.AccessController;
-import java.security.PrivilegedAction;
import java.util.Arrays;
import java.util.List;
import java.util.Set;
@@ -34,7 +32,7 @@
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.CollectionUtil;
-import org.apache.lucene.util.SuppressForbidden;
+import org.apache.lucene.util.Constants;
public class TestDocIdsWriter extends LuceneTestCase {
@@ -159,10 +157,12 @@ public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
}
// This simple test tickles a JVM C2 JIT crash on JDK's less than 21.0.1
- // Crashes only when run with C2, so with the environment variable `CI` set
+ // Crashes only when run with HotSpot C2.
// Regardless of whether C2 is enabled or not, the test should never fail.
public void testCrash() throws IOException {
- assumeTrue("Requires C2, which is only enabled when CI env is set", getCIEnv() != null);
+ assumeTrue(
+ "Requires HotSpot C2 compiler (won't work on client VM).",
+ Constants.IS_HOTSPOT_VM && !Constants.IS_CLIENT_VM);
int itrs = atLeast(100);
for (int i = 0; i < itrs; i++) {
try (Directory dir = newDirectory();
@@ -174,11 +174,4 @@ public void testCrash() throws IOException {
}
}
}
-
- @SuppressForbidden(reason = "needed to check if C2 is enabled")
- @SuppressWarnings("removal")
- private static String getCIEnv() {
- PrivilegedAction pa = () -> System.getenv("CI");
- return AccessController.doPrivileged(pa);
- }
}
diff --git a/lucene/demo/src/java/org/apache/lucene/demo/IndexFiles.java b/lucene/demo/src/java/org/apache/lucene/demo/IndexFiles.java
index 9c683d3937c9..73e7eaf3e406 100644
--- a/lucene/demo/src/java/org/apache/lucene/demo/IndexFiles.java
+++ b/lucene/demo/src/java/org/apache/lucene/demo/IndexFiles.java
@@ -77,7 +77,7 @@ public static void main(String[] args) throws Exception {
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 static void main(String[] args) throws Exception {
// 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 @@ void indexDoc(IndexWriter writer, Path file, long lastModified) throws IOExcepti
// 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
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java
index bf4867e0dda8..0d641d396ec8 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java
@@ -328,7 +328,8 @@ public int[] getBulkOrdinals(FacetLabel... categoryPaths) throws IOException {
}
// First try to find results in the cache:
int[] result = new int[categoryPaths.length];
- int[] indexesMissingFromCache = new int[10]; // initial size, will grow when required
+ // Will grow when required, but never beyond categoryPaths.length
+ int[] indexesMissingFromCache = new int[Math.min(10, categoryPaths.length)];
int numberOfMissingFromCache = 0;
FacetLabel cp;
Integer res;
@@ -352,7 +353,8 @@ public int[] getBulkOrdinals(FacetLabel... categoryPaths) throws IOException {
}
} else {
indexesMissingFromCache =
- ArrayUtil.grow(indexesMissingFromCache, numberOfMissingFromCache + 1);
+ ArrayUtil.growInRange(
+ indexesMissingFromCache, numberOfMissingFromCache + 1, categoryPaths.length);
indexesMissingFromCache[numberOfMissingFromCache++] = i;
}
}
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java
index 1d85661003e9..fd6d4d0cfb3b 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java
@@ -153,11 +153,11 @@ public String toString(String field) {
return "GlobalOrdinalsQuery{"
+ "joinField="
+ joinField
- + "min="
+ + ",min="
+ min
- + "max="
+ + ",max="
+ max
- + "fromQuery="
+ + ",fromQuery="
+ fromQuery
+ '}';
}
diff --git a/lucene/misc/src/test/org/apache/lucene/misc/TestHighFreqTerms.java b/lucene/misc/src/test/org/apache/lucene/misc/TestHighFreqTerms.java
index 5aa7687a6218..4501d7fc8269 100644
--- a/lucene/misc/src/test/org/apache/lucene/misc/TestHighFreqTerms.java
+++ b/lucene/misc/src/test/org/apache/lucene/misc/TestHighFreqTerms.java
@@ -90,7 +90,7 @@ public void testOrderedByDocFreqDescending() throws Exception {
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 void testGetHighFreqTerms() throws Exception {
}
} 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 void testOrderedByTermFreqDescending() throws Exception {
// 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 void testGetTermFreqOrdered() throws Exception {
} 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 void testGetTermFreqOrdered() throws Exception {
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 @@ private static String getContent(int i) {
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;
}
}
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/mlt/TestMoreLikeThis.java b/lucene/queries/src/test/org/apache/lucene/queries/mlt/TestMoreLikeThis.java
index e45c764eaeea..b5b51701454c 100644
--- a/lucene/queries/src/test/org/apache/lucene/queries/mlt/TestMoreLikeThis.java
+++ b/lucene/queries/src/test/org/apache/lucene/queries/mlt/TestMoreLikeThis.java
@@ -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 void testSmallSampleFromCorpus() throws Throwable {
BooleanQuery query =
(BooleanQuery) mlt.like("one_percent", new StringReader("tenth tenth all"));
- Collection clauses = query.clauses();
+ List clauses = query.clauses();
assertTrue(clauses.size() == 2);
- Term term = ((TermQuery) ((List) clauses).get(0).getQuery()).getTerm();
+ Term term = ((TermQuery) clauses.get(0).getQuery()).getTerm();
assertTrue(term.text().equals("all"));
- term = ((TermQuery) ((List) 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) clauses).get(0).getQuery()).getTerm();
+ term = ((TermQuery) clauses.get(0).getQuery()).getTerm();
assertTrue(term.text().equals("all"));
- term = ((TermQuery) ((List) clauses).get(1).getQuery()).getTerm();
+ term = ((TermQuery) clauses.get(1).getQuery()).getTerm();
assertTrue(term.text().equals("tenth"));
// clean up
@@ -287,7 +285,7 @@ public void testMultiValues() throws Exception {
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 void testTopN() throws Exception {
// check best terms are topN of highest idf
Collection 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 void testMultiFieldShouldReturnPerFieldBooleanQuery() throws Exception {
BooleanQuery query = (BooleanQuery) mlt.like(inputDocId);
Collection clauses = query.clauses();
- Collection expectedClothesShopClauses = new ArrayList();
+ Collection expectedClothesShopClauses = new ArrayList<>();
for (String itemForSale : clothesShopItemForSale) {
BooleanClause booleanClause =
new BooleanClause(
@@ -544,7 +542,7 @@ public void testMultiFieldShouldReturnPerFieldBooleanQuery() throws Exception {
}
}
- 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 void testCustomFrequecy() throws IOException {
@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) {
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/simple/TestSimpleQueryParser.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/simple/TestSimpleQueryParser.java
index da9c5fc75085..829387a2bd08 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/simple/TestSimpleQueryParser.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/simple/TestSimpleQueryParser.java
@@ -98,7 +98,8 @@ public void testFuzzy() throws Exception {
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 */
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/StrategyTestCase.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/StrategyTestCase.java
index 307f740084bf..7682ab319bc8 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/StrategyTestCase.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/StrategyTestCase.java
@@ -145,7 +145,7 @@ public void runTestQuery(SpatialMatchConcern concern, SpatialTestQuery q) {
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 void runTestQuery(SpatialMatchConcern concern, SpatialTestQuery q) {
}
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 @@ protected void checkValueSource(DoubleValuesSource vs, float[] scores, float del
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);
}
}
}
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/geo/BaseGeoPointTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/geo/BaseGeoPointTestCase.java
index d1112dcf4f44..d371292324d5 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/geo/BaseGeoPointTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/geo/BaseGeoPointTestCase.java
@@ -78,8 +78,8 @@
* test focuses on geospatial (distance queries, polygon queries, etc) indexing and search, not any
* underlying storage format or encoding: it merely supplies two hooks for the encoding so that
* tests can be exact. The [stretch] goal is for this test to be so thorough in testing a new geo
- * impl that if this test passes, then all Lucene/Solr tests should also pass. Ie, if there is some
- * bug in a given geo impl that this test fails to catch then this test needs to be improved!
+ * impl that if this test passes, then all Lucene tests should also pass. Ie, if there is some bug
+ * in a given geo impl that this test fails to catch then this test needs to be improved!
*/
public abstract class BaseGeoPointTestCase extends LuceneTestCase {
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/geo/BaseXYPointTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/geo/BaseXYPointTestCase.java
index e66fa12ce614..3a733b5462f2 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/geo/BaseXYPointTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/geo/BaseXYPointTestCase.java
@@ -650,7 +650,7 @@ public void testMultiValued() throws Exception {
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 @@ protected abstract Query newDistanceQuery(
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 PointsReader fieldsReader(SegmentReadState readState) throws IOException
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;
}
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseCompoundFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseCompoundFormatTestCase.java
index 8d22046f2963..6ad7b625a7cc 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseCompoundFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseCompoundFormatTestCase.java
@@ -51,9 +51,9 @@
/**
* Abstract class to do basic tests for a compound format. NOTE: This test focuses on the compound
* impl, nothing else. The [stretch] goal is for this test to be so thorough in testing a new
- * CompoundFormat that if this test passes, then all Lucene/Solr tests should also pass. Ie, if
- * there is some bug in a given CompoundFormat that this test fails to catch then this test needs to
- * be improved!
+ * CompoundFormat that if this test passes, then all Lucene tests should also pass. Ie, if there is
+ * some bug in a given CompoundFormat that this test fails to catch then this test needs to be
+ * improved!
*/
public abstract class BaseCompoundFormatTestCase extends BaseIndexFileFormatTestCase {
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 e5f1559b90f2..fc02a99bebc9 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
@@ -94,9 +94,9 @@
/**
* Abstract class to do basic tests for a docvalues format. NOTE: This test focuses on the docvalues
* impl, nothing else. The [stretch] goal is for this test to be so thorough in testing a new
- * DocValuesFormat that if this test passes, then all Lucene/Solr tests should also pass. Ie, if
- * there is some bug in a given DocValuesFormat that this test fails to catch then this test needs
- * to be improved!
+ * DocValuesFormat that if this test passes, then all Lucene tests should also pass. Ie, if there is
+ * some bug in a given DocValuesFormat that this test fails to catch then this test needs to be
+ * improved!
*/
public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTestCase {
@@ -1294,7 +1294,7 @@ public void testRandomSortedBytes() throws IOException {
}
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 @@ protected void doTestSortedSetVsStoredFields(
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
- Set valueSet = new HashSet();
+ Set 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 void testHighOrdsSortedSetDV() throws Exception {
}
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 {
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseFieldInfoFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseFieldInfoFormatTestCase.java
index 886fef08604a..83f8b293113c 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseFieldInfoFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseFieldInfoFormatTestCase.java
@@ -52,8 +52,8 @@
/**
* Abstract class to do basic tests for fis format. NOTE: This test focuses on the fis impl, nothing
* else. The [stretch] goal is for this test to be so thorough in testing a new fis format that if
- * this test passes, then all Lucene/Solr tests should also pass. Ie, if there is some bug in a
- * given fis Format that this test fails to catch then this test needs to be improved!
+ * this test passes, then all Lucene tests should also pass. Ie, if there is some bug in a given fis
+ * Format that this test fails to catch then this test needs to be improved!
*/
public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTestCase {
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseIndexFileFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseIndexFileFormatTestCase.java
index 968fd8ed61ed..3b73af0779f9 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseIndexFileFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseIndexFileFormatTestCase.java
@@ -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> EXCLUDED_CLASSES =
- Collections.newSetFromMap(new IdentityHashMap, 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 @@ private Map bytesUsedByExtension(Directory d) throws IOException {
* that store the same content.
*/
protected Collection excludedExtensionsFromByteCounts() {
- return new HashSet(
+ 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 @@ public void checkIntegrity() throws IOException {}
try (FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(writeState)) {
final Fields fields =
new Fields() {
- TreeSet indexedFields =
+ final TreeSet indexedFields =
new TreeSet<>(FieldInfos.getIndexedFields(oneDocReader));
@Override
@@ -622,7 +622,7 @@ public void testRandomExceptions() throws Exception {
// 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 @@ public void testRandomExceptions() throws Exception {
} 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 @@ protected final DirectoryReader maybeWrapWithMergingReader(DirectoryReader r) th
/** A directory that tracks created files that haven't been deleted. */
protected static class FileTrackingDirectoryWrapper extends FilterDirectory {
- private final Set files =
- Collections.newSetFromMap(new ConcurrentHashMap());
+ private final Set files = ConcurrentHashMap.newKeySet();
/** Sole constructor. */
FileTrackingDirectoryWrapper(Directory in) {
@@ -932,7 +931,7 @@ public IndexOutput createTempOutput(String prefix, String suffix, IOContext cont
}
/**
- * 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 @@ public void testCheckIntegrityReadsAllBytes() throws Exception {
+ unread
+ " of file "
+ name
- + "("
+ + " ("
+ unreadBytes.length()
- + "bytes) was not read.");
+ + " bytes) was not read.");
}
}
assertTrue(String.join("\n", messages), messages.isEmpty());
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseNormsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseNormsFormatTestCase.java
index 96face910c53..2c25b1315cbc 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseNormsFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseNormsFormatTestCase.java
@@ -54,8 +54,8 @@
/**
* Abstract class to do basic tests for a norms format. NOTE: This test focuses on the norms impl,
* nothing else. The [stretch] goal is for this test to be so thorough in testing a new NormsFormat
- * that if this test passes, then all Lucene/Solr tests should also pass. Ie, if there is some bug
- * in a given NormsFormat that this test fails to catch then this test needs to be improved!
+ * that if this test passes, then all Lucene tests should also pass. Ie, if there is some bug in a
+ * given NormsFormat that this test fails to catch then this test needs to be improved!
*/
public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCase {
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BasePointsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BasePointsFormatTestCase.java
index 9fedcc5eace2..265e3f073be2 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BasePointsFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BasePointsFormatTestCase.java
@@ -63,8 +63,8 @@
/**
* Abstract class to do basic tests for a points format. NOTE: This test focuses on the points impl,
* nothing else. The [stretch] goal is for this test to be so thorough in testing a new PointsFormat
- * that if this test passes, then all Lucene/Solr tests should also pass. Ie, if there is some bug
- * in a given PointsFormat that this test fails to catch then this test needs to be improved!
+ * that if this test passes, then all Lucene tests should also pass. Ie, if there is some bug in a
+ * given PointsFormat that this test fails to catch then this test needs to be improved!
*/
public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCase {
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BasePostingsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BasePostingsFormatTestCase.java
index 181be045fd1b..45e8bd8e858b 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BasePostingsFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BasePostingsFormatTestCase.java
@@ -77,9 +77,9 @@
/**
* Abstract class to do basic tests for a postings format. NOTE: This test focuses on the postings
* (docs/freqs/positions/payloads/offsets) impl, not the terms dict. The [stretch] goal is for this
- * test to be so thorough in testing a new PostingsFormat that if this test passes, then all
- * Lucene/Solr tests should also pass. Ie, if there is some bug in a given PostingsFormat that this
- * test fails to catch then this test needs to be improved!
+ * test to be so thorough in testing a new PostingsFormat that if this test passes, then all Lucene
+ * tests should also pass. Ie, if there is some bug in a given PostingsFormat that this test fails
+ * to catch then this test needs to be improved!
*/
// TODO can we make it easy for testing to pair up a "random terms dict impl" with your postings
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseSegmentInfoFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseSegmentInfoFormatTestCase.java
index 674edb89db8b..0ebb3fcffdc4 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseSegmentInfoFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseSegmentInfoFormatTestCase.java
@@ -44,8 +44,8 @@
/**
* Abstract class to do basic tests for si format. NOTE: This test focuses on the si impl, nothing
* else. The [stretch] goal is for this test to be so thorough in testing a new si format that if
- * this test passes, then all Lucene/Solr tests should also pass. Ie, if there is some bug in a
- * given si Format that this test fails to catch then this test needs to be improved!
+ * this test passes, then all Lucene tests should also pass. Ie, if there is some bug in a given si
+ * Format that this test fails to catch then this test needs to be improved!
*/
public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatTestCase {
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseStoredFieldsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseStoredFieldsFormatTestCase.java
index 2c7aadab81f6..761cf9b7387a 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseStoredFieldsFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseStoredFieldsFormatTestCase.java
@@ -72,6 +72,7 @@
import org.apache.lucene.tests.analysis.MockAnalyzer;
import org.apache.lucene.tests.store.MockDirectoryWrapper;
import org.apache.lucene.tests.store.MockDirectoryWrapper.Throttling;
+import org.apache.lucene.tests.util.LineFileDocs;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@@ -1006,4 +1007,34 @@ public void testRandomStoredFieldsWithIndexSort() throws Exception {
verifyStoreFields.run();
IOUtils.close(iw, dir);
}
+
+ /** Test realistic data, which typically compresses better than random data. */
+ public void testLineFileDocs() throws IOException {
+ // Use a FS dir and a non-randomized IWC to not slow down indexing
+ try (Directory dir = newFSDirectory(createTempDir())) {
+ try (LineFileDocs docs = new LineFileDocs(random());
+ IndexWriter w = new IndexWriter(dir, new IndexWriterConfig())) {
+ final int numDocs = atLeast(10_000);
+ for (int i = 0; i < numDocs; ++i) {
+ // Only keep stored fields
+ Document doc = docs.nextDoc();
+ Document storedDoc = new Document();
+ for (IndexableField field : doc.getFields()) {
+ if (field.fieldType().stored()) {
+ IndexableField storedField = field;
+ if (field.stringValue() != null) {
+ // Disable indexing
+ storedField = new StoredField(field.name(), field.stringValue());
+ }
+ storedDoc.add(storedField);
+ }
+ }
+
+ w.addDocument(storedDoc);
+ }
+ w.forceMerge(1);
+ }
+ TestUtil.checkIndex(dir);
+ }
+ }
}
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/RandomPostingsTester.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/RandomPostingsTester.java
index 924bc4a130c4..870c4f5548c0 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/RandomPostingsTester.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/RandomPostingsTester.java
@@ -115,7 +115,7 @@ public enum Option {
// Test w/ multiple threads
THREADS
- };
+ }
private long totalPostings;
private long totalPayloadBytes;
@@ -956,7 +956,7 @@ private void verifyEnum(
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 @@ private void testTermsOneThread(
}
TermsEnum intersected = fieldsSource.terms(field).intersect(ca, startTerm);
- Set intersectedTerms = new HashSet();
+ Set intersectedTerms = new HashSet<>();
BytesRef term;
while ((term = intersected.next()) != null) {
if (startTerm != null) {
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/similarities/BaseSimilarityTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/similarities/BaseSimilarityTestCase.java
index 84c340d516e7..763bf16e691f 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/similarities/BaseSimilarityTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/similarities/BaseSimilarityTestCase.java
@@ -42,8 +42,8 @@
/**
* Abstract class to do basic tests for a similarity. NOTE: This test focuses on the similarity
* impl, nothing else. The [stretch] goal is for this test to be so thorough in testing a new
- * Similarity that if this test passes, then all Lucene/Solr tests should also pass. Ie, if there is
- * some bug in a given Similarity that this test fails to catch then this test needs to be improved!
+ * Similarity that if this test passes, then all Lucene tests should also pass. Ie, if there is some
+ * bug in a given Similarity that this test fails to catch then this test needs to be improved!
*/
public abstract class BaseSimilarityTestCase extends LuceneTestCase {
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/store/BaseChunkedDirectoryTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/store/BaseChunkedDirectoryTestCase.java
index a2e2d519ae53..dd956c6c3fd0 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/store/BaseChunkedDirectoryTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/store/BaseChunkedDirectoryTestCase.java
@@ -50,10 +50,19 @@ protected Directory getDirectory(Path path) throws IOException {
/** Creates a new directory with the specified max chunk size */
protected abstract Directory getDirectory(Path path, int maxChunkSize) throws IOException;
+ public void testGroupVIntMultiBlocks() throws IOException {
+ final int maxChunkSize = random().nextInt(64, 512);
+ try (Directory dir = getDirectory(createTempDir(), maxChunkSize)) {
+ doTestGroupVInt(dir, 10, 1, 31, 1024);
+ }
+ }
+
public void testCloneClose() throws Exception {
Directory dir = getDirectory(createTempDir("testCloneClose"));
IndexOutput io = dir.createOutput("bytes", newIOContext(random()));
+ final long[] values = new long[] {0, 7, 11, 9};
io.writeVInt(5);
+ io.writeGroupVInts(values, values.length);
io.close();
IndexInput one = dir.openInput("bytes", IOContext.DEFAULT);
IndexInput two = one.clone();
@@ -65,6 +74,11 @@ public void testCloneClose() throws Exception {
() -> {
two.readVInt();
});
+ expectThrows(
+ AlreadyClosedException.class,
+ () -> {
+ two.readGroupVInts(values, values.length);
+ });
assertEquals(5, three.readVInt());
one.close();
three.close();
@@ -74,11 +88,13 @@ public void testCloneClose() throws Exception {
public void testCloneSliceClose() throws Exception {
Directory dir = getDirectory(createTempDir("testCloneSliceClose"));
IndexOutput io = dir.createOutput("bytes", newIOContext(random()));
+ final long[] values = new long[] {0, 7, 11, 9};
io.writeInt(1);
io.writeInt(2);
+ io.writeGroupVInts(values, values.length); // will write 5 bytes
io.close();
IndexInput slicer = dir.openInput("bytes", newIOContext(random()));
- IndexInput one = slicer.slice("first int", 0, 4);
+ IndexInput one = slicer.slice("first int", 0, 4 + 5);
IndexInput two = slicer.slice("second int", 4, 4);
one.close();
expectThrows(
@@ -86,6 +102,11 @@ public void testCloneSliceClose() throws Exception {
() -> {
one.readInt();
});
+ expectThrows(
+ AlreadyClosedException.class,
+ () -> {
+ one.readGroupVInts(values, values.length);
+ });
assertEquals(2, two.readInt());
// reopen a new slice "another":
IndexInput another = slicer.slice("first int", 0, 4);
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/store/BaseDirectoryTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/store/BaseDirectoryTestCase.java
index b98c22d8e40e..24d8db0b02f6 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/store/BaseDirectoryTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/store/BaseDirectoryTestCase.java
@@ -18,6 +18,7 @@
import com.carrotsearch.randomizedtesting.RandomizedTest;
import com.carrotsearch.randomizedtesting.generators.RandomBytes;
+import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import java.io.EOFException;
import java.io.FileNotFoundException;
@@ -58,6 +59,7 @@
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.packed.PackedInts;
import org.junit.Assert;
/** Base class for {@link Directory} implementations. */
@@ -1438,4 +1440,76 @@ public void testListAllIsSorted() throws IOException {
assertArrayEquals(expected, actual);
}
}
+
+ public void testDataTypes() throws IOException {
+ final long[] values = new long[] {43, 12345, 123456, 1234567890};
+ try (Directory dir = getDirectory(createTempDir("testDataTypes"))) {
+ IndexOutput out = dir.createOutput("test", IOContext.DEFAULT);
+ out.writeByte((byte) 43);
+ out.writeShort((short) 12345);
+ out.writeInt(1234567890);
+ out.writeGroupVInts(values, 4);
+ out.writeLong(1234567890123456789L);
+ out.close();
+
+ long[] restored = new long[4];
+ IndexInput in = dir.openInput("test", IOContext.DEFAULT);
+ assertEquals(43, in.readByte());
+ assertEquals(12345, in.readShort());
+ assertEquals(1234567890, in.readInt());
+ in.readGroupVInts(restored, 4);
+ assertArrayEquals(values, restored);
+ assertEquals(1234567890123456789L, in.readLong());
+ in.close();
+ }
+ }
+
+ public void testGroupVInt() throws IOException {
+ try (Directory dir = getDirectory(createTempDir("testGroupVInt"))) {
+ // test fallback to default implementation of readGroupVInt
+ doTestGroupVInt(dir, 5, 1, 6, 8);
+
+ // use more iterations to covers all bpv
+ doTestGroupVInt(dir, atLeast(100), 1, 31, 128);
+
+ // we use BaseChunkedDirectoryTestCase#testGroupVIntMultiBlocks cover multiple blocks for
+ // ByteBuffersDataInput and MMapDirectory
+ }
+ }
+
+ protected void doTestGroupVInt(
+ Directory dir, int iterations, int minBpv, int maxBpv, int maxNumValues) throws IOException {
+ long[] values = new long[maxNumValues];
+ int[] numValuesArray = new int[iterations];
+ IndexOutput groupVIntOut = dir.createOutput("group-varint", IOContext.DEFAULT);
+ IndexOutput vIntOut = dir.createOutput("vint", IOContext.DEFAULT);
+
+ // encode
+ for (int iter = 0; iter < iterations; iter++) {
+ final int bpv = TestUtil.nextInt(random(), minBpv, maxBpv);
+ numValuesArray[iter] = TestUtil.nextInt(random(), 1, maxNumValues);
+ for (int j = 0; j < numValuesArray[iter]; j++) {
+ values[j] = RandomNumbers.randomIntBetween(random(), 0, (int) PackedInts.maxValue(bpv));
+ vIntOut.writeVInt((int) values[j]);
+ }
+ groupVIntOut.writeGroupVInts(values, numValuesArray[iter]);
+ }
+ groupVIntOut.close();
+ vIntOut.close();
+
+ // decode
+ IndexInput groupVIntIn = dir.openInput("group-varint", IOContext.DEFAULT);
+ IndexInput vIntIn = dir.openInput("vint", IOContext.DEFAULT);
+ for (int iter = 0; iter < iterations; iter++) {
+ groupVIntIn.readGroupVInts(values, numValuesArray[iter]);
+ for (int j = 0; j < numValuesArray[iter]; j++) {
+ assertEquals(vIntIn.readVInt(), values[j]);
+ }
+ }
+
+ groupVIntIn.close();
+ vIntIn.close();
+ dir.deleteFile("group-varint");
+ dir.deleteFile("vint");
+ }
}
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/LineFileDocs.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/LineFileDocs.java
index 812582fe714c..9d02549c7de9 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/LineFileDocs.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/LineFileDocs.java
@@ -37,14 +37,14 @@
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.IntField;
import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.document.KeywordField;
import org.apache.lucene.document.NumericDocValuesField;
-import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.IOUtils;
@@ -199,17 +199,16 @@ private static final class DocState {
final Document doc;
final Field titleTokenized;
final Field title;
- final Field titleDV;
final Field body;
final Field id;
final Field idNum;
- final Field idNumDV;
final Field date;
+ final Field pageViews;
public DocState() {
doc = new Document();
- title = new StringField("title", "", Field.Store.NO);
+ title = new KeywordField("title", "", Field.Store.NO);
doc.add(title);
FieldType ft = new FieldType(TextField.TYPE_STORED);
@@ -227,16 +226,15 @@ public DocState() {
id = new StringField("docid", "", Field.Store.YES);
doc.add(id);
- idNum = new IntPoint("docid_int", 0);
+ idNum = new IntField("docid_int", 0, Field.Store.NO);
doc.add(idNum);
date = new StringField("date", "", Field.Store.YES);
doc.add(date);
- titleDV = new SortedDocValuesField("titleDV", new BytesRef());
- idNumDV = new NumericDocValuesField("docid_intDV", 0);
- doc.add(titleDV);
- doc.add(idNumDV);
+ // A numeric DV field that can be used for DV updates
+ pageViews = new NumericDocValuesField("page_views", 0L);
+ doc.add(pageViews);
}
}
@@ -277,17 +275,12 @@ public Document nextDoc() throws IOException {
docState.body.setStringValue(line.substring(1 + spot2, line.length()));
final String title = line.substring(0, spot);
docState.title.setStringValue(title);
- if (docState.titleDV != null) {
- docState.titleDV.setBytesValue(new BytesRef(title));
- }
docState.titleTokenized.setStringValue(title);
docState.date.setStringValue(line.substring(1 + spot, spot2));
final int i = id.getAndIncrement();
docState.id.setStringValue(Integer.toString(i));
docState.idNum.setIntValue(i);
- if (docState.idNumDV != null) {
- docState.idNumDV.setLongValue(i);
- }
+ docState.pageViews.setLongValue(random.nextInt(10_000));
if (random.nextInt(5) == 4) {
// Make some sparse fields
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java
index 7b6dd0646609..b9165be1bc7c 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/LuceneTestCase.java
@@ -494,8 +494,7 @@ static int defaultRandomMultiplier() {
Arrays.asList(
"tests.leaveTemporary" /* ANT tasks's (junit4) flag. */,
"tests.leavetemporary" /* lowercase */,
- "tests.leavetmpdir" /* default */,
- "solr.test.leavetmpdir" /* Solr's legacy */)) {
+ "tests.leavetmpdir" /* default */)) {
defaultValue |= systemPropertyAsBoolean(property, false);
}
LEAVE_TEMPORARY = defaultValue;
@@ -659,13 +658,7 @@ protected boolean verify(Method key) {
// We reset the default locale and timezone; these properties change as a
// side-effect
- "user.language",
- "user.timezone",
-
- // TODO: these should, ideally, be moved to Solr's base class.
- "solr.directoryFactory",
- "solr.solr.home",
- "solr.data.dir"))
+ "user.language", "user.timezone"))
.around(classEnvRule = new TestRuleSetupAndRestoreClassEnv());
}
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleAssertionsRequired.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleAssertionsRequired.java
index 6bc2020a8f6a..ae2704028785 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleAssertionsRequired.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleAssertionsRequired.java
@@ -20,7 +20,7 @@
import org.junit.runner.Description;
import org.junit.runners.model.Statement;
-/** Require assertions for Lucene/Solr packages. */
+/** Require assertions for Lucene packages. */
public class TestRuleAssertionsRequired implements TestRule {
@Override
public Statement apply(final Statement base, final Description description) {
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java
index 4b0719d72e32..c386a7ef85e6 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java
@@ -105,13 +105,6 @@ public boolean isInitialized() {
@Override
protected void before() throws Exception {
- // enable this by default, for IDE consistency with ant tests (as it's the default from ant)
- // TODO: really should be in solr base classes, but some extend LTC directly.
- // we do this in beforeClass, because some tests currently disable it
- if (System.getProperty("solr.directoryFactory") == null) {
- System.setProperty("solr.directoryFactory", "org.apache.solr.core.MockDirectoryFactory");
- }
-
// if verbose: print some debugging stuff about which codecs are loaded.
if (VERBOSE) {
System.out.println("Loaded codecs: " + Codec.availableCodecs());
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleTemporaryFilesCleanup.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleTemporaryFilesCleanup.java
index e3ff2f798d6d..6d1b6ad1f3a7 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleTemporaryFilesCleanup.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleTemporaryFilesCleanup.java
@@ -235,7 +235,6 @@ Path getPerTestClassTempDir() {
Class> clazz = ctx.getTargetClass();
String prefix = clazz.getName();
prefix = prefix.replaceFirst("^org.apache.lucene.", "lucene.");
- prefix = prefix.replaceFirst("^org.apache.solr.", "solr.");
int attempt = 0;
Path f;
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java
index f86bedafef7c..ece42e002bd8 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java
@@ -63,6 +63,8 @@
import org.apache.lucene.document.BinaryPoint;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
+import org.apache.lucene.document.IntField;
+import org.apache.lucene.document.KeywordField;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.index.CheckIndex;
@@ -1420,7 +1422,19 @@ public static Document cloneDocument(Document doc1) {
final Field field2;
final DocValuesType dvType = field1.fieldType().docValuesType();
final int dimCount = field1.fieldType().pointDimensionCount();
- if (dvType != DocValuesType.NONE) {
+ if (f instanceof KeywordField) {
+ field2 =
+ new KeywordField(
+ f.name(),
+ f.stringValue(),
+ f.fieldType().stored() ? Field.Store.YES : Field.Store.NO);
+ } else if (f instanceof IntField) {
+ field2 =
+ new IntField(
+ f.name(),
+ f.numericValue().intValue(),
+ f.fieldType().stored() ? Field.Store.YES : Field.Store.NO);
+ } else if (dvType != DocValuesType.NONE) {
switch (dvType) {
case NUMERIC:
field2 = new NumericDocValuesField(field1.name(), field1.numericValue().longValue());