mirror of https://github.com/apache/lucene.git
Clean up many small fixes (#1732)
* Abstract classes don't need public constructors since they can only be called by subclasses * Don't escape html characters in @code tags in javadoc * Fixed a few int/long arithmetic * Use explicit Term.toString instead of implicit byte[].toString * Javadoc typos * Consistent capitalization for field and parameter names
This commit is contained in:
parent
984466f31b
commit
a46316e156
|
@ -604,7 +604,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
|
|||
assert ent.isTerm: "i=" + i;
|
||||
|
||||
PendingTerm term = (PendingTerm) ent;
|
||||
assert StringHelper.startsWith(term.termBytes, prefix): "term.term=" + term.termBytes + " prefix=" + prefix;
|
||||
assert StringHelper.startsWith(term.termBytes, prefix): term + " prefix=" + prefix;
|
||||
BlockTermState state = term.state;
|
||||
final int suffix = term.termBytes.length - prefixLength;
|
||||
/*
|
||||
|
@ -640,7 +640,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
|
|||
PendingEntry ent = pending.get(i);
|
||||
if (ent.isTerm) {
|
||||
PendingTerm term = (PendingTerm) ent;
|
||||
assert StringHelper.startsWith(term.termBytes, prefix): "term.term=" + term.termBytes + " prefix=" + prefix;
|
||||
assert StringHelper.startsWith(term.termBytes, prefix): term + " prefix=" + prefix;
|
||||
BlockTermState state = term.state;
|
||||
final int suffix = term.termBytes.length - prefixLength;
|
||||
/*
|
||||
|
|
|
@ -454,7 +454,7 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
|
|||
// Field not bloom'd
|
||||
break;
|
||||
}
|
||||
assert bloomFilters.containsKey(field) == false;
|
||||
assert bloomFilters.containsKey(fieldInfo) == false;
|
||||
bloomFilters.put(fieldInfo, bloomFilter);
|
||||
}
|
||||
// Make sure there's at least one doc for this term:
|
||||
|
|
|
@ -540,10 +540,10 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
|
|||
@Override
|
||||
public int nextPosition() {
|
||||
if (positions != null) {
|
||||
assert nextPos < positions.length;
|
||||
assert nextPos < positions.length : "nextPosition() called more than freq() times!";
|
||||
return positions[nextPos++];
|
||||
} else {
|
||||
assert nextPos < startOffsets.length;
|
||||
assert nextPos < startOffsets.length : "nextPosition() called more than freq() times!";
|
||||
nextPos++;
|
||||
return -1;
|
||||
}
|
||||
|
|
|
@ -21,6 +21,10 @@ import org.apache.lucene.codecs.Codec;
|
|||
import org.apache.lucene.index.BaseTermVectorsFormatTestCase;
|
||||
|
||||
public class TestSimpleTextTermVectorsFormat extends BaseTermVectorsFormatTestCase {
|
||||
@Override
|
||||
protected Class<? extends Throwable> getReadPastLastPositionExceptionClass() {
|
||||
return AssertionError.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Codec getCodec() {
|
||||
|
|
|
@ -94,7 +94,7 @@ public abstract class Analyzer implements Closeable {
|
|||
* Create a new Analyzer, reusing the same set of components per-thread
|
||||
* across calls to {@link #tokenStream(String, Reader)}.
|
||||
*/
|
||||
public Analyzer() {
|
||||
protected Analyzer() {
|
||||
this(GLOBAL_REUSE_STRATEGY);
|
||||
}
|
||||
|
||||
|
@ -104,9 +104,9 @@ public abstract class Analyzer implements Closeable {
|
|||
* NOTE: if you just want to reuse on a per-field basis, it's easier to
|
||||
* use a subclass of {@link AnalyzerWrapper} such as
|
||||
* <a href="{@docRoot}/../analysis/common/org/apache/lucene/analysis/miscellaneous/PerFieldAnalyzerWrapper.html">
|
||||
* PerFieldAnalyerWrapper</a> instead.
|
||||
* PerFieldAnalyzerWrapper</a> instead.
|
||||
*/
|
||||
public Analyzer(ReuseStrategy reuseStrategy) {
|
||||
protected Analyzer(ReuseStrategy reuseStrategy) {
|
||||
this.reuseStrategy = reuseStrategy;
|
||||
}
|
||||
|
||||
|
@ -440,9 +440,9 @@ public abstract class Analyzer implements Closeable {
|
|||
* {@link Analyzer#tokenStream(String, java.io.Reader)}.
|
||||
*/
|
||||
public static abstract class ReuseStrategy {
|
||||
|
||||
/** Sole constructor. (For invocation by subclass constructors, typically implicit.) */
|
||||
public ReuseStrategy() {}
|
||||
// Explicitly declared so that we have non-empty javadoc
|
||||
protected ReuseStrategy() {}
|
||||
|
||||
/**
|
||||
* Gets the reusable TokenStreamComponents for the field with the given name.
|
||||
|
|
|
@ -28,12 +28,10 @@ import org.apache.lucene.store.IOContext;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class CompoundFormat {
|
||||
/** Sole constructor. (For invocation by subclass constructors, typically implicit.) */
|
||||
// Explicitly declared so that we have non-empty javadoc
|
||||
protected CompoundFormat() {}
|
||||
|
||||
/** Sole constructor. (For invocation by subclass
|
||||
* constructors, typically implicit.) */
|
||||
public CompoundFormat() {
|
||||
}
|
||||
|
||||
// TODO: this is very minimal. If we need more methods,
|
||||
// we can add 'producer' classes.
|
||||
|
||||
|
|
|
@ -709,7 +709,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
|
|||
|
||||
PendingTerm term = (PendingTerm) ent;
|
||||
|
||||
assert StringHelper.startsWith(term.termBytes, prefix): "term.term=" + term.termBytes + " prefix=" + prefix;
|
||||
assert StringHelper.startsWith(term.termBytes, prefix): term + " prefix=" + prefix;
|
||||
BlockTermState state = term.state;
|
||||
final int suffix = term.termBytes.length - prefixLength;
|
||||
//if (DEBUG2) {
|
||||
|
@ -741,7 +741,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
|
|||
if (ent.isTerm) {
|
||||
PendingTerm term = (PendingTerm) ent;
|
||||
|
||||
assert StringHelper.startsWith(term.termBytes, prefix): "term.term=" + term.termBytes + " prefix=" + prefix;
|
||||
assert StringHelper.startsWith(term.termBytes, prefix): term + " prefix=" + prefix;
|
||||
BlockTermState state = term.state;
|
||||
final int suffix = term.termBytes.length - prefixLength;
|
||||
//if (DEBUG2) {
|
||||
|
|
|
@ -225,7 +225,7 @@ final class IndexedDISI extends DocIdSetIterator {
|
|||
buffer.set(DocIdSetIterator.NO_MORE_DOCS & 0xFFFF);
|
||||
flush(DocIdSetIterator.NO_MORE_DOCS >>> 16, buffer, 1, denseRankPower, out);
|
||||
// offset+index jump-table stored at the end
|
||||
return flushBlockJumps(jumps, lastBlock+1, out, origo);
|
||||
return flushBlockJumps(jumps, lastBlock+1, out);
|
||||
}
|
||||
|
||||
// Adds entries to the offset & index jump-table for blocks
|
||||
|
@ -241,7 +241,7 @@ final class IndexedDISI extends DocIdSetIterator {
|
|||
|
||||
// Flushes the offset & index jump-table for blocks. This should be the last data written to out
|
||||
// This method returns the blockCount for the blocks reachable for the jump_table or -1 for no jump-table
|
||||
private static short flushBlockJumps(int[] jumps, int blockCount, IndexOutput out, long origo) throws IOException {
|
||||
private static short flushBlockJumps(int[] jumps, int blockCount, IndexOutput out) throws IOException {
|
||||
if (blockCount == 2) { // Jumps with a single real entry + NO_MORE_DOCS is just wasted space so we ignore that
|
||||
blockCount = 0;
|
||||
}
|
||||
|
|
|
@ -760,7 +760,7 @@ final class Lucene80DocValuesProducer extends DocValuesProducer implements Close
|
|||
}
|
||||
|
||||
// Decompresses blocks of binary values to retrieve content
|
||||
class BinaryDecoder {
|
||||
static class BinaryDecoder {
|
||||
|
||||
private final LongValues addresses;
|
||||
private final IndexInput compressedData;
|
||||
|
|
|
@ -74,7 +74,7 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
|
|||
|
||||
|
||||
/** Sole constructor. */
|
||||
public PerFieldDocValuesFormat() {
|
||||
protected PerFieldDocValuesFormat() {
|
||||
super(PER_FIELD_NAME);
|
||||
}
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.apache.lucene.index.Terms;
|
|||
* <p>
|
||||
* Warning: the input {@linkplain MergeState} instance will be updated when calling {@link #apply(Collection)}.
|
||||
* <p>
|
||||
* It should be called within a {@code try {...} finally {...}} block to make sure that the mergeState instance is
|
||||
* It should be called within a {@code try {...} finally {...}} block to make sure that the mergeState instance is
|
||||
* restored to its original state:
|
||||
* <pre>
|
||||
* PerFieldMergeState pfMergeState = new PerFieldMergeState(mergeState);
|
||||
|
|
|
@ -80,7 +80,7 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
|
|||
public static final String PER_FIELD_SUFFIX_KEY = PerFieldPostingsFormat.class.getSimpleName() + ".suffix";
|
||||
|
||||
/** Sole constructor. */
|
||||
public PerFieldPostingsFormat() {
|
||||
protected PerFieldPostingsFormat() {
|
||||
super(PER_FIELD_NAME);
|
||||
}
|
||||
|
||||
|
|
|
@ -39,7 +39,7 @@ public abstract class DocIDMerger<T extends DocIDMerger.Sub> {
|
|||
final MergeState.DocMap docMap;
|
||||
|
||||
/** Sole constructor */
|
||||
public Sub(MergeState.DocMap docMap) {
|
||||
protected Sub(MergeState.DocMap docMap) {
|
||||
this.docMap = docMap;
|
||||
}
|
||||
|
||||
|
|
|
@ -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.
|
||||
final int docIDUpto;
|
||||
final int docIDUpTo;
|
||||
final boolean hasValue;
|
||||
|
||||
/**
|
||||
|
@ -54,19 +54,19 @@ abstract class DocValuesUpdate {
|
|||
* @param term the {@link Term} which determines the documents that will be updated
|
||||
* @param field the {@link NumericDocValuesField} to update
|
||||
*/
|
||||
protected DocValuesUpdate(DocValuesType type, Term term, String field, int docIDUpto, boolean hasValue) {
|
||||
assert docIDUpto >= 0 : docIDUpto + "must be >= 0";
|
||||
protected DocValuesUpdate(DocValuesType type, Term term, String field, int docIDUpTo, boolean hasValue) {
|
||||
assert docIDUpTo >= 0 : docIDUpTo + "must be >= 0";
|
||||
this.type = type;
|
||||
this.term = term;
|
||||
this.field = field;
|
||||
this.docIDUpto = docIDUpto;
|
||||
this.docIDUpTo = docIDUpTo;
|
||||
this.hasValue = hasValue;
|
||||
}
|
||||
|
||||
abstract long valueSizeInBytes();
|
||||
|
||||
final int sizeInBytes() {
|
||||
int sizeInBytes = RAW_SIZE_IN_BYTES;
|
||||
final long sizeInBytes() {
|
||||
long sizeInBytes = RAW_SIZE_IN_BYTES;
|
||||
sizeInBytes += term.field.length() * Character.BYTES;
|
||||
sizeInBytes += term.bytes.bytes.length;
|
||||
sizeInBytes += field.length() * Character.BYTES;
|
||||
|
@ -85,7 +85,7 @@ abstract class DocValuesUpdate {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "term=" + term + ",field=" + field + ",value=" + valueToString() + ",docIDUpto=" + docIDUpto;
|
||||
return "term=" + term + ",field=" + field + ",value=" + valueToString() + ",docIDUpTo=" + docIDUpTo;
|
||||
}
|
||||
|
||||
/** An in-place update to a binary DocValues field */
|
||||
|
@ -93,7 +93,7 @@ abstract class DocValuesUpdate {
|
|||
private final BytesRef value;
|
||||
|
||||
/* Size of BytesRef: 2*INT + ARRAY_HEADER + PTR */
|
||||
private static final long RAW_VALUE_SIZE_IN_BYTES = NUM_BYTES_ARRAY_HEADER + 2*Integer.BYTES + NUM_BYTES_OBJECT_REF;
|
||||
private static final long RAW_VALUE_SIZE_IN_BYTES = 2L * Integer.BYTES + NUM_BYTES_ARRAY_HEADER + NUM_BYTES_OBJECT_REF;
|
||||
|
||||
BinaryDocValuesUpdate(Term term, String field, BytesRef value) {
|
||||
this(term, field, value, BufferedUpdates.MAX_INT);
|
||||
|
@ -104,11 +104,11 @@ abstract class DocValuesUpdate {
|
|||
this.value = value;
|
||||
}
|
||||
|
||||
BinaryDocValuesUpdate prepareForApply(int docIDUpto) {
|
||||
if (docIDUpto == this.docIDUpto) {
|
||||
BinaryDocValuesUpdate prepareForApply(int docIDUpTo) {
|
||||
if (docIDUpTo == this.docIDUpTo) {
|
||||
return this; // it's a final value so we can safely reuse this instance
|
||||
}
|
||||
return new BinaryDocValuesUpdate(term, field, value, docIDUpto);
|
||||
return new BinaryDocValuesUpdate(term, field, value, docIDUpTo);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -152,7 +152,7 @@ abstract class DocValuesUpdate {
|
|||
}
|
||||
|
||||
NumericDocValuesUpdate(Term term, String field, Long value) {
|
||||
this(term, field, value != null ? value.longValue() : -1, BufferedUpdates.MAX_INT, value != null);
|
||||
this(term, field, value != null ? value : -1, BufferedUpdates.MAX_INT, value != null);
|
||||
}
|
||||
|
||||
|
||||
|
@ -161,11 +161,11 @@ abstract class DocValuesUpdate {
|
|||
this.value = value;
|
||||
}
|
||||
|
||||
NumericDocValuesUpdate prepareForApply(int docIDUpto) {
|
||||
if (docIDUpto == this.docIDUpto) {
|
||||
NumericDocValuesUpdate prepareForApply(int docIDUpTo) {
|
||||
if (docIDUpTo == this.docIDUpTo) {
|
||||
return this;
|
||||
}
|
||||
return new NumericDocValuesUpdate(term, field, value, docIDUpto, hasValue);
|
||||
return new NumericDocValuesUpdate(term, field, value, docIDUpTo, hasValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -59,7 +59,7 @@ public abstract class FilterLeafReader extends LeafReader {
|
|||
* Creates a new FilterFields.
|
||||
* @param in the underlying Fields instance.
|
||||
*/
|
||||
public FilterFields(Fields in) {
|
||||
protected FilterFields(Fields in) {
|
||||
if (in == null) {
|
||||
throw new NullPointerException("incoming Fields must not be null");
|
||||
}
|
||||
|
@ -95,7 +95,7 @@ public abstract class FilterLeafReader extends LeafReader {
|
|||
* Creates a new FilterTerms
|
||||
* @param in the underlying Terms instance.
|
||||
*/
|
||||
public FilterTerms(Terms in) {
|
||||
protected FilterTerms(Terms in) {
|
||||
if (in == null) {
|
||||
throw new NullPointerException("incoming Terms must not be null");
|
||||
}
|
||||
|
@ -162,7 +162,7 @@ public abstract class FilterLeafReader extends LeafReader {
|
|||
* Creates a new FilterTermsEnum
|
||||
* @param in the underlying TermsEnum instance.
|
||||
*/
|
||||
public FilterTermsEnum(TermsEnum in) {
|
||||
protected FilterTermsEnum(TermsEnum in) {
|
||||
if (in == null) {
|
||||
throw new NullPointerException("incoming TermsEnum must not be null");
|
||||
}
|
||||
|
@ -244,7 +244,7 @@ public abstract class FilterLeafReader extends LeafReader {
|
|||
* Create a new FilterPostingsEnum
|
||||
* @param in the underlying PostingsEnum instance.
|
||||
*/
|
||||
public FilterPostingsEnum(PostingsEnum in) {
|
||||
protected FilterPostingsEnum(PostingsEnum in) {
|
||||
if (in == null) {
|
||||
throw new NullPointerException("incoming PostingsEnum must not be null");
|
||||
}
|
||||
|
@ -305,7 +305,7 @@ public abstract class FilterLeafReader extends LeafReader {
|
|||
* <p>Note that base reader is closed if this FilterLeafReader is closed.</p>
|
||||
* @param in specified base reader.
|
||||
*/
|
||||
public FilterLeafReader(LeafReader in) {
|
||||
protected FilterLeafReader(LeafReader in) {
|
||||
super();
|
||||
if (in == null) {
|
||||
throw new NullPointerException("incoming LeafReader must not be null");
|
||||
|
|
|
@ -73,7 +73,7 @@ public abstract class FilteredTermsEnum extends TermsEnum {
|
|||
* Creates a filtered {@link TermsEnum} on a terms enum.
|
||||
* @param tenum the terms enumeration to filter.
|
||||
*/
|
||||
public FilteredTermsEnum(final TermsEnum tenum) {
|
||||
protected FilteredTermsEnum(final TermsEnum tenum) {
|
||||
this(tenum, true);
|
||||
}
|
||||
|
||||
|
@ -81,7 +81,7 @@ public abstract class FilteredTermsEnum extends TermsEnum {
|
|||
* Creates a filtered {@link TermsEnum} on a terms enum.
|
||||
* @param tenum the terms enumeration to filter.
|
||||
*/
|
||||
public FilteredTermsEnum(final TermsEnum tenum, final boolean startWithSeek) {
|
||||
protected FilteredTermsEnum(final TermsEnum tenum, final boolean startWithSeek) {
|
||||
assert tenum != null;
|
||||
this.tenum = tenum;
|
||||
doSeek = startWithSeek;
|
||||
|
|
|
@ -40,7 +40,7 @@ import org.apache.lucene.util.Constants;
|
|||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
/*
|
||||
/**
|
||||
* This class keeps track of each SegmentInfos instance that
|
||||
* is still "live", either because it corresponds to a
|
||||
* segments_N file in the Directory (a "commit", i.e. a
|
||||
|
@ -76,7 +76,6 @@ import org.apache.lucene.util.InfoStream;
|
|||
* instantiating this class. It opens segments_N file(s)
|
||||
* directly with no retry logic.
|
||||
*/
|
||||
|
||||
final class IndexFileDeleter implements Closeable {
|
||||
|
||||
/* Reference count for all files in the index.
|
||||
|
|
|
@ -4473,7 +4473,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
|
|||
final int delCount = numDeletedDocs(info);
|
||||
assert delCount <= info.info.maxDoc();
|
||||
final double delRatio = ((double) delCount)/info.info.maxDoc();
|
||||
merge.estimatedMergeBytes += info.sizeInBytes() * (1.0 - delRatio);
|
||||
merge.estimatedMergeBytes += (long) (info.sizeInBytes() * (1.0 - delRatio));
|
||||
merge.totalMergeBytes += info.sizeInBytes();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -558,7 +558,7 @@ public abstract class MergePolicy {
|
|||
/**
|
||||
* Creates a new merge policy instance.
|
||||
*/
|
||||
public MergePolicy() {
|
||||
protected MergePolicy() {
|
||||
this(DEFAULT_NO_CFS_RATIO, DEFAULT_MAX_CFS_SEGMENT_SIZE);
|
||||
}
|
||||
|
||||
|
|
|
@ -241,9 +241,9 @@ public class MergeState {
|
|||
|
||||
/** A map of doc IDs. */
|
||||
public static abstract class DocMap {
|
||||
/** Sole constructor */
|
||||
public DocMap() {
|
||||
}
|
||||
/** Sole constructor. (For invocation by subclass constructors, typically implicit.) */
|
||||
// Explicitly declared so that we have non-empty javadoc
|
||||
protected DocMap() {}
|
||||
|
||||
/** Return the mapped docID or -1 if the given doc is not mapped. */
|
||||
public abstract int get(int docID);
|
||||
|
|
|
@ -492,7 +492,7 @@ final class ReadersAndUpdates {
|
|||
// we write approximately that many bytes (based on Lucene46DVF):
|
||||
// HEADER + FOOTER: 40
|
||||
// 90 bytes per-field (over estimating long name and attributes map)
|
||||
final long estInfosSize = 40 + 90 * fieldInfos.size();
|
||||
final long estInfosSize = 40 + 90L * fieldInfos.size();
|
||||
final IOContext infosContext = new IOContext(new FlushInfo(info.info.maxDoc(), estInfosSize));
|
||||
// separately also track which files were created for this gen
|
||||
final TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
|
||||
|
|
|
@ -166,7 +166,6 @@ final class SegmentCoreReaders {
|
|||
@SuppressWarnings("try")
|
||||
void decRef() throws IOException {
|
||||
if (ref.decrementAndGet() == 0) {
|
||||
Throwable th = null;
|
||||
try (Closeable finalizer = this::notifyCoreClosedListeners){
|
||||
IOUtils.close(termVectorsLocal, fieldsReaderLocal, fields, termVectorsReaderOrig, fieldsReaderOrig,
|
||||
cfsReader, normsProducer, pointsReader);
|
||||
|
|
|
@ -668,8 +668,8 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
|||
|
||||
final Directory directory;
|
||||
|
||||
/** Sole constructor. */
|
||||
public FindSegmentsFile(Directory directory) {
|
||||
/** Sole constructor. */
|
||||
protected FindSegmentsFile(Directory directory) {
|
||||
this.directory = directory;
|
||||
}
|
||||
|
||||
|
|
|
@ -145,7 +145,7 @@ public abstract class FieldComparator<T> {
|
|||
protected final String field;
|
||||
protected NumericDocValues currentReaderValues;
|
||||
|
||||
public NumericComparator(String field, T missingValue) {
|
||||
protected NumericComparator(String field, T missingValue) {
|
||||
this.field = field;
|
||||
this.missingValue = missingValue;
|
||||
}
|
||||
|
|
|
@ -697,7 +697,7 @@ public class IndexSearcher {
|
|||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
return collectorManager.reduce(collectors);
|
||||
return collectorManager.reduce(collectedCollectors);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -90,13 +90,10 @@ import org.apache.lucene.util.SmallFloat;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class Similarity {
|
||||
|
||||
/**
|
||||
* Sole constructor. (For invocation by subclass
|
||||
* constructors, typically implicit.)
|
||||
*/
|
||||
public Similarity() {}
|
||||
|
||||
/** Sole constructor. (For invocation by subclass constructors, typically implicit.) */
|
||||
// Explicitly declared so that we have non-empty javadoc
|
||||
protected Similarity() {}
|
||||
|
||||
/**
|
||||
* Computes the normalization value for a field, given the accumulated
|
||||
* state of term processing for this field (see {@link FieldInvertState}).
|
||||
|
@ -108,8 +105,8 @@ public abstract class Similarity {
|
|||
* <p>Note that for a given term-document frequency, greater unsigned norms
|
||||
* must produce scores that are lower or equal, ie. for two encoded norms
|
||||
* {@code n1} and {@code n2} so that
|
||||
* {@code Long.compareUnsigned(n1, n2) > 0} then
|
||||
* {@code SimScorer.score(freq, n1) <= SimScorer.score(freq, n2)}
|
||||
* {@code Long.compareUnsigned(n1, n2) > 0} then
|
||||
* {@code SimScorer.score(freq, n1) <= SimScorer.score(freq, n2)}
|
||||
* for any legal {@code freq}.
|
||||
*
|
||||
* <p>{@code 0} is not a legal norm, so {@code 1} is the norm that produces
|
||||
|
@ -154,13 +151,13 @@ public abstract class Similarity {
|
|||
* {@code 1} if norms are disabled. {@code norm} is never {@code 0}.
|
||||
* <p>
|
||||
* Score must not decrease when {@code freq} increases, ie. if
|
||||
* {@code freq1 > freq2}, then {@code score(freq1, norm) >=
|
||||
* {@code freq1 > freq2}, then {@code score(freq1, norm) >=
|
||||
* score(freq2, norm)} for any value of {@code norm} that may be produced
|
||||
* by {@link Similarity#computeNorm(FieldInvertState)}.
|
||||
* <p>
|
||||
* Score must not increase when the unsigned {@code norm} increases, ie. if
|
||||
* {@code Long.compareUnsigned(norm1, norm2) > 0} then
|
||||
* {@code score(freq, norm1) <= score(freq, norm2)} for any legal
|
||||
* {@code Long.compareUnsigned(norm1, norm2) > 0} then
|
||||
* {@code score(freq, norm1) <= score(freq, norm2)} for any legal
|
||||
* {@code freq}.
|
||||
* <p>
|
||||
* As a consequence, the maximum score that this scorer can produce is bound
|
||||
|
|
|
@ -101,7 +101,7 @@ public abstract class AttributeFactory {
|
|||
|
||||
/** <b>Expert</b>: Creates an AttributeFactory returning {@code clazz} as instance for the
|
||||
* attributes it implements and for all other attributes calls the given delegate factory. */
|
||||
public StaticImplementationAttributeFactory(AttributeFactory delegate, Class<A> clazz) {
|
||||
protected StaticImplementationAttributeFactory(AttributeFactory delegate, Class<A> clazz) {
|
||||
this.delegate = delegate;
|
||||
this.clazz = clazz;
|
||||
}
|
||||
|
|
|
@ -53,7 +53,7 @@ public final class ByteBlockPool implements Accountable {
|
|||
public abstract static class Allocator {
|
||||
protected final int blockSize;
|
||||
|
||||
public Allocator(int blockSize) {
|
||||
protected Allocator(int blockSize) {
|
||||
this.blockSize = blockSize;
|
||||
}
|
||||
|
||||
|
|
|
@ -391,7 +391,7 @@ public final class IOUtils {
|
|||
*
|
||||
* @param th The throwable to rethrow, <strong>must not be null</strong>.
|
||||
* @return This method always results in an exception, it never returns any value.
|
||||
* See method documentation for detailsa and usage example.
|
||||
* See method documentation for details and usage example.
|
||||
* @throws IOException if the argument was an instance of IOException
|
||||
* @throws RuntimeException with the {@link RuntimeException#getCause()} set
|
||||
* to the argument, if it was not an instance of IOException.
|
||||
|
|
|
@ -33,7 +33,7 @@ public final class IntBlockPool {
|
|||
public abstract static class Allocator {
|
||||
protected final int blockSize;
|
||||
|
||||
public Allocator(int blockSize) {
|
||||
protected Allocator(int blockSize) {
|
||||
this.blockSize = blockSize;
|
||||
}
|
||||
|
||||
|
|
|
@ -72,7 +72,7 @@ public final class LongsRef implements Comparable<LongsRef>, Cloneable {
|
|||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 0;
|
||||
final long end = offset + length;
|
||||
final long end = (long) offset + length;
|
||||
for(int i = offset; i < end; i++) {
|
||||
result = prime * result + (int) (longs[i] ^ (longs[i]>>>32));
|
||||
}
|
||||
|
@ -106,7 +106,7 @@ public final class LongsRef implements Comparable<LongsRef>, Cloneable {
|
|||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append('[');
|
||||
final long end = offset + length;
|
||||
final long end = (long) offset + length;
|
||||
for(int i=offset;i<end;i++) {
|
||||
if (i > offset) {
|
||||
sb.append(' ');
|
||||
|
|
|
@ -195,7 +195,7 @@ public final class RamUsageEstimator {
|
|||
|
||||
/** Approximate memory usage that we assign to a Hashtable / HashMap entry. */
|
||||
public static final long HASHTABLE_RAM_BYTES_PER_ENTRY =
|
||||
2 * NUM_BYTES_OBJECT_REF // key + value
|
||||
2L * NUM_BYTES_OBJECT_REF // key + value
|
||||
* 2; // hash tables need to be oversized to avoid collisions, assume 2x capacity
|
||||
|
||||
/** Approximate memory usage that we assign to a LinkedHashMap entry. */
|
||||
|
|
|
@ -194,7 +194,7 @@ public final class LZ4 {
|
|||
abstract void initDictionary(int dictLen);
|
||||
|
||||
/**
|
||||
* Advance the cursor to {@off} and return an index that stored the same
|
||||
* Advance the cursor to {@code off} and return an index that stored the same
|
||||
* 4 bytes as {@code b[o:o+4)}. This may only be called on strictly
|
||||
* increasing sequences of offsets. A return value of {@code -1} indicates
|
||||
* that no other index could be found. */
|
||||
|
|
|
@ -52,7 +52,7 @@ abstract class AbstractBlockPackedWriter {
|
|||
* Sole constructor.
|
||||
* @param blockSize the number of values of a single block, must be a multiple of <code>64</code>
|
||||
*/
|
||||
public AbstractBlockPackedWriter(DataOutput out, int blockSize) {
|
||||
protected AbstractBlockPackedWriter(DataOutput out, int blockSize) {
|
||||
checkBlockSize(blockSize, MIN_BLOCK_SIZE, MAX_BLOCK_SIZE);
|
||||
reset(out);
|
||||
values = new long[blockSize];
|
||||
|
|
|
@ -145,7 +145,7 @@ public class PackedInts {
|
|||
this.id = id;
|
||||
}
|
||||
|
||||
public int id;
|
||||
public final int id;
|
||||
|
||||
/**
|
||||
* Returns the ID of the format.
|
||||
|
|
|
@ -185,14 +185,14 @@ public class TestFieldUpdatesBuffer extends LuceneTestCase {
|
|||
Counter counter = Counter.newCounter();
|
||||
DocValuesUpdate.BinaryDocValuesUpdate randomUpdate = getRandomBinaryUpdate();
|
||||
updates.add(randomUpdate);
|
||||
FieldUpdatesBuffer buffer = new FieldUpdatesBuffer(counter, randomUpdate, randomUpdate.docIDUpto);
|
||||
FieldUpdatesBuffer buffer = new FieldUpdatesBuffer(counter, randomUpdate, randomUpdate.docIDUpTo);
|
||||
for (int i = 0; i < numUpdates; i++) {
|
||||
randomUpdate = getRandomBinaryUpdate();
|
||||
updates.add(randomUpdate);
|
||||
if (randomUpdate.hasValue) {
|
||||
buffer.addUpdate(randomUpdate.term, randomUpdate.getValue(), randomUpdate.docIDUpto);
|
||||
buffer.addUpdate(randomUpdate.term, randomUpdate.getValue(), randomUpdate.docIDUpTo);
|
||||
} else {
|
||||
buffer.addNoValue(randomUpdate.term, randomUpdate.docIDUpto);
|
||||
buffer.addNoValue(randomUpdate.term, randomUpdate.docIDUpTo);
|
||||
}
|
||||
}
|
||||
buffer.finish();
|
||||
|
@ -210,7 +210,7 @@ public class TestFieldUpdatesBuffer extends LuceneTestCase {
|
|||
} else {
|
||||
assertNull(value.binaryValue);
|
||||
}
|
||||
assertEquals(randomUpdate.docIDUpto, value.docUpTo);
|
||||
assertEquals(randomUpdate.docIDUpTo, value.docUpTo);
|
||||
}
|
||||
assertEquals(count, updates.size());
|
||||
}
|
||||
|
@ -221,14 +221,14 @@ public class TestFieldUpdatesBuffer extends LuceneTestCase {
|
|||
Counter counter = Counter.newCounter();
|
||||
DocValuesUpdate.NumericDocValuesUpdate randomUpdate = getRandomNumericUpdate();
|
||||
updates.add(randomUpdate);
|
||||
FieldUpdatesBuffer buffer = new FieldUpdatesBuffer(counter, randomUpdate, randomUpdate.docIDUpto);
|
||||
FieldUpdatesBuffer buffer = new FieldUpdatesBuffer(counter, randomUpdate, randomUpdate.docIDUpTo);
|
||||
for (int i = 0; i < numUpdates; i++) {
|
||||
randomUpdate = getRandomNumericUpdate();
|
||||
updates.add(randomUpdate);
|
||||
if (randomUpdate.hasValue) {
|
||||
buffer.addUpdate(randomUpdate.term, randomUpdate.getValue(), randomUpdate.docIDUpto);
|
||||
buffer.addUpdate(randomUpdate.term, randomUpdate.getValue(), randomUpdate.docIDUpTo);
|
||||
} else {
|
||||
buffer.addNoValue(randomUpdate.term, randomUpdate.docIDUpto);
|
||||
buffer.addNoValue(randomUpdate.term, randomUpdate.docIDUpTo);
|
||||
}
|
||||
}
|
||||
buffer.finish();
|
||||
|
@ -242,7 +242,7 @@ public class TestFieldUpdatesBuffer extends LuceneTestCase {
|
|||
public void testNoNumericValue() {
|
||||
DocValuesUpdate.NumericDocValuesUpdate update =
|
||||
new DocValuesUpdate.NumericDocValuesUpdate(new Term("id", "1"), "age", null);
|
||||
FieldUpdatesBuffer buffer = new FieldUpdatesBuffer(Counter.newCounter(), update, update.docIDUpto);
|
||||
FieldUpdatesBuffer buffer = new FieldUpdatesBuffer(Counter.newCounter(), update, update.docIDUpTo);
|
||||
assertEquals(0, buffer.getMinNumeric());
|
||||
assertEquals(0, buffer.getMaxNumeric());
|
||||
}
|
||||
|
@ -257,13 +257,13 @@ public class TestFieldUpdatesBuffer extends LuceneTestCase {
|
|||
new Term(termField, Integer.toString(random().nextInt(1000))), "numeric", docValue);
|
||||
randomUpdate = randomUpdate.prepareForApply(randomDocUpTo());
|
||||
updates.add(randomUpdate);
|
||||
FieldUpdatesBuffer buffer = new FieldUpdatesBuffer(counter, randomUpdate, randomUpdate.docIDUpto);
|
||||
FieldUpdatesBuffer buffer = new FieldUpdatesBuffer(counter, randomUpdate, randomUpdate.docIDUpTo);
|
||||
for (int i = 0; i < numUpdates; i++) {
|
||||
randomUpdate = new DocValuesUpdate.NumericDocValuesUpdate(
|
||||
new Term(termField, Integer.toString(random().nextInt(1000))), "numeric", docValue);
|
||||
randomUpdate = randomUpdate.prepareForApply(randomDocUpTo());
|
||||
updates.add(randomUpdate);
|
||||
buffer.addUpdate(randomUpdate.term, randomUpdate.getValue(), randomUpdate.docIDUpto);
|
||||
buffer.addUpdate(randomUpdate.term, randomUpdate.getValue(), randomUpdate.docIDUpTo);
|
||||
}
|
||||
buffer.finish();
|
||||
assertBufferUpdates(buffer, updates, true);
|
||||
|
@ -276,7 +276,7 @@ public class TestFieldUpdatesBuffer extends LuceneTestCase {
|
|||
updates.sort(Comparator.comparing(u -> u.term.bytes));
|
||||
SortedMap<BytesRef, DocValuesUpdate.NumericDocValuesUpdate> byTerms = new TreeMap<>();
|
||||
for (DocValuesUpdate.NumericDocValuesUpdate update : updates) {
|
||||
byTerms.compute(update.term.bytes, (k, v) -> v != null && v.docIDUpto >= update.docIDUpto ? v : update);
|
||||
byTerms.compute(update.term.bytes, (k, v) -> v != null && v.docIDUpTo >= update.docIDUpTo ? v : update);
|
||||
}
|
||||
updates = new ArrayList<>(byTerms.values());
|
||||
}
|
||||
|
@ -304,7 +304,7 @@ public class TestFieldUpdatesBuffer extends LuceneTestCase {
|
|||
assertEquals(0, value.numericValue);
|
||||
assertEquals(0, v);
|
||||
}
|
||||
assertEquals(expectedUpdate.docIDUpto, value.docUpTo);
|
||||
assertEquals(expectedUpdate.docIDUpTo, value.docUpTo);
|
||||
}
|
||||
if (hasAtLeastOneValue) {
|
||||
assertEquals(max, buffer.getMaxNumeric());
|
||||
|
|
|
@ -582,7 +582,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
|
|||
assert ent.isTerm: "i=" + i;
|
||||
|
||||
PendingTerm term = (PendingTerm) ent;
|
||||
assert StringHelper.startsWith(term.termBytes, prefix): "term.term=" + term.termBytes + " prefix=" + prefix;
|
||||
assert StringHelper.startsWith(term.termBytes, prefix): term + " prefix=" + prefix;
|
||||
BlockTermState state = term.state;
|
||||
maxVersionInBlock = Math.max(maxVersionInBlock, ((IDVersionTermState) state).idVersion);
|
||||
final int suffix = term.termBytes.length - prefixLength;
|
||||
|
@ -610,7 +610,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
|
|||
PendingEntry ent = pending.get(i);
|
||||
if (ent.isTerm) {
|
||||
PendingTerm term = (PendingTerm) ent;
|
||||
assert StringHelper.startsWith(term.termBytes, prefix): "term.term=" + term.termBytes + " prefix=" + prefix;
|
||||
assert StringHelper.startsWith(term.termBytes, prefix): term + " prefix=" + prefix;
|
||||
BlockTermState state = term.state;
|
||||
maxVersionInBlock = Math.max(maxVersionInBlock, ((IDVersionTermState) state).idVersion);
|
||||
final int suffix = term.termBytes.length - prefixLength;
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
package org.apache.lucene.spatial3d.geom;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
|
||||
|
@ -317,7 +318,7 @@ class GeoExactCircle extends GeoBaseCircle {
|
|||
@Override
|
||||
public String toString() {
|
||||
return "{circle plane = " + circlePlane + " plane 1 = "+plane1 +
|
||||
" plane 2 = " + plane2 + " notable edge points = " + notableEdgePoints + "}";
|
||||
" plane 2 = " + plane2 + " notable edge points = " + Arrays.toString(notableEdgePoints) + "}";
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import static org.apache.lucene.index.PostingsEnum.POSITIONS;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.EnumSet;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
|
@ -397,13 +398,12 @@ public abstract class BaseTermVectorsFormatTestCase extends BaseIndexFileFormatT
|
|||
|
||||
protected void assertEquals(RandomDocument doc, Fields fields) throws IOException {
|
||||
// compare field names
|
||||
assertEquals(doc == null, fields == null);
|
||||
assertNotNull(doc);
|
||||
assertNotNull(fields);
|
||||
assertEquals(doc.fieldNames.length, fields.size());
|
||||
final Set<String> fields1 = new HashSet<>();
|
||||
final Set<String> fields2 = new HashSet<>();
|
||||
for (int i = 0; i < doc.fieldNames.length; ++i) {
|
||||
fields1.add(doc.fieldNames[i]);
|
||||
}
|
||||
Collections.addAll(fields1, doc.fieldNames);
|
||||
for (String field : fields) {
|
||||
fields2.add(field);
|
||||
}
|
||||
|
@ -462,55 +462,48 @@ public abstract class BaseTermVectorsFormatTestCase extends BaseIndexFileFormatT
|
|||
assertEquals(0, docsAndPositionsEnum.nextDoc());
|
||||
final int freq = docsAndPositionsEnum.freq();
|
||||
assertEquals(tk.freqs.get(termsEnum.term().utf8ToString()), (Integer) freq);
|
||||
if (docsAndPositionsEnum != null) {
|
||||
for (int k = 0; k < freq; ++k) {
|
||||
final int position = docsAndPositionsEnum.nextPosition();
|
||||
final Set<Integer> indexes;
|
||||
if (terms.hasPositions()) {
|
||||
indexes = tk.positionToTerms.get(position);
|
||||
assertNotNull(indexes);
|
||||
} else {
|
||||
indexes = tk.startOffsetToTerms.get(docsAndPositionsEnum.startOffset());
|
||||
assertNotNull(indexes);
|
||||
}
|
||||
if (terms.hasPositions()) {
|
||||
boolean foundPosition = false;
|
||||
for (int index : indexes) {
|
||||
if (tk.termBytes[index].equals(termsEnum.term()) && tk.positions[index] == position) {
|
||||
foundPosition = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertTrue(foundPosition);
|
||||
}
|
||||
if (terms.hasOffsets()) {
|
||||
boolean foundOffset = false;
|
||||
for (int index : indexes) {
|
||||
if (tk.termBytes[index].equals(termsEnum.term()) && tk.startOffsets[index] == docsAndPositionsEnum.startOffset() && tk.endOffsets[index] == docsAndPositionsEnum.endOffset()) {
|
||||
foundOffset = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertTrue(foundOffset);
|
||||
}
|
||||
if (terms.hasPayloads()) {
|
||||
boolean foundPayload = false;
|
||||
for (int index : indexes) {
|
||||
if (tk.termBytes[index].equals(termsEnum.term()) && equals(tk.payloads[index], docsAndPositionsEnum.getPayload())) {
|
||||
foundPayload = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertTrue(foundPayload);
|
||||
}
|
||||
for (int k = 0; k < freq; ++k) {
|
||||
final int position = docsAndPositionsEnum.nextPosition();
|
||||
final Set<Integer> indexes;
|
||||
if (terms.hasPositions()) {
|
||||
indexes = tk.positionToTerms.get(position);
|
||||
assertNotNull(indexes);
|
||||
} else {
|
||||
indexes = tk.startOffsetToTerms.get(docsAndPositionsEnum.startOffset());
|
||||
assertNotNull(indexes);
|
||||
}
|
||||
try {
|
||||
docsAndPositionsEnum.nextPosition();
|
||||
fail();
|
||||
} catch (Exception | AssertionError e) {
|
||||
// ok
|
||||
if (terms.hasPositions()) {
|
||||
boolean foundPosition = false;
|
||||
for (int index : indexes) {
|
||||
if (tk.termBytes[index].equals(termsEnum.term()) && tk.positions[index] == position) {
|
||||
foundPosition = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertTrue(foundPosition);
|
||||
}
|
||||
if (terms.hasOffsets()) {
|
||||
boolean foundOffset = false;
|
||||
for (int index : indexes) {
|
||||
if (tk.termBytes[index].equals(termsEnum.term()) && tk.startOffsets[index] == docsAndPositionsEnum.startOffset() && tk.endOffsets[index] == docsAndPositionsEnum.endOffset()) {
|
||||
foundOffset = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertTrue(foundOffset);
|
||||
}
|
||||
if (terms.hasPayloads()) {
|
||||
boolean foundPayload = false;
|
||||
for (int index : indexes) {
|
||||
if (tk.termBytes[index].equals(termsEnum.term()) && equals(tk.payloads[index], docsAndPositionsEnum.getPayload())) {
|
||||
foundPayload = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertTrue(foundPayload);
|
||||
}
|
||||
}
|
||||
expectThrows(getReadPastLastPositionExceptionClass(), docsAndPositionsEnum::nextPosition);
|
||||
assertEquals(PostingsEnum.NO_MORE_DOCS, docsAndPositionsEnum.nextDoc());
|
||||
}
|
||||
this.docsEnum.set(docsAndPositionsEnum);
|
||||
|
@ -525,6 +518,10 @@ public abstract class BaseTermVectorsFormatTestCase extends BaseIndexFileFormatT
|
|||
}
|
||||
}
|
||||
|
||||
protected Class<? extends Throwable> getReadPastLastPositionExceptionClass() {
|
||||
return IllegalStateException.class;
|
||||
}
|
||||
|
||||
protected Document addId(Document doc, String id) {
|
||||
doc.add(new StringField("id", id, Store.NO));
|
||||
return doc;
|
||||
|
|
|
@ -163,7 +163,7 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
|
|||
ranges[id][0].setMax(d, ranges[oldID][0].getMax(d));
|
||||
}
|
||||
if (VERBOSE) {
|
||||
System.out.println(" id=" + id + " box=" + ranges[id] + " (same box as doc=" + oldID + ")");
|
||||
System.out.println(" id=" + id + " box=" + Arrays.toString(ranges[id]) + " (same box as doc=" + oldID + ")");
|
||||
}
|
||||
} else {
|
||||
for (int m = 0, even = dimensions % 2; m < dimensions * 2; ++m) {
|
||||
|
@ -173,12 +173,12 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
|
|||
if (even == 0) { // even is min
|
||||
ranges[id][0].setMin(d, ranges[oldID][0].getMin(d));
|
||||
if (VERBOSE) {
|
||||
System.out.println(" id=" + id + " box=" + ranges[id] + " (same min[" + d + "] as doc=" + oldID + ")");
|
||||
System.out.println(" id=" + id + " box=" + Arrays.toString(ranges[id]) + " (same min[" + d + "] as doc=" + oldID + ")");
|
||||
}
|
||||
} else { // odd is max
|
||||
ranges[id][0].setMax(d, ranges[oldID][0].getMax(d));
|
||||
if (VERBOSE) {
|
||||
System.out.println(" id=" + id + " box=" + ranges[id] + " (same max[" + d + "] as doc=" + oldID + ")");
|
||||
System.out.println(" id=" + id + " box=" + Arrays.toString(ranges[id]) + " (same max[" + d + "] as doc=" + oldID + ")");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,7 +22,12 @@ import org.apache.lucene.index.BaseTermVectorsFormatTestCase;
|
|||
/** Test AssertingTermVectorsFormat directly */
|
||||
public class TestAssertingTermVectorsFormat extends BaseTermVectorsFormatTestCase {
|
||||
private final Codec codec = new AssertingCodec();
|
||||
|
||||
|
||||
@Override
|
||||
protected Class<? extends Throwable> getReadPastLastPositionExceptionClass() {
|
||||
return AssertionError.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Codec getCodec() {
|
||||
return codec;
|
||||
|
|
Loading…
Reference in New Issue