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:
Mike Drob 2020-09-08 10:27:53 -05:00 committed by GitHub
parent 984466f31b
commit a46316e156
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
41 changed files with 150 additions and 150 deletions

View File

@ -604,7 +604,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
assert ent.isTerm: "i=" + i; assert ent.isTerm: "i=" + i;
PendingTerm term = (PendingTerm) ent; 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; BlockTermState state = term.state;
final int suffix = term.termBytes.length - prefixLength; final int suffix = term.termBytes.length - prefixLength;
/* /*
@ -640,7 +640,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
PendingEntry ent = pending.get(i); PendingEntry ent = pending.get(i);
if (ent.isTerm) { if (ent.isTerm) {
PendingTerm term = (PendingTerm) ent; 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; BlockTermState state = term.state;
final int suffix = term.termBytes.length - prefixLength; final int suffix = term.termBytes.length - prefixLength;
/* /*

View File

@ -454,7 +454,7 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
// Field not bloom'd // Field not bloom'd
break; break;
} }
assert bloomFilters.containsKey(field) == false; assert bloomFilters.containsKey(fieldInfo) == false;
bloomFilters.put(fieldInfo, bloomFilter); bloomFilters.put(fieldInfo, bloomFilter);
} }
// Make sure there's at least one doc for this term: // Make sure there's at least one doc for this term:

View File

@ -540,10 +540,10 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
@Override @Override
public int nextPosition() { public int nextPosition() {
if (positions != null) { if (positions != null) {
assert nextPos < positions.length; assert nextPos < positions.length : "nextPosition() called more than freq() times!";
return positions[nextPos++]; return positions[nextPos++];
} else { } else {
assert nextPos < startOffsets.length; assert nextPos < startOffsets.length : "nextPosition() called more than freq() times!";
nextPos++; nextPos++;
return -1; return -1;
} }

View File

@ -21,6 +21,10 @@ import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.BaseTermVectorsFormatTestCase; import org.apache.lucene.index.BaseTermVectorsFormatTestCase;
public class TestSimpleTextTermVectorsFormat extends BaseTermVectorsFormatTestCase { public class TestSimpleTextTermVectorsFormat extends BaseTermVectorsFormatTestCase {
@Override
protected Class<? extends Throwable> getReadPastLastPositionExceptionClass() {
return AssertionError.class;
}
@Override @Override
protected Codec getCodec() { protected Codec getCodec() {

View File

@ -94,7 +94,7 @@ public abstract class Analyzer implements Closeable {
* Create a new Analyzer, reusing the same set of components per-thread * Create a new Analyzer, reusing the same set of components per-thread
* across calls to {@link #tokenStream(String, Reader)}. * across calls to {@link #tokenStream(String, Reader)}.
*/ */
public Analyzer() { protected Analyzer() {
this(GLOBAL_REUSE_STRATEGY); 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 * NOTE: if you just want to reuse on a per-field basis, it's easier to
* use a subclass of {@link AnalyzerWrapper} such as * use a subclass of {@link AnalyzerWrapper} such as
* <a href="{@docRoot}/../analysis/common/org/apache/lucene/analysis/miscellaneous/PerFieldAnalyzerWrapper.html"> * <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; this.reuseStrategy = reuseStrategy;
} }
@ -440,9 +440,9 @@ public abstract class Analyzer implements Closeable {
* {@link Analyzer#tokenStream(String, java.io.Reader)}. * {@link Analyzer#tokenStream(String, java.io.Reader)}.
*/ */
public static abstract class ReuseStrategy { public static abstract class ReuseStrategy {
/** Sole constructor. (For invocation by subclass constructors, typically implicit.) */ /** 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. * Gets the reusable TokenStreamComponents for the field with the given name.

View File

@ -28,12 +28,10 @@ import org.apache.lucene.store.IOContext;
* @lucene.experimental * @lucene.experimental
*/ */
public abstract class CompoundFormat { 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, // TODO: this is very minimal. If we need more methods,
// we can add 'producer' classes. // we can add 'producer' classes.

View File

@ -709,7 +709,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
PendingTerm term = (PendingTerm) ent; 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; BlockTermState state = term.state;
final int suffix = term.termBytes.length - prefixLength; final int suffix = term.termBytes.length - prefixLength;
//if (DEBUG2) { //if (DEBUG2) {
@ -741,7 +741,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
if (ent.isTerm) { if (ent.isTerm) {
PendingTerm term = (PendingTerm) ent; 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; BlockTermState state = term.state;
final int suffix = term.termBytes.length - prefixLength; final int suffix = term.termBytes.length - prefixLength;
//if (DEBUG2) { //if (DEBUG2) {

View File

@ -225,7 +225,7 @@ final class IndexedDISI extends DocIdSetIterator {
buffer.set(DocIdSetIterator.NO_MORE_DOCS & 0xFFFF); buffer.set(DocIdSetIterator.NO_MORE_DOCS & 0xFFFF);
flush(DocIdSetIterator.NO_MORE_DOCS >>> 16, buffer, 1, denseRankPower, out); flush(DocIdSetIterator.NO_MORE_DOCS >>> 16, buffer, 1, denseRankPower, out);
// offset+index jump-table stored at the end // 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 // 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 // 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 // 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 if (blockCount == 2) { // Jumps with a single real entry + NO_MORE_DOCS is just wasted space so we ignore that
blockCount = 0; blockCount = 0;
} }

View File

@ -760,7 +760,7 @@ final class Lucene80DocValuesProducer extends DocValuesProducer implements Close
} }
// Decompresses blocks of binary values to retrieve content // Decompresses blocks of binary values to retrieve content
class BinaryDecoder { static class BinaryDecoder {
private final LongValues addresses; private final LongValues addresses;
private final IndexInput compressedData; private final IndexInput compressedData;

View File

@ -74,7 +74,7 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
/** Sole constructor. */ /** Sole constructor. */
public PerFieldDocValuesFormat() { protected PerFieldDocValuesFormat() {
super(PER_FIELD_NAME); super(PER_FIELD_NAME);
} }

View File

@ -37,7 +37,7 @@ import org.apache.lucene.index.Terms;
* <p> * <p>
* Warning: the input {@linkplain MergeState} instance will be updated when calling {@link #apply(Collection)}. * Warning: the input {@linkplain MergeState} instance will be updated when calling {@link #apply(Collection)}.
* <p> * <p>
* It should be called within a {@code try &#123;...&#125; finally &#123;...&#125;} 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: * restored to its original state:
* <pre> * <pre>
* PerFieldMergeState pfMergeState = new PerFieldMergeState(mergeState); * PerFieldMergeState pfMergeState = new PerFieldMergeState(mergeState);

View File

@ -80,7 +80,7 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
public static final String PER_FIELD_SUFFIX_KEY = PerFieldPostingsFormat.class.getSimpleName() + ".suffix"; public static final String PER_FIELD_SUFFIX_KEY = PerFieldPostingsFormat.class.getSimpleName() + ".suffix";
/** Sole constructor. */ /** Sole constructor. */
public PerFieldPostingsFormat() { protected PerFieldPostingsFormat() {
super(PER_FIELD_NAME); super(PER_FIELD_NAME);
} }

View File

@ -39,7 +39,7 @@ public abstract class DocIDMerger<T extends DocIDMerger.Sub> {
final MergeState.DocMap docMap; final MergeState.DocMap docMap;
/** Sole constructor */ /** Sole constructor */
public Sub(MergeState.DocMap docMap) { protected Sub(MergeState.DocMap docMap) {
this.docMap = docMap; this.docMap = docMap;
} }

View File

@ -45,7 +45,7 @@ abstract class DocValuesUpdate {
final String field; final String field;
// used in BufferedDeletes to apply this update only to a slice of docs. It's initialized to BufferedUpdates.MAX_INT // 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 safe object creations.
final int docIDUpto; final int docIDUpTo;
final boolean hasValue; final boolean hasValue;
/** /**
@ -54,19 +54,19 @@ abstract class DocValuesUpdate {
* @param term the {@link Term} which determines the documents that will be updated * @param term the {@link Term} which determines the documents that will be updated
* @param field the {@link NumericDocValuesField} to update * @param field the {@link NumericDocValuesField} to update
*/ */
protected DocValuesUpdate(DocValuesType type, Term term, String field, int docIDUpto, boolean hasValue) { 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.type = type;
this.term = term; this.term = term;
this.field = field; this.field = field;
this.docIDUpto = docIDUpto; this.docIDUpTo = docIDUpTo;
this.hasValue = hasValue; this.hasValue = hasValue;
} }
abstract long valueSizeInBytes(); abstract long valueSizeInBytes();
final int sizeInBytes() { final long sizeInBytes() {
int sizeInBytes = RAW_SIZE_IN_BYTES; long sizeInBytes = RAW_SIZE_IN_BYTES;
sizeInBytes += term.field.length() * Character.BYTES; sizeInBytes += term.field.length() * Character.BYTES;
sizeInBytes += term.bytes.bytes.length; sizeInBytes += term.bytes.bytes.length;
sizeInBytes += field.length() * Character.BYTES; sizeInBytes += field.length() * Character.BYTES;
@ -85,7 +85,7 @@ abstract class DocValuesUpdate {
@Override @Override
public String toString() { 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 */ /** An in-place update to a binary DocValues field */
@ -93,7 +93,7 @@ abstract class DocValuesUpdate {
private final BytesRef value; private final BytesRef value;
/* Size of BytesRef: 2*INT + ARRAY_HEADER + PTR */ /* 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) { BinaryDocValuesUpdate(Term term, String field, BytesRef value) {
this(term, field, value, BufferedUpdates.MAX_INT); this(term, field, value, BufferedUpdates.MAX_INT);
@ -104,11 +104,11 @@ abstract class DocValuesUpdate {
this.value = value; this.value = value;
} }
BinaryDocValuesUpdate prepareForApply(int docIDUpto) { BinaryDocValuesUpdate prepareForApply(int docIDUpTo) {
if (docIDUpto == this.docIDUpto) { if (docIDUpTo == this.docIDUpTo) {
return this; // it's a final value so we can safely reuse this instance 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 @Override
@ -152,7 +152,7 @@ abstract class DocValuesUpdate {
} }
NumericDocValuesUpdate(Term term, String field, Long value) { 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; this.value = value;
} }
NumericDocValuesUpdate prepareForApply(int docIDUpto) { NumericDocValuesUpdate prepareForApply(int docIDUpTo) {
if (docIDUpto == this.docIDUpto) { if (docIDUpTo == this.docIDUpTo) {
return this; return this;
} }
return new NumericDocValuesUpdate(term, field, value, docIDUpto, hasValue); return new NumericDocValuesUpdate(term, field, value, docIDUpTo, hasValue);
} }
@Override @Override

View File

@ -59,7 +59,7 @@ public abstract class FilterLeafReader extends LeafReader {
* Creates a new FilterFields. * Creates a new FilterFields.
* @param in the underlying Fields instance. * @param in the underlying Fields instance.
*/ */
public FilterFields(Fields in) { protected FilterFields(Fields in) {
if (in == null) { if (in == null) {
throw new NullPointerException("incoming Fields must not be null"); throw new NullPointerException("incoming Fields must not be null");
} }
@ -95,7 +95,7 @@ public abstract class FilterLeafReader extends LeafReader {
* Creates a new FilterTerms * Creates a new FilterTerms
* @param in the underlying Terms instance. * @param in the underlying Terms instance.
*/ */
public FilterTerms(Terms in) { protected FilterTerms(Terms in) {
if (in == null) { if (in == null) {
throw new NullPointerException("incoming Terms must not be null"); throw new NullPointerException("incoming Terms must not be null");
} }
@ -162,7 +162,7 @@ public abstract class FilterLeafReader extends LeafReader {
* Creates a new FilterTermsEnum * Creates a new FilterTermsEnum
* @param in the underlying TermsEnum instance. * @param in the underlying TermsEnum instance.
*/ */
public FilterTermsEnum(TermsEnum in) { protected FilterTermsEnum(TermsEnum in) {
if (in == null) { if (in == null) {
throw new NullPointerException("incoming TermsEnum must not be null"); throw new NullPointerException("incoming TermsEnum must not be null");
} }
@ -244,7 +244,7 @@ public abstract class FilterLeafReader extends LeafReader {
* Create a new FilterPostingsEnum * Create a new FilterPostingsEnum
* @param in the underlying PostingsEnum instance. * @param in the underlying PostingsEnum instance.
*/ */
public FilterPostingsEnum(PostingsEnum in) { protected FilterPostingsEnum(PostingsEnum in) {
if (in == null) { if (in == null) {
throw new NullPointerException("incoming PostingsEnum must not be 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> * <p>Note that base reader is closed if this FilterLeafReader is closed.</p>
* @param in specified base reader. * @param in specified base reader.
*/ */
public FilterLeafReader(LeafReader in) { protected FilterLeafReader(LeafReader in) {
super(); super();
if (in == null) { if (in == null) {
throw new NullPointerException("incoming LeafReader must not be null"); throw new NullPointerException("incoming LeafReader must not be null");

View File

@ -73,7 +73,7 @@ public abstract class FilteredTermsEnum extends TermsEnum {
* Creates a filtered {@link TermsEnum} on a terms enum. * Creates a filtered {@link TermsEnum} on a terms enum.
* @param tenum the terms enumeration to filter. * @param tenum the terms enumeration to filter.
*/ */
public FilteredTermsEnum(final TermsEnum tenum) { protected FilteredTermsEnum(final TermsEnum tenum) {
this(tenum, true); this(tenum, true);
} }
@ -81,7 +81,7 @@ public abstract class FilteredTermsEnum extends TermsEnum {
* Creates a filtered {@link TermsEnum} on a terms enum. * Creates a filtered {@link TermsEnum} on a terms enum.
* @param tenum the terms enumeration to filter. * @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; assert tenum != null;
this.tenum = tenum; this.tenum = tenum;
doSeek = startWithSeek; doSeek = startWithSeek;

View File

@ -40,7 +40,7 @@ import org.apache.lucene.util.Constants;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.InfoStream; import org.apache.lucene.util.InfoStream;
/* /**
* This class keeps track of each SegmentInfos instance that * This class keeps track of each SegmentInfos instance that
* is still "live", either because it corresponds to a * is still "live", either because it corresponds to a
* segments_N file in the Directory (a "commit", i.e. 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) * instantiating this class. It opens segments_N file(s)
* directly with no retry logic. * directly with no retry logic.
*/ */
final class IndexFileDeleter implements Closeable { final class IndexFileDeleter implements Closeable {
/* Reference count for all files in the index. /* Reference count for all files in the index.

View File

@ -4473,7 +4473,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
final int delCount = numDeletedDocs(info); final int delCount = numDeletedDocs(info);
assert delCount <= info.info.maxDoc(); assert delCount <= info.info.maxDoc();
final double delRatio = ((double) 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(); merge.totalMergeBytes += info.sizeInBytes();
} }
} }

View File

@ -558,7 +558,7 @@ public abstract class MergePolicy {
/** /**
* Creates a new merge policy instance. * Creates a new merge policy instance.
*/ */
public MergePolicy() { protected MergePolicy() {
this(DEFAULT_NO_CFS_RATIO, DEFAULT_MAX_CFS_SEGMENT_SIZE); this(DEFAULT_NO_CFS_RATIO, DEFAULT_MAX_CFS_SEGMENT_SIZE);
} }

View File

@ -241,9 +241,9 @@ public class MergeState {
/** A map of doc IDs. */ /** A map of doc IDs. */
public static abstract class DocMap { public static abstract class DocMap {
/** Sole constructor */ /** Sole constructor. (For invocation by subclass constructors, typically implicit.) */
public DocMap() { // Explicitly declared so that we have non-empty javadoc
} protected DocMap() {}
/** Return the mapped docID or -1 if the given doc is not mapped. */ /** Return the mapped docID or -1 if the given doc is not mapped. */
public abstract int get(int docID); public abstract int get(int docID);

View File

@ -492,7 +492,7 @@ final class ReadersAndUpdates {
// we write approximately that many bytes (based on Lucene46DVF): // we write approximately that many bytes (based on Lucene46DVF):
// HEADER + FOOTER: 40 // HEADER + FOOTER: 40
// 90 bytes per-field (over estimating long name and attributes map) // 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)); final IOContext infosContext = new IOContext(new FlushInfo(info.info.maxDoc(), estInfosSize));
// separately also track which files were created for this gen // separately also track which files were created for this gen
final TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir); final TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);

View File

@ -166,7 +166,6 @@ final class SegmentCoreReaders {
@SuppressWarnings("try") @SuppressWarnings("try")
void decRef() throws IOException { void decRef() throws IOException {
if (ref.decrementAndGet() == 0) { if (ref.decrementAndGet() == 0) {
Throwable th = null;
try (Closeable finalizer = this::notifyCoreClosedListeners){ try (Closeable finalizer = this::notifyCoreClosedListeners){
IOUtils.close(termVectorsLocal, fieldsReaderLocal, fields, termVectorsReaderOrig, fieldsReaderOrig, IOUtils.close(termVectorsLocal, fieldsReaderLocal, fields, termVectorsReaderOrig, fieldsReaderOrig,
cfsReader, normsProducer, pointsReader); cfsReader, normsProducer, pointsReader);

View File

@ -668,8 +668,8 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
final Directory directory; final Directory directory;
/** Sole constructor. */ /** Sole constructor. */
public FindSegmentsFile(Directory directory) { protected FindSegmentsFile(Directory directory) {
this.directory = directory; this.directory = directory;
} }

View File

@ -145,7 +145,7 @@ public abstract class FieldComparator<T> {
protected final String field; protected final String field;
protected NumericDocValues currentReaderValues; protected NumericDocValues currentReaderValues;
public NumericComparator(String field, T missingValue) { protected NumericComparator(String field, T missingValue) {
this.field = field; this.field = field;
this.missingValue = missingValue; this.missingValue = missingValue;
} }

View File

@ -697,7 +697,7 @@ public class IndexSearcher {
throw new RuntimeException(e); throw new RuntimeException(e);
} }
} }
return collectorManager.reduce(collectors); return collectorManager.reduce(collectedCollectors);
} }
} }

View File

@ -90,13 +90,10 @@ import org.apache.lucene.util.SmallFloat;
* @lucene.experimental * @lucene.experimental
*/ */
public abstract class Similarity { public abstract class Similarity {
/** Sole constructor. (For invocation by subclass constructors, typically implicit.) */
/** // Explicitly declared so that we have non-empty javadoc
* Sole constructor. (For invocation by subclass protected Similarity() {}
* constructors, typically implicit.)
*/
public Similarity() {}
/** /**
* Computes the normalization value for a field, given the accumulated * Computes the normalization value for a field, given the accumulated
* state of term processing for this field (see {@link FieldInvertState}). * 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 * <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 * must produce scores that are lower or equal, ie. for two encoded norms
* {@code n1} and {@code n2} so that * {@code n1} and {@code n2} so that
* {@code Long.compareUnsigned(n1, n2) &gt; 0} then * {@code Long.compareUnsigned(n1, n2) > 0} then
* {@code SimScorer.score(freq, n1) &lt;= SimScorer.score(freq, n2)} * {@code SimScorer.score(freq, n1) <= SimScorer.score(freq, n2)}
* for any legal {@code freq}. * for any legal {@code freq}.
* *
* <p>{@code 0} is not a legal norm, so {@code 1} is the norm that produces * <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}. * {@code 1} if norms are disabled. {@code norm} is never {@code 0}.
* <p> * <p>
* Score must not decrease when {@code freq} increases, ie. if * Score must not decrease when {@code freq} increases, ie. if
* {@code freq1 &gt; freq2}, then {@code score(freq1, norm) &gt;= * {@code freq1 > freq2}, then {@code score(freq1, norm) >=
* score(freq2, norm)} for any value of {@code norm} that may be produced * score(freq2, norm)} for any value of {@code norm} that may be produced
* by {@link Similarity#computeNorm(FieldInvertState)}. * by {@link Similarity#computeNorm(FieldInvertState)}.
* <p> * <p>
* Score must not increase when the unsigned {@code norm} increases, ie. if * Score must not increase when the unsigned {@code norm} increases, ie. if
* {@code Long.compareUnsigned(norm1, norm2) &gt; 0} then * {@code Long.compareUnsigned(norm1, norm2) > 0} then
* {@code score(freq, norm1) &lt;= score(freq, norm2)} for any legal * {@code score(freq, norm1) <= score(freq, norm2)} for any legal
* {@code freq}. * {@code freq}.
* <p> * <p>
* As a consequence, the maximum score that this scorer can produce is bound * As a consequence, the maximum score that this scorer can produce is bound

View File

@ -101,7 +101,7 @@ public abstract class AttributeFactory {
/** <b>Expert</b>: Creates an AttributeFactory returning {@code clazz} as instance for the /** <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. */ * 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.delegate = delegate;
this.clazz = clazz; this.clazz = clazz;
} }

View File

@ -53,7 +53,7 @@ public final class ByteBlockPool implements Accountable {
public abstract static class Allocator { public abstract static class Allocator {
protected final int blockSize; protected final int blockSize;
public Allocator(int blockSize) { protected Allocator(int blockSize) {
this.blockSize = blockSize; this.blockSize = blockSize;
} }

View File

@ -391,7 +391,7 @@ public final class IOUtils {
* *
* @param th The throwable to rethrow, <strong>must not be null</strong>. * @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. * @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 IOException if the argument was an instance of IOException
* @throws RuntimeException with the {@link RuntimeException#getCause()} set * @throws RuntimeException with the {@link RuntimeException#getCause()} set
* to the argument, if it was not an instance of IOException. * to the argument, if it was not an instance of IOException.

View File

@ -33,7 +33,7 @@ public final class IntBlockPool {
public abstract static class Allocator { public abstract static class Allocator {
protected final int blockSize; protected final int blockSize;
public Allocator(int blockSize) { protected Allocator(int blockSize) {
this.blockSize = blockSize; this.blockSize = blockSize;
} }

View File

@ -72,7 +72,7 @@ public final class LongsRef implements Comparable<LongsRef>, Cloneable {
public int hashCode() { public int hashCode() {
final int prime = 31; final int prime = 31;
int result = 0; int result = 0;
final long end = offset + length; final long end = (long) offset + length;
for(int i = offset; i < end; i++) { for(int i = offset; i < end; i++) {
result = prime * result + (int) (longs[i] ^ (longs[i]>>>32)); result = prime * result + (int) (longs[i] ^ (longs[i]>>>32));
} }
@ -106,7 +106,7 @@ public final class LongsRef implements Comparable<LongsRef>, Cloneable {
public String toString() { public String toString() {
StringBuilder sb = new StringBuilder(); StringBuilder sb = new StringBuilder();
sb.append('['); sb.append('[');
final long end = offset + length; final long end = (long) offset + length;
for(int i=offset;i<end;i++) { for(int i=offset;i<end;i++) {
if (i > offset) { if (i > offset) {
sb.append(' '); sb.append(' ');

View File

@ -195,7 +195,7 @@ public final class RamUsageEstimator {
/** Approximate memory usage that we assign to a Hashtable / HashMap entry. */ /** Approximate memory usage that we assign to a Hashtable / HashMap entry. */
public static final long HASHTABLE_RAM_BYTES_PER_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 * 2; // hash tables need to be oversized to avoid collisions, assume 2x capacity
/** Approximate memory usage that we assign to a LinkedHashMap entry. */ /** Approximate memory usage that we assign to a LinkedHashMap entry. */

View File

@ -194,7 +194,7 @@ public final class LZ4 {
abstract void initDictionary(int dictLen); 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 * 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 * increasing sequences of offsets. A return value of {@code -1} indicates
* that no other index could be found. */ * that no other index could be found. */

View File

@ -52,7 +52,7 @@ abstract class AbstractBlockPackedWriter {
* Sole constructor. * Sole constructor.
* @param blockSize the number of values of a single block, must be a multiple of <code>64</code> * @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); checkBlockSize(blockSize, MIN_BLOCK_SIZE, MAX_BLOCK_SIZE);
reset(out); reset(out);
values = new long[blockSize]; values = new long[blockSize];

View File

@ -145,7 +145,7 @@ public class PackedInts {
this.id = id; this.id = id;
} }
public int id; public final int id;
/** /**
* Returns the ID of the format. * Returns the ID of the format.

View File

@ -185,14 +185,14 @@ public class TestFieldUpdatesBuffer extends LuceneTestCase {
Counter counter = Counter.newCounter(); Counter counter = Counter.newCounter();
DocValuesUpdate.BinaryDocValuesUpdate randomUpdate = getRandomBinaryUpdate(); DocValuesUpdate.BinaryDocValuesUpdate randomUpdate = getRandomBinaryUpdate();
updates.add(randomUpdate); 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++) { for (int i = 0; i < numUpdates; i++) {
randomUpdate = getRandomBinaryUpdate(); randomUpdate = getRandomBinaryUpdate();
updates.add(randomUpdate); updates.add(randomUpdate);
if (randomUpdate.hasValue) { if (randomUpdate.hasValue) {
buffer.addUpdate(randomUpdate.term, randomUpdate.getValue(), randomUpdate.docIDUpto); buffer.addUpdate(randomUpdate.term, randomUpdate.getValue(), randomUpdate.docIDUpTo);
} else { } else {
buffer.addNoValue(randomUpdate.term, randomUpdate.docIDUpto); buffer.addNoValue(randomUpdate.term, randomUpdate.docIDUpTo);
} }
} }
buffer.finish(); buffer.finish();
@ -210,7 +210,7 @@ public class TestFieldUpdatesBuffer extends LuceneTestCase {
} else { } else {
assertNull(value.binaryValue); assertNull(value.binaryValue);
} }
assertEquals(randomUpdate.docIDUpto, value.docUpTo); assertEquals(randomUpdate.docIDUpTo, value.docUpTo);
} }
assertEquals(count, updates.size()); assertEquals(count, updates.size());
} }
@ -221,14 +221,14 @@ public class TestFieldUpdatesBuffer extends LuceneTestCase {
Counter counter = Counter.newCounter(); Counter counter = Counter.newCounter();
DocValuesUpdate.NumericDocValuesUpdate randomUpdate = getRandomNumericUpdate(); DocValuesUpdate.NumericDocValuesUpdate randomUpdate = getRandomNumericUpdate();
updates.add(randomUpdate); 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++) { for (int i = 0; i < numUpdates; i++) {
randomUpdate = getRandomNumericUpdate(); randomUpdate = getRandomNumericUpdate();
updates.add(randomUpdate); updates.add(randomUpdate);
if (randomUpdate.hasValue) { if (randomUpdate.hasValue) {
buffer.addUpdate(randomUpdate.term, randomUpdate.getValue(), randomUpdate.docIDUpto); buffer.addUpdate(randomUpdate.term, randomUpdate.getValue(), randomUpdate.docIDUpTo);
} else { } else {
buffer.addNoValue(randomUpdate.term, randomUpdate.docIDUpto); buffer.addNoValue(randomUpdate.term, randomUpdate.docIDUpTo);
} }
} }
buffer.finish(); buffer.finish();
@ -242,7 +242,7 @@ public class TestFieldUpdatesBuffer extends LuceneTestCase {
public void testNoNumericValue() { public void testNoNumericValue() {
DocValuesUpdate.NumericDocValuesUpdate update = DocValuesUpdate.NumericDocValuesUpdate update =
new DocValuesUpdate.NumericDocValuesUpdate(new Term("id", "1"), "age", null); 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.getMinNumeric());
assertEquals(0, buffer.getMaxNumeric()); assertEquals(0, buffer.getMaxNumeric());
} }
@ -257,13 +257,13 @@ public class TestFieldUpdatesBuffer extends LuceneTestCase {
new Term(termField, Integer.toString(random().nextInt(1000))), "numeric", docValue); new Term(termField, Integer.toString(random().nextInt(1000))), "numeric", docValue);
randomUpdate = randomUpdate.prepareForApply(randomDocUpTo()); randomUpdate = randomUpdate.prepareForApply(randomDocUpTo());
updates.add(randomUpdate); 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++) { for (int i = 0; i < numUpdates; i++) {
randomUpdate = new DocValuesUpdate.NumericDocValuesUpdate( randomUpdate = new DocValuesUpdate.NumericDocValuesUpdate(
new Term(termField, Integer.toString(random().nextInt(1000))), "numeric", docValue); new Term(termField, Integer.toString(random().nextInt(1000))), "numeric", docValue);
randomUpdate = randomUpdate.prepareForApply(randomDocUpTo()); randomUpdate = randomUpdate.prepareForApply(randomDocUpTo());
updates.add(randomUpdate); updates.add(randomUpdate);
buffer.addUpdate(randomUpdate.term, randomUpdate.getValue(), randomUpdate.docIDUpto); buffer.addUpdate(randomUpdate.term, randomUpdate.getValue(), randomUpdate.docIDUpTo);
} }
buffer.finish(); buffer.finish();
assertBufferUpdates(buffer, updates, true); assertBufferUpdates(buffer, updates, true);
@ -276,7 +276,7 @@ public class TestFieldUpdatesBuffer extends LuceneTestCase {
updates.sort(Comparator.comparing(u -> u.term.bytes)); updates.sort(Comparator.comparing(u -> u.term.bytes));
SortedMap<BytesRef, DocValuesUpdate.NumericDocValuesUpdate> byTerms = new TreeMap<>(); SortedMap<BytesRef, DocValuesUpdate.NumericDocValuesUpdate> byTerms = new TreeMap<>();
for (DocValuesUpdate.NumericDocValuesUpdate update : updates) { 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()); updates = new ArrayList<>(byTerms.values());
} }
@ -304,7 +304,7 @@ public class TestFieldUpdatesBuffer extends LuceneTestCase {
assertEquals(0, value.numericValue); assertEquals(0, value.numericValue);
assertEquals(0, v); assertEquals(0, v);
} }
assertEquals(expectedUpdate.docIDUpto, value.docUpTo); assertEquals(expectedUpdate.docIDUpTo, value.docUpTo);
} }
if (hasAtLeastOneValue) { if (hasAtLeastOneValue) {
assertEquals(max, buffer.getMaxNumeric()); assertEquals(max, buffer.getMaxNumeric());

View File

@ -582,7 +582,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
assert ent.isTerm: "i=" + i; assert ent.isTerm: "i=" + i;
PendingTerm term = (PendingTerm) ent; 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; BlockTermState state = term.state;
maxVersionInBlock = Math.max(maxVersionInBlock, ((IDVersionTermState) state).idVersion); maxVersionInBlock = Math.max(maxVersionInBlock, ((IDVersionTermState) state).idVersion);
final int suffix = term.termBytes.length - prefixLength; final int suffix = term.termBytes.length - prefixLength;
@ -610,7 +610,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
PendingEntry ent = pending.get(i); PendingEntry ent = pending.get(i);
if (ent.isTerm) { if (ent.isTerm) {
PendingTerm term = (PendingTerm) ent; 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; BlockTermState state = term.state;
maxVersionInBlock = Math.max(maxVersionInBlock, ((IDVersionTermState) state).idVersion); maxVersionInBlock = Math.max(maxVersionInBlock, ((IDVersionTermState) state).idVersion);
final int suffix = term.termBytes.length - prefixLength; final int suffix = term.termBytes.length - prefixLength;

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.lucene.spatial3d.geom; package org.apache.lucene.spatial3d.geom;
import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.ArrayList; import java.util.ArrayList;
@ -317,7 +318,7 @@ class GeoExactCircle extends GeoBaseCircle {
@Override @Override
public String toString() { public String toString() {
return "{circle plane = " + circlePlane + " plane 1 = "+plane1 + return "{circle plane = " + circlePlane + " plane 1 = "+plane1 +
" plane 2 = " + plane2 + " notable edge points = " + notableEdgePoints + "}"; " plane 2 = " + plane2 + " notable edge points = " + Arrays.toString(notableEdgePoints) + "}";
} }
} }
} }

View File

@ -26,6 +26,7 @@ import static org.apache.lucene.index.PostingsEnum.POSITIONS;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.EnumSet; import java.util.EnumSet;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
@ -397,13 +398,12 @@ public abstract class BaseTermVectorsFormatTestCase extends BaseIndexFileFormatT
protected void assertEquals(RandomDocument doc, Fields fields) throws IOException { protected void assertEquals(RandomDocument doc, Fields fields) throws IOException {
// compare field names // compare field names
assertEquals(doc == null, fields == null); assertNotNull(doc);
assertNotNull(fields);
assertEquals(doc.fieldNames.length, fields.size()); assertEquals(doc.fieldNames.length, fields.size());
final Set<String> fields1 = new HashSet<>(); final Set<String> fields1 = new HashSet<>();
final Set<String> fields2 = new HashSet<>(); final Set<String> fields2 = new HashSet<>();
for (int i = 0; i < doc.fieldNames.length; ++i) { Collections.addAll(fields1, doc.fieldNames);
fields1.add(doc.fieldNames[i]);
}
for (String field : fields) { for (String field : fields) {
fields2.add(field); fields2.add(field);
} }
@ -462,55 +462,48 @@ public abstract class BaseTermVectorsFormatTestCase extends BaseIndexFileFormatT
assertEquals(0, docsAndPositionsEnum.nextDoc()); assertEquals(0, docsAndPositionsEnum.nextDoc());
final int freq = docsAndPositionsEnum.freq(); final int freq = docsAndPositionsEnum.freq();
assertEquals(tk.freqs.get(termsEnum.term().utf8ToString()), (Integer) freq); assertEquals(tk.freqs.get(termsEnum.term().utf8ToString()), (Integer) freq);
if (docsAndPositionsEnum != null) { for (int k = 0; k < freq; ++k) {
for (int k = 0; k < freq; ++k) { final int position = docsAndPositionsEnum.nextPosition();
final int position = docsAndPositionsEnum.nextPosition(); final Set<Integer> indexes;
final Set<Integer> indexes; if (terms.hasPositions()) {
if (terms.hasPositions()) { indexes = tk.positionToTerms.get(position);
indexes = tk.positionToTerms.get(position); assertNotNull(indexes);
assertNotNull(indexes); } else {
} else { indexes = tk.startOffsetToTerms.get(docsAndPositionsEnum.startOffset());
indexes = tk.startOffsetToTerms.get(docsAndPositionsEnum.startOffset()); assertNotNull(indexes);
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);
}
} }
try { if (terms.hasPositions()) {
docsAndPositionsEnum.nextPosition(); boolean foundPosition = false;
fail(); for (int index : indexes) {
} catch (Exception | AssertionError e) { if (tk.termBytes[index].equals(termsEnum.term()) && tk.positions[index] == position) {
// ok 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()); assertEquals(PostingsEnum.NO_MORE_DOCS, docsAndPositionsEnum.nextDoc());
} }
this.docsEnum.set(docsAndPositionsEnum); 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) { protected Document addId(Document doc, String id) {
doc.add(new StringField("id", id, Store.NO)); doc.add(new StringField("id", id, Store.NO));
return doc; return doc;

View File

@ -163,7 +163,7 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
ranges[id][0].setMax(d, ranges[oldID][0].getMax(d)); ranges[id][0].setMax(d, ranges[oldID][0].getMax(d));
} }
if (VERBOSE) { 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 { } else {
for (int m = 0, even = dimensions % 2; m < dimensions * 2; ++m) { 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 if (even == 0) { // even is min
ranges[id][0].setMin(d, ranges[oldID][0].getMin(d)); ranges[id][0].setMin(d, ranges[oldID][0].getMin(d));
if (VERBOSE) { 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 } else { // odd is max
ranges[id][0].setMax(d, ranges[oldID][0].getMax(d)); ranges[id][0].setMax(d, ranges[oldID][0].getMax(d));
if (VERBOSE) { 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 + ")");
} }
} }
} }

View File

@ -22,7 +22,12 @@ import org.apache.lucene.index.BaseTermVectorsFormatTestCase;
/** Test AssertingTermVectorsFormat directly */ /** Test AssertingTermVectorsFormat directly */
public class TestAssertingTermVectorsFormat extends BaseTermVectorsFormatTestCase { public class TestAssertingTermVectorsFormat extends BaseTermVectorsFormatTestCase {
private final Codec codec = new AssertingCodec(); private final Codec codec = new AssertingCodec();
@Override
protected Class<? extends Throwable> getReadPastLastPositionExceptionClass() {
return AssertionError.class;
}
@Override @Override
protected Codec getCodec() { protected Codec getCodec() {
return codec; return codec;