LUCENE-7070: insert missing 's' in PointFormat/Reader/Writer

This commit is contained in:
Mike McCandless 2016-03-06 08:27:02 -05:00
parent 4d5a33e9ba
commit e2ebbdf638
43 changed files with 321 additions and 321 deletions

View File

@ -21,12 +21,12 @@ import java.util.Objects;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat; import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.PointFormat;
import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat; import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.LiveDocsFormat; import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat; import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.StoredFieldsFormat;
@ -154,8 +154,8 @@ public class Lucene50Codec extends Codec {
} }
@Override @Override
public final PointFormat pointFormat() { public final PointsFormat pointsFormat() {
return PointFormat.EMPTY; return PointsFormat.EMPTY;
} }
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50"); private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");

View File

@ -21,12 +21,12 @@ import java.util.Objects;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat; import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.PointFormat;
import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat; import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.LiveDocsFormat; import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat; import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.StoredFieldsFormat;
@ -35,8 +35,8 @@ import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
import org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat; import org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat;
import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat; import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat; import org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode; import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat; import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
@ -160,8 +160,8 @@ public class Lucene53Codec extends Codec {
} }
@Override @Override
public final PointFormat pointFormat() { public final PointsFormat pointsFormat() {
return PointFormat.EMPTY; return PointsFormat.EMPTY;
} }
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50"); private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");

View File

@ -21,12 +21,12 @@ import java.util.Objects;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat; import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.PointFormat;
import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat; import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.LiveDocsFormat; import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat; import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.StoredFieldsFormat;
@ -35,8 +35,8 @@ import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
import org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat; import org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat;
import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat; import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat; import org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode; import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat; import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.lucene53.Lucene53NormsFormat; import org.apache.lucene.codecs.lucene53.Lucene53NormsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
@ -160,8 +160,8 @@ public class Lucene54Codec extends Codec {
} }
@Override @Override
public final PointFormat pointFormat() { public final PointsFormat pointsFormat() {
return PointFormat.EMPTY; return PointsFormat.EMPTY;
} }
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50"); private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");

View File

