mirror of
https://github.com/apache/lucene.git
synced 2025-02-08 02:58:58 +00:00
Merge remote-tracking branch 'origin/branch_6x' into branch_6x
This commit is contained in:
commit
c187c46437
@ -187,7 +187,7 @@ def pushLocal(version, root, rev, rcNum, localDir):
|
||||
os.remove('%s/solr/package/solr.tar.bz2' % root)
|
||||
|
||||
print(' KEYS')
|
||||
run('wget http://people.apache.org/keys/group/lucene.asc')
|
||||
run('wget http://home.apache.org/keys/group/lucene.asc')
|
||||
os.rename('lucene.asc', 'KEYS')
|
||||
run('chmod a+r-w KEYS')
|
||||
run('cp KEYS ../lucene')
|
||||
|
@ -118,6 +118,9 @@ API Changes
|
||||
* LUCENE-7064: MultiPhraseQuery is now immutable and should be constructed
|
||||
with MultiPhraseQuery.Builder. (Luc Vanlerberghe via Adrien Grand)
|
||||
|
||||
* LUCENE-7072: Geo3DPoint always uses WGS84 planet model.
|
||||
(Robert Muir, Mike McCandless)
|
||||
|
||||
Optimizations
|
||||
|
||||
* LUCENE-6891: Use prefix coding when writing points in
|
||||
@ -135,6 +138,9 @@ Optimizations
|
||||
* LUCENE-7050: TermsQuery is now cached more aggressively by the default
|
||||
query caching policy. (Adrien Grand)
|
||||
|
||||
* LUCENE-7066: PointRangeQuery got optimized for the case that all documents
|
||||
have a value and all points from the segment match. (Adrien Grand)
|
||||
|
||||
Changes in Runtime Behavior
|
||||
|
||||
* LUCENE-6789: IndexSearcher's default Similarity is changed to BM25Similarity.
|
||||
|
@ -21,12 +21,12 @@ import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.CompoundFormat;
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
@ -154,8 +154,8 @@ public class Lucene50Codec extends Codec {
|
||||
}
|
||||
|
||||
@Override
|
||||
public final PointFormat pointFormat() {
|
||||
return PointFormat.EMPTY;
|
||||
public final PointsFormat pointsFormat() {
|
||||
return PointsFormat.EMPTY;
|
||||
}
|
||||
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
|
||||
|
@ -21,12 +21,12 @@ import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.CompoundFormat;
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfoFormat;
|
||||
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.Lucene50LiveDocsFormat;
|
||||
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;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
@ -160,8 +160,8 @@ public class Lucene53Codec extends Codec {
|
||||
}
|
||||
|
||||
@Override
|
||||
public final PointFormat pointFormat() {
|
||||
return PointFormat.EMPTY;
|
||||
public final PointsFormat pointsFormat() {
|
||||
return PointsFormat.EMPTY;
|
||||
}
|
||||
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
|
||||
|
@ -21,12 +21,12 @@ import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.CompoundFormat;
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfoFormat;
|
||||
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.Lucene50LiveDocsFormat;
|
||||
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;
|
||||
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene53.Lucene53NormsFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
|
||||
@ -160,8 +160,8 @@ public class Lucene54Codec extends Codec {
|
||||
}
|
||||
|
||||
@Override
|
||||
public final PointFormat pointFormat() {
|
||||
return PointFormat.EMPTY;
|
||||
public final PointsFormat pointsFormat() {
|
||||
return PointsFormat.EMPTY;
|
||||
}
|
||||
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
|
||||
|
@ -395,7 +395,7 @@
|
||||
|
||||
<!-- copy KEYS to the release folder -->
|
||||
<target name="-dist-keys">
|
||||
<get src="http://people.apache.org/keys/group/lucene.asc"
|
||||
<get src="http://home.apache.org/keys/group/lucene.asc"
|
||||
dest="${dist.dir}/KEYS"/>
|
||||
</target>
|
||||
|
||||
|
@ -27,9 +27,9 @@ import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.bkd.BKDReader;
|
||||
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.BLOCK_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.BLOCK_DOC_ID;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.BLOCK_VALUE;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.BLOCK_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.BLOCK_DOC_ID;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.BLOCK_VALUE;
|
||||
|
||||
class SimpleTextBKDReader extends BKDReader {
|
||||
|
||||
|
@ -19,11 +19,11 @@ package org.apache.lucene.codecs.simpletext;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.CompoundFormat;
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
@ -45,7 +45,7 @@ public final class SimpleTextCodec extends Codec {
|
||||
private final LiveDocsFormat liveDocs = new SimpleTextLiveDocsFormat();
|
||||
private final DocValuesFormat dvFormat = new SimpleTextDocValuesFormat();
|
||||
private final CompoundFormat compoundFormat = new SimpleTextCompoundFormat();
|
||||
private final PointFormat pointFormat = new SimpleTextPointFormat();
|
||||
private final PointsFormat pointsFormat = new SimpleTextPointsFormat();
|
||||
|
||||
public SimpleTextCodec() {
|
||||
super("SimpleText");
|
||||
@ -97,7 +97,7 @@ public final class SimpleTextCodec extends Codec {
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointFormat pointFormat() {
|
||||
return pointFormat;
|
||||
public PointsFormat pointsFormat() {
|
||||
return pointsFormat;
|
||||
}
|
||||
}
|
||||
|
@ -19,9 +19,9 @@ package org.apache.lucene.codecs.simpletext;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PointsReader;
|
||||
import org.apache.lucene.codecs.PointsWriter;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
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.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
public final class SimpleTextPointFormat extends PointFormat {
|
||||
public final class SimpleTextPointsFormat extends PointsFormat {
|
||||
|
||||
@Override
|
||||
public PointWriter fieldsWriter(SegmentWriteState state) throws IOException {
|
||||
return new SimpleTextPointWriter(state);
|
||||
public PointsWriter fieldsWriter(SegmentWriteState state) throws IOException {
|
||||
return new SimpleTextPointsWriter(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointReader fieldsReader(SegmentReadState state) throws IOException {
|
||||
return new SimpleTextPointReader(state);
|
||||
public PointsReader fieldsReader(SegmentReadState state) throws IOException {
|
||||
return new SimpleTextPointsReader(state);
|
||||
}
|
||||
|
||||
/** Extension of points data file */
|
@ -22,7 +22,7 @@ import java.nio.charset.StandardCharsets;
|
||||
import java.util.HashMap;
|
||||
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.FieldInfo;
|
||||
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.bkd.BKDReader;
|
||||
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.BLOCK_FP;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.BYTES_PER_DIM;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.DOC_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.FIELD_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.FIELD_FP;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.FIELD_FP_NAME;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.INDEX_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.MAX_LEAF_POINTS;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.MAX_VALUE;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.MIN_VALUE;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.NUM_DIMS;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.POINT_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.SPLIT_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.SPLIT_DIM;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointWriter.SPLIT_VALUE;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.BLOCK_FP;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.BYTES_PER_DIM;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.DOC_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.FIELD_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.FIELD_FP;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.FIELD_FP_NAME;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.INDEX_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.MAX_LEAF_POINTS;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.MAX_VALUE;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.MIN_VALUE;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.NUM_DIMS;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.POINT_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.SPLIT_COUNT;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.SPLIT_DIM;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextPointsWriter.SPLIT_VALUE;
|
||||
|
||||
class SimpleTextPointReader extends PointReader {
|
||||
class SimpleTextPointsReader extends PointsReader {
|
||||
|
||||
private final IndexInput dataIn;
|
||||
final SegmentReadState readState;
|
||||
final Map<String,BKDReader> readers = new HashMap<>();
|
||||
final BytesRefBuilder scratch = new BytesRefBuilder();
|
||||
|
||||
public SimpleTextPointReader(SegmentReadState readState) throws IOException {
|
||||
public SimpleTextPointsReader(SegmentReadState readState) throws IOException {
|
||||
// Initialize readers now:
|
||||
|
||||
// Read index:
|
||||
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)) {
|
||||
readLine(in);
|
||||
int count = parseInt(FIELD_COUNT);
|
||||
@ -81,7 +81,7 @@ class SimpleTextPointReader extends PointReader {
|
||||
}
|
||||
|
||||
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);
|
||||
try {
|
||||
for(Map.Entry<String,Long> ent : fieldToFileOffset.entrySet()) {
|
||||
@ -98,7 +98,7 @@ class SimpleTextPointReader extends PointReader {
|
||||
}
|
||||
|
||||
private BKDReader initReader(long fp) throws IOException {
|
||||
// NOTE: matches what writeIndex does in SimpleTextPointWriter
|
||||
// NOTE: matches what writeIndex does in SimpleTextPointsWriter
|
||||
dataIn.seek(fp);
|
||||
readLine(dataIn);
|
||||
int numDims = parseInt(NUM_DIMS);
|
||||
@ -231,7 +231,7 @@ class SimpleTextPointReader extends PointReader {
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "SimpleTextPointReader(segment=" + readState.segmentInfo.name + " maxDoc=" + readState.segmentInfo.maxDoc() + ")";
|
||||
return "SimpleTextPointsReader(segment=" + readState.segmentInfo.name + " maxDoc=" + readState.segmentInfo.maxDoc() + ")";
|
||||
}
|
||||
|
||||
@Override
|
@ -21,19 +21,19 @@ import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.codecs.PointsReader;
|
||||
import org.apache.lucene.codecs.PointsWriter;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
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 BYTES_PER_DIM = new BytesRef("bytes per dim ");
|
||||
@ -53,20 +53,21 @@ class SimpleTextPointWriter extends PointWriter {
|
||||
final static BytesRef MAX_VALUE = new BytesRef("max value ");
|
||||
final static BytesRef POINT_COUNT = new BytesRef("point count ");
|
||||
final static BytesRef DOC_COUNT = new BytesRef("doc count ");
|
||||
final static BytesRef END = new BytesRef("END");
|
||||
|
||||
private IndexOutput dataOut;
|
||||
final BytesRefBuilder scratch = new BytesRefBuilder();
|
||||
final SegmentWriteState writeState;
|
||||
final Map<String,Long> indexFPs = new HashMap<>();
|
||||
|
||||
public SimpleTextPointWriter(SegmentWriteState writeState) throws IOException {
|
||||
String fileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name, writeState.segmentSuffix, SimpleTextPointFormat.POINT_EXTENSION);
|
||||
public SimpleTextPointsWriter(SegmentWriteState writeState) throws IOException {
|
||||
String fileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name, writeState.segmentSuffix, SimpleTextPointsFormat.POINT_EXTENSION);
|
||||
dataOut = writeState.directory.createOutput(fileName, writeState.context);
|
||||
this.writeState = writeState;
|
||||
}
|
||||
|
||||
@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:
|
||||
try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
|
||||
@ -210,6 +211,8 @@ class SimpleTextPointWriter extends PointWriter {
|
||||
|
||||
@Override
|
||||
public void finish() throws IOException {
|
||||
SimpleTextUtil.write(dataOut, END);
|
||||
SimpleTextUtil.writeNewline(dataOut);
|
||||
SimpleTextUtil.writeChecksum(dataOut, scratch);
|
||||
}
|
||||
|
||||
@ -220,7 +223,7 @@ class SimpleTextPointWriter extends PointWriter {
|
||||
dataOut = null;
|
||||
|
||||
// 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)) {
|
||||
int count = indexFPs.size();
|
||||
write(indexOut, FIELD_COUNT);
|
@ -18,12 +18,12 @@ package org.apache.lucene.codecs.simpletext;
|
||||
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.index.BasePointFormatTestCase;
|
||||
import org.apache.lucene.index.BasePointsFormatTestCase;
|
||||
|
||||
/**
|
||||
* Tests SimpleText's point format
|
||||
*/
|
||||
public class TestSimpleTextPointFormat extends BasePointFormatTestCase {
|
||||
public class TestSimpleTextPointsFormat extends BasePointsFormatTestCase {
|
||||
private final Codec codec = new SimpleTextCodec();
|
||||
|
||||
@Override
|
@ -109,7 +109,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
|
||||
public abstract CompoundFormat compoundFormat();
|
||||
|
||||
/** Encodes/decodes points index */
|
||||
public abstract PointFormat pointFormat();
|
||||
public abstract PointsFormat pointsFormat();
|
||||
|
||||
/** looks up a codec by name */
|
||||
public static Codec forName(String name) {
|
||||
|
@ -105,7 +105,7 @@ public abstract class FilterCodec extends Codec {
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointFormat pointFormat() {
|
||||
return delegate.pointFormat();
|
||||
public PointsFormat pointsFormat() {
|
||||
return delegate.pointsFormat();
|
||||
}
|
||||
}
|
||||
|
@ -26,16 +26,16 @@ import org.apache.lucene.index.SegmentWriteState;
|
||||
* Encodes/decodes indexed points.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
public abstract class PointFormat {
|
||||
public abstract class PointsFormat {
|
||||
|
||||
/**
|
||||
* Creates a new point format.
|
||||
*/
|
||||
protected PointFormat() {
|
||||
protected PointsFormat() {
|
||||
}
|
||||
|
||||
/** 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
|
||||
* 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
|
||||
* 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 */
|
||||
public static final PointFormat EMPTY = new PointFormat() {
|
||||
/** A {@code PointsFormat} that has nothing indexed */
|
||||
public static final PointsFormat EMPTY = new PointsFormat() {
|
||||
@Override
|
||||
public PointWriter fieldsWriter(SegmentWriteState state) {
|
||||
public PointsWriter fieldsWriter(SegmentWriteState state) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointReader fieldsReader(SegmentReadState state) {
|
||||
return new PointReader() {
|
||||
public PointsReader fieldsReader(SegmentReadState state) {
|
||||
return new PointsReader() {
|
||||
@Override
|
||||
public void close() {
|
||||
}
|
@ -27,10 +27,10 @@ import org.apache.lucene.util.Accountable;
|
||||
*
|
||||
* @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.) */
|
||||
protected PointReader() {}
|
||||
protected PointsReader() {}
|
||||
|
||||
/**
|
||||
* Checks consistency of this reader.
|
||||
@ -45,7 +45,7 @@ public abstract class PointReader extends PointValues implements Closeable, Acco
|
||||
* Returns an instance optimized for merging.
|
||||
* <p>
|
||||
* The default implementation returns {@code this} */
|
||||
public PointReader getMergeInstance() throws IOException {
|
||||
public PointsReader getMergeInstance() throws IOException {
|
||||
return this;
|
||||
}
|
||||
}
|
@ -28,57 +28,57 @@ import org.apache.lucene.index.MergeState;
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
||||
public abstract class PointWriter implements Closeable {
|
||||
public abstract class PointsWriter implements Closeable {
|
||||
/** Sole constructor. (For invocation by subclass
|
||||
* constructors, typically implicit.) */
|
||||
protected PointWriter() {
|
||||
protected PointsWriter() {
|
||||
}
|
||||
|
||||
/** 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
|
||||
* from the incoming segment. The default codec overrides this for 1D fields and uses
|
||||
* a faster but more complex implementation. */
|
||||
protected void mergeOneField(MergeState mergeState, FieldInfo fieldInfo) throws IOException {
|
||||
writeField(fieldInfo,
|
||||
new PointReader() {
|
||||
new PointsReader() {
|
||||
@Override
|
||||
public void intersect(String fieldName, IntersectVisitor mergedVisitor) throws IOException {
|
||||
if (fieldName.equals(fieldInfo.name) == false) {
|
||||
throw new IllegalArgumentException("field name must match the field being merged");
|
||||
}
|
||||
for (int i=0;i<mergeState.pointReaders.length;i++) {
|
||||
PointReader pointReader = mergeState.pointReaders[i];
|
||||
if (pointReader == null) {
|
||||
for (int i=0;i<mergeState.pointsReaders.length;i++) {
|
||||
PointsReader pointsReader = mergeState.pointsReaders[i];
|
||||
if (pointsReader == null) {
|
||||
// This segment has no points
|
||||
continue;
|
||||
}
|
||||
MergeState.DocMap docMap = mergeState.docMaps[i];
|
||||
int docBase = mergeState.docBase[i];
|
||||
pointReader.intersect(fieldInfo.name,
|
||||
new IntersectVisitor() {
|
||||
@Override
|
||||
public void visit(int docID) {
|
||||
// Should never be called because our compare method never returns Relation.CELL_INSIDE_QUERY
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
pointsReader.intersect(fieldInfo.name,
|
||||
new IntersectVisitor() {
|
||||
@Override
|
||||
public void visit(int docID) {
|
||||
// Should never be called because our compare method never returns Relation.CELL_INSIDE_QUERY
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {
|
||||
int newDocID = docMap.get(docID);
|
||||
if (newDocID != -1) {
|
||||
// Not deleted:
|
||||
mergedVisitor.visit(docBase + newDocID, packedValue);
|
||||
}
|
||||
}
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {
|
||||
int newDocID = docMap.get(docID);
|
||||
if (newDocID != -1) {
|
||||
// Not deleted:
|
||||
mergedVisitor.visit(docBase + newDocID, packedValue);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
// Forces this segment's PointReader to always visit all docs + values:
|
||||
return Relation.CELL_CROSSES_QUERY;
|
||||
}
|
||||
});
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
// Forces this segment's PointsReader to always visit all docs + values:
|
||||
return Relation.CELL_CROSSES_QUERY;
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
@ -132,7 +132,7 @@ public abstract class PointWriter implements Closeable {
|
||||
* adding to this writer */
|
||||
public void merge(MergeState mergeState) throws IOException {
|
||||
// check each incoming reader
|
||||
for (PointReader reader : mergeState.pointReaders) {
|
||||
for (PointsReader reader : mergeState.pointsReaders) {
|
||||
if (reader != null) {
|
||||
reader.checkIntegrity();
|
||||
}
|
@ -21,12 +21,12 @@ import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.CompoundFormat;
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
@ -129,8 +129,8 @@ public class Lucene60Codec extends Codec {
|
||||
}
|
||||
|
||||
@Override
|
||||
public final PointFormat pointFormat() {
|
||||
return new Lucene60PointFormat();
|
||||
public final PointsFormat pointsFormat() {
|
||||
return new Lucene60PointsFormat();
|
||||
}
|
||||
|
||||
/** Returns the postings format that should be used for writing
|
||||
|
@ -20,9 +20,9 @@ package org.apache.lucene.codecs.lucene60;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PointsReader;
|
||||
import org.apache.lucene.codecs.PointsWriter;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
||||
@ -69,10 +69,10 @@ import org.apache.lucene.index.SegmentWriteState;
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
||||
public final class Lucene60PointFormat extends PointFormat {
|
||||
public final class Lucene60PointsFormat extends PointsFormat {
|
||||
|
||||
static final String DATA_CODEC_NAME = "Lucene60PointFormatData";
|
||||
static final String META_CODEC_NAME = "Lucene60PointFormatMeta";
|
||||
static final String DATA_CODEC_NAME = "Lucene60PointsFormatData";
|
||||
static final String META_CODEC_NAME = "Lucene60PointsFormatMeta";
|
||||
|
||||
/**
|
||||
* 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;
|
||||
|
||||
/** Sole constructor */
|
||||
public Lucene60PointFormat() {
|
||||
public Lucene60PointsFormat() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointWriter fieldsWriter(SegmentWriteState state) throws IOException {
|
||||
return new Lucene60PointWriter(state);
|
||||
public PointsWriter fieldsWriter(SegmentWriteState state) throws IOException {
|
||||
return new Lucene60PointsWriter(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointReader fieldsReader(SegmentReadState state) throws IOException {
|
||||
return new Lucene60PointReader(state);
|
||||
public PointsReader fieldsReader(SegmentReadState state) throws IOException {
|
||||
return new Lucene60PointsReader(state);
|
||||
}
|
||||
}
|
@ -27,7 +27,7 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
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.IndexFileNames;
|
||||
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.bkd.BKDReader;
|
||||
|
||||
/** Reads point values previously written with {@link Lucene60PointWriter} */
|
||||
public class Lucene60PointReader extends PointReader implements Closeable {
|
||||
/** Reads point values previously written with {@link Lucene60PointsWriter} */
|
||||
public class Lucene60PointsReader extends PointsReader implements Closeable {
|
||||
final IndexInput dataIn;
|
||||
final SegmentReadState readState;
|
||||
final Map<Integer,BKDReader> readers = new HashMap<>();
|
||||
|
||||
/** Sole constructor */
|
||||
public Lucene60PointReader(SegmentReadState readState) throws IOException {
|
||||
public Lucene60PointsReader(SegmentReadState readState) throws IOException {
|
||||
this.readState = readState;
|
||||
|
||||
|
||||
String indexFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name,
|
||||
readState.segmentSuffix,
|
||||
Lucene60PointFormat.INDEX_EXTENSION);
|
||||
Lucene60PointsFormat.INDEX_EXTENSION);
|
||||
|
||||
Map<Integer,Long> fieldToFileOffset = new HashMap<>();
|
||||
|
||||
@ -60,9 +60,9 @@ public class Lucene60PointReader extends PointReader implements Closeable {
|
||||
Throwable priorE = null;
|
||||
try {
|
||||
CodecUtil.checkIndexHeader(indexIn,
|
||||
Lucene60PointFormat.META_CODEC_NAME,
|
||||
Lucene60PointFormat.INDEX_VERSION_START,
|
||||
Lucene60PointFormat.INDEX_VERSION_START,
|
||||
Lucene60PointsFormat.META_CODEC_NAME,
|
||||
Lucene60PointsFormat.INDEX_VERSION_START,
|
||||
Lucene60PointsFormat.INDEX_VERSION_START,
|
||||
readState.segmentInfo.getId(),
|
||||
readState.segmentSuffix);
|
||||
int count = indexIn.readVInt();
|
||||
@ -80,15 +80,15 @@ public class Lucene60PointReader extends PointReader implements Closeable {
|
||||
|
||||
String dataFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name,
|
||||
readState.segmentSuffix,
|
||||
Lucene60PointFormat.DATA_EXTENSION);
|
||||
Lucene60PointsFormat.DATA_EXTENSION);
|
||||
boolean success = false;
|
||||
dataIn = readState.directory.openInput(dataFileName, readState.context);
|
||||
try {
|
||||
|
||||
CodecUtil.checkIndexHeader(dataIn,
|
||||
Lucene60PointFormat.DATA_CODEC_NAME,
|
||||
Lucene60PointFormat.DATA_VERSION_START,
|
||||
Lucene60PointFormat.DATA_VERSION_START,
|
||||
Lucene60PointsFormat.DATA_CODEC_NAME,
|
||||
Lucene60PointsFormat.DATA_VERSION_START,
|
||||
Lucene60PointsFormat.DATA_VERSION_START,
|
||||
readState.segmentInfo.getId(),
|
||||
readState.segmentSuffix);
|
||||
|
@ -25,8 +25,8 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.codecs.PointsReader;
|
||||
import org.apache.lucene.codecs.PointsWriter;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
@ -40,7 +40,7 @@ import org.apache.lucene.util.bkd.BKDReader;
|
||||
import org.apache.lucene.util.bkd.BKDWriter;
|
||||
|
||||
/** Writes dimensional values */
|
||||
public class Lucene60PointWriter extends PointWriter implements Closeable {
|
||||
public class Lucene60PointsWriter extends PointsWriter implements Closeable {
|
||||
|
||||
final IndexOutput dataOut;
|
||||
final Map<String,Long> indexFPs = new HashMap<>();
|
||||
@ -50,20 +50,20 @@ public class Lucene60PointWriter extends PointWriter implements Closeable {
|
||||
private boolean finished;
|
||||
|
||||
/** 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();
|
||||
this.writeState = writeState;
|
||||
this.maxPointsInLeafNode = maxPointsInLeafNode;
|
||||
this.maxMBSortInHeap = maxMBSortInHeap;
|
||||
String dataFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name,
|
||||
writeState.segmentSuffix,
|
||||
Lucene60PointFormat.DATA_EXTENSION);
|
||||
Lucene60PointsFormat.DATA_EXTENSION);
|
||||
dataOut = writeState.directory.createOutput(dataFileName, writeState.context);
|
||||
boolean success = false;
|
||||
try {
|
||||
CodecUtil.writeIndexHeader(dataOut,
|
||||
Lucene60PointFormat.DATA_CODEC_NAME,
|
||||
Lucene60PointFormat.DATA_VERSION_CURRENT,
|
||||
Lucene60PointsFormat.DATA_CODEC_NAME,
|
||||
Lucene60PointsFormat.DATA_VERSION_CURRENT,
|
||||
writeState.segmentInfo.getId(),
|
||||
writeState.segmentSuffix);
|
||||
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) */
|
||||
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);
|
||||
}
|
||||
|
||||
@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(),
|
||||
writeState.directory,
|
||||
@ -115,14 +115,14 @@ public class Lucene60PointWriter extends PointWriter implements Closeable {
|
||||
|
||||
@Override
|
||||
public void merge(MergeState mergeState) throws IOException {
|
||||
for(PointReader reader : mergeState.pointReaders) {
|
||||
if (reader instanceof Lucene60PointReader == false) {
|
||||
for(PointsReader reader : mergeState.pointsReaders) {
|
||||
if (reader instanceof Lucene60PointsReader == false) {
|
||||
// We can only bulk merge when all to-be-merged segments use our format:
|
||||
super.merge(mergeState);
|
||||
return;
|
||||
}
|
||||
}
|
||||
for (PointReader reader : mergeState.pointReaders) {
|
||||
for (PointsReader reader : mergeState.pointsReaders) {
|
||||
if (reader != null) {
|
||||
reader.checkIntegrity();
|
||||
}
|
||||
@ -145,14 +145,14 @@ public class Lucene60PointWriter extends PointWriter implements Closeable {
|
||||
List<BKDReader> bkdReaders = new ArrayList<>();
|
||||
List<MergeState.DocMap> docMaps = new ArrayList<>();
|
||||
List<Integer> docIDBases = new ArrayList<>();
|
||||
for(int i=0;i<mergeState.pointReaders.length;i++) {
|
||||
PointReader reader = mergeState.pointReaders[i];
|
||||
for(int i=0;i<mergeState.pointsReaders.length;i++) {
|
||||
PointsReader reader = mergeState.pointsReaders[i];
|
||||
|
||||
if (reader != null) {
|
||||
|
||||
// we confirmed this up above
|
||||
assert reader instanceof Lucene60PointReader;
|
||||
Lucene60PointReader reader60 = (Lucene60PointReader) reader;
|
||||
assert reader instanceof Lucene60PointsReader;
|
||||
Lucene60PointsReader reader60 = (Lucene60PointsReader) reader;
|
||||
|
||||
// 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
|
||||
@ -196,12 +196,12 @@ public class Lucene60PointWriter extends PointWriter implements Closeable {
|
||||
|
||||
String indexFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name,
|
||||
writeState.segmentSuffix,
|
||||
Lucene60PointFormat.INDEX_EXTENSION);
|
||||
Lucene60PointsFormat.INDEX_EXTENSION);
|
||||
// Write index file
|
||||
try (IndexOutput indexOut = writeState.directory.createOutput(indexFileName, writeState.context)) {
|
||||
CodecUtil.writeIndexHeader(indexOut,
|
||||
Lucene60PointFormat.META_CODEC_NAME,
|
||||
Lucene60PointFormat.INDEX_VERSION_CURRENT,
|
||||
Lucene60PointsFormat.META_CODEC_NAME,
|
||||
Lucene60PointsFormat.INDEX_VERSION_CURRENT,
|
||||
writeState.segmentInfo.getId(),
|
||||
writeState.segmentSuffix);
|
||||
int count = indexFPs.size();
|
@ -193,7 +193,7 @@
|
||||
* An optional file indicating which documents are live.
|
||||
* </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
|
||||
* numeric range filtering and large numeric values like BigInteger and BigDecimal (1D)
|
||||
* and geo shape intersection (2D, 3D).
|
||||
@ -322,7 +322,7 @@
|
||||
* <td>Info about what files are live</td>
|
||||
* </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>Holds indexed points, if any</td>
|
||||
* </tr>
|
||||
|
@ -33,9 +33,9 @@ import java.util.Locale;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
import org.apache.lucene.codecs.PointsReader;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.codecs.TermVectorsReader;
|
||||
@ -567,8 +567,6 @@ public final class CheckIndex implements Closeable {
|
||||
input.close();
|
||||
}
|
||||
|
||||
String sFormat = "";
|
||||
|
||||
result.segmentsFileName = segmentsFileName;
|
||||
result.numSegments = numSegments;
|
||||
result.userData = sis.getUserData();
|
||||
@ -591,7 +589,7 @@ public final class CheckIndex implements Closeable {
|
||||
}
|
||||
|
||||
msg(infoStream, "Segments file=" + segmentsFileName + " numSegments=" + numSegments
|
||||
+ " " + versionString + " id=" + StringHelper.idToString(sis.getId()) + " format=" + sFormat + userDataString);
|
||||
+ " " + versionString + " id=" + StringHelper.idToString(sis.getId()) + userDataString);
|
||||
|
||||
if (onlySegments != null) {
|
||||
result.partial = true;
|
||||
@ -1689,9 +1687,9 @@ public final class CheckIndex implements Closeable {
|
||||
Status.PointsStatus status = new Status.PointsStatus();
|
||||
try {
|
||||
if (fieldInfos.hasPointValues()) {
|
||||
PointReader values = reader.getPointReader();
|
||||
PointsReader values = reader.getPointsReader();
|
||||
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) {
|
||||
if (fieldInfo.getPointDimensionCount() > 0) {
|
||||
|
@ -25,10 +25,10 @@ import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
import org.apache.lucene.codecs.PointsReader;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.codecs.TermVectorsReader;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
@ -77,10 +77,10 @@ public abstract class CodecReader extends LeafReader implements Accountable {
|
||||
public abstract FieldsProducer getPostingsReader();
|
||||
|
||||
/**
|
||||
* Expert: retrieve underlying PointReader
|
||||
* Expert: retrieve underlying PointsReader
|
||||
* @lucene.internal
|
||||
*/
|
||||
public abstract PointReader getPointReader();
|
||||
public abstract PointsReader getPointsReader();
|
||||
|
||||
@Override
|
||||
public final void document(int docID, StoredFieldVisitor visitor) throws IOException {
|
||||
@ -323,8 +323,8 @@ public abstract class CodecReader extends LeafReader implements Accountable {
|
||||
}
|
||||
|
||||
// points
|
||||
if (getPointReader() != null) {
|
||||
ramBytesUsed += getPointReader().ramBytesUsed();
|
||||
if (getPointsReader() != null) {
|
||||
ramBytesUsed += getPointsReader().ramBytesUsed();
|
||||
}
|
||||
|
||||
return ramBytesUsed;
|
||||
@ -359,8 +359,8 @@ public abstract class CodecReader extends LeafReader implements Accountable {
|
||||
}
|
||||
|
||||
// points
|
||||
if (getPointReader() != null) {
|
||||
resources.add(Accountables.namedAccountable("points", getPointReader()));
|
||||
if (getPointsReader() != null) {
|
||||
resources.add(Accountables.namedAccountable("points", getPointsReader()));
|
||||
}
|
||||
|
||||
return Collections.unmodifiableList(resources);
|
||||
@ -394,8 +394,8 @@ public abstract class CodecReader extends LeafReader implements Accountable {
|
||||
}
|
||||
|
||||
// points
|
||||
if (getPointReader() != null) {
|
||||
getPointReader().checkIntegrity();
|
||||
if (getPointsReader() != null) {
|
||||
getPointsReader().checkIntegrity();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -23,12 +23,12 @@ import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
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.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.NormsConsumer;
|
||||
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.document.FieldType;
|
||||
import org.apache.lucene.search.similarities.Similarity;
|
||||
@ -149,7 +149,7 @@ final class DefaultIndexingChain extends DocConsumer {
|
||||
|
||||
/** Writes all buffered points. */
|
||||
private void writePoints(SegmentWriteState state) throws IOException {
|
||||
PointWriter pointWriter = null;
|
||||
PointsWriter pointsWriter = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
for (int i=0;i<fieldHash.length;i++) {
|
||||
@ -160,16 +160,16 @@ final class DefaultIndexingChain extends DocConsumer {
|
||||
// BUG
|
||||
throw new AssertionError("segment=" + state.segmentInfo + ": field=\"" + perField.fieldInfo.name + "\" has no points but wrote them");
|
||||
}
|
||||
if (pointWriter == null) {
|
||||
if (pointsWriter == null) {
|
||||
// lazy init
|
||||
PointFormat fmt = state.segmentInfo.getCodec().pointFormat();
|
||||
PointsFormat fmt = state.segmentInfo.getCodec().pointsFormat();
|
||||
if (fmt == null) {
|
||||
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;
|
||||
} else if (perField.fieldInfo.getPointDimensionCount() != 0) {
|
||||
// BUG
|
||||
@ -178,15 +178,15 @@ final class DefaultIndexingChain extends DocConsumer {
|
||||
perField = perField.next;
|
||||
}
|
||||
}
|
||||
if (pointWriter != null) {
|
||||
pointWriter.finish();
|
||||
if (pointsWriter != null) {
|
||||
pointsWriter.finish();
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(pointWriter);
|
||||
IOUtils.close(pointsWriter);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(pointWriter);
|
||||
IOUtils.closeWhileHandlingException(pointsWriter);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -49,7 +49,7 @@ public final class FieldInfo {
|
||||
private long dvGen;
|
||||
|
||||
/** 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 pointNumBytes;
|
||||
|
||||
|
@ -19,10 +19,10 @@ package org.apache.lucene.index;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
import org.apache.lucene.codecs.PointsReader;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.codecs.TermVectorsReader;
|
||||
import org.apache.lucene.util.Bits;
|
||||
@ -82,8 +82,8 @@ public class FilterCodecReader extends CodecReader {
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointReader getPointReader() {
|
||||
return in.getPointReader();
|
||||
public PointsReader getPointsReader() {
|
||||
return in.getPointsReader();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -300,7 +300,7 @@ public abstract class LeafReader extends IndexReader {
|
||||
*/
|
||||
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. */
|
||||
public abstract PointValues getPointValues();
|
||||
|
||||
|
@ -20,10 +20,10 @@ package org.apache.lucene.index;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
import org.apache.lucene.codecs.PointsReader;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.codecs.TermVectorsReader;
|
||||
import org.apache.lucene.util.Bits;
|
||||
@ -67,7 +67,7 @@ public class MergeState {
|
||||
public final FieldsProducer[] fieldsProducers;
|
||||
|
||||
/** Point readers to merge */
|
||||
public final PointReader[] pointReaders;
|
||||
public final PointsReader[] pointsReaders;
|
||||
|
||||
/** New docID base per reader. */
|
||||
public final int[] docBase;
|
||||
@ -90,7 +90,7 @@ public class MergeState {
|
||||
storedFieldsReaders = new StoredFieldsReader[numReaders];
|
||||
termVectorsReaders = new TermVectorsReader[numReaders];
|
||||
docValuesProducers = new DocValuesProducer[numReaders];
|
||||
pointReaders = new PointReader[numReaders];
|
||||
pointsReaders = new PointsReader[numReaders];
|
||||
fieldInfos = new FieldInfos[numReaders];
|
||||
liveDocs = new Bits[numReaders];
|
||||
|
||||
@ -122,9 +122,9 @@ public class MergeState {
|
||||
}
|
||||
|
||||
fieldsProducers[i] = reader.getPostingsReader().getMergeInstance();
|
||||
pointReaders[i] = reader.getPointReader();
|
||||
if (pointReaders[i] != null) {
|
||||
pointReaders[i] = pointReaders[i].getMergeInstance();
|
||||
pointsReaders[i] = reader.getPointsReader();
|
||||
if (pointsReaders[i] != null) {
|
||||
pointsReaders[i] = pointsReaders[i].getMergeInstance();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -18,8 +18,8 @@ package org.apache.lucene.index;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.codecs.PointsReader;
|
||||
import org.apache.lucene.codecs.PointsWriter;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.ByteBlockPool;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
@ -60,10 +60,10 @@ class PointValuesWriter {
|
||||
numDocs++;
|
||||
}
|
||||
|
||||
public void flush(SegmentWriteState state, PointWriter writer) throws IOException {
|
||||
public void flush(SegmentWriteState state, PointsWriter writer) throws IOException {
|
||||
|
||||
writer.writeField(fieldInfo,
|
||||
new PointReader() {
|
||||
new PointsReader() {
|
||||
@Override
|
||||
public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
|
||||
if (fieldName.equals(fieldInfo.name) == false) {
|
||||
|
@ -24,9 +24,9 @@ import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
import org.apache.lucene.codecs.PointsReader;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.codecs.TermVectorsReader;
|
||||
@ -54,7 +54,7 @@ final class SegmentCoreReaders {
|
||||
|
||||
final StoredFieldsReader fieldsReaderOrig;
|
||||
final TermVectorsReader termVectorsReaderOrig;
|
||||
final PointReader pointReader;
|
||||
final PointsReader pointsReader;
|
||||
final Directory cfsReader;
|
||||
/**
|
||||
* fieldinfos for this core: means gen=-1.
|
||||
@ -125,9 +125,9 @@ final class SegmentCoreReaders {
|
||||
}
|
||||
|
||||
if (coreFieldInfos.hasPointValues()) {
|
||||
pointReader = codec.pointFormat().fieldsReader(segmentReadState);
|
||||
pointsReader = codec.pointsFormat().fieldsReader(segmentReadState);
|
||||
} else {
|
||||
pointReader = null;
|
||||
pointsReader = null;
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
@ -157,7 +157,7 @@ final class SegmentCoreReaders {
|
||||
Throwable th = null;
|
||||
try {
|
||||
IOUtils.close(termVectorsLocal, fieldsReaderLocal, fields, termVectorsReaderOrig, fieldsReaderOrig,
|
||||
cfsReader, normsProducer, pointReader);
|
||||
cfsReader, normsProducer, pointsReader);
|
||||
} catch (Throwable throwable) {
|
||||
th = throwable;
|
||||
} finally {
|
||||
|
@ -21,10 +21,10 @@ import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.codecs.NormsConsumer;
|
||||
import org.apache.lucene.codecs.PointsWriter;
|
||||
import org.apache.lucene.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.codecs.TermVectorsWriter;
|
||||
import org.apache.lucene.store.Directory;
|
||||
@ -164,7 +164,7 @@ final class SegmentMerger {
|
||||
}
|
||||
|
||||
private void mergePoints(SegmentWriteState segmentWriteState) throws IOException {
|
||||
try (PointWriter writer = codec.pointFormat().fieldsWriter(segmentWriteState)) {
|
||||
try (PointsWriter writer = codec.pointsFormat().fieldsWriter(segmentWriteState)) {
|
||||
writer.merge(mergeState);
|
||||
}
|
||||
}
|
||||
|
@ -21,11 +21,11 @@ import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
import org.apache.lucene.codecs.PointsReader;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.codecs.TermVectorsReader;
|
||||
import org.apache.lucene.store.Directory;
|
||||
@ -220,7 +220,7 @@ public final class SegmentReader extends CodecReader {
|
||||
@Override
|
||||
public PointValues getPointValues() {
|
||||
ensureOpen();
|
||||
return core.pointReader;
|
||||
return core.pointsReader;
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -242,9 +242,9 @@ public final class SegmentReader extends CodecReader {
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointReader getPointReader() {
|
||||
public PointsReader getPointsReader() {
|
||||
ensureOpen();
|
||||
return core.pointReader;
|
||||
return core.pointsReader;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -20,10 +20,10 @@ package org.apache.lucene.index;
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
import org.apache.lucene.codecs.PointsReader;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.codecs.TermVectorsReader;
|
||||
import org.apache.lucene.util.Bits;
|
||||
@ -97,7 +97,7 @@ public final class SlowCodecReaderWrapper {
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointReader getPointReader() {
|
||||
public PointsReader getPointsReader() {
|
||||
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) {
|
||||
return null;
|
||||
}
|
||||
return new PointReader() {
|
||||
return new PointsReader() {
|
||||
@Override
|
||||
public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
|
||||
values.intersect(fieldName, visitor);
|
||||
|
@ -125,6 +125,68 @@ public abstract class PointRangeQuery extends Query {
|
||||
|
||||
return new ConstantScoreWeight(this) {
|
||||
|
||||
private DocIdSet buildMatchingDocIdSet(LeafReader reader, PointValues values,
|
||||
byte[] packedLower, byte[] packedUpper) throws IOException {
|
||||
DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc());
|
||||
|
||||
values.intersect(field,
|
||||
new IntersectVisitor() {
|
||||
|
||||
@Override
|
||||
public void grow(int count) {
|
||||
result.grow(count);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID) {
|
||||
result.add(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) {
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
int offset = dim*bytesPerDim;
|
||||
if (StringHelper.compare(bytesPerDim, packedValue, offset, packedLower, offset) < 0) {
|
||||
// Doc's value is too low, in this dimension
|
||||
return;
|
||||
}
|
||||
if (StringHelper.compare(bytesPerDim, packedValue, offset, packedUpper, offset) > 0) {
|
||||
// Doc's value is too high, in this dimension
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
// Doc is in-bounds
|
||||
result.add(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
|
||||
boolean crosses = false;
|
||||
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
int offset = dim*bytesPerDim;
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, minPackedValue, offset, packedUpper, offset) > 0 ||
|
||||
StringHelper.compare(bytesPerDim, maxPackedValue, offset, packedLower, offset) < 0) {
|
||||
return Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
|
||||
crosses |= StringHelper.compare(bytesPerDim, minPackedValue, offset, packedLower, offset) < 0 ||
|
||||
StringHelper.compare(bytesPerDim, maxPackedValue, offset, packedUpper, offset) > 0;
|
||||
}
|
||||
|
||||
if (crosses) {
|
||||
return Relation.CELL_CROSSES_QUERY;
|
||||
} else {
|
||||
return Relation.CELL_INSIDE_QUERY;
|
||||
}
|
||||
}
|
||||
});
|
||||
return result.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
LeafReader reader = context.reader();
|
||||
@ -155,67 +217,32 @@ public abstract class PointRangeQuery extends Query {
|
||||
System.arraycopy(upperPoint[dim], 0, packedUpper, dim*bytesPerDim, bytesPerDim);
|
||||
}
|
||||
|
||||
// Now packedLowerIncl and packedUpperIncl are inclusive, and non-empty space:
|
||||
boolean allDocsMatch;
|
||||
if (values.getDocCount(field) == reader.maxDoc()) {
|
||||
final byte[] fieldPackedLower = values.getMinPackedValue(field);
|
||||
final byte[] fieldPackedUpper = values.getMaxPackedValue(field);
|
||||
allDocsMatch = true;
|
||||
for (int i = 0; i < numDims; ++i) {
|
||||
int offset = i * bytesPerDim;
|
||||
if (StringHelper.compare(bytesPerDim, packedLower, offset, fieldPackedLower, offset) > 0
|
||||
|| StringHelper.compare(bytesPerDim, packedUpper, offset, fieldPackedUpper, offset) < 0) {
|
||||
allDocsMatch = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
allDocsMatch = false;
|
||||
}
|
||||
|
||||
DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc());
|
||||
DocIdSetIterator iterator;
|
||||
if (allDocsMatch) {
|
||||
// all docs have a value and all points are within bounds, so everything matches
|
||||
iterator = DocIdSetIterator.all(reader.maxDoc());
|
||||
} else {
|
||||
iterator = buildMatchingDocIdSet(reader, values, packedLower, packedUpper).iterator();
|
||||
}
|
||||
|
||||
values.intersect(field,
|
||||
new IntersectVisitor() {
|
||||
|
||||
@Override
|
||||
public void grow(int count) {
|
||||
result.grow(count);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID) {
|
||||
result.add(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) {
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
int offset = dim*bytesPerDim;
|
||||
if (StringHelper.compare(bytesPerDim, packedValue, offset, packedLower, offset) < 0) {
|
||||
// Doc's value is too low, in this dimension
|
||||
return;
|
||||
}
|
||||
if (StringHelper.compare(bytesPerDim, packedValue, offset, packedUpper, offset) > 0) {
|
||||
// Doc's value is too high, in this dimension
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
// Doc is in-bounds
|
||||
result.add(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
|
||||
boolean crosses = false;
|
||||
|
||||
for(int dim=0;dim<numDims;dim++) {
|
||||
int offset = dim*bytesPerDim;
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, minPackedValue, offset, packedUpper, offset) > 0 ||
|
||||
StringHelper.compare(bytesPerDim, maxPackedValue, offset, packedLower, offset) < 0) {
|
||||
return Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
|
||||
crosses |= StringHelper.compare(bytesPerDim, minPackedValue, offset, packedLower, offset) < 0 ||
|
||||
StringHelper.compare(bytesPerDim, maxPackedValue, offset, packedUpper, offset) > 0;
|
||||
}
|
||||
|
||||
if (crosses) {
|
||||
return Relation.CELL_CROSSES_QUERY;
|
||||
} else {
|
||||
return Relation.CELL_INSIDE_QUERY;
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
return new ConstantScoreScorer(this, score(), result.build().iterator());
|
||||
return new ConstantScoreScorer(this, score(), iterator);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -21,21 +21,21 @@ import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.index.BasePointFormatTestCase;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PointsReader;
|
||||
import org.apache.lucene.codecs.PointsWriter;
|
||||
import org.apache.lucene.index.BasePointsFormatTestCase;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
/**
|
||||
* Tests Lucene60PointFormat
|
||||
* Tests Lucene60PointsFormat
|
||||
*/
|
||||
public class TestLucene60PointFormat extends BasePointFormatTestCase {
|
||||
public class TestLucene60PointsFormat extends BasePointsFormatTestCase {
|
||||
private final Codec codec;
|
||||
|
||||
public TestLucene60PointFormat() {
|
||||
public TestLucene60PointsFormat() {
|
||||
// standard issue
|
||||
Codec defaultCodec = TestUtil.getDefaultCodec();
|
||||
if (random().nextBoolean()) {
|
||||
@ -43,22 +43,22 @@ public class TestLucene60PointFormat extends BasePointFormatTestCase {
|
||||
int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 500);
|
||||
double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
|
||||
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!
|
||||
codec = new FilterCodec(defaultCodec.getName(), defaultCodec) {
|
||||
@Override
|
||||
public PointFormat pointFormat() {
|
||||
return new PointFormat() {
|
||||
public PointsFormat pointsFormat() {
|
||||
return new PointsFormat() {
|
||||
@Override
|
||||
public PointWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
|
||||
return new Lucene60PointWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
|
||||
public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
|
||||
return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointReader fieldsReader(SegmentReadState readState) throws IOException {
|
||||
return new Lucene60PointReader(readState);
|
||||
public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
|
||||
return new Lucene60PointsReader(readState);
|
||||
}
|
||||
};
|
||||
}
|
@ -22,11 +22,11 @@ import java.io.IOException;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60PointReader;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60PointWriter;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
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.BinaryPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.DoublePoint;
|
||||
|
@ -32,11 +32,11 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60PointReader;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60PointWriter;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
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.BinaryPoint;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.DoublePoint;
|
||||
@ -53,13 +53,11 @@ import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.MultiDocValues;
|
||||
import org.apache.lucene.index.MultiReader;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SlowCompositeReaderWrapper;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
@ -353,12 +351,12 @@ public class TestPointQueries extends LuceneTestCase {
|
||||
|
||||
@Nightly
|
||||
public void testRandomLongsBig() throws Exception {
|
||||
doTestRandomLongs(200000);
|
||||
doTestRandomLongs(100000);
|
||||
}
|
||||
|
||||
private void doTestRandomLongs(int count) throws Exception {
|
||||
|
||||
int numValues = atLeast(count);
|
||||
int numValues = TestUtil.nextInt(random(), count, count*2);
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: numValues=" + numValues);
|
||||
@ -1157,21 +1155,21 @@ public class TestPointQueries extends LuceneTestCase {
|
||||
int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
|
||||
double maxMBSortInHeap = 4.0 + (3*random().nextDouble());
|
||||
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()) {
|
||||
@Override
|
||||
public PointFormat pointFormat() {
|
||||
return new PointFormat() {
|
||||
public PointsFormat pointsFormat() {
|
||||
return new PointsFormat() {
|
||||
@Override
|
||||
public PointWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
|
||||
return new Lucene60PointWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
|
||||
public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
|
||||
return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointReader fieldsReader(SegmentReadState readState) throws IOException {
|
||||
return new Lucene60PointReader(readState);
|
||||
public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
|
||||
return new Lucene60PointsReader(readState);
|
||||
}
|
||||
};
|
||||
}
|
||||
@ -1847,4 +1845,45 @@ public class TestPointQueries extends LuceneTestCase {
|
||||
// binary
|
||||
assertEquals("bytes:{[12] [2a]}", BinaryPoint.newSetQuery("bytes", new byte[] {42}, new byte[] {18}).toString());
|
||||
}
|
||||
|
||||
public void testRangeOptimizesIfAllPointsMatch() throws IOException {
|
||||
final int numDims = TestUtil.nextInt(random(), 1, 3);
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
int[] value = new int[numDims];
|
||||
for (int i = 0; i < numDims; ++i) {
|
||||
value[i] = TestUtil.nextInt(random(), 1, 10);
|
||||
}
|
||||
doc.add(new IntPoint("point", value));
|
||||
w.addDocument(doc);
|
||||
IndexReader reader = w.getReader();
|
||||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
searcher.setQueryCache(null);
|
||||
int[] lowerBound = new int[numDims];
|
||||
int[] upperBound = new int[numDims];
|
||||
for (int i = 0; i < numDims; ++i) {
|
||||
lowerBound[i] = value[i] - random().nextInt(1);
|
||||
upperBound[i] = value[i] + random().nextInt(1);
|
||||
}
|
||||
Query query = IntPoint.newRangeQuery("point", lowerBound, upperBound);
|
||||
Weight weight = searcher.createNormalizedWeight(query, false);
|
||||
Scorer scorer = weight.scorer(searcher.getIndexReader().leaves().get(0));
|
||||
assertEquals(DocIdSetIterator.all(1).getClass(), scorer.iterator().getClass());
|
||||
|
||||
// When not all documents in the query have a value, the optimization is not applicable
|
||||
reader.close();
|
||||
w.addDocument(new Document());
|
||||
w.forceMerge(1);
|
||||
reader = w.getReader();
|
||||
searcher = new IndexSearcher(reader);
|
||||
searcher.setQueryCache(null);
|
||||
weight = searcher.createNormalizedWeight(query, false);
|
||||
scorer = weight.scorer(searcher.getIndexReader().leaves().get(0));
|
||||
assertFalse(DocIdSetIterator.all(1).getClass().equals(scorer.iterator().getClass()));
|
||||
|
||||
reader.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
@ -421,7 +421,10 @@ public class TestRangeFacetCounts extends FacetTestCase {
|
||||
}
|
||||
boolean minIncl;
|
||||
boolean maxIncl;
|
||||
if (min == max) {
|
||||
|
||||
// NOTE: max - min >= 0 is here to handle the common overflow case!
|
||||
if (max - min >= 0 && max - min < 2) {
|
||||
// If max == min or max == min+1, we always do inclusive, else we might pass an empty range and hit exc from LongRange's ctor:
|
||||
minIncl = true;
|
||||
maxIncl = true;
|
||||
} else {
|
||||
|
@ -20,11 +20,11 @@ import java.io.IOException;
|
||||
import java.util.BitSet;
|
||||
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60PointReader;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60PointWriter;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
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.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
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.spatial.util.GeoDistanceUtils;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.lucene.util.bkd.BKDWriter;
|
||||
|
||||
/** Simple tests for {@link LatLonPoint#newDistanceQuery} */
|
||||
@ -119,16 +119,16 @@ public class TestLatLonPointDistanceQuery extends LuceneTestCase {
|
||||
int pointsInLeaf = 2 + random().nextInt(4);
|
||||
iwc.setCodec(new FilterCodec("Lucene60", TestUtil.getDefaultCodec()) {
|
||||
@Override
|
||||
public PointFormat pointFormat() {
|
||||
return new PointFormat() {
|
||||
public PointsFormat pointsFormat() {
|
||||
return new PointsFormat() {
|
||||
@Override
|
||||
public PointWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
|
||||
return new Lucene60PointWriter(writeState, pointsInLeaf, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
|
||||
public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
|
||||
return new Lucene60PointsWriter(writeState, pointsInLeaf, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointReader fieldsReader(SegmentReadState readState) throws IOException {
|
||||
return new Lucene60PointReader(readState);
|
||||
public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
|
||||
return new Lucene60PointsReader(readState);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -34,6 +34,11 @@ public abstract class BasePlanetObject {
|
||||
public BasePlanetObject(final PlanetModel planetModel) {
|
||||
this.planetModel = planetModel;
|
||||
}
|
||||
|
||||
/** Returns the {@link PlanetModel} provided when this shape was created. */
|
||||
public PlanetModel getPlanetModel() {
|
||||
return planetModel;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
|
@ -36,8 +36,6 @@ import org.apache.lucene.util.RamUsageEstimator;
|
||||
* @lucene.experimental */
|
||||
public final class Geo3DPoint extends Field {
|
||||
|
||||
private final PlanetModel planetModel;
|
||||
|
||||
/** Indexing {@link FieldType}. */
|
||||
public static final FieldType TYPE = new FieldType();
|
||||
static {
|
||||
@ -46,16 +44,15 @@ public final class Geo3DPoint extends Field {
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new Geo3DPoint field with the specified lat, lon (in radians), given a planet model.
|
||||
* Creates a new Geo3DPoint field with the specified lat, lon (in radians).
|
||||
*
|
||||
* @throws IllegalArgumentException if the field name is null or lat or lon are out of bounds
|
||||
*/
|
||||
public Geo3DPoint(String name, PlanetModel planetModel, double lat, double lon) {
|
||||
public Geo3DPoint(String name, double lat, double lon) {
|
||||
super(name, TYPE);
|
||||
this.planetModel = planetModel;
|
||||
// Translate lat/lon to x,y,z:
|
||||
final GeoPoint point = new GeoPoint(planetModel, lat, lon);
|
||||
fillFieldsData(planetModel, point.x, point.y, point.z);
|
||||
final GeoPoint point = new GeoPoint(PlanetModel.WGS84, lat, lon);
|
||||
fillFieldsData(point.x, point.y, point.z);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -63,40 +60,38 @@ public final class Geo3DPoint extends Field {
|
||||
*
|
||||
* @throws IllegalArgumentException if the field name is null or lat or lon are out of bounds
|
||||
*/
|
||||
public Geo3DPoint(String name, PlanetModel planetModel, double x, double y, double z) {
|
||||
public Geo3DPoint(String name, double x, double y, double z) {
|
||||
super(name, TYPE);
|
||||
this.planetModel = planetModel;
|
||||
fillFieldsData(planetModel, x, y, z);
|
||||
fillFieldsData(x, y, z);
|
||||
}
|
||||
|
||||
private void fillFieldsData(PlanetModel planetModel, double x, double y, double z) {
|
||||
private void fillFieldsData(double x, double y, double z) {
|
||||
byte[] bytes = new byte[12];
|
||||
encodeDimension(planetModel, x, bytes, 0);
|
||||
encodeDimension(planetModel, y, bytes, Integer.BYTES);
|
||||
encodeDimension(planetModel, z, bytes, 2*Integer.BYTES);
|
||||
encodeDimension(x, bytes, 0);
|
||||
encodeDimension(y, bytes, Integer.BYTES);
|
||||
encodeDimension(z, bytes, 2*Integer.BYTES);
|
||||
fieldsData = new BytesRef(bytes);
|
||||
}
|
||||
|
||||
// public helper methods (e.g. for queries)
|
||||
|
||||
/** Encode single dimension */
|
||||
public static void encodeDimension(PlanetModel planetModel, double value, byte bytes[], int offset) {
|
||||
NumericUtils.intToSortableBytes(Geo3DUtil.encodeValue(planetModel.getMaximumMagnitude(), value), bytes, offset);
|
||||
public static void encodeDimension(double value, byte bytes[], int offset) {
|
||||
NumericUtils.intToSortableBytes(Geo3DUtil.encodeValue(PlanetModel.WGS84.getMaximumMagnitude(), value), bytes, offset);
|
||||
}
|
||||
|
||||
/** Decode single dimension */
|
||||
public static double decodeDimension(PlanetModel planetModel, byte value[], int offset) {
|
||||
return Geo3DUtil.decodeValueCenter(planetModel.getMaximumMagnitude(), NumericUtils.sortableBytesToInt(value, offset));
|
||||
public static double decodeDimension(byte value[], int offset) {
|
||||
return Geo3DUtil.decodeValueCenter(PlanetModel.WGS84.getMaximumMagnitude(), NumericUtils.sortableBytesToInt(value, offset));
|
||||
}
|
||||
|
||||
/** Returns a query matching all points inside the provided shape.
|
||||
*
|
||||
* @param planetModel The {@link PlanetModel} to use, which must match what was used during indexing
|
||||
* @param field field name. must not be {@code null}.
|
||||
* @param shape Which {@link GeoShape} to match
|
||||
*/
|
||||
public static Query newShapeQuery(PlanetModel planetModel, String field, GeoShape shape) {
|
||||
return new PointInGeo3DShapeQuery(planetModel, field, shape);
|
||||
public static Query newShapeQuery(String field, GeoShape shape) {
|
||||
return new PointInGeo3DShapeQuery(field, shape);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -108,9 +103,9 @@ public final class Geo3DPoint extends Field {
|
||||
result.append(':');
|
||||
|
||||
BytesRef bytes = (BytesRef) fieldsData;
|
||||
result.append(" x=" + decodeDimension(planetModel, bytes.bytes, bytes.offset));
|
||||
result.append(" y=" + decodeDimension(planetModel, bytes.bytes, bytes.offset + Integer.BYTES));
|
||||
result.append(" z=" + decodeDimension(planetModel, bytes.bytes, bytes.offset + 2*Integer.BYTES));
|
||||
result.append(" x=" + decodeDimension(bytes.bytes, bytes.offset));
|
||||
result.append(" y=" + decodeDimension(bytes.bytes, bytes.offset + Integer.BYTES));
|
||||
result.append(" z=" + decodeDimension(bytes.bytes, bytes.offset + 2*Integer.BYTES));
|
||||
result.append('>');
|
||||
return result.toString();
|
||||
}
|
||||
|
@ -40,14 +40,19 @@ import org.apache.lucene.util.NumericUtils;
|
||||
|
||||
class PointInGeo3DShapeQuery extends Query {
|
||||
final String field;
|
||||
final PlanetModel planetModel;
|
||||
final GeoShape shape;
|
||||
|
||||
/** The lats/lons must be clockwise or counter-clockwise. */
|
||||
public PointInGeo3DShapeQuery(PlanetModel planetModel, String field, GeoShape shape) {
|
||||
public PointInGeo3DShapeQuery(String field, GeoShape shape) {
|
||||
this.field = field;
|
||||
this.planetModel = planetModel;
|
||||
this.shape = shape;
|
||||
|
||||
if (shape instanceof BasePlanetObject) {
|
||||
BasePlanetObject planetObject = (BasePlanetObject) shape;
|
||||
if (planetObject.getPlanetModel().equals(PlanetModel.WGS84) == false) {
|
||||
throw new IllegalArgumentException("this qurey requires PlanetModel.WGS84, but got: " + planetObject.getPlanetModel());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -88,7 +93,7 @@ class PointInGeo3DShapeQuery extends Query {
|
||||
assert xyzSolid.getRelationship(shape) == GeoArea.WITHIN || xyzSolid.getRelationship(shape) == GeoArea.OVERLAPS: "expected WITHIN (1) or OVERLAPS (2) but got " + xyzSolid.getRelationship(shape) + "; shape="+shape+"; XYZSolid="+xyzSolid;
|
||||
*/
|
||||
|
||||
double planetMax = planetModel.getMaximumMagnitude();
|
||||
double planetMax = PlanetModel.WGS84.getMaximumMagnitude();
|
||||
|
||||
DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc());
|
||||
|
||||
@ -103,9 +108,9 @@ class PointInGeo3DShapeQuery extends Query {
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) {
|
||||
assert packedValue.length == 12;
|
||||
double x = Geo3DPoint.decodeDimension(planetModel, packedValue, 0);
|
||||
double y = Geo3DPoint.decodeDimension(planetModel, packedValue, Integer.BYTES);
|
||||
double z = Geo3DPoint.decodeDimension(planetModel, packedValue, 2 * Integer.BYTES);
|
||||
double x = Geo3DPoint.decodeDimension(packedValue, 0);
|
||||
double y = Geo3DPoint.decodeDimension(packedValue, Integer.BYTES);
|
||||
double z = Geo3DPoint.decodeDimension(packedValue, 2 * Integer.BYTES);
|
||||
if (shape.isWithin(x, y, z)) {
|
||||
result.add(docID);
|
||||
}
|
||||
@ -129,7 +134,7 @@ class PointInGeo3DShapeQuery extends Query {
|
||||
assert yMin <= yMax;
|
||||
assert zMin <= zMax;
|
||||
|
||||
GeoArea xyzSolid = GeoAreaFactory.makeGeoArea(planetModel, xMin, xMax, yMin, yMax, zMin, zMax);
|
||||
GeoArea xyzSolid = GeoAreaFactory.makeGeoArea(PlanetModel.WGS84, xMin, xMax, yMin, yMax, zMin, zMax);
|
||||
|
||||
switch(xyzSolid.getRelationship(shape)) {
|
||||
case GeoArea.CONTAINS:
|
||||
@ -165,10 +170,6 @@ class PointInGeo3DShapeQuery extends Query {
|
||||
return field;
|
||||
}
|
||||
|
||||
public PlanetModel getPlanetModel() {
|
||||
return planetModel;
|
||||
}
|
||||
|
||||
public GeoShape getShape() {
|
||||
return shape;
|
||||
}
|
||||
@ -182,13 +183,12 @@ class PointInGeo3DShapeQuery extends Query {
|
||||
|
||||
PointInGeo3DShapeQuery that = (PointInGeo3DShapeQuery) o;
|
||||
|
||||
return planetModel.equals(that.planetModel) && shape.equals(that.shape);
|
||||
return shape.equals(that.shape);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final int hashCode() {
|
||||
int result = super.hashCode();
|
||||
result = 31 * result + planetModel.hashCode();
|
||||
result = 31 * result + shape.hashCode();
|
||||
return result;
|
||||
}
|
||||
@ -203,8 +203,6 @@ class PointInGeo3DShapeQuery extends Query {
|
||||
sb.append(this.field);
|
||||
sb.append(':');
|
||||
}
|
||||
sb.append(" PlanetModel: ");
|
||||
sb.append(planetModel);
|
||||
sb.append(" Shape: ");
|
||||
sb.append(shape);
|
||||
return sb.toString();
|
||||
|
@ -27,12 +27,12 @@ import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
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.lucene60.Lucene60PointReader;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60PointWriter;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
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.Field;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
@ -76,21 +76,21 @@ public class TestGeo3DPoint extends LuceneTestCase {
|
||||
int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
|
||||
double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
|
||||
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()) {
|
||||
@Override
|
||||
public PointFormat pointFormat() {
|
||||
return new PointFormat() {
|
||||
public PointsFormat pointsFormat() {
|
||||
return new PointsFormat() {
|
||||
@Override
|
||||
public PointWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
|
||||
return new Lucene60PointWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
|
||||
public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
|
||||
return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointReader fieldsReader(SegmentReadState readState) throws IOException {
|
||||
return new Lucene60PointReader(readState);
|
||||
public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
|
||||
return new Lucene60PointsReader(readState);
|
||||
}
|
||||
};
|
||||
}
|
||||
@ -106,13 +106,12 @@ public class TestGeo3DPoint extends LuceneTestCase {
|
||||
iwc.setCodec(getCodec());
|
||||
IndexWriter w = new IndexWriter(dir, iwc);
|
||||
Document doc = new Document();
|
||||
doc.add(new Geo3DPoint("field", PlanetModel.WGS84, toRadians(50.7345267), toRadians(-97.5303555)));
|
||||
doc.add(new Geo3DPoint("field", toRadians(50.7345267), toRadians(-97.5303555)));
|
||||
w.addDocument(doc);
|
||||
IndexReader r = DirectoryReader.open(w);
|
||||
// We can't wrap with "exotic" readers because the query must see the BKD3DDVFormat:
|
||||
IndexSearcher s = newSearcher(r, false);
|
||||
assertEquals(1, s.search(Geo3DPoint.newShapeQuery(PlanetModel.WGS84,
|
||||
"field",
|
||||
assertEquals(1, s.search(Geo3DPoint.newShapeQuery("field",
|
||||
GeoCircleFactory.makeGeoCircle(PlanetModel.WGS84, toRadians(50), toRadians(-97), Math.PI/180.)), 1).totalHits);
|
||||
w.close();
|
||||
r.close();
|
||||
@ -640,8 +639,6 @@ public class TestGeo3DPoint extends LuceneTestCase {
|
||||
private static void verify(double[] lats, double[] lons) throws Exception {
|
||||
IndexWriterConfig iwc = newIndexWriterConfig();
|
||||
|
||||
PlanetModel planetModel = getPlanetModel();
|
||||
|
||||
// Else we can get O(N^2) merging:
|
||||
int mbd = iwc.getMaxBufferedDocs();
|
||||
if (mbd != -1 && mbd < lats.length/100) {
|
||||
@ -662,7 +659,7 @@ public class TestGeo3DPoint extends LuceneTestCase {
|
||||
doc.add(newStringField("id", ""+id, Field.Store.NO));
|
||||
doc.add(new NumericDocValuesField("id", id));
|
||||
if (Double.isNaN(lats[id]) == false) {
|
||||
doc.add(new Geo3DPoint("point", planetModel, lats[id], lons[id]));
|
||||
doc.add(new Geo3DPoint("point", lats[id], lons[id]));
|
||||
}
|
||||
w.addDocument(doc);
|
||||
if (id > 0 && random().nextInt(100) == 42) {
|
||||
@ -710,13 +707,13 @@ public class TestGeo3DPoint extends LuceneTestCase {
|
||||
|
||||
for (int iter=0;iter<iters && failed.get() == false;iter++) {
|
||||
|
||||
GeoShape shape = randomShape(planetModel);
|
||||
GeoShape shape = randomShape(PlanetModel.WGS84);
|
||||
|
||||
if (VERBOSE) {
|
||||
System.err.println("\n" + Thread.currentThread() + ": TEST: iter=" + iter + " shape="+shape);
|
||||
}
|
||||
|
||||
Query query = Geo3DPoint.newShapeQuery(planetModel, "point", shape);
|
||||
Query query = Geo3DPoint.newShapeQuery("point", shape);
|
||||
|
||||
if (VERBOSE) {
|
||||
System.err.println(" using query: " + query);
|
||||
@ -753,10 +750,10 @@ public class TestGeo3DPoint extends LuceneTestCase {
|
||||
if (Double.isNaN(lats[id]) == false) {
|
||||
|
||||
// Accurate point:
|
||||
GeoPoint point1 = new GeoPoint(planetModel, lats[id], lons[id]);
|
||||
GeoPoint point1 = new GeoPoint(PlanetModel.WGS84, lats[id], lons[id]);
|
||||
|
||||
// Quantized point (32 bits per dim):
|
||||
GeoPoint point2 = quantize(planetModel.getMaximumMagnitude(), point1);
|
||||
GeoPoint point2 = quantize(PlanetModel.WGS84.getMaximumMagnitude(), point1);
|
||||
|
||||
if (shape.isWithin(point1) != shape.isWithin(point2)) {
|
||||
if (VERBOSE) {
|
||||
@ -789,13 +786,13 @@ public class TestGeo3DPoint extends LuceneTestCase {
|
||||
}
|
||||
|
||||
public void testToString() {
|
||||
Geo3DPoint point = new Geo3DPoint("point", PlanetModel.SPHERE, toRadians(44.244272), toRadians(7.769736));
|
||||
assertEquals("Geo3DPoint <point: x=0.9242545719837093 y=0.06276412683667808 z=0.37658219569203544>", point.toString());
|
||||
Geo3DPoint point = new Geo3DPoint("point", toRadians(44.244272), toRadians(7.769736));
|
||||
assertEquals("Geo3DPoint <point: x=0.9248467864160119 y=0.06280434265368656 z=0.37682349005486243>", point.toString());
|
||||
}
|
||||
|
||||
public void testShapeQueryToString() {
|
||||
assertEquals("PointInGeo3DShapeQuery: field=point: PlanetModel: PlanetModel.SPHERE Shape: GeoStandardCircle: {planetmodel=PlanetModel.SPHERE, center=[lat=0.3861041107739683, lon=0.06780373760536706], radius=0.1(5.729577951308232)}",
|
||||
Geo3DPoint.newShapeQuery(PlanetModel.SPHERE, "point", GeoCircleFactory.makeGeoCircle(PlanetModel.SPHERE, toRadians(44.244272), toRadians(7.769736), 0.1)).toString());
|
||||
assertEquals("PointInGeo3DShapeQuery: field=point: Shape: GeoStandardCircle: {planetmodel=PlanetModel.WGS84, center=[lat=0.3861041107739683, lon=0.06780373760536706], radius=0.1(5.729577951308232)}",
|
||||
Geo3DPoint.newShapeQuery("point", GeoCircleFactory.makeGeoCircle(PlanetModel.WGS84, toRadians(44.244272), toRadians(7.769736), 0.1)).toString());
|
||||
}
|
||||
|
||||
private static Directory getDirectory() {
|
||||
|
@ -16,11 +16,11 @@
|
||||
*/
|
||||
package org.apache.lucene.codecs.asserting;
|
||||
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
@ -53,7 +53,7 @@ public class AssertingCodec extends FilterCodec {
|
||||
private final LiveDocsFormat liveDocs = new AssertingLiveDocsFormat();
|
||||
private final PostingsFormat defaultFormat = new AssertingPostingsFormat();
|
||||
private final DocValuesFormat defaultDVFormat = new AssertingDocValuesFormat();
|
||||
private final PointFormat pointFormat = new AssertingPointFormat();
|
||||
private final PointsFormat pointsFormat = new AssertingPointsFormat();
|
||||
|
||||
public AssertingCodec() {
|
||||
super("Asserting", TestUtil.getDefaultCodec());
|
||||
@ -90,8 +90,8 @@ public class AssertingCodec extends FilterCodec {
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointFormat pointFormat() {
|
||||
return pointFormat;
|
||||
public PointsFormat pointsFormat() {
|
||||
return pointsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -20,9 +20,9 @@ import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PointsReader;
|
||||
import org.apache.lucene.codecs.PointsWriter;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
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.
|
||||
*/
|
||||
|
||||
public final class AssertingPointFormat extends PointFormat {
|
||||
private final PointFormat in;
|
||||
public final class AssertingPointsFormat extends PointsFormat {
|
||||
private final PointsFormat in;
|
||||
|
||||
/** Create a new AssertingPointFormat */
|
||||
public AssertingPointFormat() {
|
||||
this(TestUtil.getDefaultCodec().pointFormat());
|
||||
/** Create a new AssertingPointsFormat */
|
||||
public AssertingPointsFormat() {
|
||||
this(TestUtil.getDefaultCodec().pointsFormat());
|
||||
}
|
||||
|
||||
/**
|
||||
* Expert: Create an AssertingPointFormat.
|
||||
* Expert: Create an AssertingPointsFormat.
|
||||
* 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
|
||||
// this with a separate codec...
|
||||
public AssertingPointFormat(PointFormat in) {
|
||||
public AssertingPointsFormat(PointsFormat in) {
|
||||
this.in = in;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointWriter fieldsWriter(SegmentWriteState state) throws IOException {
|
||||
return new AssertingPointWriter(state, in.fieldsWriter(state));
|
||||
public PointsWriter fieldsWriter(SegmentWriteState state) throws IOException {
|
||||
return new AssertingPointsWriter(state, in.fieldsWriter(state));
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointReader fieldsReader(SegmentReadState state) throws IOException {
|
||||
return new AssertingPointReader(state.segmentInfo.maxDoc(), in.fieldsReader(state));
|
||||
public PointsReader fieldsReader(SegmentReadState state) throws IOException {
|
||||
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 */
|
||||
@ -144,11 +144,11 @@ public final class AssertingPointFormat extends PointFormat {
|
||||
}
|
||||
}
|
||||
|
||||
static class AssertingPointReader extends PointReader {
|
||||
private final PointReader in;
|
||||
static class AssertingPointsReader extends PointsReader {
|
||||
private final PointsReader in;
|
||||
private final int maxDoc;
|
||||
|
||||
AssertingPointReader(int maxDoc, PointReader in) {
|
||||
AssertingPointsReader(int maxDoc, PointsReader in) {
|
||||
this.in = in;
|
||||
this.maxDoc = maxDoc;
|
||||
// do a few simple checks on init
|
||||
@ -189,8 +189,8 @@ public final class AssertingPointFormat extends PointFormat {
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointReader getMergeInstance() throws IOException {
|
||||
return new AssertingPointReader(maxDoc, in.getMergeInstance());
|
||||
public PointsReader getMergeInstance() throws IOException {
|
||||
return new AssertingPointsReader(maxDoc, in.getMergeInstance());
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -242,15 +242,15 @@ public final class AssertingPointFormat extends PointFormat {
|
||||
}
|
||||
}
|
||||
|
||||
static class AssertingPointWriter extends PointWriter {
|
||||
private final PointWriter in;
|
||||
static class AssertingPointsWriter extends PointsWriter {
|
||||
private final PointsWriter in;
|
||||
|
||||
AssertingPointWriter(SegmentWriteState writeState, PointWriter in) {
|
||||
AssertingPointsWriter(SegmentWriteState writeState, PointsWriter in) {
|
||||
this.in = in;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeField(FieldInfo fieldInfo, PointReader values) throws IOException {
|
||||
public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
|
||||
if (fieldInfo.getPointDimensionCount() == 0) {
|
||||
throw new IllegalArgumentException("writing field=\"" + fieldInfo.name + "\" but pointDimensionalCount is 0");
|
||||
}
|
@ -25,7 +25,7 @@ import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.LiveDocsFormat;
|
||||
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.SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
@ -92,8 +92,8 @@ public class CrankyCodec extends FilterCodec {
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointFormat pointFormat() {
|
||||
return new CrankyPointFormat(delegate.pointFormat(), random);
|
||||
public PointsFormat pointsFormat() {
|
||||
return new CrankyPointsFormat(delegate.pointsFormat(), random);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -19,9 +19,9 @@ package org.apache.lucene.codecs.cranky;
|
||||
import java.io.IOException;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PointsReader;
|
||||
import org.apache.lucene.codecs.PointsWriter;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
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.IOContext;
|
||||
|
||||
class CrankyPointFormat extends PointFormat {
|
||||
PointFormat delegate;
|
||||
class CrankyPointsFormat extends PointsFormat {
|
||||
PointsFormat delegate;
|
||||
Random random;
|
||||
|
||||
CrankyPointFormat(PointFormat delegate, Random random) {
|
||||
CrankyPointsFormat(PointsFormat delegate, Random random) {
|
||||
this.delegate = delegate;
|
||||
this.random = random;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointWriter fieldsWriter(SegmentWriteState state) throws IOException {
|
||||
return new CrankyPointWriter(delegate.fieldsWriter(state), random);
|
||||
public PointsWriter fieldsWriter(SegmentWriteState state) throws IOException {
|
||||
return new CrankyPointsWriter(delegate.fieldsWriter(state), random);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointReader fieldsReader(SegmentReadState state) throws IOException {
|
||||
return new CrankyPointReader(delegate.fieldsReader(state), random);
|
||||
public PointsReader fieldsReader(SegmentReadState state) throws IOException {
|
||||
return new CrankyPointsReader(delegate.fieldsReader(state), random);
|
||||
}
|
||||
|
||||
static class CrankyPointWriter extends PointWriter {
|
||||
final PointWriter delegate;
|
||||
static class CrankyPointsWriter extends PointsWriter {
|
||||
final PointsWriter delegate;
|
||||
final Random random;
|
||||
|
||||
public CrankyPointWriter(PointWriter delegate, Random random) {
|
||||
public CrankyPointsWriter(PointsWriter delegate, Random random) {
|
||||
this.delegate = delegate;
|
||||
this.random = random;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeField(FieldInfo fieldInfo, PointReader values) throws IOException {
|
||||
public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
|
||||
if (random.nextInt(100) == 0) {
|
||||
throw new IOException("Fake IOException");
|
||||
}
|
||||
@ -97,10 +97,10 @@ class CrankyPointFormat extends PointFormat {
|
||||
}
|
||||
}
|
||||
|
||||
static class CrankyPointReader extends PointReader {
|
||||
final PointReader delegate;
|
||||
static class CrankyPointsReader extends PointsReader {
|
||||
final PointsReader delegate;
|
||||
final Random random;
|
||||
public CrankyPointReader(PointReader delegate, Random random) {
|
||||
public CrankyPointsReader(PointsReader delegate, Random random) {
|
||||
this.delegate = delegate;
|
||||
this.random = random;
|
||||
}
|
@ -47,11 +47,11 @@ import org.apache.lucene.util.TestUtil;
|
||||
* Abstract class to do basic tests for a points format.
|
||||
* NOTE: This test focuses on the points impl, nothing else.
|
||||
* The [stretch] goal is for this test to be
|
||||
* so thorough in testing a new 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,
|
||||
* 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! */
|
||||
public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCase {
|
||||
public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCase {
|
||||
|
||||
@Override
|
||||
protected void addRandomFields(Document doc) {
|
@ -28,21 +28,21 @@ import java.util.Random;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.PointFormat;
|
||||
import org.apache.lucene.codecs.PointReader;
|
||||
import org.apache.lucene.codecs.PointWriter;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PointsReader;
|
||||
import org.apache.lucene.codecs.PointsWriter;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.asserting.AssertingCodec;
|
||||
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.blockterms.LuceneFixedGap;
|
||||
import org.apache.lucene.codecs.blockterms.LuceneVarGapDocFreqInterval;
|
||||
import org.apache.lucene.codecs.blockterms.LuceneVarGapFixedInterval;
|
||||
import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat;
|
||||
import org.apache.lucene.codecs.bloom.TestBloomFilteredLucenePostings;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60PointReader;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60PointWriter;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60PointsReader;
|
||||
import org.apache.lucene.codecs.lucene60.Lucene60PointsWriter;
|
||||
import org.apache.lucene.codecs.memory.DirectDocValuesFormat;
|
||||
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.FSTOrdPostingsFormat;
|
||||
@ -95,16 +95,16 @@ public class RandomCodec extends AssertingCodec {
|
||||
private final double maxMBSortInHeap;
|
||||
|
||||
@Override
|
||||
public PointFormat pointFormat() {
|
||||
return new AssertingPointFormat(new PointFormat() {
|
||||
public PointsFormat pointsFormat() {
|
||||
return new AssertingPointsFormat(new PointsFormat() {
|
||||
@Override
|
||||
public PointWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
|
||||
return new Lucene60PointWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
|
||||
public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
|
||||
return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointReader fieldsReader(SegmentReadState readState) throws IOException {
|
||||
return new Lucene60PointReader(readState);
|
||||
public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
|
||||
return new Lucene60PointsReader(readState);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -17,10 +17,10 @@
|
||||
package org.apache.lucene.codecs.asserting;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.index.BasePointFormatTestCase;
|
||||
import org.apache.lucene.index.BasePointsFormatTestCase;
|
||||
|
||||
/** Test AssertingPointFormat directly */
|
||||
public class TestAssertingPointFormat extends BasePointFormatTestCase {
|
||||
/** Test AssertingPointsFormat directly */
|
||||
public class TestAssertingPointsFormat extends BasePointsFormatTestCase {
|
||||
private final Codec codec = new AssertingCodec();
|
||||
|
||||
@Override
|
@ -266,6 +266,14 @@ Bug Fixes
|
||||
|
||||
* SOLR-8779: Fix missing InterruptedException handling in ZkStateReader.java (Varun Thacker)
|
||||
|
||||
* SOLR-8449: Fix the core restore functionality to allow restoring multiple times on the same core
|
||||
(Johannes Brucher, Varun Thacker)
|
||||
|
||||
* SOLR-8155: JSON Facet API - field faceting on a multi-valued string field without
|
||||
docValues (i.e. UnInvertedField implementation), but with a prefix or with a sort
|
||||
other than count, resulted in incorrect results. This has been fixed, and facet.prefix
|
||||
support for facet.method=uif has been enabled. (Mikhail Khludnev, yonik)
|
||||
|
||||
Optimizations
|
||||
----------------------
|
||||
* SOLR-7876: Speed up queries and operations that use many terms when timeAllowed has not been
|
||||
@ -290,6 +298,9 @@ Optimizations
|
||||
|
||||
* SOLR-8720: ZkController#publishAndWaitForDownStates should use #publishNodeAsDown. (Mark Miller)
|
||||
|
||||
* SOLR-8745: Deprecate costly ZkStateReader.updateClusterState(), replace with a narrow
|
||||
forceUpdateCollection(collection) (Scott Blum via shalin)
|
||||
|
||||
Other Changes
|
||||
----------------------
|
||||
|
||||
|
@ -444,7 +444,7 @@
|
||||
|
||||
<!-- copy KEYS to the release folder -->
|
||||
<target name="-dist-keys">
|
||||
<get src="http://people.apache.org/keys/group/lucene.asc"
|
||||
<get src="http://home.apache.org/keys/group/lucene.asc"
|
||||
dest="${package.dir}/KEYS"/>
|
||||
</target>
|
||||
|
||||
|
@ -646,7 +646,6 @@ public class MorphlineGoLiveMiniMRTest extends AbstractFullDistribZkTestBase {
|
||||
}
|
||||
|
||||
Thread.sleep(200);
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
}
|
||||
|
||||
if (TEST_NIGHTLY) {
|
||||
|
@ -462,7 +462,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
|
||||
public void publishActiveIfRegisteredAndNotActive(SolrCore core) throws KeeperException, InterruptedException {
|
||||
if (core.getCoreDescriptor().getCloudDescriptor().hasRegistered()) {
|
||||
ZkStateReader zkStateReader = zkController.getZkStateReader();
|
||||
zkStateReader.updateClusterState();
|
||||
zkStateReader.forceUpdateCollection(collection);
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
Replica rep = (clusterState == null) ? null
|
||||
: clusterState.getReplica(collection, leaderProps.getStr(ZkStateReader.CORE_NODE_NAME_PROP));
|
||||
|
@ -244,12 +244,6 @@ public class LeaderInitiatedRecoveryThread extends Thread {
|
||||
|
||||
// see if the replica's node is still live, if not, no need to keep doing this loop
|
||||
ZkStateReader zkStateReader = zkController.getZkStateReader();
|
||||
try {
|
||||
zkStateReader.updateClusterState();
|
||||
} catch (Exception exc) {
|
||||
log.warn("Error when updating cluster state: "+exc);
|
||||
}
|
||||
|
||||
if (!zkStateReader.getClusterState().liveNodesContain(replicaNodeName)) {
|
||||
log.warn("Node "+replicaNodeName+" hosting core "+coreNeedingRecovery+
|
||||
" is no longer live. No need to keep trying to tell it to recover!");
|
||||
|
@ -1371,7 +1371,6 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
|
||||
return;
|
||||
}
|
||||
Thread.sleep(1000);
|
||||
zkStateReader.updateClusterState();
|
||||
}
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR,
|
||||
"Could not find new slice " + sliceName + " in collection " + collectionName
|
||||
|
@ -883,7 +883,7 @@ public final class ZkController {
|
||||
}
|
||||
|
||||
// make sure we have an update cluster state right away
|
||||
zkStateReader.updateClusterState();
|
||||
zkStateReader.forceUpdateCollection(collection);
|
||||
return shardId;
|
||||
} finally {
|
||||
MDCLoggingContext.clear();
|
||||
|
@ -361,7 +361,7 @@ public class CdcrRequestHandler extends RequestHandlerBase implements SolrCoreAw
|
||||
throws IOException, SolrServerException {
|
||||
ZkController zkController = core.getCoreDescriptor().getCoreContainer().getZkController();
|
||||
try {
|
||||
zkController.getZkStateReader().updateClusterState();
|
||||
zkController.getZkStateReader().forceUpdateCollection(collection);
|
||||
} catch (Exception e) {
|
||||
log.warn("Error when updating cluster state", e);
|
||||
}
|
||||
|
@ -19,6 +19,9 @@ package org.apache.solr.handler;
|
||||
import java.lang.invoke.MethodHandles;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Date;
|
||||
import java.util.Locale;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.Future;
|
||||
|
||||
@ -55,7 +58,8 @@ public class RestoreCore implements Callable<Boolean> {
|
||||
private boolean doRestore() throws Exception {
|
||||
|
||||
Path backupPath = Paths.get(backupLocation).resolve(backupName);
|
||||
String restoreIndexName = "restore." + backupName;
|
||||
SimpleDateFormat dateFormat = new SimpleDateFormat(SnapShooter.DATE_FMT, Locale.ROOT);
|
||||
String restoreIndexName = "restore." + dateFormat.format(new Date());
|
||||
String restoreIndexPath = core.getDataDir() + restoreIndexName;
|
||||
|
||||
Directory restoreIndexDir = null;
|
||||
|
@ -57,9 +57,6 @@ public class ClusterStatus {
|
||||
@SuppressWarnings("unchecked")
|
||||
public void getClusterStatus(NamedList results)
|
||||
throws KeeperException, InterruptedException {
|
||||
zkStateReader.updateClusterState();
|
||||
|
||||
|
||||
// read aliases
|
||||
Aliases aliases = zkStateReader.getAliases();
|
||||
Map<String, List<String>> collectionVsAliases = new HashMap<>();
|
||||
|
@ -920,8 +920,6 @@ public class CollectionsHandler extends RequestHandlerBase {
|
||||
+ (checkLeaderOnly ? "leaders" : "replicas"));
|
||||
ZkStateReader zkStateReader = cc.getZkController().getZkStateReader();
|
||||
for (int i = 0; i < numRetries; i++) {
|
||||
|
||||
zkStateReader.updateClusterState();
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
|
||||
Collection<Slice> shards = clusterState.getSlices(collectionName);
|
||||
|
@ -461,6 +461,7 @@ enum CoreAdminOperation {
|
||||
// to accept updates
|
||||
CloudDescriptor cloudDescriptor = core.getCoreDescriptor()
|
||||
.getCloudDescriptor();
|
||||
String collection = cloudDescriptor.getCollectionName();
|
||||
|
||||
if (retry % 15 == 0) {
|
||||
if (retry > 0 && log.isInfoEnabled())
|
||||
@ -470,7 +471,7 @@ enum CoreAdminOperation {
|
||||
waitForState + "; forcing ClusterState update from ZooKeeper");
|
||||
|
||||
// force a cluster state update
|
||||
coreContainer.getZkController().getZkStateReader().updateClusterState();
|
||||
coreContainer.getZkController().getZkStateReader().forceUpdateCollection(collection);
|
||||
}
|
||||
|
||||
if (maxTries == 0) {
|
||||
@ -483,7 +484,6 @@ enum CoreAdminOperation {
|
||||
}
|
||||
|
||||
ClusterState clusterState = coreContainer.getZkController().getClusterState();
|
||||
String collection = cloudDescriptor.getCollectionName();
|
||||
Slice slice = clusterState.getSlice(collection, cloudDescriptor.getShardId());
|
||||
if (slice != null) {
|
||||
final Replica replica = slice.getReplicasMap().get(coreNodeName);
|
||||
@ -937,4 +937,4 @@ enum CoreAdminOperation {
|
||||
return size;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
@ -79,7 +79,7 @@ class RebalanceLeaders {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
String.format(Locale.ROOT, "The " + COLLECTION_PROP + " is required for the Rebalance Leaders command."));
|
||||
}
|
||||
coreContainer.getZkController().getZkStateReader().updateClusterState();
|
||||
coreContainer.getZkController().getZkStateReader().forceUpdateCollection(collectionName);
|
||||
ClusterState clusterState = coreContainer.getZkController().getClusterState();
|
||||
DocCollection dc = clusterState.getCollection(collectionName);
|
||||
if (dc == null) {
|
||||
|
@ -487,17 +487,7 @@ public class SimpleFacets {
|
||||
jsonFacet.put("limit", limit);
|
||||
jsonFacet.put("mincount", mincount);
|
||||
jsonFacet.put("missing", missing);
|
||||
|
||||
if (prefix!=null) {
|
||||
// presumably it supports single-value, but at least now returns wrong results on multi-value
|
||||
throw new SolrException (
|
||||
SolrException.ErrorCode.BAD_REQUEST,
|
||||
FacetParams.FACET_PREFIX+"="+prefix+
|
||||
" are not supported by "+FacetParams.FACET_METHOD+"="+FacetParams.FACET_METHOD_uif+
|
||||
" for field:"+ field
|
||||
//jsonFacet.put("prefix", prefix);
|
||||
);
|
||||
}
|
||||
jsonFacet.put("prefix", prefix);
|
||||
jsonFacet.put("numBuckets", params.getFieldBool(field, "numBuckets", false));
|
||||
jsonFacet.put("allBuckets", params.getFieldBool(field, "allBuckets", false));
|
||||
jsonFacet.put("method", "uif");
|
||||
|
@ -499,10 +499,11 @@ public class UnInvertedField extends DocTermOrds {
|
||||
if (delta==0) break;
|
||||
tnum += delta - TNUM_OFFSET;
|
||||
int arrIdx = tnum - startTermIndex;
|
||||
if (arrIdx < 0) continue;
|
||||
if (arrIdx >= nTerms) break;
|
||||
countAcc.incrementCount(arrIdx, 1);
|
||||
processor.collectFirstPhase(segDoc, arrIdx);
|
||||
if (arrIdx >= 0) {
|
||||
if (arrIdx >= nTerms) break;
|
||||
countAcc.incrementCount(arrIdx, 1);
|
||||
processor.collectFirstPhase(segDoc, arrIdx);
|
||||
}
|
||||
delta = 0;
|
||||
}
|
||||
code >>>= 8;
|
||||
|
@ -215,9 +215,6 @@ public class TestRandomDVFaceting extends SolrTestCaseJ4 {
|
||||
List<String> methods = multiValued ? multiValuedMethods : singleValuedMethods;
|
||||
List<String> responses = new ArrayList<>(methods.size());
|
||||
for (String method : methods) {
|
||||
if (method.equals("uif") && params.get("facet.prefix")!=null) {
|
||||
continue; // it's not supported there
|
||||
}
|
||||
if (method.equals("dv")) {
|
||||
params.set("facet.field", "{!key="+facet_field+"}"+facet_field+"_dv");
|
||||
params.set("facet.method",(String) null);
|
||||
|
@ -635,7 +635,6 @@ public class BaseCdcrDistributedZkTest extends AbstractDistribZkTestBase {
|
||||
try {
|
||||
cloudClient.connect();
|
||||
ZkStateReader zkStateReader = cloudClient.getZkStateReader();
|
||||
zkStateReader.updateClusterState();
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
DocCollection coll = clusterState.getCollection(collection);
|
||||
|
||||
|
@ -552,7 +552,7 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
|
||||
|
||||
Thread.sleep(5000);
|
||||
ChaosMonkey.start(cloudJettys.get(0).jetty);
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
cloudClient.getZkStateReader().forceUpdateCollection("multiunload2");
|
||||
try {
|
||||
cloudClient.getZkStateReader().getLeaderRetry("multiunload2", "shard1", 30000);
|
||||
} catch (SolrException e) {
|
||||
@ -830,7 +830,7 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
|
||||
|
||||
// we added a role of none on these creates - check for it
|
||||
ZkStateReader zkStateReader = getCommonCloudSolrClient().getZkStateReader();
|
||||
zkStateReader.updateClusterState();
|
||||
zkStateReader.forceUpdateCollection(oneInstanceCollection2);
|
||||
Map<String,Slice> slices = zkStateReader.getClusterState().getSlicesMap(oneInstanceCollection2);
|
||||
assertNotNull(slices);
|
||||
String roles = slices.get("slice1").getReplicasMap().values().iterator().next().getStr(ZkStateReader.ROLES_PROP);
|
||||
|
@ -205,7 +205,7 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
|
||||
|
||||
// TODO: assert we didnt kill everyone
|
||||
|
||||
zkStateReader.updateClusterState();
|
||||
zkStateReader.updateLiveNodes();
|
||||
assertTrue(zkStateReader.getClusterState().getLiveNodes().size() > 0);
|
||||
|
||||
|
||||
|
@ -206,7 +206,7 @@ public class ChaosMonkeyShardSplitTest extends ShardSplitTest {
|
||||
for (int i = 0; i < 30; i++) {
|
||||
Thread.sleep(3000);
|
||||
ZkStateReader zkStateReader = cloudClient.getZkStateReader();
|
||||
zkStateReader.updateClusterState();
|
||||
zkStateReader.forceUpdateCollection("collection1");
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
DocCollection collection1 = clusterState.getCollection("collection1");
|
||||
Slice slice = collection1.getSlice("shard1");
|
||||
|
@ -103,7 +103,7 @@ public class CollectionReloadTest extends AbstractFullDistribZkTestBase {
|
||||
timeout = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeoutSecs, TimeUnit.SECONDS);
|
||||
while (System.nanoTime() < timeout) {
|
||||
// state of leader should be active after session loss recovery - see SOLR-7338
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
cloudClient.getZkStateReader().forceUpdateCollection(testCollectionName);
|
||||
ClusterState cs = cloudClient.getZkStateReader().getClusterState();
|
||||
Slice slice = cs.getSlice(testCollectionName, shardId);
|
||||
replicaState = slice.getReplica(leader.getName()).getStr(ZkStateReader.STATE_PROP);
|
||||
|
@ -97,7 +97,7 @@ public class CollectionTooManyReplicasTest extends AbstractFullDistribZkTestBase
|
||||
assertEquals(0, response.getStatus());
|
||||
|
||||
ZkStateReader zkStateReader = getCommonCloudSolrClient().getZkStateReader();
|
||||
zkStateReader.updateClusterState();
|
||||
zkStateReader.forceUpdateCollection(collectionName);
|
||||
Slice slice = zkStateReader.getClusterState().getSlicesMap(collectionName).get("shard1");
|
||||
|
||||
Replica rep = null;
|
||||
@ -194,7 +194,7 @@ public class CollectionTooManyReplicasTest extends AbstractFullDistribZkTestBase
|
||||
// And finally, insure that there are all the replcias we expect. We should have shards 1, 2 and 4 and each
|
||||
// should have exactly two replicas
|
||||
ZkStateReader zkStateReader = getCommonCloudSolrClient().getZkStateReader();
|
||||
zkStateReader.updateClusterState();
|
||||
zkStateReader.forceUpdateCollection(collectionName);
|
||||
Map<String, Slice> slices = zkStateReader.getClusterState().getSlicesMap(collectionName);
|
||||
assertEquals("There should be exaclty four slices", slices.size(), 4);
|
||||
assertNotNull("shardstart should exist", slices.get("shardstart"));
|
||||
@ -275,7 +275,7 @@ public class CollectionTooManyReplicasTest extends AbstractFullDistribZkTestBase
|
||||
|
||||
private List<String> getAllNodeNames(String collectionName) throws KeeperException, InterruptedException {
|
||||
ZkStateReader zkStateReader = getCommonCloudSolrClient().getZkStateReader();
|
||||
zkStateReader.updateClusterState();
|
||||
zkStateReader.forceUpdateCollection(collectionName);
|
||||
Slice slice = zkStateReader.getClusterState().getSlicesMap(collectionName).get("shard1");
|
||||
|
||||
List<String> nodes = new ArrayList<>();
|
||||
|
@ -368,7 +368,6 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa
|
||||
}
|
||||
|
||||
Thread.sleep(200);
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
}
|
||||
|
||||
assertFalse("Still found collection that should be gone", cloudClient.getZkStateReader().getClusterState().hasCollection("halfdeletedcollection2"));
|
||||
@ -540,8 +539,6 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa
|
||||
}
|
||||
|
||||
private void testNoCollectionSpecified() throws Exception {
|
||||
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
assertFalse(cloudClient.getZkStateReader().getClusterState().hasCollection("corewithnocollection"));
|
||||
assertFalse(cloudClient.getZkStateReader().getClusterState().hasCollection("corewithnocollection2"));
|
||||
|
||||
@ -565,13 +562,13 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa
|
||||
makeRequest(getBaseUrl((HttpSolrClient) clients.get(1)), createCmd);
|
||||
|
||||
// in both cases, the collection should have default to the core name
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
cloudClient.getZkStateReader().forceUpdateCollection("corewithnocollection");
|
||||
cloudClient.getZkStateReader().forceUpdateCollection("corewithnocollection2");
|
||||
assertTrue(cloudClient.getZkStateReader().getClusterState().hasCollection("corewithnocollection"));
|
||||
assertTrue(cloudClient.getZkStateReader().getClusterState().hasCollection("corewithnocollection2"));
|
||||
}
|
||||
|
||||
private void testNoConfigSetExist() throws Exception {
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
assertFalse(cloudClient.getZkStateReader().getClusterState().hasCollection("corewithnocollection3"));
|
||||
|
||||
// try and create a SolrCore with no collection name
|
||||
@ -592,7 +589,7 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa
|
||||
assertTrue(gotExp);
|
||||
TimeUnit.MILLISECONDS.sleep(200);
|
||||
// in both cases, the collection should have default to the core name
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
cloudClient.getZkStateReader().forceUpdateCollection("corewithnocollection3");
|
||||
|
||||
Collection<Slice> slices = cloudClient.getZkStateReader().getClusterState().getActiveSlices("corewithnocollection3");
|
||||
int replicaCount = 0;
|
||||
|
@ -409,7 +409,6 @@ public class CustomCollectionTest extends AbstractFullDistribZkTestBase {
|
||||
int attempts = 0;
|
||||
while (true) {
|
||||
if (attempts > 30) fail("Not enough active replicas in the shard 'x'");
|
||||
zkStateReader.updateClusterState();
|
||||
attempts++;
|
||||
replicaCount = zkStateReader.getClusterState().getSlice(collectionName, "x").getReplicas().size();
|
||||
if (replicaCount >= 1) break;
|
||||
|
@ -96,7 +96,6 @@ public class DeleteShardTest extends AbstractFullDistribZkTestBase {
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
int counter = 10;
|
||||
while (counter-- > 0) {
|
||||
zkStateReader.updateClusterState();
|
||||
clusterState = zkStateReader.getClusterState();
|
||||
if (clusterState.getSlice("collection1", shard) == null) {
|
||||
break;
|
||||
@ -142,7 +141,6 @@ public class DeleteShardTest extends AbstractFullDistribZkTestBase {
|
||||
boolean transition = false;
|
||||
|
||||
for (int counter = 10; counter > 0; counter--) {
|
||||
zkStateReader.updateClusterState();
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
State sliceState = clusterState.getSlice("collection1", slice).getState();
|
||||
if (sliceState == state) {
|
||||
|
@ -89,7 +89,7 @@ public class ForceLeaderTest extends HttpPartitionTest {
|
||||
|
||||
putNonLeadersIntoLIR(testCollectionName, SHARD1, zkController, leader, notLeaders);
|
||||
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
cloudClient.getZkStateReader().forceUpdateCollection(testCollectionName);
|
||||
ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
|
||||
int numActiveReplicas = getNumberOfActiveReplicas(clusterState, testCollectionName, SHARD1);
|
||||
assertEquals("Expected only 0 active replica but found " + numActiveReplicas +
|
||||
@ -114,7 +114,7 @@ public class ForceLeaderTest extends HttpPartitionTest {
|
||||
// By now we have an active leader. Wait for recoveries to begin
|
||||
waitForRecoveriesToFinish(testCollectionName, cloudClient.getZkStateReader(), true);
|
||||
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
cloudClient.getZkStateReader().forceUpdateCollection(testCollectionName);
|
||||
clusterState = cloudClient.getZkStateReader().getClusterState();
|
||||
log.info("After forcing leader: " + clusterState.getSlice(testCollectionName, SHARD1));
|
||||
// we have a leader
|
||||
@ -187,7 +187,7 @@ public class ForceLeaderTest extends HttpPartitionTest {
|
||||
setReplicaState(testCollectionName, SHARD1, rep, State.DOWN);
|
||||
}
|
||||
|
||||
zkController.getZkStateReader().updateClusterState();
|
||||
zkController.getZkStateReader().forceUpdateCollection(testCollectionName);
|
||||
// Assert all replicas are down and that there is no leader
|
||||
assertEquals(0, getActiveOrRecoveringReplicas(testCollectionName, SHARD1).size());
|
||||
|
||||
@ -224,7 +224,6 @@ public class ForceLeaderTest extends HttpPartitionTest {
|
||||
ClusterState clusterState = null;
|
||||
boolean transition = false;
|
||||
for (int counter = 10; counter > 0; counter--) {
|
||||
zkStateReader.updateClusterState();
|
||||
clusterState = zkStateReader.getClusterState();
|
||||
Replica newLeader = clusterState.getSlice(collection, slice).getLeader();
|
||||
if (newLeader == null) {
|
||||
@ -259,7 +258,6 @@ public class ForceLeaderTest extends HttpPartitionTest {
|
||||
|
||||
Replica.State replicaState = null;
|
||||
for (int counter = 10; counter > 0; counter--) {
|
||||
zkStateReader.updateClusterState();
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
replicaState = clusterState.getSlice(collection, slice).getReplica(replica.getName()).getState();
|
||||
if (replicaState == state) {
|
||||
@ -355,7 +353,6 @@ public class ForceLeaderTest extends HttpPartitionTest {
|
||||
for (int j = 0; j < notLeaders.size(); j++)
|
||||
lirStates[j] = zkController.getLeaderInitiatedRecoveryState(collectionName, shard, notLeaders.get(j).getName());
|
||||
|
||||
zkController.getZkStateReader().updateClusterState();
|
||||
ClusterState clusterState = zkController.getZkStateReader().getClusterState();
|
||||
boolean allDown = true;
|
||||
for (State lirState : lirStates)
|
||||
@ -391,7 +388,7 @@ public class ForceLeaderTest extends HttpPartitionTest {
|
||||
JettySolrRunner leaderJetty = getJettyOnPort(getReplicaPort(leader));
|
||||
leaderJetty.start();
|
||||
waitForRecoveriesToFinish(collection, cloudClient.getZkStateReader(), true);
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
cloudClient.getZkStateReader().forceUpdateCollection(collection);
|
||||
ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
|
||||
log.info("After bringing back leader: " + clusterState.getSlice(collection, SHARD1));
|
||||
int numActiveReplicas = getNumberOfActiveReplicas(clusterState, collection, SHARD1);
|
||||
|
@ -215,7 +215,7 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
|
||||
|
||||
// Verify that the partitioned replica is DOWN
|
||||
ZkStateReader zkr = cloudClient.getZkStateReader();
|
||||
zkr.updateClusterState(); // force the state to be fresh
|
||||
zkr.forceUpdateCollection(testCollectionName);; // force the state to be fresh
|
||||
ClusterState cs = zkr.getClusterState();
|
||||
Collection<Slice> slices = cs.getActiveSlices(testCollectionName);
|
||||
Slice slice = slices.iterator().next();
|
||||
@ -645,18 +645,13 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
|
||||
final RTimer timer = new RTimer();
|
||||
|
||||
ZkStateReader zkr = cloudClient.getZkStateReader();
|
||||
zkr.updateClusterState(); // force the state to be fresh
|
||||
|
||||
zkr.forceUpdateCollection(testCollectionName);
|
||||
ClusterState cs = zkr.getClusterState();
|
||||
Collection<Slice> slices = cs.getActiveSlices(testCollectionName);
|
||||
boolean allReplicasUp = false;
|
||||
long waitMs = 0L;
|
||||
long maxWaitMs = maxWaitSecs * 1000L;
|
||||
while (waitMs < maxWaitMs && !allReplicasUp) {
|
||||
// refresh state every 2 secs
|
||||
if (waitMs % 2000 == 0)
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
|
||||
cs = cloudClient.getZkStateReader().getClusterState();
|
||||
assertNotNull(cs);
|
||||
Slice shard = cs.getSlice(testCollectionName, shardId);
|
||||
|
@ -159,8 +159,6 @@ public class LeaderFailoverAfterPartitionTest extends HttpPartitionTest {
|
||||
|
||||
long timeout = System.nanoTime() + TimeUnit.NANOSECONDS.convert(60, TimeUnit.SECONDS);
|
||||
while (System.nanoTime() < timeout) {
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
|
||||
List<Replica> activeReps = getActiveOrRecoveringReplicas(testCollectionName, "shard1");
|
||||
if (activeReps.size() >= 2) break;
|
||||
Thread.sleep(1000);
|
||||
|
@ -80,7 +80,7 @@ public class LeaderInitiatedRecoveryOnCommitTest extends BasicDistributedZkTest
|
||||
|
||||
Thread.sleep(sleepMsBeforeHealPartition);
|
||||
|
||||
cloudClient.getZkStateReader().updateClusterState(); // get the latest state
|
||||
cloudClient.getZkStateReader().forceUpdateCollection(testCollectionName); // get the latest state
|
||||
leader = cloudClient.getZkStateReader().getLeaderRetry(testCollectionName, "shard1");
|
||||
assertSame("Leader was not active", Replica.State.ACTIVE, leader.getState());
|
||||
|
||||
@ -128,7 +128,7 @@ public class LeaderInitiatedRecoveryOnCommitTest extends BasicDistributedZkTest
|
||||
sendCommitWithRetry(replica);
|
||||
Thread.sleep(sleepMsBeforeHealPartition);
|
||||
|
||||
cloudClient.getZkStateReader().updateClusterState(); // get the latest state
|
||||
cloudClient.getZkStateReader().forceUpdateCollection(testCollectionName); // get the latest state
|
||||
leader = cloudClient.getZkStateReader().getLeaderRetry(testCollectionName, "shard1");
|
||||
assertSame("Leader was not active", Replica.State.ACTIVE, leader.getState());
|
||||
|
||||
|
@ -72,7 +72,7 @@ public class MigrateRouteKeyTest extends BasicDistributedZkTest {
|
||||
boolean ruleRemoved = false;
|
||||
long expiryTime = finishTime + TimeUnit.NANOSECONDS.convert(60, TimeUnit.SECONDS);
|
||||
while (System.nanoTime() < expiryTime) {
|
||||
getCommonCloudSolrClient().getZkStateReader().updateClusterState();
|
||||
getCommonCloudSolrClient().getZkStateReader().forceUpdateCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION);
|
||||
state = getCommonCloudSolrClient().getZkStateReader().getClusterState();
|
||||
slice = state.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD2);
|
||||
Map<String,RoutingRule> routingRules = slice.getRoutingRules();
|
||||
@ -186,7 +186,7 @@ public class MigrateRouteKeyTest extends BasicDistributedZkTest {
|
||||
log.info("Response from target collection: " + response);
|
||||
assertEquals("DocCount on target collection does not match", splitKeyCount[0], response.getResults().getNumFound());
|
||||
|
||||
getCommonCloudSolrClient().getZkStateReader().updateClusterState();
|
||||
getCommonCloudSolrClient().getZkStateReader().forceUpdateCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION);
|
||||
ClusterState state = getCommonCloudSolrClient().getZkStateReader().getClusterState();
|
||||
Slice slice = state.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD2);
|
||||
assertNotNull("Routing rule map is null", slice.getRoutingRules());
|
||||
|
@ -439,7 +439,6 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||
int cloudStateSliceCount = 0;
|
||||
for (int i = 0; i < 40; i++) {
|
||||
cloudStateSliceCount = 0;
|
||||
reader.updateClusterState();
|
||||
ClusterState state = reader.getClusterState();
|
||||
final Map<String,Slice> slices = state.getSlicesMap(collection);
|
||||
if (slices != null) {
|
||||
@ -524,7 +523,6 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||
private void waitForCollections(ZkStateReader stateReader, String... collections) throws InterruptedException, KeeperException {
|
||||
int maxIterations = 100;
|
||||
while (0 < maxIterations--) {
|
||||
stateReader.updateClusterState();
|
||||
final ClusterState state = stateReader.getClusterState();
|
||||
Set<String> availableCollections = state.getCollections();
|
||||
int availableCount = 0;
|
||||
@ -605,7 +603,6 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||
private void verifyShardLeader(ZkStateReader reader, String collection, String shard, String expectedCore) throws InterruptedException, KeeperException {
|
||||
int maxIterations = 200;
|
||||
while(maxIterations-->0) {
|
||||
reader.updateClusterState(); // poll state
|
||||
ZkNodeProps props = reader.getClusterState().getLeader(collection, shard);
|
||||
if(props!=null) {
|
||||
if(expectedCore.equals(props.getStr(ZkStateReader.CORE_NAME_PROP))) {
|
||||
@ -832,7 +829,8 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||
killerThread = new Thread(killer);
|
||||
killerThread.start();
|
||||
|
||||
reader = new ZkStateReader(controllerClient); //no watches, we'll poll
|
||||
reader = new ZkStateReader(controllerClient);
|
||||
reader.createClusterStateWatchersAndUpdate();
|
||||
|
||||
for (int i = 0; i < atLeast(4); i++) {
|
||||
killCounter.incrementAndGet(); //for each round allow 1 kill
|
||||
@ -905,9 +903,10 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||
mockController = new MockZKController(server.getZkAddress(), "node1");
|
||||
mockController.publishState(collection, "core1", "core_node1", Replica.State.RECOVERING, 1);
|
||||
|
||||
while (version == getClusterStateVersion(controllerClient));
|
||||
while (version == reader.getClusterState().getZkClusterStateVersion()) {
|
||||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
reader.updateClusterState();
|
||||
ClusterState state = reader.getClusterState();
|
||||
|
||||
int numFound = 0;
|
||||
@ -1048,7 +1047,6 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
||||
assertTrue(overseers.size() > 0);
|
||||
|
||||
while (true) {
|
||||
reader.updateClusterState();
|
||||
ClusterState state = reader.getClusterState();
|
||||
if (state.hasCollection("perf_sentinel")) {
|
||||
break;
|
||||
|
@ -56,7 +56,6 @@ public abstract class ReplicaPropertiesBase extends AbstractFullDistribZkTestBas
|
||||
ClusterState clusterState = null;
|
||||
Replica replica = null;
|
||||
for (int idx = 0; idx < 300; ++idx) {
|
||||
client.getZkStateReader().updateClusterState();
|
||||
clusterState = client.getZkStateReader().getClusterState();
|
||||
replica = clusterState.getReplica(collectionName, replicaName);
|
||||
if (replica == null) {
|
||||
@ -82,7 +81,6 @@ public abstract class ReplicaPropertiesBase extends AbstractFullDistribZkTestBas
|
||||
ClusterState clusterState = null;
|
||||
|
||||
for (int idx = 0; idx < 300; ++idx) { // Keep trying while Overseer writes the ZK state for up to 30 seconds.
|
||||
client.getZkStateReader().updateClusterState();
|
||||
clusterState = client.getZkStateReader().getClusterState();
|
||||
replica = clusterState.getReplica(collectionName, replicaName);
|
||||
if (replica == null) {
|
||||
@ -116,7 +114,6 @@ public abstract class ReplicaPropertiesBase extends AbstractFullDistribZkTestBas
|
||||
|
||||
DocCollection col = null;
|
||||
for (int idx = 0; idx < 300; ++idx) {
|
||||
client.getZkStateReader().updateClusterState();
|
||||
ClusterState clusterState = client.getZkStateReader().getClusterState();
|
||||
|
||||
col = clusterState.getCollection(collectionName);
|
||||
|
@ -416,7 +416,6 @@ public class ShardSplitTest extends BasicDistributedZkTest {
|
||||
int i = 0;
|
||||
for (i = 0; i < 10; i++) {
|
||||
ZkStateReader zkStateReader = cloudClient.getZkStateReader();
|
||||
zkStateReader.updateClusterState();
|
||||
clusterState = zkStateReader.getClusterState();
|
||||
slice1_0 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, "shard1_0");
|
||||
slice1_1 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, "shard1_1");
|
||||
|
@ -218,7 +218,6 @@ public class SyncSliceTest extends AbstractFullDistribZkTestBase {
|
||||
for (int i = 0; i < 60; i++) {
|
||||
Thread.sleep(3000);
|
||||
ZkStateReader zkStateReader = cloudClient.getZkStateReader();
|
||||
zkStateReader.updateClusterState();
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
DocCollection collection1 = clusterState.getCollection("collection1");
|
||||
Slice slice = collection1.getSlice("shard1");
|
||||
|
@ -119,7 +119,6 @@ public class TestCloudDeleteByQuery extends SolrCloudTestCase {
|
||||
String nodeKey = jettyURL.getHost() + ":" + jettyURL.getPort() + jettyURL.getPath().replace("/","_");
|
||||
urlMap.put(nodeKey, jettyURL.toString());
|
||||
}
|
||||
zkStateReader.updateClusterState();
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
for (Slice slice : clusterState.getSlices(COLLECTION_NAME)) {
|
||||
String shardName = slice.getName();
|
||||
|
@ -625,7 +625,7 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
|
||||
.setCollectionName("testClusterStateMigration")
|
||||
.process(client);
|
||||
|
||||
client.getZkStateReader().updateClusterState();
|
||||
client.getZkStateReader().forceUpdateCollection("testClusterStateMigration");
|
||||
|
||||
assertEquals(2, client.getZkStateReader().getClusterState().getCollection("testClusterStateMigration").getStateFormat());
|
||||
|
||||
@ -735,7 +735,7 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
|
||||
private Map<String, String> getProps(CloudSolrClient client, String collectionName, String replicaName, String... props)
|
||||
throws KeeperException, InterruptedException {
|
||||
|
||||
client.getZkStateReader().updateClusterState();
|
||||
client.getZkStateReader().forceUpdateCollection(collectionName);
|
||||
ClusterState clusterState = client.getZkStateReader().getClusterState();
|
||||
Replica replica = clusterState.getReplica(collectionName, replicaName);
|
||||
if (replica == null) {
|
||||
|
@ -175,7 +175,6 @@ public class TestLeaderInitiatedRecoveryThread extends AbstractFullDistribZkTest
|
||||
|
||||
timeOut = new TimeOut(30, TimeUnit.SECONDS);
|
||||
while (!timeOut.hasTimedOut()) {
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
Replica r = cloudClient.getZkStateReader().getClusterState().getReplica(DEFAULT_COLLECTION, replica.getName());
|
||||
if (r.getState() == Replica.State.DOWN) {
|
||||
break;
|
||||
|
@ -176,7 +176,7 @@ public class TestMiniSolrCloudCluster extends LuceneTestCase {
|
||||
assertEquals(1, rsp.getResults().getNumFound());
|
||||
|
||||
// remove a server not hosting any replicas
|
||||
zkStateReader.updateClusterState();
|
||||
zkStateReader.forceUpdateCollection(collectionName);
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
HashMap<String, JettySolrRunner> jettyMap = new HashMap<String, JettySolrRunner>();
|
||||
for (JettySolrRunner jetty : miniCluster.getJettySolrRunners()) {
|
||||
@ -321,7 +321,8 @@ public class TestMiniSolrCloudCluster extends LuceneTestCase {
|
||||
try (SolrZkClient zkClient = new SolrZkClient
|
||||
(miniCluster.getZkServer().getZkAddress(), AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT, null);
|
||||
ZkStateReader zkStateReader = new ZkStateReader(zkClient)) {
|
||||
|
||||
zkStateReader.createClusterStateWatchersAndUpdate();
|
||||
|
||||
// wait for collection to appear
|
||||
AbstractDistribZkTestBase.waitForRecoveriesToFinish(collectionName, zkStateReader, true, true, 330);
|
||||
|
||||
@ -368,6 +369,7 @@ public class TestMiniSolrCloudCluster extends LuceneTestCase {
|
||||
try (SolrZkClient zkClient = new SolrZkClient
|
||||
(miniCluster.getZkServer().getZkAddress(), AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT, null);
|
||||
ZkStateReader zkStateReader = new ZkStateReader(zkClient)) {
|
||||
zkStateReader.createClusterStateWatchersAndUpdate();
|
||||
AbstractDistribZkTestBase.waitForRecoveriesToFinish(collectionName, zkStateReader, true, true, 330);
|
||||
|
||||
// modify collection
|
||||
@ -385,7 +387,7 @@ public class TestMiniSolrCloudCluster extends LuceneTestCase {
|
||||
}
|
||||
|
||||
// the test itself
|
||||
zkStateReader.updateClusterState();
|
||||
zkStateReader.forceUpdateCollection(collectionName);
|
||||
final ClusterState clusterState = zkStateReader.getClusterState();
|
||||
|
||||
final HashSet<Integer> leaderIndices = new HashSet<Integer>();
|
||||
@ -444,7 +446,7 @@ public class TestMiniSolrCloudCluster extends LuceneTestCase {
|
||||
}
|
||||
AbstractDistribZkTestBase.waitForRecoveriesToFinish(collectionName, zkStateReader, true, true, 330);
|
||||
|
||||
zkStateReader.updateClusterState();
|
||||
zkStateReader.forceUpdateCollection(collectionName);
|
||||
|
||||
// re-query collection
|
||||
{
|
||||
@ -489,32 +491,29 @@ public class TestMiniSolrCloudCluster extends LuceneTestCase {
|
||||
}
|
||||
}
|
||||
|
||||
try (SolrZkClient zkClient = new SolrZkClient
|
||||
(miniCluster.getZkServer().getZkAddress(), AbstractZkTestCase.TIMEOUT, 45000, null);
|
||||
ZkStateReader zkStateReader = new ZkStateReader(zkClient)) {
|
||||
AbstractDistribZkTestBase.waitForRecoveriesToFinish(collectionName, zkStateReader, true, true, 330);
|
||||
ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
|
||||
AbstractDistribZkTestBase.waitForRecoveriesToFinish(collectionName, zkStateReader, true, true, 330);
|
||||
|
||||
// add some documents, then optimize to get merged-sorted segments
|
||||
tstes.addDocuments(cloudSolrClient, 10, 10, true);
|
||||
// add some documents, then optimize to get merged-sorted segments
|
||||
tstes.addDocuments(cloudSolrClient, 10, 10, true);
|
||||
|
||||
// CommonParams.SEGMENT_TERMINATE_EARLY parameter intentionally absent
|
||||
tstes.queryTimestampDescending(cloudSolrClient);
|
||||
// CommonParams.SEGMENT_TERMINATE_EARLY parameter intentionally absent
|
||||
tstes.queryTimestampDescending(cloudSolrClient);
|
||||
|
||||
// add a few more documents, but don't optimize to have some not-merge-sorted segments
|
||||
tstes.addDocuments(cloudSolrClient, 2, 10, false);
|
||||
// add a few more documents, but don't optimize to have some not-merge-sorted segments
|
||||
tstes.addDocuments(cloudSolrClient, 2, 10, false);
|
||||
|
||||
// CommonParams.SEGMENT_TERMINATE_EARLY parameter now present
|
||||
tstes.queryTimestampDescendingSegmentTerminateEarlyYes(cloudSolrClient);
|
||||
tstes.queryTimestampDescendingSegmentTerminateEarlyNo(cloudSolrClient);
|
||||
// CommonParams.SEGMENT_TERMINATE_EARLY parameter now present
|
||||
tstes.queryTimestampDescendingSegmentTerminateEarlyYes(cloudSolrClient);
|
||||
tstes.queryTimestampDescendingSegmentTerminateEarlyNo(cloudSolrClient);
|
||||
|
||||
// CommonParams.SEGMENT_TERMINATE_EARLY parameter present but it won't be used
|
||||
tstes.queryTimestampDescendingSegmentTerminateEarlyYesGrouped(cloudSolrClient);
|
||||
tstes.queryTimestampAscendingSegmentTerminateEarlyYes(cloudSolrClient); // uses a sort order that is _not_ compatible with the merge sort order
|
||||
// CommonParams.SEGMENT_TERMINATE_EARLY parameter present but it won't be used
|
||||
tstes.queryTimestampDescendingSegmentTerminateEarlyYesGrouped(cloudSolrClient);
|
||||
tstes.queryTimestampAscendingSegmentTerminateEarlyYes(cloudSolrClient); // uses a sort order that is _not_ compatible with the merge sort order
|
||||
|
||||
// delete the collection we created earlier
|
||||
miniCluster.deleteCollection(collectionName);
|
||||
AbstractDistribZkTestBase.waitForCollectionToDisappear(collectionName, zkStateReader, true, true, 330);
|
||||
}
|
||||
// delete the collection we created earlier
|
||||
miniCluster.deleteCollection(collectionName);
|
||||
AbstractDistribZkTestBase.waitForCollectionToDisappear(collectionName, zkStateReader, true, true, 330);
|
||||
}
|
||||
finally {
|
||||
miniCluster.shutdown();
|
||||
|
@ -146,6 +146,7 @@ public class TestMiniSolrCloudClusterBase extends LuceneTestCase {
|
||||
try (SolrZkClient zkClient = new SolrZkClient
|
||||
(miniCluster.getZkServer().getZkAddress(), AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT, null);
|
||||
ZkStateReader zkStateReader = new ZkStateReader(zkClient)) {
|
||||
zkStateReader.createClusterStateWatchersAndUpdate();
|
||||
AbstractDistribZkTestBase.waitForRecoveriesToFinish(collectionName, zkStateReader, true, true, 330);
|
||||
|
||||
// modify/query collection
|
||||
@ -160,7 +161,7 @@ public class TestMiniSolrCloudClusterBase extends LuceneTestCase {
|
||||
assertEquals(1, rsp.getResults().getNumFound());
|
||||
|
||||
// remove a server not hosting any replicas
|
||||
zkStateReader.updateClusterState();
|
||||
zkStateReader.forceUpdateCollection(collectionName);
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
HashMap<String, JettySolrRunner> jettyMap = new HashMap<String, JettySolrRunner>();
|
||||
for (JettySolrRunner jetty : miniCluster.getJettySolrRunners()) {
|
||||
|
@ -88,7 +88,7 @@ public class TestRandomRequestDistribution extends AbstractFullDistribZkTestBase
|
||||
waitForRecoveriesToFinish("a1x2", true);
|
||||
waitForRecoveriesToFinish("b1x1", true);
|
||||
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
cloudClient.getZkStateReader().forceUpdateCollection("b1x1");
|
||||
|
||||
ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
|
||||
DocCollection b1x1 = clusterState.getCollection("b1x1");
|
||||
@ -137,7 +137,7 @@ public class TestRandomRequestDistribution extends AbstractFullDistribZkTestBase
|
||||
|
||||
waitForRecoveriesToFinish("football", true);
|
||||
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
cloudClient.getZkStateReader().forceUpdateCollection("football");
|
||||
|
||||
Replica leader = null;
|
||||
Replica notLeader = null;
|
||||
|
@ -310,7 +310,6 @@ public class TestRebalanceLeaders extends AbstractFullDistribZkTestBase {
|
||||
TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS);
|
||||
while (! timeout.hasTimedOut()) {
|
||||
goAgain = false;
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
Map<String, Slice> slices = cloudClient.getZkStateReader().getClusterState().getCollection(COLLECTION_NAME).getSlicesMap();
|
||||
|
||||
for (Map.Entry<String, Replica> ent : expected.entrySet()) {
|
||||
|
@ -192,7 +192,6 @@ public class TestReplicaProperties extends ReplicaPropertiesBase {
|
||||
String lastFailMsg = "";
|
||||
for (int idx = 0; idx < 300; ++idx) { // Keep trying while Overseer writes the ZK state for up to 30 seconds.
|
||||
lastFailMsg = "";
|
||||
client.getZkStateReader().updateClusterState();
|
||||
ClusterState clusterState = client.getZkStateReader().getClusterState();
|
||||
for (Slice slice : clusterState.getSlices(collectionName)) {
|
||||
Boolean foundLeader = false;
|
||||
|
@ -205,6 +205,7 @@ public class TestSolrCloudWithKerberosAlt extends LuceneTestCase {
|
||||
try (SolrZkClient zkClient = new SolrZkClient
|
||||
(miniCluster.getZkServer().getZkAddress(), AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT, null);
|
||||
ZkStateReader zkStateReader = new ZkStateReader(zkClient)) {
|
||||
zkStateReader.createClusterStateWatchersAndUpdate();
|
||||
AbstractDistribZkTestBase.waitForRecoveriesToFinish(collectionName, zkStateReader, true, true, 330);
|
||||
|
||||
// modify/query collection
|
||||
|
@ -187,7 +187,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
|
||||
}
|
||||
ZkStateReader zkStateReader = getCommonCloudSolrClient().getZkStateReader();
|
||||
|
||||
zkStateReader.updateClusterState();
|
||||
zkStateReader.forceUpdateCollection("unloadcollection");
|
||||
|
||||
int slices = zkStateReader.getClusterState().getCollection("unloadcollection").getSlices().size();
|
||||
assertEquals(1, slices);
|
||||
@ -203,7 +203,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
|
||||
createCmd.setDataDir(getDataDir(core2dataDir));
|
||||
adminClient.request(createCmd);
|
||||
}
|
||||
zkStateReader.updateClusterState();
|
||||
zkStateReader.forceUpdateCollection("unloadcollection");
|
||||
slices = zkStateReader.getClusterState().getCollection("unloadcollection").getSlices().size();
|
||||
assertEquals(1, slices);
|
||||
|
||||
|
@ -296,7 +296,7 @@ public class ZkControllerTest extends SolrTestCaseJ4 {
|
||||
byte[] bytes = Utils.toJSON(state);
|
||||
zkController.getZkClient().makePath(ZkStateReader.getCollectionPath("testPublishAndWaitForDownStates"), bytes, CreateMode.PERSISTENT, true);
|
||||
|
||||
zkController.getZkStateReader().updateClusterState();
|
||||
zkController.getZkStateReader().forceUpdateCollection("testPublishAndWaitForDownStates");
|
||||
assertTrue(zkController.getZkStateReader().getClusterState().hasCollection("testPublishAndWaitForDownStates"));
|
||||
assertNotNull(zkController.getZkStateReader().getClusterState().getCollection("testPublishAndWaitForDownStates"));
|
||||
|
||||
|
@ -154,7 +154,7 @@ public class StressHdfsTest extends BasicDistributedZkTest {
|
||||
|
||||
waitForRecoveriesToFinish(DELETE_DATA_DIR_COLLECTION, false);
|
||||
cloudClient.setDefaultCollection(DELETE_DATA_DIR_COLLECTION);
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
cloudClient.getZkStateReader().forceUpdateCollection(DELETE_DATA_DIR_COLLECTION);
|
||||
|
||||
for (int i = 1; i < nShards + 1; i++) {
|
||||
cloudClient.getZkStateReader().getLeaderRetry(DELETE_DATA_DIR_COLLECTION, "shard" + i, 30000);
|
||||
@ -211,7 +211,6 @@ public class StressHdfsTest extends BasicDistributedZkTest {
|
||||
}
|
||||
|
||||
Thread.sleep(200);
|
||||
cloudClient.getZkStateReader().updateClusterState();
|
||||
}
|
||||
|
||||
// check that all dirs are gone
|
||||
|
@ -94,7 +94,7 @@ public class ZkStateReaderTest extends SolrTestCaseJ4 {
|
||||
assertFalse(exists);
|
||||
|
||||
if (explicitRefresh) {
|
||||
reader.updateClusterState();
|
||||
reader.forceUpdateCollection("c1");
|
||||
} else {
|
||||
for (int i = 0; i < 100; ++i) {
|
||||
if (reader.getClusterState().hasCollection("c1")) {
|
||||
@ -122,7 +122,7 @@ public class ZkStateReaderTest extends SolrTestCaseJ4 {
|
||||
assertTrue(exists);
|
||||
|
||||
if (explicitRefresh) {
|
||||
reader.updateClusterState();
|
||||
reader.forceUpdateCollection("c1");
|
||||
} else {
|
||||
for (int i = 0; i < 100; ++i) {
|
||||
if (reader.getClusterState().getCollection("c1").getStateFormat() == 2) {
|
||||
@ -167,7 +167,7 @@ public class ZkStateReaderTest extends SolrTestCaseJ4 {
|
||||
new DocCollection("c1", new HashMap<String, Slice>(), new HashMap<String, Object>(), DocRouter.DEFAULT, 0, ZkStateReader.COLLECTIONS_ZKNODE + "/c1/state.json"));
|
||||
writer.enqueueUpdate(reader.getClusterState(), c1, null);
|
||||
writer.writePendingUpdates();
|
||||
reader.updateClusterState();
|
||||
reader.forceUpdateCollection("c1");
|
||||
|
||||
assertTrue(reader.getClusterState().getCollectionRef("c1").isLazilyLoaded());
|
||||
reader.addCollectionWatch("c1");
|
||||
|
@ -233,7 +233,8 @@ public class ZkStateWriterTest extends SolrTestCaseJ4 {
|
||||
writer.enqueueUpdate(reader.getClusterState(), c1, null);
|
||||
writer.writePendingUpdates();
|
||||
|
||||
reader.updateClusterState();
|
||||
reader.forceUpdateCollection("c1");
|
||||
reader.forceUpdateCollection("c2");
|
||||
ClusterState clusterState = reader.getClusterState(); // keep a reference to the current cluster state object
|
||||
assertTrue(clusterState.hasCollection("c1"));
|
||||
assertFalse(clusterState.hasCollection("c2"));
|
||||
@ -257,7 +258,6 @@ public class ZkStateWriterTest extends SolrTestCaseJ4 {
|
||||
// expected
|
||||
}
|
||||
|
||||
reader.updateClusterState();
|
||||
try {
|
||||
writer.enqueueUpdate(reader.getClusterState(), c2, null);
|
||||
fail("enqueueUpdate after BadVersionException should not have suceeded");
|
||||
@ -317,7 +317,7 @@ public class ZkStateWriterTest extends SolrTestCaseJ4 {
|
||||
zkClient.setData(ZkStateReader.getCollectionPath("c2"), data, true);
|
||||
|
||||
// get the most up-to-date state
|
||||
reader.updateClusterState();
|
||||
reader.forceUpdateCollection("c2");
|
||||
state = reader.getClusterState();
|
||||
assertTrue(state.hasCollection("c2"));
|
||||
assertEquals(sharedClusterStateVersion, (int) state.getZkClusterStateVersion());
|
||||
@ -328,7 +328,7 @@ public class ZkStateWriterTest extends SolrTestCaseJ4 {
|
||||
assertTrue(writer.hasPendingUpdates());
|
||||
|
||||
// get the most up-to-date state
|
||||
reader.updateClusterState();
|
||||
reader.forceUpdateCollection("c2");
|
||||
state = reader.getClusterState();
|
||||
|
||||
// enqueue a stateFormat=1 collection which should cause a flush
|
||||
@ -336,7 +336,7 @@ public class ZkStateWriterTest extends SolrTestCaseJ4 {
|
||||
new DocCollection("c1", new HashMap<String, Slice>(), new HashMap<String, Object>(), DocRouter.DEFAULT, 0, ZkStateReader.CLUSTER_STATE));
|
||||
|
||||
try {
|
||||
state = writer.enqueueUpdate(state, c1, null);
|
||||
writer.enqueueUpdate(state, c1, null);
|
||||
fail("Enqueue should not have succeeded");
|
||||
} catch (KeeperException.BadVersionException bve) {
|
||||
// expected
|
||||
|
@ -138,36 +138,43 @@ public class TestRestoreCore extends SolrJettyTestBase {
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
|
||||
//Modify existing index before we call restore.
|
||||
|
||||
//Delete a few docs
|
||||
int numDeletes = TestUtil.nextInt(random(), 1, nDocs);
|
||||
for(int i=0; i<numDeletes; i++) {
|
||||
masterClient.deleteByQuery("id:" + i);
|
||||
}
|
||||
masterClient.commit();
|
||||
|
||||
//Add a few more
|
||||
int moreAdds = TestUtil.nextInt(random(), 1, 100);
|
||||
for (int i=0; i<moreAdds; i++) {
|
||||
SolrInputDocument doc = new SolrInputDocument();
|
||||
doc.addField("id", i + nDocs);
|
||||
doc.addField("name", "name = " + (i + nDocs));
|
||||
masterClient.add(doc);
|
||||
}
|
||||
//Purposely not calling commit once in a while. There can be some docs which are not committed
|
||||
if (usually()) {
|
||||
int numRestoreTests = TestUtil.nextInt(random(), 1, 5);
|
||||
|
||||
for (int attempts=0; attempts<numRestoreTests; attempts++) {
|
||||
//Modify existing index before we call restore.
|
||||
|
||||
//Delete a few docs
|
||||
int numDeletes = TestUtil.nextInt(random(), 1, nDocs);
|
||||
for(int i=0; i<numDeletes; i++) {
|
||||
masterClient.deleteByQuery("id:" + i);
|
||||
}
|
||||
masterClient.commit();
|
||||
|
||||
//Add a few more
|
||||
int moreAdds = TestUtil.nextInt(random(), 1, 100);
|
||||
for (int i=0; i<moreAdds; i++) {
|
||||
SolrInputDocument doc = new SolrInputDocument();
|
||||
doc.addField("id", i + nDocs);
|
||||
doc.addField("name", "name = " + (i + nDocs));
|
||||
masterClient.add(doc);
|
||||
}
|
||||
//Purposely not calling commit once in a while. There can be some docs which are not committed
|
||||
if (usually()) {
|
||||
masterClient.commit();
|
||||
}
|
||||
|
||||
TestReplicationHandlerBackup.runBackupCommand(masterJetty, ReplicationHandler.CMD_RESTORE, params);
|
||||
|
||||
while (!fetchRestoreStatus()) {
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
|
||||
//See if restore was successful by checking if all the docs are present again
|
||||
verifyDocs(nDocs);
|
||||
}
|
||||
|
||||
TestReplicationHandlerBackup.runBackupCommand(masterJetty, ReplicationHandler.CMD_RESTORE, params);
|
||||
|
||||
while (!fetchRestoreStatus()) {
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
|
||||
//See if restore was successful by checking if all the docs are present again
|
||||
verifyDocs(nDocs);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
x
Reference in New Issue
Block a user