@ -27,9 +27,9 @@ import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.StringHelper; import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.bkd.BKDReader; import org.apache.lucene.util.bkd.BKDReader;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.BLOCK_COUNT; import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.BLOCK_COUNT;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.BLOCK_DOC_ID; import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.BLOCK_DOC_ID;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.BLOCK_VALUE; import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.BLOCK_VALUE;
class SimpleTextBKDReader extends BKDReader { class SimpleTextBKDReader extends BKDReader {

View File

@ -19,11 +19,11 @@ package org.apache.lucene.codecs.simpletext;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat; import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.PointFormat;
import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat; import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat; import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat; import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.StoredFieldsFormat;
@ -45,7 +45,7 @@ public final class SimpleTextCodec extends Codec {
private final LiveDocsFormat liveDocs = new SimpleTextLiveDocsFormat(); private final LiveDocsFormat liveDocs = new SimpleTextLiveDocsFormat();
private final DocValuesFormat dvFormat = new SimpleTextDocValuesFormat(); private final DocValuesFormat dvFormat = new SimpleTextDocValuesFormat();
private final CompoundFormat compoundFormat = new SimpleTextCompoundFormat(); private final CompoundFormat compoundFormat = new SimpleTextCompoundFormat();
private final PointFormat pointFormat = new SimpleTextPointFormat(); private final PointsFormat pointsFormat = new SimpleTextPointsFormat();
public SimpleTextCodec() { public SimpleTextCodec() {
super("SimpleText"); super("SimpleText");
@ -97,7 +97,7 @@ public final class SimpleTextCodec extends Codec {
} }
@Override @Override
public PointFormat pointFormat() { public PointsFormat pointsFormat() {
return pointFormat; return pointsFormat;
} }
} }

View File

@ -19,9 +19,9 @@ package org.apache.lucene.codecs.simpletext;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.codecs.PointFormat; import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointReader; import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointWriter; import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
@ -33,16 +33,16 @@ import org.apache.lucene.index.SegmentWriteState;
* any text editor, and even edit it to alter your index. * any text editor, and even edit it to alter your index.
* *
* @lucene.experimental */ * @lucene.experimental */
public final class SimpleTextPointFormat extends PointFormat { public final class SimpleTextPointsFormat extends PointsFormat {
@Override @Override
public PointWriter fieldsWriter(SegmentWriteState state) throws IOException { public PointsWriter fieldsWriter(SegmentWriteState state) throws IOException {
return new SimpleTextPointWriter(state); return new SimpleTextPointsWriter(state);
} }
@Override @Override
public PointReader fieldsReader(SegmentReadState state) throws IOException { public PointsReader fieldsReader(SegmentReadState state) throws IOException {
return new SimpleTextPointReader(state); return new SimpleTextPointsReader(state);
} }
/** Extension of points data file */ /** Extension of points data file */

View File

@ -22,7 +22,7 @@ import java.nio.charset.StandardCharsets;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import org.apache.lucene.codecs.PointReader; import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
@ -37,36 +37,36 @@ import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.StringHelper; import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.bkd.BKDReader; import org.apache.lucene.util.bkd.BKDReader;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.BLOCK_FP; import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.BLOCK_FP;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.BYTES_PER_DIM; import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.BYTES_PER_DIM;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.DOC_COUNT; import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.DOC_COUNT;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.FIELD_COUNT; import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.FIELD_COUNT;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.FIELD_FP; import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.FIELD_FP;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.FIELD_FP_NAME; import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.FIELD_FP_NAME;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.INDEX_COUNT; import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.INDEX_COUNT;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.MAX_LEAF_POINTS; import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.MAX_LEAF_POINTS;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.MAX_VALUE; import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.MAX_VALUE;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.MIN_VALUE; import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.MIN_VALUE;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.NUM_DIMS; import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.NUM_DIMS;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.POINT_COUNT; import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.POINT_COUNT;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.SPLIT_COUNT; import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.SPLIT_COUNT;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.SPLIT_DIM; import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.SPLIT_DIM;
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.SPLIT_VALUE; import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.SPLIT_VALUE;
class SimpleTextPointReader extends PointReader { class SimpleTextPointsReader extends PointsReader {
private final IndexInput dataIn; private final IndexInput dataIn;
final SegmentReadState readState; final SegmentReadState readState;
final Map<String,BKDReader> readers = new HashMap<>(); final Map<String,BKDReader> readers = new HashMap<>();
final BytesRefBuilder scratch = new BytesRefBuilder(); final BytesRefBuilder scratch = new BytesRefBuilder();
public SimpleTextPointReader(SegmentReadState readState) throws IOException { public SimpleTextPointsReader(SegmentReadState readState) throws IOException {
// Initialize readers now: // Initialize readers now:
// Read index: // Read index:
Map<String,Long> fieldToFileOffset = new HashMap<>(); Map<String,Long> fieldToFileOffset = new HashMap<>();
String indexFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.segmentSuffix, SimpleTextPointFormat.POINT_INDEX_EXTENSION); String indexFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.segmentSuffix, SimpleTextPointsFormat.POINT_INDEX_EXTENSION);
try (ChecksumIndexInput in = readState.directory.openChecksumInput(indexFileName, IOContext.DEFAULT)) { try (ChecksumIndexInput in = readState.directory.openChecksumInput(indexFileName, IOContext.DEFAULT)) {
readLine(in); readLine(in);
int count = parseInt(FIELD_COUNT); int count = parseInt(FIELD_COUNT);
@ -81,7 +81,7 @@ class SimpleTextPointReader extends PointReader {
} }
boolean success = false; boolean success = false;
String fileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.segmentSuffix, SimpleTextPointFormat.POINT_EXTENSION); String fileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.segmentSuffix, SimpleTextPointsFormat.POINT_EXTENSION);
dataIn = readState.directory.openInput(fileName, IOContext.DEFAULT); dataIn = readState.directory.openInput(fileName, IOContext.DEFAULT);
try { try {
for(Map.Entry<String,Long> ent : fieldToFileOffset.entrySet()) { for(Map.Entry<String,Long> ent : fieldToFileOffset.entrySet()) {
@ -98,7 +98,7 @@ class SimpleTextPointReader extends PointReader {
} }
private BKDReader initReader(long fp) throws IOException { private BKDReader initReader(long fp) throws IOException {
// NOTE: matches what writeIndex does in SimpleTextPointWriter // NOTE: matches what writeIndex does in SimpleTextPointsWriter
dataIn.seek(fp); dataIn.seek(fp);
readLine(dataIn); readLine(dataIn);
int numDims = parseInt(NUM_DIMS); int numDims = parseInt(NUM_DIMS);
@ -231,7 +231,7 @@ class SimpleTextPointReader extends PointReader {
@Override @Override
public String toString() { public String toString() {
return "SimpleTextPointReader(segment=" + readState.segmentInfo.name + " maxDoc=" + readState.segmentInfo.maxDoc() + ")"; return "SimpleTextPointsReader(segment=" + readState.segmentInfo.name + " maxDoc=" + readState.segmentInfo.maxDoc() + ")";
} }
@Override @Override

View File

@ -21,8 +21,8 @@ import java.io.IOException;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import org.apache.lucene.codecs.PointReader; import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointWriter; import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.PointValues.IntersectVisitor; import org.apache.lucene.index.PointValues.IntersectVisitor;
@ -33,7 +33,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder; import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.bkd.BKDWriter; import org.apache.lucene.util.bkd.BKDWriter;
class SimpleTextPointWriter extends PointWriter { class SimpleTextPointsWriter extends PointsWriter {
final static BytesRef NUM_DIMS = new BytesRef("num dims "); final static BytesRef NUM_DIMS = new BytesRef("num dims ");
final static BytesRef BYTES_PER_DIM = new BytesRef("bytes per dim "); final static BytesRef BYTES_PER_DIM = new BytesRef("bytes per dim ");
@ -60,14 +60,14 @@ class SimpleTextPointWriter extends PointWriter {
final SegmentWriteState writeState; final SegmentWriteState writeState;
final Map<String,Long> indexFPs = new HashMap<>(); final Map<String,Long> indexFPs = new HashMap<>();
public SimpleTextPointWriter(SegmentWriteState writeState) throws IOException { public SimpleTextPointsWriter(SegmentWriteState writeState) throws IOException {
String fileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name, writeState.segmentSuffix, SimpleTextPointFormat.POINT_EXTENSION); String fileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name, writeState.segmentSuffix, SimpleTextPointsFormat.POINT_EXTENSION);
dataOut = writeState.directory.createOutput(fileName, writeState.context); dataOut = writeState.directory.createOutput(fileName, writeState.context);
this.writeState = writeState; this.writeState = writeState;
} }
@Override @Override
public void writeField(FieldInfo fieldInfo, PointReader values) throws IOException { public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
// We use the normal BKDWriter, but subclass to customize how it writes the index and blocks to disk: // We use the normal BKDWriter, but subclass to customize how it writes the index and blocks to disk:
try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(), try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
@ -223,7 +223,7 @@ class SimpleTextPointWriter extends PointWriter {
dataOut = null; dataOut = null;
// Write index file // Write index file
String fileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name, writeState.segmentSuffix, SimpleTextPointFormat.POINT_INDEX_EXTENSION); String fileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name, writeState.segmentSuffix, SimpleTextPointsFormat.POINT_INDEX_EXTENSION);
try (IndexOutput indexOut = writeState.directory.createOutput(fileName, writeState.context)) { try (IndexOutput indexOut = writeState.directory.createOutput(fileName, writeState.context)) {
int count = indexFPs.size(); int count = indexFPs.size();
write(indexOut, FIELD_COUNT); write(indexOut, FIELD_COUNT);

View File

@ -18,12 +18,12 @@ package org.apache.lucene.codecs.simpletext;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.BasePointFormatTestCase; import org.apache.lucene.index.BasePointsFormatTestCase;
/** /**
* Tests SimpleText's point format * Tests SimpleText's point format
*/ */
public class TestSimpleTextPointFormat extends BasePointFormatTestCase { public class TestSimpleTextPointsFormat extends BasePointsFormatTestCase {
private final Codec codec = new SimpleTextCodec(); private final Codec codec = new SimpleTextCodec();
@Override @Override

View File

@ -109,7 +109,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
public abstract CompoundFormat compoundFormat(); public abstract CompoundFormat compoundFormat();
/** Encodes/decodes points index */ /** Encodes/decodes points index */
public abstract PointFormat pointFormat(); public abstract PointsFormat pointsFormat();
/** looks up a codec by name */ /** looks up a codec by name */
public static Codec forName(String name) { public static Codec forName(String name) {

View File

@ -105,7 +105,7 @@ public abstract class FilterCodec extends Codec {
} }
@Override @Override
public PointFormat pointFormat() { public PointsFormat pointsFormat() {
return delegate.pointFormat(); return delegate.pointsFormat();
} }
} }

View File

@ -26,16 +26,16 @@ import org.apache.lucene.index.SegmentWriteState;
* Encodes/decodes indexed points. * Encodes/decodes indexed points.
* *
* @lucene.experimental */ * @lucene.experimental */
public abstract class PointFormat { public abstract class PointsFormat {
/** /**
* Creates a new point format. * Creates a new point format.
*/ */
protected PointFormat() { protected PointsFormat() {
} }
/** Writes a new segment */ /** Writes a new segment */
public abstract PointWriter fieldsWriter(SegmentWriteState state) throws IOException; public abstract PointsWriter fieldsWriter(SegmentWriteState state) throws IOException;
/** Reads a segment. NOTE: by the time this call /** Reads a segment. NOTE: by the time this call
* returns, it must hold open any files it will need to * returns, it must hold open any files it will need to
@ -46,18 +46,18 @@ public abstract class PointFormat {
* IOExceptions are expected and will automatically cause a retry of the * IOExceptions are expected and will automatically cause a retry of the
* segment opening logic with the newly revised segments. * segment opening logic with the newly revised segments.
* */ * */
public abstract PointReader fieldsReader(SegmentReadState state) throws IOException; public abstract PointsReader fieldsReader(SegmentReadState state) throws IOException;
/** A {@code PointFormat} that has nothing indexed */ /** A {@code PointsFormat} that has nothing indexed */
public static final PointFormat EMPTY = new PointFormat() { public static final PointsFormat EMPTY = new PointsFormat() {
@Override @Override
public PointWriter fieldsWriter(SegmentWriteState state) { public PointsWriter fieldsWriter(SegmentWriteState state) {
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }
@Override @Override
public PointReader fieldsReader(SegmentReadState state) { public PointsReader fieldsReader(SegmentReadState state) {
return new PointReader() { return new PointsReader() {
@Override @Override
public void close() { public void close() {
} }

View File

@ -27,10 +27,10 @@ import org.apache.lucene.util.Accountable;
* *
* @lucene.experimental * @lucene.experimental
*/ */
public abstract class PointReader extends PointValues implements Closeable, Accountable { public abstract class PointsReader extends PointValues implements Closeable, Accountable {
/** Sole constructor. (For invocation by subclass constructors, typically implicit.) */ /** Sole constructor. (For invocation by subclass constructors, typically implicit.) */
protected PointReader() {} protected PointsReader() {}
/** /**
* Checks consistency of this reader. * Checks consistency of this reader.
@ -45,7 +45,7 @@ public abstract class PointReader extends PointValues implements Closeable, Acco
* Returns an instance optimized for merging. * Returns an instance optimized for merging.
* <p> * <p>
* The default implementation returns {@code this} */ * The default implementation returns {@code this} */
public PointReader getMergeInstance() throws IOException { public PointsReader getMergeInstance() throws IOException {
return this; return this;
} }
} }

View File

@ -28,35 +28,35 @@ import org.apache.lucene.index.MergeState;
* @lucene.experimental * @lucene.experimental
*/ */
public abstract class PointWriter implements Closeable { public abstract class PointsWriter implements Closeable {
/** Sole constructor. (For invocation by subclass /** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */ * constructors, typically implicit.) */
protected PointWriter() { protected PointsWriter() {
} }
/** Write all values contained in the provided reader */ /** Write all values contained in the provided reader */
public abstract void writeField(FieldInfo fieldInfo, PointReader values) throws IOException; public abstract void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException;
/** Default naive merge implementation for one field: it just re-indexes all the values /** Default naive merge implementation for one field: it just re-indexes all the values
* from the incoming segment. The default codec overrides this for 1D fields and uses * from the incoming segment. The default codec overrides this for 1D fields and uses
* a faster but more complex implementation. */ * a faster but more complex implementation. */
protected void mergeOneField(MergeState mergeState, FieldInfo fieldInfo) throws IOException { protected void mergeOneField(MergeState mergeState, FieldInfo fieldInfo) throws IOException {
writeField(fieldInfo, writeField(fieldInfo,
new PointReader() { new PointsReader() {
@Override @Override
public void intersect(String fieldName, IntersectVisitor mergedVisitor) throws IOException { public void intersect(String fieldName, IntersectVisitor mergedVisitor) throws IOException {
if (fieldName.equals(fieldInfo.name) == false) { if (fieldName.equals(fieldInfo.name) == false) {
throw new IllegalArgumentException("field name must match the field being merged"); throw new IllegalArgumentException("field name must match the field being merged");
} }
for (int i=0;i<mergeState.pointReaders.length;i++) { for (int i=0;i<mergeState.pointsReaders.length;i++) {
PointReader pointReader = mergeState.pointReaders[i]; PointsReader pointsReader = mergeState.pointsReaders[i];
if (pointReader == null) { if (pointsReader == null) {
// This segment has no points // This segment has no points
continue; continue;
} }
MergeState.DocMap docMap = mergeState.docMaps[i]; MergeState.DocMap docMap = mergeState.docMaps[i];
int docBase = mergeState.docBase[i]; int docBase = mergeState.docBase[i];
pointReader.intersect(fieldInfo.name, pointsReader.intersect(fieldInfo.name,
new IntersectVisitor() { new IntersectVisitor() {
@Override @Override
public void visit(int docID) { public void visit(int docID) {
@ -75,7 +75,7 @@ public abstract class PointWriter implements Closeable {
@Override @Override
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) { public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
// Forces this segment's PointReader to always visit all docs + values: // Forces this segment's PointsReader to always visit all docs + values:
return Relation.CELL_CROSSES_QUERY; return Relation.CELL_CROSSES_QUERY;
} }
}); });
@ -132,7 +132,7 @@ public abstract class PointWriter implements Closeable {
* adding to this writer */ * adding to this writer */
public void merge(MergeState mergeState) throws IOException { public void merge(MergeState mergeState) throws IOException {
// check each incoming reader // check each incoming reader
for (PointReader reader : mergeState.pointReaders) { for (PointsReader reader : mergeState.pointsReaders) {
if (reader != null) { if (reader != null) {
reader.checkIntegrity(); reader.checkIntegrity();
} }

View File

@ -21,12 +21,12 @@ import java.util.Objects;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat; import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.PointFormat;
import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat; import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.LiveDocsFormat; import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat; import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.StoredFieldsFormat;
@ -129,8 +129,8 @@ public class Lucene60Codec extends Codec {
} }
@Override @Override
public final PointFormat pointFormat() { public final PointsFormat pointsFormat() {
return new Lucene60PointFormat(); return new Lucene60PointsFormat();
} }
/** Returns the postings format that should be used for writing /** Returns the postings format that should be used for writing

View File

@ -20,9 +20,9 @@ package org.apache.lucene.codecs.lucene60;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.PointFormat; import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointReader; import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointWriter; import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
@ -69,10 +69,10 @@ import org.apache.lucene.index.SegmentWriteState;
* @lucene.experimental * @lucene.experimental
*/ */
public final class Lucene60PointFormat extends PointFormat { public final class Lucene60PointsFormat extends PointsFormat {
static final String DATA_CODEC_NAME = "Lucene60PointFormatData"; static final String DATA_CODEC_NAME = "Lucene60PointsFormatData";
static final String META_CODEC_NAME = "Lucene60PointFormatMeta"; static final String META_CODEC_NAME = "Lucene60PointsFormatMeta";
/** /**
* Filename extension for the leaf blocks * Filename extension for the leaf blocks
@ -91,16 +91,16 @@ public final class Lucene60PointFormat extends PointFormat {
static final int INDEX_VERSION_CURRENT = INDEX_VERSION_START; static final int INDEX_VERSION_CURRENT = INDEX_VERSION_START;
/** Sole constructor */ /** Sole constructor */
public Lucene60PointFormat() { public Lucene60PointsFormat() {
} }
@Override @Override
public PointWriter fieldsWriter(SegmentWriteState state) throws IOException { public PointsWriter fieldsWriter(SegmentWriteState state) throws IOException {
return new Lucene60PointWriter(state); return new Lucene60PointsWriter(state);
} }
@Override @Override
public PointReader fieldsReader(SegmentReadState state) throws IOException { public PointsReader fieldsReader(SegmentReadState state) throws IOException {
return new Lucene60PointReader(state); return new Lucene60PointsReader(state);
} }
} }

View File

@ -27,7 +27,7 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.PointReader; import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
@ -38,20 +38,20 @@ import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.bkd.BKDReader; import org.apache.lucene.util.bkd.BKDReader;
/** Reads point values previously written with {@link Lucene60PointWriter} */ /** Reads point values previously written with {@link Lucene60PointsWriter} */
public class Lucene60PointReader extends PointReader implements Closeable { public class Lucene60PointsReader extends PointsReader implements Closeable {
final IndexInput dataIn; final IndexInput dataIn;
final SegmentReadState readState; final SegmentReadState readState;
final Map<Integer,BKDReader> readers = new HashMap<>(); final Map<Integer,BKDReader> readers = new HashMap<>();
/** Sole constructor */ /** Sole constructor */
public Lucene60PointReader(SegmentReadState readState) throws IOException { public Lucene60PointsReader(SegmentReadState readState) throws IOException {
this.readState = readState; this.readState = readState;
String indexFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, String indexFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name,
readState.segmentSuffix, readState.segmentSuffix,
Lucene60PointFormat.INDEX_EXTENSION); Lucene60PointsFormat.INDEX_EXTENSION);
Map<Integer,Long> fieldToFileOffset = new HashMap<>(); Map<Integer,Long> fieldToFileOffset = new HashMap<>();
@ -60,9 +60,9 @@ public class Lucene60PointReader extends PointReader implements Closeable {
Throwable priorE = null; Throwable priorE = null;
try { try {
CodecUtil.checkIndexHeader(indexIn, CodecUtil.checkIndexHeader(indexIn,
Lucene60PointFormat.META_CODEC_NAME, Lucene60PointsFormat.META_CODEC_NAME,
Lucene60PointFormat.INDEX_VERSION_START, Lucene60PointsFormat.INDEX_VERSION_START,
Lucene60PointFormat.INDEX_VERSION_START, Lucene60PointsFormat.INDEX_VERSION_START,
readState.segmentInfo.getId(), readState.segmentInfo.getId(),
readState.segmentSuffix); readState.segmentSuffix);
int count = indexIn.readVInt(); int count = indexIn.readVInt();
@ -80,15 +80,15 @@ public class Lucene60PointReader extends PointReader implements Closeable {
String dataFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, String dataFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name,
readState.segmentSuffix, readState.segmentSuffix,
Lucene60PointFormat.DATA_EXTENSION); Lucene60PointsFormat.DATA_EXTENSION);
boolean success = false; boolean success = false;
dataIn = readState.directory.openInput(dataFileName, readState.context); dataIn = readState.directory.openInput(dataFileName, readState.context);
try { try {
CodecUtil.checkIndexHeader(dataIn, CodecUtil.checkIndexHeader(dataIn,
Lucene60PointFormat.DATA_CODEC_NAME, Lucene60PointsFormat.DATA_CODEC_NAME,
Lucene60PointFormat.DATA_VERSION_START, Lucene60PointsFormat.DATA_VERSION_START,
Lucene60PointFormat.DATA_VERSION_START, Lucene60PointsFormat.DATA_VERSION_START,
readState.segmentInfo.getId(), readState.segmentInfo.getId(),
readState.segmentSuffix); readState.segmentSuffix);

View File

@ -25,8 +25,8 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.PointReader; import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointWriter; import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFileNames;
@ -40,7 +40,7 @@ import org.apache.lucene.util.bkd.BKDReader;
import org.apache.lucene.util.bkd.BKDWriter; import org.apache.lucene.util.bkd.BKDWriter;
/** Writes dimensional values */ /** Writes dimensional values */
public class Lucene60PointWriter extends PointWriter implements Closeable { public class Lucene60PointsWriter extends PointsWriter implements Closeable {
final IndexOutput dataOut; final IndexOutput dataOut;
final Map<String,Long> indexFPs = new HashMap<>(); final Map<String,Long> indexFPs = new HashMap<>();
@ -50,20 +50,20 @@ public class Lucene60PointWriter extends PointWriter implements Closeable {
private boolean finished; private boolean finished;
/** Full constructor */ /** Full constructor */
public Lucene60PointWriter(SegmentWriteState writeState, int maxPointsInLeafNode, double maxMBSortInHeap) throws IOException { public Lucene60PointsWriter(SegmentWriteState writeState, int maxPointsInLeafNode, double maxMBSortInHeap) throws IOException {
assert writeState.fieldInfos.hasPointValues(); assert writeState.fieldInfos.hasPointValues();
this.writeState = writeState; this.writeState = writeState;
this.maxPointsInLeafNode = maxPointsInLeafNode; this.maxPointsInLeafNode = maxPointsInLeafNode;
this.maxMBSortInHeap = maxMBSortInHeap; this.maxMBSortInHeap = maxMBSortInHeap;
String dataFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name, String dataFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name,
writeState.segmentSuffix, writeState.segmentSuffix,
Lucene60PointFormat.DATA_EXTENSION); Lucene60PointsFormat.DATA_EXTENSION);
dataOut = writeState.directory.createOutput(dataFileName, writeState.context); dataOut = writeState.directory.createOutput(dataFileName, writeState.context);
boolean success = false; boolean success = false;
try { try {
CodecUtil.writeIndexHeader(dataOut, CodecUtil.writeIndexHeader(dataOut,
Lucene60PointFormat.DATA_CODEC_NAME, Lucene60PointsFormat.DATA_CODEC_NAME,
Lucene60PointFormat.DATA_VERSION_CURRENT, Lucene60PointsFormat.DATA_VERSION_CURRENT,
writeState.segmentInfo.getId(), writeState.segmentInfo.getId(),
writeState.segmentSuffix); writeState.segmentSuffix);
success = true; success = true;
@ -75,12 +75,12 @@ public class Lucene60PointWriter extends PointWriter implements Closeable {
} }
/** Uses the defaults values for {@code maxPointsInLeafNode} (1024) and {@code maxMBSortInHeap} (16.0) */ /** Uses the defaults values for {@code maxPointsInLeafNode} (1024) and {@code maxMBSortInHeap} (16.0) */
public Lucene60PointWriter(SegmentWriteState writeState) throws IOException { public Lucene60PointsWriter(SegmentWriteState writeState) throws IOException {
this(writeState, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP); this(writeState, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
} }
@Override @Override
public void writeField(FieldInfo fieldInfo, PointReader values) throws IOException { public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(), try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
writeState.directory, writeState.directory,
@ -115,14 +115,14 @@ public class Lucene60PointWriter extends PointWriter implements Closeable {
@Override @Override
public void merge(MergeState mergeState) throws IOException { public void merge(MergeState mergeState) throws IOException {
for(PointReader reader : mergeState.pointReaders) { for(PointsReader reader : mergeState.pointsReaders) {
if (reader instanceof Lucene60PointReader == false) { if (reader instanceof Lucene60PointsReader == false) {
// We can only bulk merge when all to-be-merged segments use our format: // We can only bulk merge when all to-be-merged segments use our format:
super.merge(mergeState); super.merge(mergeState);
return; return;
} }
} }
for (PointReader reader : mergeState.pointReaders) { for (PointsReader reader : mergeState.pointsReaders) {
if (reader != null) { if (reader != null) {
reader.checkIntegrity(); reader.checkIntegrity();
} }
@ -145,14 +145,14 @@ public class Lucene60PointWriter extends PointWriter implements Closeable {
List<BKDReader> bkdReaders = new ArrayList<>(); List<BKDReader> bkdReaders = new ArrayList<>();
List<MergeState.DocMap> docMaps = new ArrayList<>(); List<MergeState.DocMap> docMaps = new ArrayList<>();
List<Integer> docIDBases = new ArrayList<>(); List<Integer> docIDBases = new ArrayList<>();
for(int i=0;i<mergeState.pointReaders.length;i++) { for(int i=0;i<mergeState.pointsReaders.length;i++) {
PointReader reader = mergeState.pointReaders[i]; PointsReader reader = mergeState.pointsReaders[i];
if (reader != null) { if (reader != null) {
// we confirmed this up above // we confirmed this up above
assert reader instanceof Lucene60PointReader; assert reader instanceof Lucene60PointsReader;
Lucene60PointReader reader60 = (Lucene60PointReader) reader; Lucene60PointsReader reader60 = (Lucene60PointsReader) reader;
// NOTE: we cannot just use the merged fieldInfo.number (instead of resolving to this // NOTE: we cannot just use the merged fieldInfo.number (instead of resolving to this
// reader's FieldInfo as we do below) because field numbers can easily be different // reader's FieldInfo as we do below) because field numbers can easily be different
@ -196,12 +196,12 @@ public class Lucene60PointWriter extends PointWriter implements Closeable {
String indexFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name, String indexFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name,
writeState.segmentSuffix, writeState.segmentSuffix,
Lucene60PointFormat.INDEX_EXTENSION); Lucene60PointsFormat.INDEX_EXTENSION);
// Write index file // Write index file
try (IndexOutput indexOut = writeState.directory.createOutput(indexFileName, writeState.context)) { try (IndexOutput indexOut = writeState.directory.createOutput(indexFileName, writeState.context)) {
CodecUtil.writeIndexHeader(indexOut, CodecUtil.writeIndexHeader(indexOut,
Lucene60PointFormat.META_CODEC_NAME, Lucene60PointsFormat.META_CODEC_NAME,
Lucene60PointFormat.INDEX_VERSION_CURRENT, Lucene60PointsFormat.INDEX_VERSION_CURRENT,
writeState.segmentInfo.getId(), writeState.segmentInfo.getId(),
writeState.segmentSuffix); writeState.segmentSuffix);
int count = indexFPs.size(); int count = indexFPs.size();

View File

@ -193,7 +193,7 @@
* An optional file indicating which documents are live. * An optional file indicating which documents are live.
* </li> * </li>
* <li> * <li>
* {@link org.apache.lucene.codecs.lucene60.Lucene60PointFormat Point values}. * {@link org.apache.lucene.codecs.lucene60.Lucene60PointsFormat Point values}.
* Optional pair of files, recording dimesionally indexed fields, to enable fast * Optional pair of files, recording dimesionally indexed fields, to enable fast
* numeric range filtering and large numeric values like BigInteger and BigDecimal (1D) * numeric range filtering and large numeric values like BigInteger and BigDecimal (1D)
* and geo shape intersection (2D, 3D). * and geo shape intersection (2D, 3D).
@ -322,7 +322,7 @@
* <td>Info about what files are live</td> * <td>Info about what files are live</td>
* </tr> * </tr>
* <tr> * <tr>
* <td>{@link org.apache.lucene.codecs.lucene60.Lucene60PointFormat Point values}</td> * <td>{@link org.apache.lucene.codecs.lucene60.Lucene60PointsFormat Point values}</td>
* <td>.dii, .dim</td> * <td>.dii, .dim</td>
* <td>Holds indexed points, if any</td> * <td>Holds indexed points, if any</td>
* </tr> * </tr>

View File

@ -33,9 +33,9 @@ import java.util.Locale;
import java.util.Map; import java.util.Map;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PointReader;
import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.NormsProducer; import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.StoredFieldsReader; import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader; import org.apache.lucene.codecs.TermVectorsReader;
@ -1687,9 +1687,9 @@ public final class CheckIndex implements Closeable {
Status.PointsStatus status = new Status.PointsStatus(); Status.PointsStatus status = new Status.PointsStatus();
try { try {
if (fieldInfos.hasPointValues()) { if (fieldInfos.hasPointValues()) {
PointReader values = reader.getPointReader(); PointsReader values = reader.getPointsReader();
if (values == null) { if (values == null) {
throw new RuntimeException("there are fields with points, but reader.getPointReader() is null"); throw new RuntimeException("there are fields with points, but reader.getPointsReader() is null");
} }
for (FieldInfo fieldInfo : fieldInfos) { for (FieldInfo fieldInfo : fieldInfos) {
if (fieldInfo.getPointDimensionCount() > 0) { if (fieldInfo.getPointDimensionCount() > 0) {

View File

@ -25,10 +25,10 @@ import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import org.apache.lucene.codecs.PointReader;
import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.NormsProducer; import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.StoredFieldsReader; import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader; import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.util.Accountable; import org.apache.lucene.util.Accountable;
@ -77,10 +77,10 @@ public abstract class CodecReader extends LeafReader implements Accountable {
public abstract FieldsProducer getPostingsReader(); public abstract FieldsProducer getPostingsReader();
/** /**
* Expert: retrieve underlying PointReader * Expert: retrieve underlying PointsReader
* @lucene.internal * @lucene.internal
*/ */
public abstract PointReader getPointReader(); public abstract PointsReader getPointsReader();
@Override @Override
public final void document(int docID, StoredFieldVisitor visitor) throws IOException { public final void document(int docID, StoredFieldVisitor visitor) throws IOException {
@ -323,8 +323,8 @@ public abstract class CodecReader extends LeafReader implements Accountable {
} }
// points // points
if (getPointReader() != null) { if (getPointsReader() != null) {
ramBytesUsed += getPointReader().ramBytesUsed(); ramBytesUsed += getPointsReader().ramBytesUsed();
} }
return ramBytesUsed; return ramBytesUsed;
@ -359,8 +359,8 @@ public abstract class CodecReader extends LeafReader implements Accountable {
} }
// points // points
if (getPointReader() != null) { if (getPointsReader() != null) {
resources.add(Accountables.namedAccountable("points", getPointReader())); resources.add(Accountables.namedAccountable("points", getPointsReader()));
} }
return Collections.unmodifiableList(resources); return Collections.unmodifiableList(resources);
@ -394,8 +394,8 @@ public abstract class CodecReader extends LeafReader implements Accountable {
} }
// points // points
if (getPointReader() != null) { if (getPointsReader() != null) {
getPointReader().checkIntegrity(); getPointsReader().checkIntegrity();
} }
} }
} }

View File

@ -23,12 +23,12 @@ import java.util.HashMap;
import java.util.Map; import java.util.Map;
import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.codecs.PointFormat;
import org.apache.lucene.codecs.PointWriter;
import org.apache.lucene.codecs.DocValuesConsumer; import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.NormsConsumer; import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.codecs.StoredFieldsWriter; import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.document.FieldType; import org.apache.lucene.document.FieldType;
import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.search.similarities.Similarity;
@ -149,7 +149,7 @@ final class DefaultIndexingChain extends DocConsumer {
/** Writes all buffered points. */ /** Writes all buffered points. */
private void writePoints(SegmentWriteState state) throws IOException { private void writePoints(SegmentWriteState state) throws IOException {
PointWriter pointWriter = null; PointsWriter pointsWriter = null;
boolean success = false; boolean success = false;
try { try {
for (int i=0;i<fieldHash.length;i++) { for (int i=0;i<fieldHash.length;i++) {
@ -160,16 +160,16 @@ final class DefaultIndexingChain extends DocConsumer {
// BUG // BUG
throw new AssertionError("segment=" + state.segmentInfo + ": field=\"" + perField.fieldInfo.name + "\" has no points but wrote them"); throw new AssertionError("segment=" + state.segmentInfo + ": field=\"" + perField.fieldInfo.name + "\" has no points but wrote them");
} }
if (pointWriter == null) { if (pointsWriter == null) {
// lazy init // lazy init
PointFormat fmt = state.segmentInfo.getCodec().pointFormat(); PointsFormat fmt = state.segmentInfo.getCodec().pointsFormat();
if (fmt == null) { if (fmt == null) {
throw new IllegalStateException("field=\"" + perField.fieldInfo.name + "\" was indexed as points but codec does not support points"); throw new IllegalStateException("field=\"" + perField.fieldInfo.name + "\" was indexed as points but codec does not support points");
} }
pointWriter = fmt.fieldsWriter(state); pointsWriter = fmt.fieldsWriter(state);
} }
perField.pointValuesWriter.flush(state, pointWriter); perField.pointValuesWriter.flush(state, pointsWriter);
perField.pointValuesWriter = null; perField.pointValuesWriter = null;
} else if (perField.fieldInfo.getPointDimensionCount() != 0) { } else if (perField.fieldInfo.getPointDimensionCount() != 0) {
// BUG // BUG
@ -178,15 +178,15 @@ final class DefaultIndexingChain extends DocConsumer {
perField = perField.next; perField = perField.next;
} }
} }
if (pointWriter != null) { if (pointsWriter != null) {
pointWriter.finish(); pointsWriter.finish();
} }
success = true; success = true;
} finally { } finally {
if (success) { if (success) {
IOUtils.close(pointWriter); IOUtils.close(pointsWriter);
} else { } else {
IOUtils.closeWhileHandlingException(pointWriter); IOUtils.closeWhileHandlingException(pointsWriter);
} }
} }
} }

View File

@ -49,7 +49,7 @@ public final class FieldInfo {
private long dvGen; private long dvGen;
/** If both of these are positive it means this field indexed points /** If both of these are positive it means this field indexed points
* (see {@link org.apache.lucene.codecs.PointFormat}). */ * (see {@link org.apache.lucene.codecs.PointsFormat}). */
private int pointDimensionCount; private int pointDimensionCount;
private int pointNumBytes; private int pointNumBytes;

View File

@ -19,10 +19,10 @@ package org.apache.lucene.index;
import java.util.Objects; import java.util.Objects;
import org.apache.lucene.codecs.PointReader;
import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.NormsProducer; import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.StoredFieldsReader; import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader; import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
@ -82,8 +82,8 @@ public class FilterCodecReader extends CodecReader {
} }
@Override @Override
public PointReader getPointReader() { public PointsReader getPointsReader() {
return in.getPointReader(); return in.getPointsReader();
} }
@Override @Override

View File

@ -300,7 +300,7 @@ public abstract class LeafReader extends IndexReader {
*/ */
public abstract Bits getLiveDocs(); public abstract Bits getLiveDocs();
/** Returns the {@link org.apache.lucene.codecs.PointReader} used for numeric or /** Returns the {@link PointValues} used for numeric or
* spatial searches, or null if there are no point fields. */ * spatial searches, or null if there are no point fields. */
public abstract PointValues getPointValues(); public abstract PointValues getPointValues();

View File

@ -20,10 +20,10 @@ package org.apache.lucene.index;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
import org.apache.lucene.codecs.PointReader;
import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.NormsProducer; import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.StoredFieldsReader; import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader; import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
@ -67,7 +67,7 @@ public class MergeState {
public final FieldsProducer[] fieldsProducers; public final FieldsProducer[] fieldsProducers;
/** Point readers to merge */ /** Point readers to merge */
public final PointReader[] pointReaders; public final PointsReader[] pointsReaders;
/** New docID base per reader. */ /** New docID base per reader. */
public final int[] docBase; public final int[] docBase;
@ -90,7 +90,7 @@ public class MergeState {
storedFieldsReaders = new StoredFieldsReader[numReaders]; storedFieldsReaders = new StoredFieldsReader[numReaders];
termVectorsReaders = new TermVectorsReader[numReaders]; termVectorsReaders = new TermVectorsReader[numReaders];
docValuesProducers = new DocValuesProducer[numReaders]; docValuesProducers = new DocValuesProducer[numReaders];
pointReaders = new PointReader[numReaders]; pointsReaders = new PointsReader[numReaders];
fieldInfos = new FieldInfos[numReaders]; fieldInfos = new FieldInfos[numReaders];
liveDocs = new Bits[numReaders]; liveDocs = new Bits[numReaders];
@ -122,9 +122,9 @@ public class MergeState {
} }
fieldsProducers[i] = reader.getPostingsReader().getMergeInstance(); fieldsProducers[i] = reader.getPostingsReader().getMergeInstance();
pointReaders[i] = reader.getPointReader(); pointsReaders[i] = reader.getPointsReader();
if (pointReaders[i] != null) { if (pointsReaders[i] != null) {
pointReaders[i] = pointReaders[i].getMergeInstance(); pointsReaders[i] = pointsReaders[i].getMergeInstance();
} }
} }

View File

@ -18,8 +18,8 @@ package org.apache.lucene.index;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.codecs.PointReader; import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointWriter; import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.ByteBlockPool; import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
@ -60,10 +60,10 @@ class PointValuesWriter {
numDocs++; numDocs++;
} }
public void flush(SegmentWriteState state, PointWriter writer) throws IOException { public void flush(SegmentWriteState state, PointsWriter writer) throws IOException {
writer.writeField(fieldInfo, writer.writeField(fieldInfo,
new PointReader() { new PointsReader() {
@Override @Override
public void intersect(String fieldName, IntersectVisitor visitor) throws IOException { public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
if (fieldName.equals(fieldInfo.name) == false) { if (fieldName.equals(fieldInfo.name) == false) {

View File

@ -24,9 +24,9 @@ import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PointReader;
import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.NormsProducer; import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.StoredFieldsReader; import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader; import org.apache.lucene.codecs.TermVectorsReader;
@ -54,7 +54,7 @@ final class SegmentCoreReaders {
final StoredFieldsReader fieldsReaderOrig; final StoredFieldsReader fieldsReaderOrig;
final TermVectorsReader termVectorsReaderOrig; final TermVectorsReader termVectorsReaderOrig;
final PointReader pointReader; final PointsReader pointsReader;
final Directory cfsReader; final Directory cfsReader;
/** /**
* fieldinfos for this core: means gen=-1. * fieldinfos for this core: means gen=-1.
@ -125,9 +125,9 @@ final class SegmentCoreReaders {
} }
if (coreFieldInfos.hasPointValues()) { if (coreFieldInfos.hasPointValues()) {
pointReader = codec.pointFormat().fieldsReader(segmentReadState); pointsReader = codec.pointsFormat().fieldsReader(segmentReadState);
} else { } else {
pointReader = null; pointsReader = null;
} }
success = true; success = true;
} finally { } finally {
@ -157,7 +157,7 @@ final class SegmentCoreReaders {
Throwable th = null; Throwable th = null;
try { try {
IOUtils.close(termVectorsLocal, fieldsReaderLocal, fields, termVectorsReaderOrig, fieldsReaderOrig, IOUtils.close(termVectorsLocal, fieldsReaderLocal, fields, termVectorsReaderOrig, fieldsReaderOrig,
cfsReader, normsProducer, pointReader); cfsReader, normsProducer, pointsReader);
} catch (Throwable throwable) { } catch (Throwable throwable) {
th = throwable; th = throwable;
} finally { } finally {

View File

@ -21,10 +21,10 @@ import java.io.IOException;
import java.util.List; import java.util.List;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PointWriter;
import org.apache.lucene.codecs.DocValuesConsumer; import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.NormsConsumer; import org.apache.lucene.codecs.NormsConsumer;
import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.codecs.StoredFieldsWriter; import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.TermVectorsWriter; import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
@ -164,7 +164,7 @@ final class SegmentMerger {
} }
private void mergePoints(SegmentWriteState segmentWriteState) throws IOException { private void mergePoints(SegmentWriteState segmentWriteState) throws IOException {
try (PointWriter writer = codec.pointFormat().fieldsWriter(segmentWriteState)) { try (PointsWriter writer = codec.pointsFormat().fieldsWriter(segmentWriteState)) {
writer.merge(mergeState); writer.merge(mergeState);
} }
} }

View File

@ -21,11 +21,11 @@ import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PointReader;
import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.FieldInfosFormat; import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.NormsProducer; import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.StoredFieldsReader; import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader; import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
@ -220,7 +220,7 @@ public final class SegmentReader extends CodecReader {
@Override @Override
public PointValues getPointValues() { public PointValues getPointValues() {
ensureOpen(); ensureOpen();
return core.pointReader; return core.pointsReader;
} }
@Override @Override
@ -242,9 +242,9 @@ public final class SegmentReader extends CodecReader {
} }
@Override @Override
public PointReader getPointReader() { public PointsReader getPointsReader() {
ensureOpen(); ensureOpen();
return core.pointReader; return core.pointsReader;
} }
@Override @Override

View File

@ -20,10 +20,10 @@ package org.apache.lucene.index;
import java.io.IOException; import java.io.IOException;
import java.util.Iterator; import java.util.Iterator;
import org.apache.lucene.codecs.PointReader;
import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.NormsProducer; import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.StoredFieldsReader; import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader; import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
@ -97,7 +97,7 @@ public final class SlowCodecReaderWrapper {
} }
@Override @Override
public PointReader getPointReader() { public PointsReader getPointsReader() {
return pointValuesToReader(reader.getPointValues()); return pointValuesToReader(reader.getPointValues());
} }
@ -129,11 +129,11 @@ public final class SlowCodecReaderWrapper {
} }
} }
private static PointReader pointValuesToReader(PointValues values) { private static PointsReader pointValuesToReader(PointValues values) {
if (values == null) { if (values == null) {
return null; return null;
} }
return new PointReader() { return new PointsReader() {
@Override @Override
public void intersect(String fieldName, IntersectVisitor visitor) throws IOException { public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
values.intersect(fieldName, visitor); values.intersect(fieldName, visitor);

View File

@ -21,21 +21,21 @@ import java.io.IOException;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.PointFormat; import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointReader; import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointWriter; import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.index.BasePointFormatTestCase; import org.apache.lucene.index.BasePointsFormatTestCase;
import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.TestUtil; import org.apache.lucene.util.TestUtil;
/** /**
* Tests Lucene60PointFormat * Tests Lucene60PointsFormat
*/ */
public class TestLucene60PointFormat extends BasePointFormatTestCase { public class TestLucene60PointsFormat extends BasePointsFormatTestCase {
private final Codec codec; private final Codec codec;
public TestLucene60PointFormat() { public TestLucene60PointsFormat() {
// standard issue // standard issue
Codec defaultCodec = TestUtil.getDefaultCodec(); Codec defaultCodec = TestUtil.getDefaultCodec();
if (random().nextBoolean()) { if (random().nextBoolean()) {
@ -43,22 +43,22 @@ public class TestLucene60PointFormat extends BasePointFormatTestCase {
int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 500); int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 500);
double maxMBSortInHeap = 3.0 + (3*random().nextDouble()); double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
if (VERBOSE) { if (VERBOSE) {
System.out.println("TEST: using Lucene60PointFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap); System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
} }
// sneaky impersonation! // sneaky impersonation!
codec = new FilterCodec(defaultCodec.getName(), defaultCodec) { codec = new FilterCodec(defaultCodec.getName(), defaultCodec) {
@Override @Override
public PointFormat pointFormat() { public PointsFormat pointsFormat() {
return new PointFormat() { return new PointsFormat() {
@Override @Override
public PointWriter fieldsWriter(SegmentWriteState writeState) throws IOException { public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
return new Lucene60PointWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap); return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
} }
@Override @Override
public PointReader fieldsReader(SegmentReadState readState) throws IOException { public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
return new Lucene60PointReader(readState); return new Lucene60PointsReader(readState);
} }
}; };
} }

View File

@ -22,11 +22,11 @@ import java.io.IOException;
import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.PointFormat; import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointReader; import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointWriter; import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.codecs.lucene60.Lucene60PointReader; import org.apache.lucene.codecs.lucene60.Lucene60PointsReader;
import org.apache.lucene.codecs.lucene60.Lucene60PointWriter; import org.apache.lucene.codecs.lucene60.Lucene60PointsWriter;
import org.apache.lucene.document.BinaryPoint; import org.apache.lucene.document.BinaryPoint;
import org.apache.lucene.document.Document; import org.apache.lucene.document.Document;
import org.apache.lucene.document.DoublePoint; import org.apache.lucene.document.DoublePoint;

View File

@ -32,11 +32,11 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.PointFormat; import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointReader; import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointWriter; import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.codecs.lucene60.Lucene60PointReader; import org.apache.lucene.codecs.lucene60.Lucene60PointsReader;
import org.apache.lucene.codecs.lucene60.Lucene60PointWriter; import org.apache.lucene.codecs.lucene60.Lucene60PointsWriter;
import org.apache.lucene.document.BinaryPoint; import org.apache.lucene.document.BinaryPoint;
import org.apache.lucene.document.Document; import org.apache.lucene.document.Document;
import org.apache.lucene.document.DoublePoint; import org.apache.lucene.document.DoublePoint;
@ -1157,21 +1157,21 @@ public class TestPointQueries extends LuceneTestCase {
int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048); int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
double maxMBSortInHeap = 4.0 + (3*random().nextDouble()); double maxMBSortInHeap = 4.0 + (3*random().nextDouble());
if (VERBOSE) { if (VERBOSE) {
System.out.println("TEST: using Lucene60PointFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap); System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
} }
return new FilterCodec("Lucene60", Codec.getDefault()) { return new FilterCodec("Lucene60", Codec.getDefault()) {
@Override @Override
public PointFormat pointFormat() { public PointsFormat pointsFormat() {
return new PointFormat() { return new PointsFormat() {
@Override @Override
public PointWriter fieldsWriter(SegmentWriteState writeState) throws IOException { public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
return new Lucene60PointWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap); return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
} }
@Override @Override
public PointReader fieldsReader(SegmentReadState readState) throws IOException { public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
return new Lucene60PointReader(readState); return new Lucene60PointsReader(readState);
} }
}; };
} }

View File

@ -20,11 +20,11 @@ import java.io.IOException;
import java.util.BitSet; import java.util.BitSet;
import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.PointFormat; import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointReader; import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointWriter; import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.codecs.lucene60.Lucene60PointReader; import org.apache.lucene.codecs.lucene60.Lucene60PointsReader;
import org.apache.lucene.codecs.lucene60.Lucene60PointWriter; import org.apache.lucene.codecs.lucene60.Lucene60PointsWriter;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.RandomIndexWriter;
@ -36,8 +36,8 @@ import org.apache.lucene.search.Sort;
import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocs;
import org.apache.lucene.spatial.util.GeoDistanceUtils; import org.apache.lucene.spatial.util.GeoDistanceUtils;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.bkd.BKDWriter; import org.apache.lucene.util.bkd.BKDWriter;
/** Simple tests for {@link LatLonPoint#newDistanceQuery} */ /** Simple tests for {@link LatLonPoint#newDistanceQuery} */
@ -119,16 +119,16 @@ public class TestLatLonPointDistanceQuery extends LuceneTestCase {
int pointsInLeaf = 2 + random().nextInt(4); int pointsInLeaf = 2 + random().nextInt(4);
iwc.setCodec(new FilterCodec("Lucene60", TestUtil.getDefaultCodec()) { iwc.setCodec(new FilterCodec("Lucene60", TestUtil.getDefaultCodec()) {
@Override @Override
public PointFormat pointFormat() { public PointsFormat pointsFormat() {
return new PointFormat() { return new PointsFormat() {
@Override @Override
public PointWriter fieldsWriter(SegmentWriteState writeState) throws IOException { public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
return new Lucene60PointWriter(writeState, pointsInLeaf, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP); return new Lucene60PointsWriter(writeState, pointsInLeaf, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
} }
@Override @Override
public PointReader fieldsReader(SegmentReadState readState) throws IOException { public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
return new Lucene60PointReader(readState); return new Lucene60PointsReader(readState);
} }
}; };
} }

View File

@ -27,12 +27,12 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PointFormat;
import org.apache.lucene.codecs.PointReader;
import org.apache.lucene.codecs.PointWriter;
import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.lucene60.Lucene60PointReader; import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.lucene60.Lucene60PointWriter; import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.codecs.lucene60.Lucene60PointsReader;
import org.apache.lucene.codecs.lucene60.Lucene60PointsWriter;
import org.apache.lucene.document.Document; import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field; import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.NumericDocValuesField;
@ -76,21 +76,21 @@ public class TestGeo3DPoint extends LuceneTestCase {
int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048); int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
double maxMBSortInHeap = 3.0 + (3*random().nextDouble()); double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
if (VERBOSE) { if (VERBOSE) {
System.out.println("TEST: using Lucene60PointFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap); System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
} }
return new FilterCodec("Lucene60", Codec.getDefault()) { return new FilterCodec("Lucene60", Codec.getDefault()) {
@Override @Override
public PointFormat pointFormat() { public PointsFormat pointsFormat() {
return new PointFormat() { return new PointsFormat() {
@Override @Override
public PointWriter fieldsWriter(SegmentWriteState writeState) throws IOException { public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
return new Lucene60PointWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap); return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
} }
@Override @Override
public PointReader fieldsReader(SegmentReadState readState) throws IOException { public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
return new Lucene60PointReader(readState); return new Lucene60PointsReader(readState);
} }
}; };
} }

View File

@ -16,11 +16,11 @@
*/ */
package org.apache.lucene.codecs.asserting; package org.apache.lucene.codecs.asserting;
import org.apache.lucene.codecs.PointFormat;
import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.LiveDocsFormat; import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat; import org.apache.lucene.codecs.TermVectorsFormat;
@ -53,7 +53,7 @@ public class AssertingCodec extends FilterCodec {
private final LiveDocsFormat liveDocs = new AssertingLiveDocsFormat(); private final LiveDocsFormat liveDocs = new AssertingLiveDocsFormat();
private final PostingsFormat defaultFormat = new AssertingPostingsFormat(); private final PostingsFormat defaultFormat = new AssertingPostingsFormat();
private final DocValuesFormat defaultDVFormat = new AssertingDocValuesFormat(); private final DocValuesFormat defaultDVFormat = new AssertingDocValuesFormat();
private final PointFormat pointFormat = new AssertingPointFormat(); private final PointsFormat pointsFormat = new AssertingPointsFormat();
public AssertingCodec() { public AssertingCodec() {
super("Asserting", TestUtil.getDefaultCodec()); super("Asserting", TestUtil.getDefaultCodec());
@ -90,8 +90,8 @@ public class AssertingCodec extends FilterCodec {
} }
@Override @Override
public PointFormat pointFormat() { public PointsFormat pointsFormat() {
return pointFormat; return pointsFormat;
} }
@Override @Override

View File

@ -20,9 +20,9 @@ import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import org.apache.lucene.codecs.PointFormat; import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointReader; import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointWriter; import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.MergeState; import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.PointValues.IntersectVisitor; import org.apache.lucene.index.PointValues.IntersectVisitor;
@ -39,32 +39,32 @@ import org.apache.lucene.util.TestUtil;
* Just like the default point format but with additional asserts. * Just like the default point format but with additional asserts.
*/ */
public final class AssertingPointFormat extends PointFormat { public final class AssertingPointsFormat extends PointsFormat {
private final PointFormat in; private final PointsFormat in;
/** Create a new AssertingPointFormat */ /** Create a new AssertingPointsFormat */
public AssertingPointFormat() { public AssertingPointsFormat() {
this(TestUtil.getDefaultCodec().pointFormat()); this(TestUtil.getDefaultCodec().pointsFormat());
} }
/** /**
* Expert: Create an AssertingPointFormat. * Expert: Create an AssertingPointsFormat.
* This is only intended to pass special parameters for testing. * This is only intended to pass special parameters for testing.
*/ */
// TODO: can we randomize this a cleaner way? e.g. stored fields and vectors do // TODO: can we randomize this a cleaner way? e.g. stored fields and vectors do
// this with a separate codec... // this with a separate codec...
public AssertingPointFormat(PointFormat in) { public AssertingPointsFormat(PointsFormat in) {
this.in = in; this.in = in;
} }
@Override @Override
public PointWriter fieldsWriter(SegmentWriteState state) throws IOException { public PointsWriter fieldsWriter(SegmentWriteState state) throws IOException {
return new AssertingPointWriter(state, in.fieldsWriter(state)); return new AssertingPointsWriter(state, in.fieldsWriter(state));
} }
@Override @Override
public PointReader fieldsReader(SegmentReadState state) throws IOException { public PointsReader fieldsReader(SegmentReadState state) throws IOException {
return new AssertingPointReader(state.segmentInfo.maxDoc(), in.fieldsReader(state)); return new AssertingPointsReader(state.segmentInfo.maxDoc(), in.fieldsReader(state));
} }
/** Validates in the 1D case that all points are visited in order, and point values are in bounds of the last cell checked */ /** Validates in the 1D case that all points are visited in order, and point values are in bounds of the last cell checked */
@ -144,11 +144,11 @@ public final class AssertingPointFormat extends PointFormat {
} }
} }
static class AssertingPointReader extends PointReader { static class AssertingPointsReader extends PointsReader {
private final PointReader in; private final PointsReader in;
private final int maxDoc; private final int maxDoc;
AssertingPointReader(int maxDoc, PointReader in) { AssertingPointsReader(int maxDoc, PointsReader in) {
this.in = in; this.in = in;
this.maxDoc = maxDoc; this.maxDoc = maxDoc;
// do a few simple checks on init // do a few simple checks on init
@ -189,8 +189,8 @@ public final class AssertingPointFormat extends PointFormat {
} }
@Override @Override
public PointReader getMergeInstance() throws IOException { public PointsReader getMergeInstance() throws IOException {
return new AssertingPointReader(maxDoc, in.getMergeInstance()); return new AssertingPointsReader(maxDoc, in.getMergeInstance());
} }
@Override @Override
@ -242,15 +242,15 @@ public final class AssertingPointFormat extends PointFormat {
} }
} }
static class AssertingPointWriter extends PointWriter { static class AssertingPointsWriter extends PointsWriter {
private final PointWriter in; private final PointsWriter in;
AssertingPointWriter(SegmentWriteState writeState, PointWriter in) { AssertingPointsWriter(SegmentWriteState writeState, PointsWriter in) {
this.in = in; this.in = in;
} }
@Override @Override
public void writeField(FieldInfo fieldInfo, PointReader values) throws IOException { public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
if (fieldInfo.getPointDimensionCount() == 0) { if (fieldInfo.getPointDimensionCount() == 0) {
throw new IllegalArgumentException("writing field=\"" + fieldInfo.name + "\" but pointDimensionalCount is 0"); throw new IllegalArgumentException("writing field=\"" + fieldInfo.name + "\" but pointDimensionalCount is 0");
} }

View File

@ -25,7 +25,7 @@ import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.LiveDocsFormat; import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PointFormat; import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat; import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.StoredFieldsFormat;
@ -92,8 +92,8 @@ public class CrankyCodec extends FilterCodec {
} }
@Override @Override
public PointFormat pointFormat() { public PointsFormat pointsFormat() {
return new CrankyPointFormat(delegate.pointFormat(), random); return new CrankyPointsFormat(delegate.pointsFormat(), random);
} }
@Override @Override

View File

@ -19,9 +19,9 @@ package org.apache.lucene.codecs.cranky;
import java.io.IOException; import java.io.IOException;
import java.util.Random; import java.util.Random;
import org.apache.lucene.codecs.PointFormat; import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointReader; import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointWriter; import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.MergeState; import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentInfo;
@ -30,36 +30,36 @@ import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IOContext;
class CrankyPointFormat extends PointFormat { class CrankyPointsFormat extends PointsFormat {
PointFormat delegate; PointsFormat delegate;
Random random; Random random;
CrankyPointFormat(PointFormat delegate, Random random) { CrankyPointsFormat(PointsFormat delegate, Random random) {
this.delegate = delegate; this.delegate = delegate;
this.random = random; this.random = random;
} }
@Override @Override
public PointWriter fieldsWriter(SegmentWriteState state) throws IOException { public PointsWriter fieldsWriter(SegmentWriteState state) throws IOException {
return new CrankyPointWriter(delegate.fieldsWriter(state), random); return new CrankyPointsWriter(delegate.fieldsWriter(state), random);
} }
@Override @Override
public PointReader fieldsReader(SegmentReadState state) throws IOException { public PointsReader fieldsReader(SegmentReadState state) throws IOException {
return new CrankyPointReader(delegate.fieldsReader(state), random); return new CrankyPointsReader(delegate.fieldsReader(state), random);
} }
static class CrankyPointWriter extends PointWriter { static class CrankyPointsWriter extends PointsWriter {
final PointWriter delegate; final PointsWriter delegate;
final Random random; final Random random;
public CrankyPointWriter(PointWriter delegate, Random random) { public CrankyPointsWriter(PointsWriter delegate, Random random) {
this.delegate = delegate; this.delegate = delegate;
this.random = random; this.random = random;
} }
@Override @Override
public void writeField(FieldInfo fieldInfo, PointReader values) throws IOException { public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
if (random.nextInt(100) == 0) { if (random.nextInt(100) == 0) {
throw new IOException("Fake IOException"); throw new IOException("Fake IOException");
} }
@ -97,10 +97,10 @@ class CrankyPointFormat extends PointFormat {
} }
} }
static class CrankyPointReader extends PointReader { static class CrankyPointsReader extends PointsReader {
final PointReader delegate; final PointsReader delegate;
final Random random; final Random random;
public CrankyPointReader(PointReader delegate, Random random) { public CrankyPointsReader(PointsReader delegate, Random random) {
this.delegate = delegate; this.delegate = delegate;
this.random = random; this.random = random;
} }

View File

@ -47,11 +47,11 @@ import org.apache.lucene.util.TestUtil;
* Abstract class to do basic tests for a points format. * Abstract class to do basic tests for a points format.
* NOTE: This test focuses on the points impl, nothing else. * NOTE: This test focuses on the points impl, nothing else.
* The [stretch] goal is for this test to be * The [stretch] goal is for this test to be
* so thorough in testing a new PointFormat that if this * so thorough in testing a new PointsFormat that if this
* test passes, then all Lucene/Solr tests should also pass. Ie, * test passes, then all Lucene/Solr tests should also pass. Ie,
* if there is some bug in a given PointFormat that this * if there is some bug in a given PointsFormat that this
* test fails to catch then this test needs to be improved! */ * test fails to catch then this test needs to be improved! */
public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCase { public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCase {
@Override @Override
protected void addRandomFields(Document doc) { protected void addRandomFields(Document doc) {

View File

@ -28,21 +28,21 @@ import java.util.Random;
import java.util.Set; import java.util.Set;
import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PointFormat; import org.apache.lucene.codecs.PointsFormat;
import org.apache.lucene.codecs.PointReader; import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.PointWriter; import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.asserting.AssertingCodec; import org.apache.lucene.codecs.asserting.AssertingCodec;
import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat; import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat;
import org.apache.lucene.codecs.asserting.AssertingPointFormat; import org.apache.lucene.codecs.asserting.AssertingPointsFormat;
import org.apache.lucene.codecs.asserting.AssertingPostingsFormat; import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
import org.apache.lucene.codecs.blockterms.LuceneFixedGap; import org.apache.lucene.codecs.blockterms.LuceneFixedGap;
import org.apache.lucene.codecs.blockterms.LuceneVarGapDocFreqInterval; import org.apache.lucene.codecs.blockterms.LuceneVarGapDocFreqInterval;
import org.apache.lucene.codecs.blockterms.LuceneVarGapFixedInterval; import org.apache.lucene.codecs.blockterms.LuceneVarGapFixedInterval;
import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat; import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat;
import org.apache.lucene.codecs.bloom.TestBloomFilteredLucenePostings; import org.apache.lucene.codecs.bloom.TestBloomFilteredLucenePostings;
import org.apache.lucene.codecs.lucene60.Lucene60PointReader; import org.apache.lucene.codecs.lucene60.Lucene60PointsReader;
import org.apache.lucene.codecs.lucene60.Lucene60PointWriter; import org.apache.lucene.codecs.lucene60.Lucene60PointsWriter;
import org.apache.lucene.codecs.memory.DirectDocValuesFormat; import org.apache.lucene.codecs.memory.DirectDocValuesFormat;
import org.apache.lucene.codecs.memory.DirectPostingsFormat; import org.apache.lucene.codecs.memory.DirectPostingsFormat;
import org.apache.lucene.codecs.memory.FSTOrdPostingsFormat; import org.apache.lucene.codecs.memory.FSTOrdPostingsFormat;
@ -95,16 +95,16 @@ public class RandomCodec extends AssertingCodec {
private final double maxMBSortInHeap; private final double maxMBSortInHeap;
@Override @Override
public PointFormat pointFormat() { public PointsFormat pointsFormat() {
return new AssertingPointFormat(new PointFormat() { return new AssertingPointsFormat(new PointsFormat() {
@Override @Override
public PointWriter fieldsWriter(SegmentWriteState writeState) throws IOException { public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
return new Lucene60PointWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap); return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
} }
@Override @Override
public PointReader fieldsReader(SegmentReadState readState) throws IOException { public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
return new Lucene60PointReader(readState); return new Lucene60PointsReader(readState);
} }
}); });
} }

View File

@ -17,10 +17,10 @@
package org.apache.lucene.codecs.asserting; package org.apache.lucene.codecs.asserting;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.BasePointFormatTestCase; import org.apache.lucene.index.BasePointsFormatTestCase;
/** Test AssertingPointFormat directly */ /** Test AssertingPointsFormat directly */
public class TestAssertingPointFormat extends BasePointFormatTestCase { public class TestAssertingPointsFormat extends BasePointsFormatTestCase {
private final Codec codec = new AssertingCodec(); private final Codec codec = new AssertingCodec();
@Override @Override