mirror of https://github.com/apache/lucene.git
SOLR-2452: Merged with trunk up to r1137265
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/solr2452@1137267 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
a9b8ed0002
|
@ -429,6 +429,11 @@ New features
|
|||
|
||||
(Mike McCandless, Simon Willnauer)
|
||||
|
||||
* LUCENE-3209: Added MemoryCodec, which stores all terms & postings in
|
||||
RAM as an FST; this is good for primary-key fields if you frequently
|
||||
need to lookup by that field or perform deletions against it, for
|
||||
example in a near-real-time setting. (Mike McCandless)
|
||||
|
||||
Optimizations
|
||||
|
||||
* LUCENE-2588: Don't store unnecessary suffixes when writing the terms
|
||||
|
@ -522,6 +527,10 @@ Bug fixes
|
|||
in the superclass Searcher were not using it, leading to strange bugs.
|
||||
(Uwe Schindler, Robert Muir)
|
||||
|
||||
* LUCENE-3197: Fix core merge policies to not over-merge during
|
||||
background optimize when documents are still being deleted
|
||||
concurrently with the optimize (Mike McCandless)
|
||||
|
||||
API Changes
|
||||
|
||||
* LUCENE-3208: Renamed protected IndexSearcher.createWeight() to expert
|
||||
|
@ -531,6 +540,11 @@ API Changes
|
|||
IndexSearcher. Both deprecated methods will be removed in Lucene 4.0.
|
||||
(Uwe Schindler, Robert Muir, Yonik Seeley)
|
||||
|
||||
* LUCENE-3197: MergePolicy.findMergesForOptimize now takes
|
||||
Map<SegmentInfo,Boolean> instead of Set<SegmentInfo> as the second
|
||||
argument, so the merge policy knows which segments were originally
|
||||
present vs produced by an optimizing merge (Mike McCandless)
|
||||
|
||||
New Features
|
||||
|
||||
* LUCENE-3140: Added experimental FST implementation to Lucene.
|
||||
|
|
|
@ -81,7 +81,7 @@ New Features
|
|||
|
||||
* LUCENE-2919: Added PKIndexSplitter, that splits an index according
|
||||
to a middle term in a specified field. (Jason Rutherglen via Mike
|
||||
McCandless)
|
||||
McCandless, Uwe Schindler)
|
||||
|
||||
API Changes
|
||||
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Set;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Merge policy that tries to balance not doing large
|
||||
|
@ -105,7 +105,7 @@ public class BalancedSegmentMergePolicy extends LogByteSizeMergePolicy {
|
|||
}
|
||||
|
||||
@Override
|
||||
public MergeSpecification findMergesForOptimize(SegmentInfos infos, int maxNumSegments, Set<SegmentInfo> segmentsToOptimize) throws IOException {
|
||||
public MergeSpecification findMergesForOptimize(SegmentInfos infos, int maxNumSegments, Map<SegmentInfo,Boolean> segmentsToOptimize) throws IOException {
|
||||
|
||||
assert maxNumSegments > 0;
|
||||
|
||||
|
@ -120,8 +120,7 @@ public class BalancedSegmentMergePolicy extends LogByteSizeMergePolicy {
|
|||
while(last > 0) {
|
||||
|
||||
final SegmentInfo info = infos.info(--last);
|
||||
if (segmentsToOptimize.contains(info)) {
|
||||
|
||||
if (segmentsToOptimize.containsKey(info)) {
|
||||
last++;
|
||||
break;
|
||||
}
|
||||
|
|
|
@ -19,113 +19,114 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.search.DocIdSet;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Filter;
|
||||
import org.apache.lucene.search.TermRangeFilter;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.OpenBitSet;
|
||||
import org.apache.lucene.util.OpenBitSetDISI;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
||||
/**
|
||||
* Split an index based on a given primary key term
|
||||
* and a 'middle' term. If the middle term is present, it's
|
||||
* sent to dir2.
|
||||
* Split an index based on a {@link Filter}.
|
||||
*/
|
||||
public class PKIndexSplitter {
|
||||
private Term midTerm;
|
||||
Directory input;
|
||||
Directory dir1;
|
||||
Directory dir2;
|
||||
private final Filter docsInFirstIndex;
|
||||
private final Directory input;
|
||||
private final Directory dir1;
|
||||
private final Directory dir2;
|
||||
|
||||
public PKIndexSplitter(Term midTerm, Directory input,
|
||||
Directory dir1, Directory dir2) {
|
||||
this.midTerm = midTerm;
|
||||
/**
|
||||
* Split an index based on a {@link Filter}. All documents that match the filter
|
||||
* are sent to dir1, remaining ones to dir2.
|
||||
*/
|
||||
public PKIndexSplitter(Directory input, Directory dir1, Directory dir2, Filter docsInFirstIndex) {
|
||||
this.input = input;
|
||||
this.dir1 = dir1;
|
||||
this.dir2 = dir2;
|
||||
this.docsInFirstIndex = docsInFirstIndex;
|
||||
}
|
||||
|
||||
/**
|
||||
* Split an index based on a given primary key term
|
||||
* and a 'middle' term. If the middle term is present, it's
|
||||
* sent to dir2.
|
||||
*/
|
||||
public PKIndexSplitter(Directory input, Directory dir1, Directory dir2, Term midTerm) {
|
||||
this(input, dir1, dir2,
|
||||
new TermRangeFilter(midTerm.field(), null, midTerm.bytes(), true, false));
|
||||
}
|
||||
|
||||
public void split() throws IOException {
|
||||
boolean success = false;
|
||||
IndexReader reader = IndexReader.open(input);
|
||||
OpenBitSet lowDels = setDeletes(reader, null, midTerm.bytes());
|
||||
OpenBitSet hiDels = setDeletes(reader, midTerm.bytes(), null);
|
||||
|
||||
createIndex(dir1, reader, lowDels);
|
||||
createIndex(dir2, reader, hiDels);
|
||||
reader.close();
|
||||
try {
|
||||
createIndex(dir1, reader, docsInFirstIndex, false);
|
||||
createIndex(dir2, reader, docsInFirstIndex, true);
|
||||
success = true;
|
||||
} finally {
|
||||
IOUtils.closeSafely(!success, reader);
|
||||
}
|
||||
}
|
||||
|
||||
private void createIndex(Directory target, IndexReader reader, OpenBitSet bv) throws IOException {
|
||||
private void createIndex(Directory target, IndexReader reader, Filter preserveFilter, boolean negateFilter) throws IOException {
|
||||
boolean success = false;
|
||||
IndexWriter w = new IndexWriter(target, new IndexWriterConfig(
|
||||
Version.LUCENE_CURRENT,
|
||||
new WhitespaceAnalyzer(Version.LUCENE_CURRENT))
|
||||
.setOpenMode(OpenMode.CREATE));
|
||||
w.addIndexes(new DeletesIndexReader(reader, bv));
|
||||
w.close();
|
||||
Version.LUCENE_CURRENT, null).setOpenMode(OpenMode.CREATE));
|
||||
try {
|
||||
w.addIndexes(new DocumentFilteredIndexReader(reader, preserveFilter, negateFilter));
|
||||
success = true;
|
||||
} finally {
|
||||
IOUtils.closeSafely(!success, w);
|
||||
}
|
||||
}
|
||||
|
||||
private OpenBitSet setDeletes(IndexReader reader, BytesRef startTerm,
|
||||
BytesRef endTermExcl) throws IOException {
|
||||
OpenBitSet incl = new OpenBitSet(reader.maxDoc());
|
||||
Terms terms = MultiFields.getTerms(reader, midTerm.field());
|
||||
TermsEnum te = terms.iterator();
|
||||
if (startTerm != null) {
|
||||
te.seek(startTerm);
|
||||
}
|
||||
while (true) {
|
||||
final BytesRef term = te.next();
|
||||
if (term == null) {
|
||||
break;
|
||||
}
|
||||
if (endTermExcl != null && term.compareTo(endTermExcl) >= 0) {
|
||||
break;
|
||||
}
|
||||
DocsEnum docs = MultiFields.getTermDocsEnum(reader,
|
||||
MultiFields.getDeletedDocs(reader), midTerm.field(), term);
|
||||
while (true) {
|
||||
final int doc = docs.nextDoc();
|
||||
if (doc != DocsEnum.NO_MORE_DOCS) {
|
||||
incl.set(doc);
|
||||
} else break;
|
||||
}
|
||||
}
|
||||
OpenBitSet dels = new OpenBitSet(reader.maxDoc());
|
||||
for (int x=0; x < reader.maxDoc(); x++) {
|
||||
if (!incl.get(x)) {
|
||||
dels.set(x);
|
||||
}
|
||||
}
|
||||
return dels;
|
||||
}
|
||||
|
||||
public static class DeletesIndexReader extends FilterIndexReader {
|
||||
OpenBitSet readerDels;
|
||||
|
||||
public DeletesIndexReader(IndexReader reader, OpenBitSet deletes) {
|
||||
public static class DocumentFilteredIndexReader extends FilterIndexReader {
|
||||
final Bits readerDels;
|
||||
final int numDocs;
|
||||
|
||||
public DocumentFilteredIndexReader(IndexReader reader, Filter preserveFilter, boolean negateFilter) throws IOException {
|
||||
super(new SlowMultiReaderWrapper(reader));
|
||||
readerDels = new OpenBitSet(reader.maxDoc());
|
||||
|
||||
final OpenBitSetDISI bits = new OpenBitSetDISI(in.maxDoc());
|
||||
final DocIdSet docs = preserveFilter.getDocIdSet((AtomicReaderContext) in.getTopReaderContext());
|
||||
if (docs != null) {
|
||||
final DocIdSetIterator it = docs.iterator();
|
||||
if (it != null) {
|
||||
bits.inPlaceOr(it);
|
||||
}
|
||||
}
|
||||
// this is somehow inverse, if we negate the filter, we delete all documents it matches!
|
||||
if (!negateFilter) {
|
||||
bits.flip(0, in.maxDoc());
|
||||
}
|
||||
|
||||
if (in.hasDeletions()) {
|
||||
final Bits oldDelBits = MultiFields.getDeletedDocs(in);
|
||||
final Bits oldDelBits = in.getDeletedDocs();
|
||||
assert oldDelBits != null;
|
||||
for (int i = 0; i < in.maxDoc(); i++) {
|
||||
if (oldDelBits.get(i) || deletes.get(i)) {
|
||||
readerDels.set(i);
|
||||
if (oldDelBits.get(i)) {
|
||||
bits.set(i);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
readerDels = deletes;
|
||||
}
|
||||
|
||||
this.readerDels = bits;
|
||||
this.numDocs = in.maxDoc() - (int) bits.cardinality();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numDocs() {
|
||||
return in.maxDoc() - (int)readerDels.cardinality();
|
||||
return numDocs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasDeletions() {
|
||||
return (int)readerDels.cardinality() > 0;
|
||||
return (in.maxDoc() != numDocs);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -20,73 +20,95 @@ package org.apache.lucene.index;
|
|||
import java.text.DecimalFormat;
|
||||
import java.text.NumberFormat;
|
||||
|
||||
import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.Field.Index;
|
||||
import org.apache.lucene.document.Field.Store;
|
||||
import org.apache.lucene.document.Field.TermVector;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.Version;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
|
||||
public class TestPKIndexSplitter extends LuceneTestCase {
|
||||
public void testSplit() throws Exception {
|
||||
|
||||
public void testSplit() throws Exception {
|
||||
NumberFormat format = new DecimalFormat("000000000");
|
||||
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(
|
||||
Version.LUCENE_CURRENT,
|
||||
new WhitespaceAnalyzer(Version.LUCENE_CURRENT))
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random, MockTokenizer.WHITESPACE, false))
|
||||
.setOpenMode(OpenMode.CREATE));
|
||||
for (int x=0; x < 10; x++) {
|
||||
for (int x = 0; x < 11; x++) {
|
||||
Document doc = createDocument(x, "1", 3, format);
|
||||
w.addDocument(doc);
|
||||
}
|
||||
for (int x=15; x < 20; x++) {
|
||||
for (int x = 11; x < 20; x++) {
|
||||
Document doc = createDocument(x, "2", 3, format);
|
||||
w.addDocument(doc);
|
||||
}
|
||||
w.close();
|
||||
|
||||
final Term midTerm = new Term("id", format.format(11));
|
||||
|
||||
checkSplitting(dir, midTerm, 11, 9);
|
||||
|
||||
// delete some documents
|
||||
w = new IndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random, MockTokenizer.WHITESPACE, false))
|
||||
.setOpenMode(OpenMode.APPEND));
|
||||
w.deleteDocuments(midTerm);
|
||||
w.deleteDocuments(new Term("id", format.format(2)));
|
||||
w.close();
|
||||
|
||||
checkSplitting(dir, midTerm, 10, 8);
|
||||
|
||||
dir.close();
|
||||
}
|
||||
|
||||
private void checkSplitting(Directory dir, Term splitTerm, int leftCount, int rightCount) throws Exception {
|
||||
Directory dir1 = newDirectory();
|
||||
Directory dir2 = newDirectory();
|
||||
Term splitTerm = new Term("id", new BytesRef(format.format(11)));
|
||||
PKIndexSplitter splitter = new PKIndexSplitter(splitTerm,
|
||||
dir, dir1, dir2);
|
||||
PKIndexSplitter splitter = new PKIndexSplitter(dir, dir1, dir2, splitTerm);
|
||||
splitter.split();
|
||||
|
||||
IndexReader ir1 = IndexReader.open(dir1);
|
||||
IndexReader ir2 = IndexReader.open(dir2);
|
||||
assertEquals(10, ir1.maxDoc());
|
||||
assertEquals(4, ir2.maxDoc());
|
||||
assertEquals(leftCount, ir1.numDocs());
|
||||
assertEquals(rightCount, ir2.numDocs());
|
||||
|
||||
checkContents(ir1, "1");
|
||||
checkContents(ir2, "2");
|
||||
|
||||
ir1.close();
|
||||
ir2.close();
|
||||
|
||||
dir1.close();
|
||||
dir2.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public Document createDocument(int n, String indexName,
|
||||
private void checkContents(IndexReader ir, String indexname) throws Exception {
|
||||
final Bits delDocs = MultiFields.getDeletedDocs(ir);
|
||||
for (int i = 0; i < ir.maxDoc(); i++) {
|
||||
if (delDocs == null || !delDocs.get(i)) {
|
||||
assertEquals(indexname, ir.document(i).get("indexname"));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private Document createDocument(int n, String indexName,
|
||||
int numFields, NumberFormat format) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
Document doc = new Document();
|
||||
String id = format.format(n);
|
||||
doc.add(new Field("id", id, Store.YES, Index.NOT_ANALYZED, TermVector.WITH_POSITIONS_OFFSETS));
|
||||
doc.add(new Field("indexname", indexName, Store.YES, Index.NOT_ANALYZED, TermVector.WITH_POSITIONS_OFFSETS));
|
||||
doc.add(newField("id", id, Store.YES, Index.NOT_ANALYZED));
|
||||
doc.add(newField("indexname", indexName, Store.YES, Index.NOT_ANALYZED));
|
||||
sb.append("a");
|
||||
sb.append(n);
|
||||
doc.add(new Field("field1", sb.toString(), Store.YES, Index.ANALYZED, TermVector.WITH_POSITIONS_OFFSETS));
|
||||
doc.add(newField("field1", sb.toString(), Store.YES, Index.ANALYZED));
|
||||
sb.append(" b");
|
||||
sb.append(n);
|
||||
for (int i = 1; i < numFields; i++) {
|
||||
doc.add(new Field("field" + (i + 1), sb.toString(), Store.YES,
|
||||
Index.ANALYZED, TermVector.WITH_POSITIONS_OFFSETS));
|
||||
doc.add(newField("field" + (i + 1), sb.toString(), Store.YES, Index.ANALYZED));
|
||||
}
|
||||
return doc;
|
||||
}
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.lucene.document.Document;
|
|||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.NumericField;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
|
@ -222,8 +222,8 @@ public class TestCartesian extends LuceneTestCase {
|
|||
CustomScoreQuery customScore = new CustomScoreQuery(dq.getQuery(tq),fsQuery){
|
||||
|
||||
@Override
|
||||
protected CustomScoreProvider getCustomScoreProvider(IndexReader reader) {
|
||||
return new CustomScoreProvider(reader) {
|
||||
protected CustomScoreProvider getCustomScoreProvider(AtomicReaderContext context) {
|
||||
return new CustomScoreProvider(context) {
|
||||
@Override // TODO: broken, as reader is not used!
|
||||
public float customScore(int doc, float subQueryScore, float valSrcScore){
|
||||
if (VERBOSE) System.out.println(doc);
|
||||
|
@ -318,8 +318,8 @@ public class TestCartesian extends LuceneTestCase {
|
|||
CustomScoreQuery customScore = new CustomScoreQuery(dq.getQuery(tq),fsQuery){
|
||||
|
||||
@Override
|
||||
protected CustomScoreProvider getCustomScoreProvider(IndexReader reader) {
|
||||
return new CustomScoreProvider(reader) {
|
||||
protected CustomScoreProvider getCustomScoreProvider(AtomicReaderContext context) {
|
||||
return new CustomScoreProvider(context) {
|
||||
@Override // TODO: broken, as reader is not used!
|
||||
public float customScore(int doc, float subQueryScore, float valSrcScore){
|
||||
if (VERBOSE) System.out.println(doc);
|
||||
|
@ -415,8 +415,8 @@ public class TestCartesian extends LuceneTestCase {
|
|||
|
||||
CustomScoreQuery customScore = new CustomScoreQuery(dq.getQuery(tq),fsQuery){
|
||||
@Override
|
||||
protected CustomScoreProvider getCustomScoreProvider(IndexReader reader) {
|
||||
return new CustomScoreProvider(reader) {
|
||||
protected CustomScoreProvider getCustomScoreProvider(AtomicReaderContext context) {
|
||||
return new CustomScoreProvider(context) {
|
||||
@Override // TODO: broken, as reader is not used!
|
||||
public float customScore(int doc, float subQueryScore, float valSrcScore){
|
||||
if (VERBOSE) System.out.println(doc);
|
||||
|
@ -510,8 +510,8 @@ public class TestCartesian extends LuceneTestCase {
|
|||
FieldScoreQuery fsQuery = new FieldScoreQuery("geo_distance", Type.FLOAT);
|
||||
CustomScoreQuery customScore = new CustomScoreQuery(tq,fsQuery){
|
||||
@Override
|
||||
protected CustomScoreProvider getCustomScoreProvider(IndexReader reader) {
|
||||
return new CustomScoreProvider(reader) {
|
||||
protected CustomScoreProvider getCustomScoreProvider(AtomicReaderContext context) {
|
||||
return new CustomScoreProvider(context) {
|
||||
@Override // TODO: broken, as reader is not used!
|
||||
public float customScore(int doc, float subQueryScore, float valSrcScore){
|
||||
if (VERBOSE) System.out.println(doc);
|
||||
|
|
|
@ -236,7 +236,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
private DocumentsWriter docWriter;
|
||||
final IndexFileDeleter deleter;
|
||||
|
||||
private Set<SegmentInfo> segmentsToOptimize = new HashSet<SegmentInfo>(); // used by optimize to note those needing optimization
|
||||
private Map<SegmentInfo,Boolean> segmentsToOptimize = new HashMap<SegmentInfo,Boolean>(); // used by optimize to note those needing optimization
|
||||
private int optimizeMaxNumSegments;
|
||||
|
||||
private Lock writeLock;
|
||||
|
@ -1664,7 +1664,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
synchronized(this) {
|
||||
resetMergeExceptions();
|
||||
segmentsToOptimize.clear();
|
||||
segmentsToOptimize.addAll(segmentInfos.asSet());
|
||||
for(SegmentInfo info : segmentInfos) {
|
||||
segmentsToOptimize.put(info, Boolean.TRUE);
|
||||
}
|
||||
optimizeMaxNumSegments = maxNumSegments;
|
||||
|
||||
// Now mark all pending & running merges as optimize
|
||||
|
@ -1888,7 +1890,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
final MergePolicy.MergeSpecification spec;
|
||||
if (optimize) {
|
||||
spec = mergePolicy.findMergesForOptimize(segmentInfos, maxNumSegmentsOptimize, Collections.unmodifiableSet(segmentsToOptimize));
|
||||
spec = mergePolicy.findMergesForOptimize(segmentInfos, maxNumSegmentsOptimize, Collections.unmodifiableMap(segmentsToOptimize));
|
||||
|
||||
if (spec != null) {
|
||||
final int numMerges = spec.merges.size();
|
||||
|
@ -3042,7 +3044,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
if (merge.optimize) {
|
||||
// cascade the optimize:
|
||||
segmentsToOptimize.add(merge.info);
|
||||
segmentsToOptimize.put(merge.info, Boolean.FALSE);
|
||||
}
|
||||
|
||||
return true;
|
||||
|
@ -3086,7 +3088,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final void merge(MergePolicy.OneMerge merge)
|
||||
public void merge(MergePolicy.OneMerge merge)
|
||||
throws CorruptIndexException, IOException {
|
||||
|
||||
boolean success = false;
|
||||
|
@ -3167,7 +3169,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
if (info.dir != directory) {
|
||||
isExternal = true;
|
||||
}
|
||||
if (segmentsToOptimize.contains(info)) {
|
||||
if (segmentsToOptimize.containsKey(info)) {
|
||||
merge.optimize = true;
|
||||
merge.maxNumSegmentsOptimize = optimizeMaxNumSegments;
|
||||
}
|
||||
|
|
|
@ -21,7 +21,7 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.Map;
|
||||
|
||||
/** <p>This class implements a {@link MergePolicy} that tries
|
||||
* to merge segments into levels of exponentially
|
||||
|
@ -201,20 +201,23 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
}
|
||||
}
|
||||
|
||||
protected boolean isOptimized(SegmentInfos infos, int maxNumSegments, Set<SegmentInfo> segmentsToOptimize) throws IOException {
|
||||
protected boolean isOptimized(SegmentInfos infos, int maxNumSegments, Map<SegmentInfo,Boolean> segmentsToOptimize) throws IOException {
|
||||
final int numSegments = infos.size();
|
||||
int numToOptimize = 0;
|
||||
SegmentInfo optimizeInfo = null;
|
||||
boolean segmentIsOriginal = false;
|
||||
for(int i=0;i<numSegments && numToOptimize <= maxNumSegments;i++) {
|
||||
final SegmentInfo info = infos.info(i);
|
||||
if (segmentsToOptimize.contains(info)) {
|
||||
final Boolean isOriginal = segmentsToOptimize.get(info);
|
||||
if (isOriginal != null) {
|
||||
segmentIsOriginal = isOriginal;
|
||||
numToOptimize++;
|
||||
optimizeInfo = info;
|
||||
}
|
||||
}
|
||||
|
||||
return numToOptimize <= maxNumSegments &&
|
||||
(numToOptimize != 1 || isOptimized(optimizeInfo));
|
||||
(numToOptimize != 1 || !segmentIsOriginal || isOptimized(optimizeInfo));
|
||||
}
|
||||
|
||||
/** Returns true if this single info is optimized (has no
|
||||
|
@ -346,7 +349,7 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
* in use may make use of concurrency. */
|
||||
@Override
|
||||
public MergeSpecification findMergesForOptimize(SegmentInfos infos,
|
||||
int maxNumSegments, Set<SegmentInfo> segmentsToOptimize) throws IOException {
|
||||
int maxNumSegments, Map<SegmentInfo,Boolean> segmentsToOptimize) throws IOException {
|
||||
|
||||
assert maxNumSegments > 0;
|
||||
if (verbose()) {
|
||||
|
@ -368,7 +371,7 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
int last = infos.size();
|
||||
while (last > 0) {
|
||||
final SegmentInfo info = infos.info(--last);
|
||||
if (segmentsToOptimize.contains(info)) {
|
||||
if (segmentsToOptimize.get(info) != null) {
|
||||
last++;
|
||||
break;
|
||||
}
|
||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.lucene.util.SetOnce.AlreadySetException;
|
|||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Set;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* <p>Expert: a MergePolicy determines the sequence of
|
||||
|
@ -297,10 +297,15 @@ public abstract class MergePolicy implements java.io.Closeable {
|
|||
* is always 1)
|
||||
* @param segmentsToOptimize
|
||||
* contains the specific SegmentInfo instances that must be merged
|
||||
* away. This may be a subset of all SegmentInfos.
|
||||
* away. This may be a subset of all
|
||||
* SegmentInfos. If the value is True for a
|
||||
* given SegmentInfo, that means this segment was
|
||||
* an original segment present in the
|
||||
* to-be-optimized index; else, it was a segment
|
||||
* produced by a cascaded merge.
|
||||
*/
|
||||
public abstract MergeSpecification findMergesForOptimize(
|
||||
SegmentInfos segmentInfos, int maxSegmentCount, Set<SegmentInfo> segmentsToOptimize)
|
||||
SegmentInfos segmentInfos, int maxSegmentCount, Map<SegmentInfo,Boolean> segmentsToOptimize)
|
||||
throws CorruptIndexException, IOException;
|
||||
|
||||
/**
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* A {@link MergePolicy} which never returns merges to execute (hence it's
|
||||
|
@ -59,7 +59,7 @@ public final class NoMergePolicy extends MergePolicy {
|
|||
|
||||
@Override
|
||||
public MergeSpecification findMergesForOptimize(SegmentInfos segmentInfos,
|
||||
int maxSegmentCount, Set<SegmentInfo> segmentsToOptimize)
|
||||
int maxSegmentCount, Map<SegmentInfo,Boolean> segmentsToOptimize)
|
||||
throws CorruptIndexException, IOException { return null; }
|
||||
|
||||
@Override
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
import java.util.Map;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
|
@ -472,7 +472,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
}
|
||||
|
||||
@Override
|
||||
public MergeSpecification findMergesForOptimize(SegmentInfos infos, int maxSegmentCount, Set<SegmentInfo> segmentsToOptimize) throws IOException {
|
||||
public MergeSpecification findMergesForOptimize(SegmentInfos infos, int maxSegmentCount, Map<SegmentInfo,Boolean> segmentsToOptimize) throws IOException {
|
||||
if (verbose()) {
|
||||
message("findMergesForOptimize maxSegmentCount=" + maxSegmentCount + " infos=" + writer.get().segString(infos) + " segmentsToOptimize=" + segmentsToOptimize);
|
||||
}
|
||||
|
@ -480,8 +480,11 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
List<SegmentInfo> eligible = new ArrayList<SegmentInfo>();
|
||||
boolean optimizeMergeRunning = false;
|
||||
final Collection<SegmentInfo> merging = writer.get().getMergingSegments();
|
||||
boolean segmentIsOriginal = false;
|
||||
for(SegmentInfo info : infos) {
|
||||
if (segmentsToOptimize.contains(info)) {
|
||||
final Boolean isOriginal = segmentsToOptimize.get(info);
|
||||
if (isOriginal != null) {
|
||||
segmentIsOriginal = isOriginal;
|
||||
if (!merging.contains(info)) {
|
||||
eligible.add(info);
|
||||
} else {
|
||||
|
@ -495,7 +498,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
}
|
||||
|
||||
if ((maxSegmentCount > 1 && eligible.size() <= maxSegmentCount) ||
|
||||
(maxSegmentCount == 1 && eligible.size() == 1 && isOptimized(eligible.get(0)))) {
|
||||
(maxSegmentCount == 1 && eligible.size() == 1 && (!segmentIsOriginal || isOptimized(eligible.get(0))))) {
|
||||
if (verbose()) {
|
||||
message("already optimized");
|
||||
}
|
||||
|
|
|
@ -21,9 +21,9 @@ import org.apache.lucene.util.Constants;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.Map;
|
||||
import java.util.HashMap;
|
||||
|
||||
/** This {@link MergePolicy} is used for upgrading all existing segments of
|
||||
* an index when calling {@link IndexWriter#optimize()}.
|
||||
|
@ -79,12 +79,13 @@ public class UpgradeIndexMergePolicy extends MergePolicy {
|
|||
}
|
||||
|
||||
@Override
|
||||
public MergeSpecification findMergesForOptimize(SegmentInfos segmentInfos, int maxSegmentCount, Set<SegmentInfo> segmentsToOptimize) throws CorruptIndexException, IOException {
|
||||
public MergeSpecification findMergesForOptimize(SegmentInfos segmentInfos, int maxSegmentCount, Map<SegmentInfo,Boolean> segmentsToOptimize) throws CorruptIndexException, IOException {
|
||||
// first find all old segments
|
||||
final HashSet<SegmentInfo> oldSegments = new HashSet<SegmentInfo>();
|
||||
final Map<SegmentInfo,Boolean> oldSegments = new HashMap<SegmentInfo,Boolean>();
|
||||
for (final SegmentInfo si : segmentInfos) {
|
||||
if (segmentsToOptimize.contains(si) && shouldUpgradeSegment(si)) {
|
||||
oldSegments.add(si);
|
||||
final Boolean v =segmentsToOptimize.get(si);
|
||||
if (v != null && shouldUpgradeSegment(si)) {
|
||||
oldSegments.put(si, v);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -93,14 +94,16 @@ public class UpgradeIndexMergePolicy extends MergePolicy {
|
|||
if (oldSegments.isEmpty())
|
||||
return null;
|
||||
|
||||
MergeSpecification spec = base.findMergesForOptimize(segmentInfos, maxSegmentCount, oldSegments);
|
||||
MergeSpecification spec = base.findMergesForOptimize(segmentInfos, maxSegmentCount, oldSegments);
|
||||
|
||||
if (spec != null) {
|
||||
// remove all segments that are in merge specification from oldSegments,
|
||||
// the resulting set contains all segments that are left over
|
||||
// and will be merged to one additional segment:
|
||||
for (final OneMerge om : spec.merges) {
|
||||
oldSegments.removeAll(om.segments);
|
||||
for(SegmentInfo info : om.segments) {
|
||||
oldSegments.remove(info);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -110,7 +113,7 @@ public class UpgradeIndexMergePolicy extends MergePolicy {
|
|||
" does not want to merge all old segments, merge remaining ones into new segment: " + oldSegments);
|
||||
final List<SegmentInfo> newInfos = new ArrayList<SegmentInfo>();
|
||||
for (final SegmentInfo si : segmentInfos) {
|
||||
if (oldSegments.contains(si)) {
|
||||
if (oldSegments.containsKey(si)) {
|
||||
newInfos.add(si);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,7 +22,6 @@ import java.util.HashMap;
|
|||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.Map.Entry;
|
||||
|
||||
/** Holds a set of codecs, keyed by name. You subclass
|
||||
* this, instantiate it, and register your codecs, then
|
||||
|
@ -44,7 +43,7 @@ public class CodecProvider {
|
|||
private final Set<String> knownExtensions = new HashSet<String>();
|
||||
|
||||
|
||||
public final static String[] CORE_CODECS = new String[] {"Standard", "Pulsing", "PreFlex", "SimpleText"};
|
||||
public final static String[] CORE_CODECS = new String[] {"Standard", "Pulsing", "PreFlex", "SimpleText", "Memory"};
|
||||
|
||||
public synchronized void register(Codec codec) {
|
||||
if (codec.name == null) {
|
||||
|
@ -84,8 +83,9 @@ public class CodecProvider {
|
|||
|
||||
public synchronized Codec lookup(String name) {
|
||||
final Codec codec = codecs.get(name);
|
||||
if (codec == null)
|
||||
if (codec == null) {
|
||||
throw new IllegalArgumentException("required codec '" + name + "' not found");
|
||||
}
|
||||
return codec;
|
||||
}
|
||||
|
||||
|
|
|
@ -17,6 +17,7 @@ package org.apache.lucene.index.codecs;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.codecs.memory.MemoryCodec;
|
||||
import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
|
||||
import org.apache.lucene.index.codecs.pulsing.PulsingCodec;
|
||||
import org.apache.lucene.index.codecs.simpletext.SimpleTextCodec;
|
||||
|
@ -45,5 +46,6 @@ public class CoreCodecProvider extends CodecProvider {
|
|||
register(new PreFlexCodec());
|
||||
register(new PulsingCodec(1));
|
||||
register(new SimpleTextCodec());
|
||||
register(new MemoryCodec());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,780 @@
|
|||
package org.apache.lucene.index.codecs.memory;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.Set;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.FieldsEnum;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.codecs.PostingsConsumer;
|
||||
import org.apache.lucene.index.codecs.TermStats;
|
||||
import org.apache.lucene.index.codecs.TermsConsumer;
|
||||
import org.apache.lucene.store.ByteArrayDataInput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.store.RAMOutputStream;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.fst.Builder;
|
||||
import org.apache.lucene.util.fst.ByteSequenceOutputs;
|
||||
import org.apache.lucene.util.fst.BytesRefFSTEnum;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
|
||||
// TODO: would be nice to somehow allow this to act like
|
||||
// InstantiatedIndex, by never writing to disk; ie you write
|
||||
// to this Codec in RAM only and then when you open a reader
|
||||
// it pulls the FST directly from what you wrote w/o going
|
||||
// to disk.
|
||||
|
||||
/** Stores terms & postings (docs, positions, payloads) in
|
||||
* RAM, using an FST.
|
||||
*
|
||||
* <p>Note that this codec implements advance as a linear
|
||||
* scan! This means if you store large fields in here,
|
||||
* queries that rely on advance will (AND BooleanQuery,
|
||||
* PhraseQuery) will be relatively slow!
|
||||
*
|
||||
* @lucene.experimental */
|
||||
|
||||
public class MemoryCodec extends Codec {
|
||||
|
||||
public MemoryCodec() {
|
||||
name = "Memory";
|
||||
}
|
||||
|
||||
private static final boolean VERBOSE = false;
|
||||
|
||||
private final static class TermsWriter extends TermsConsumer {
|
||||
private final IndexOutput out;
|
||||
private final FieldInfo field;
|
||||
private final Builder<BytesRef> builder;
|
||||
private final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
|
||||
private int termCount;
|
||||
|
||||
public TermsWriter(IndexOutput out, FieldInfo field) {
|
||||
this.out = out;
|
||||
this.field = field;
|
||||
builder = new Builder<BytesRef>(FST.INPUT_TYPE.BYTE1, outputs);
|
||||
|
||||
// The byte[] output we create can easily be > 255 bytes:
|
||||
builder.setAllowArrayArcs(false);
|
||||
}
|
||||
|
||||
private class PostingsWriter extends PostingsConsumer {
|
||||
private int lastDocID;
|
||||
private int lastPos;
|
||||
private int lastPayloadLen;
|
||||
|
||||
// NOTE: not private so we don't pay access check at runtime:
|
||||
int docCount;
|
||||
RAMOutputStream buffer = new RAMOutputStream();
|
||||
|
||||
@Override
|
||||
public void startDoc(int docID, int termDocFreq) throws IOException {
|
||||
if (VERBOSE) System.out.println(" startDoc docID=" + docID + " freq=" + termDocFreq);
|
||||
final int delta = docID - lastDocID;
|
||||
assert docID == 0 || delta > 0;
|
||||
lastDocID = docID;
|
||||
docCount++;
|
||||
|
||||
if (field.omitTermFreqAndPositions) {
|
||||
buffer.writeVInt(delta);
|
||||
} else if (termDocFreq == 1) {
|
||||
buffer.writeVInt((delta<<1) | 1);
|
||||
} else {
|
||||
buffer.writeVInt(delta<<1);
|
||||
assert termDocFreq > 0;
|
||||
buffer.writeVInt(termDocFreq);
|
||||
}
|
||||
|
||||
lastPos = 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addPosition(int pos, BytesRef payload) throws IOException {
|
||||
assert payload == null || field.storePayloads;
|
||||
|
||||
if (VERBOSE) System.out.println(" addPos pos=" + pos + " payload=" + payload);
|
||||
|
||||
final int delta = pos - lastPos;
|
||||
assert delta >= 0;
|
||||
lastPos = pos;
|
||||
|
||||
if (field.storePayloads) {
|
||||
final int payloadLen = payload == null ? 0 : payload.length;
|
||||
if (payloadLen != lastPayloadLen) {
|
||||
lastPayloadLen = payloadLen;
|
||||
buffer.writeVInt((delta<<1)|1);
|
||||
buffer.writeVInt(payloadLen);
|
||||
} else {
|
||||
buffer.writeVInt(delta<<1);
|
||||
}
|
||||
|
||||
if (payloadLen > 0) {
|
||||
buffer.writeBytes(payload.bytes, payload.offset, payloadLen);
|
||||
}
|
||||
} else {
|
||||
buffer.writeVInt(delta);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finishDoc() {
|
||||
}
|
||||
|
||||
public PostingsWriter reset() {
|
||||
assert buffer.getFilePointer() == 0;
|
||||
lastDocID = 0;
|
||||
docCount = 0;
|
||||
lastPayloadLen = 0;
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
private final PostingsWriter postingsWriter = new PostingsWriter();
|
||||
|
||||
@Override
|
||||
public PostingsConsumer startTerm(BytesRef text) {
|
||||
if (VERBOSE) System.out.println(" startTerm term=" + text.utf8ToString());
|
||||
return postingsWriter.reset();
|
||||
}
|
||||
|
||||
private final RAMOutputStream buffer2 = new RAMOutputStream();
|
||||
private final BytesRef spare = new BytesRef();
|
||||
private byte[] finalBuffer = new byte[128];
|
||||
|
||||
@Override
|
||||
public void finishTerm(BytesRef text, TermStats stats) throws IOException {
|
||||
|
||||
assert postingsWriter.docCount == stats.docFreq;
|
||||
|
||||
assert buffer2.getFilePointer() == 0;
|
||||
|
||||
buffer2.writeVInt(stats.docFreq);
|
||||
if (!field.omitTermFreqAndPositions) {
|
||||
buffer2.writeVLong(stats.totalTermFreq-stats.docFreq);
|
||||
}
|
||||
int pos = (int) buffer2.getFilePointer();
|
||||
buffer2.writeTo(finalBuffer, 0);
|
||||
buffer2.reset();
|
||||
|
||||
final int totalBytes = pos + (int) postingsWriter.buffer.getFilePointer();
|
||||
if (totalBytes > finalBuffer.length) {
|
||||
finalBuffer = ArrayUtil.grow(finalBuffer, totalBytes);
|
||||
}
|
||||
postingsWriter.buffer.writeTo(finalBuffer, pos);
|
||||
postingsWriter.buffer.reset();
|
||||
|
||||
spare.bytes = finalBuffer;
|
||||
spare.length = totalBytes;
|
||||
if (VERBOSE) {
|
||||
System.out.println(" finishTerm term=" + text.utf8ToString() + " " + totalBytes + " bytes totalTF=" + stats.totalTermFreq);
|
||||
for(int i=0;i<totalBytes;i++) {
|
||||
System.out.println(" " + Integer.toHexString(finalBuffer[i]&0xFF));
|
||||
}
|
||||
}
|
||||
|
||||
builder.add(text, new BytesRef(spare));
|
||||
termCount++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(long sumTotalTermFreq) throws IOException {
|
||||
if (termCount > 0) {
|
||||
out.writeVInt(termCount);
|
||||
out.writeVInt(field.number);
|
||||
if (!field.omitTermFreqAndPositions) {
|
||||
out.writeVLong(sumTotalTermFreq);
|
||||
}
|
||||
builder.finish().save(out);
|
||||
if (VERBOSE) System.out.println("finish field=" + field.name + " fp=" + out.getFilePointer());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
}
|
||||
|
||||
private static String EXTENSION = "ram";
|
||||
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
|
||||
final String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, EXTENSION);
|
||||
final IndexOutput out = state.directory.createOutput(fileName);
|
||||
|
||||
return new FieldsConsumer() {
|
||||
@Override
|
||||
public TermsConsumer addField(FieldInfo field) {
|
||||
if (VERBOSE) System.out.println("\naddField field=" + field.name);
|
||||
return new TermsWriter(out, field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
// EOF marker:
|
||||
try {
|
||||
out.writeVInt(0);
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private final static class FSTDocsEnum extends DocsEnum {
|
||||
private final boolean omitTFAP;
|
||||
private final boolean storePayloads;
|
||||
private byte[] buffer = new byte[16];
|
||||
private final ByteArrayDataInput in = new ByteArrayDataInput(buffer);
|
||||
|
||||
private Bits skipDocs;
|
||||
private int docUpto;
|
||||
private int docID;
|
||||
private int freq;
|
||||
private int payloadLen;
|
||||
private int numDocs;
|
||||
|
||||
public FSTDocsEnum(boolean omitTFAP, boolean storePayloads) {
|
||||
this.omitTFAP = omitTFAP;
|
||||
this.storePayloads = storePayloads;
|
||||
}
|
||||
|
||||
public boolean canReuse(boolean omitTFAP, boolean storePayloads) {
|
||||
return omitTFAP == this.omitTFAP && storePayloads == this.storePayloads;
|
||||
}
|
||||
|
||||
public FSTDocsEnum reset(BytesRef bufferIn, Bits skipDocs, int numDocs) {
|
||||
assert numDocs > 0;
|
||||
if (buffer.length < bufferIn.length - bufferIn.offset) {
|
||||
buffer = ArrayUtil.grow(buffer, bufferIn.length - bufferIn.offset);
|
||||
}
|
||||
in.reset(buffer, 0, bufferIn.length - bufferIn.offset);
|
||||
System.arraycopy(bufferIn.bytes, bufferIn.offset, buffer, 0, bufferIn.length - bufferIn.offset);
|
||||
this.skipDocs = skipDocs;
|
||||
docID = 0;
|
||||
docUpto = 0;
|
||||
payloadLen = 0;
|
||||
this.numDocs = numDocs;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() {
|
||||
while(true) {
|
||||
if (VERBOSE) System.out.println(" nextDoc cycle docUpto=" + docUpto + " numDocs=" + numDocs + " fp=" + in.getPosition() + " this=" + this);
|
||||
if (docUpto == numDocs) {
|
||||
if (VERBOSE) {
|
||||
System.out.println(" END");
|
||||
}
|
||||
return docID = NO_MORE_DOCS;
|
||||
}
|
||||
docUpto++;
|
||||
if (omitTFAP) {
|
||||
docID += in.readVInt();
|
||||
freq = 1;
|
||||
} else {
|
||||
final int code = in.readVInt();
|
||||
docID += code >>> 1;
|
||||
if (VERBOSE) System.out.println(" docID=" + docID + " code=" + code);
|
||||
if ((code & 1) != 0) {
|
||||
freq = 1;
|
||||
} else {
|
||||
freq = in.readVInt();
|
||||
assert freq > 0;
|
||||
}
|
||||
|
||||
// Skip positions
|
||||
for(int posUpto=0;posUpto<freq;posUpto++) {
|
||||
if (!storePayloads) {
|
||||
in.readVInt();
|
||||
} else {
|
||||
final int posCode = in.readVInt();
|
||||
if ((posCode & 1) != 0) {
|
||||
payloadLen = in.readVInt();
|
||||
}
|
||||
in.skipBytes(payloadLen);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (skipDocs == null || !skipDocs.get(docID)) {
|
||||
if (VERBOSE) System.out.println(" return docID=" + docID + " freq=" + freq);
|
||||
return docID;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return docID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) {
|
||||
// TODO: we could make more efficient version, but, it
|
||||
// should be rare that this will matter in practice
|
||||
// since usually apps will not store "big" fields in
|
||||
// this codec!
|
||||
//System.out.println("advance start docID=" + docID + " target=" + target);
|
||||
while(nextDoc() < target) {
|
||||
}
|
||||
return docID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() {
|
||||
return freq;
|
||||
}
|
||||
}
|
||||
|
||||
private final static class FSTDocsAndPositionsEnum extends DocsAndPositionsEnum {
|
||||
private final boolean omitTFAP;
|
||||
private final boolean storePayloads;
|
||||
private byte[] buffer = new byte[16];
|
||||
private final ByteArrayDataInput in = new ByteArrayDataInput(buffer);
|
||||
|
||||
private Bits skipDocs;
|
||||
private int docUpto;
|
||||
private int docID;
|
||||
private int freq;
|
||||
private int numDocs;
|
||||
private int posPending;
|
||||
private int payloadLength;
|
||||
private boolean payloadRetrieved;
|
||||
|
||||
private int pos;
|
||||
private final BytesRef payload = new BytesRef();
|
||||
|
||||
public FSTDocsAndPositionsEnum(boolean omitTFAP, boolean storePayloads) {
|
||||
this.omitTFAP = omitTFAP;
|
||||
this.storePayloads = storePayloads;
|
||||
}
|
||||
|
||||
public boolean canReuse(boolean omitTFAP, boolean storePayloads) {
|
||||
return omitTFAP == this.omitTFAP && storePayloads == this.storePayloads;
|
||||
}
|
||||
|
||||
public FSTDocsAndPositionsEnum reset(BytesRef bufferIn, Bits skipDocs, int numDocs) {
|
||||
assert numDocs > 0;
|
||||
if (VERBOSE) {
|
||||
System.out.println("D&P reset bytes this=" + this);
|
||||
for(int i=bufferIn.offset;i<bufferIn.length;i++) {
|
||||
System.out.println(" " + Integer.toHexString(bufferIn.bytes[i]&0xFF));
|
||||
}
|
||||
}
|
||||
if (buffer.length < bufferIn.length - bufferIn.offset) {
|
||||
buffer = ArrayUtil.grow(buffer, bufferIn.length - bufferIn.offset);
|
||||
}
|
||||
in.reset(buffer, 0, bufferIn.length - bufferIn.offset);
|
||||
System.arraycopy(bufferIn.bytes, bufferIn.offset, buffer, 0, bufferIn.length - bufferIn.offset);
|
||||
this.skipDocs = skipDocs;
|
||||
docID = 0;
|
||||
docUpto = 0;
|
||||
payload.bytes = buffer;
|
||||
payloadLength = 0;
|
||||
this.numDocs = numDocs;
|
||||
posPending = 0;
|
||||
payloadRetrieved = false;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() {
|
||||
while (posPending > 0) {
|
||||
nextPosition();
|
||||
}
|
||||
while(true) {
|
||||
if (VERBOSE) System.out.println(" nextDoc cycle docUpto=" + docUpto + " numDocs=" + numDocs + " fp=" + in.getPosition() + " this=" + this);
|
||||
if (docUpto == numDocs) {
|
||||
if (VERBOSE) System.out.println(" END");
|
||||
return docID = NO_MORE_DOCS;
|
||||
}
|
||||
docUpto++;
|
||||
if (omitTFAP) {
|
||||
docID += in.readVInt();
|
||||
freq = 1;
|
||||
} else {
|
||||
final int code = in.readVInt();
|
||||
docID += code >>> 1;
|
||||
if ((code & 1) != 0) {
|
||||
freq = 1;
|
||||
} else {
|
||||
freq = in.readVInt();
|
||||
assert freq > 0;
|
||||
}
|
||||
}
|
||||
|
||||
if (skipDocs == null || !skipDocs.get(docID)) {
|
||||
pos = 0;
|
||||
posPending = freq;
|
||||
if (VERBOSE) System.out.println(" return docID=" + docID + " freq=" + freq);
|
||||
return docID;
|
||||
}
|
||||
|
||||
// Skip positions
|
||||
for(int posUpto=0;posUpto<freq;posUpto++) {
|
||||
if (!storePayloads) {
|
||||
in.readVInt();
|
||||
} else {
|
||||
final int code = in.readVInt();
|
||||
if ((code & 1) != 0) {
|
||||
payloadLength = in.readVInt();
|
||||
if (VERBOSE) System.out.println(" new payloadLen=" + payloadLength);
|
||||
}
|
||||
in.skipBytes(payloadLength);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextPosition() {
|
||||
if (VERBOSE) System.out.println(" nextPos storePayloads=" + storePayloads);
|
||||
assert posPending > 0;
|
||||
posPending--;
|
||||
if (!storePayloads) {
|
||||
pos += in.readVInt();
|
||||
} else {
|
||||
final int code = in.readVInt();
|
||||
pos += code >>> 1;
|
||||
if ((code & 1) != 0) {
|
||||
payloadLength = in.readVInt();
|
||||
//System.out.println(" new payloadLen=" + payloadLength);
|
||||
//} else {
|
||||
//System.out.println(" same payloadLen=" + payloadLength);
|
||||
}
|
||||
payload.offset = in.getPosition();
|
||||
in.skipBytes(payloadLength);
|
||||
payload.length = payloadLength;
|
||||
payloadRetrieved = false;
|
||||
}
|
||||
|
||||
if (VERBOSE) System.out.println(" pos=" + pos + " payload=" + payload + " fp=" + in.getPosition());
|
||||
return pos;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getPayload() {
|
||||
payloadRetrieved = true;
|
||||
return payload;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasPayload() {
|
||||
return !payloadRetrieved && payload.length > 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return docID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) {
|
||||
// TODO: we could make more efficient version, but, it
|
||||
// should be rare that this will matter in practice
|
||||
// since usually apps will not store "big" fields in
|
||||
// this codec!
|
||||
//System.out.println("advance target=" + target);
|
||||
while(nextDoc() < target) {
|
||||
}
|
||||
//System.out.println(" return " + docID);
|
||||
return docID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() {
|
||||
return freq;
|
||||
}
|
||||
}
|
||||
|
||||
private final static class FSTTermsEnum extends TermsEnum {
|
||||
private final FieldInfo field;
|
||||
private final BytesRefFSTEnum<BytesRef> fstEnum;
|
||||
private final ByteArrayDataInput buffer = new ByteArrayDataInput(null);
|
||||
|
||||
private int docFreq;
|
||||
private long totalTermFreq;
|
||||
private BytesRefFSTEnum.InputOutput<BytesRef> current;
|
||||
|
||||
public FSTTermsEnum(FieldInfo field, FST<BytesRef> fst) {
|
||||
this.field = field;
|
||||
fstEnum = new BytesRefFSTEnum<BytesRef>(fst);
|
||||
}
|
||||
|
||||
private void readTermStats() throws IOException {
|
||||
buffer.reset(current.output.bytes, 0, current.output.length);
|
||||
docFreq = buffer.readVInt();
|
||||
if (!field.omitTermFreqAndPositions) {
|
||||
totalTermFreq = docFreq + buffer.readVLong();
|
||||
} else {
|
||||
totalTermFreq = 0;
|
||||
}
|
||||
current.output.offset = buffer.getPosition();
|
||||
if (VERBOSE) System.out.println(" df=" + docFreq + " totTF=" + totalTermFreq + " offset=" + buffer.getPosition() + " len=" + current.output.length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SeekStatus seek(BytesRef text, boolean useCache /* ignored */) throws IOException {
|
||||
if (VERBOSE) System.out.println("te.seek text=" + field.name + ":" + text.utf8ToString() + " this=" + this);
|
||||
current = fstEnum.seekCeil(text);
|
||||
if (current == null) {
|
||||
return SeekStatus.END;
|
||||
} else {
|
||||
if (VERBOSE) {
|
||||
System.out.println(" got term=" + current.input.utf8ToString());
|
||||
for(int i=0;i<current.output.length;i++) {
|
||||
System.out.println(" " + Integer.toHexString(current.output.bytes[i]&0xFF));
|
||||
}
|
||||
}
|
||||
|
||||
readTermStats();
|
||||
if (text.equals(current.input)) {
|
||||
if (VERBOSE) System.out.println(" found!");
|
||||
return SeekStatus.FOUND;
|
||||
} else {
|
||||
if (VERBOSE) System.out.println(" not found: " + current.input.utf8ToString());
|
||||
return SeekStatus.NOT_FOUND;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsEnum docs(Bits skipDocs, DocsEnum reuse) {
|
||||
FSTDocsEnum docsEnum;
|
||||
if (reuse == null || !(reuse instanceof FSTDocsEnum)) {
|
||||
docsEnum = new FSTDocsEnum(field.omitTermFreqAndPositions, field.storePayloads);
|
||||
} else {
|
||||
docsEnum = (FSTDocsEnum) reuse;
|
||||
if (!docsEnum.canReuse(field.omitTermFreqAndPositions, field.storePayloads)) {
|
||||
docsEnum = new FSTDocsEnum(field.omitTermFreqAndPositions, field.storePayloads);
|
||||
}
|
||||
}
|
||||
return docsEnum.reset(current.output, skipDocs, docFreq);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse) {
|
||||
if (field.omitTermFreqAndPositions) {
|
||||
return null;
|
||||
}
|
||||
FSTDocsAndPositionsEnum docsAndPositionsEnum;
|
||||
if (reuse == null || !(reuse instanceof FSTDocsAndPositionsEnum)) {
|
||||
docsAndPositionsEnum = new FSTDocsAndPositionsEnum(field.omitTermFreqAndPositions, field.storePayloads);
|
||||
} else {
|
||||
docsAndPositionsEnum = (FSTDocsAndPositionsEnum) reuse;
|
||||
if (!docsAndPositionsEnum.canReuse(field.omitTermFreqAndPositions, field.storePayloads)) {
|
||||
docsAndPositionsEnum = new FSTDocsAndPositionsEnum(field.omitTermFreqAndPositions, field.storePayloads);
|
||||
}
|
||||
}
|
||||
if (VERBOSE) System.out.println("D&P reset this=" + this);
|
||||
return docsAndPositionsEnum.reset(current.output, skipDocs, docFreq);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef term() {
|
||||
return current.input;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef next() throws IOException {
|
||||
if (VERBOSE) System.out.println("te.next");
|
||||
current = fstEnum.next();
|
||||
if (current == null) {
|
||||
if (VERBOSE) System.out.println(" END");
|
||||
return null;
|
||||
}
|
||||
readTermStats();
|
||||
if (VERBOSE) System.out.println(" term=" + field.name + ":" + current.input.utf8ToString());
|
||||
return current.input;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docFreq() {
|
||||
return docFreq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long totalTermFreq() {
|
||||
return totalTermFreq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public SeekStatus seek(long ord) {
|
||||
// NOTE: we could add this...
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ord() {
|
||||
// NOTE: we could add this...
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private final static class TermsReader extends Terms {
|
||||
|
||||
private final long sumTotalTermFreq;
|
||||
private FST<BytesRef> fst;
|
||||
private final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
|
||||
private final FieldInfo field;
|
||||
|
||||
public TermsReader(FieldInfos fieldInfos, IndexInput in) throws IOException {
|
||||
final int fieldNumber = in.readVInt();
|
||||
field = fieldInfos.fieldInfo(fieldNumber);
|
||||
if (!field.omitTermFreqAndPositions) {
|
||||
sumTotalTermFreq = in.readVInt();
|
||||
} else {
|
||||
sumTotalTermFreq = 0;
|
||||
}
|
||||
|
||||
fst = new FST<BytesRef>(in, outputs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSumTotalTermFreq() {
|
||||
return sumTotalTermFreq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum iterator() {
|
||||
return new FSTTermsEnum(field, fst);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.codecId, EXTENSION);
|
||||
final IndexInput in = state.dir.openInput(fileName);
|
||||
|
||||
final SortedMap<String,TermsReader> fields = new TreeMap<String,TermsReader>();
|
||||
|
||||
try {
|
||||
while(true) {
|
||||
final int termCount = in.readVInt();
|
||||
if (termCount == 0) {
|
||||
break;
|
||||
}
|
||||
final TermsReader termsReader = new TermsReader(state.fieldInfos, in);
|
||||
fields.put(termsReader.field.name, termsReader);
|
||||
}
|
||||
} finally {
|
||||
in.close();
|
||||
}
|
||||
|
||||
return new FieldsProducer() {
|
||||
@Override
|
||||
public FieldsEnum iterator() {
|
||||
final Iterator<TermsReader> iter = fields.values().iterator();
|
||||
|
||||
return new FieldsEnum() {
|
||||
|
||||
private TermsReader current;
|
||||
|
||||
@Override
|
||||
public String next() {
|
||||
current = iter.next();
|
||||
return current.field.name;
|
||||
}
|
||||
|
||||
public TermsEnum terms() {
|
||||
return current.iterator();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Terms terms(String field) {
|
||||
return fields.get(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void loadTermsIndex(int indexDivisor) {
|
||||
// no op
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
// Drop ref to FST:
|
||||
for(TermsReader termsReader : fields.values()) {
|
||||
termsReader.fst = null;
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, EXTENSION));
|
||||
DefaultDocValuesConsumer.files(dir, segmentInfo, id, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getExtensions(Set<String> extensions) {
|
||||
extensions.add(EXTENSION);
|
||||
DefaultDocValuesConsumer.getDocValuesExtensions(extensions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
return new DefaultDocValuesConsumer(state, BytesRef.getUTF8SortedAsUnicodeComparator());
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
|
||||
return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId);
|
||||
}
|
||||
}
|
|
@ -19,7 +19,8 @@ package org.apache.lucene.search.function;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
||||
import org.apache.lucene.index.IndexReader; // for javadocs
|
||||
import org.apache.lucene.search.Explanation;
|
||||
import org.apache.lucene.search.FieldCache; // for javadocs
|
||||
|
||||
|
@ -28,7 +29,7 @@ import org.apache.lucene.search.FieldCache; // for javadocs
|
|||
* {@link CustomScoreQuery#getCustomScoreProvider}, if you want
|
||||
* to modify the custom score calculation of a {@link CustomScoreQuery}.
|
||||
* <p>Since Lucene 2.9, queries operate on each segment of an index separately,
|
||||
* so the protected {@link #reader} field can be used to resolve doc IDs,
|
||||
* so the protected {@link #context} field can be used to resolve doc IDs,
|
||||
* as the supplied <code>doc</code> ID is per-segment and without knowledge
|
||||
* of the IndexReader you cannot access the document or {@link FieldCache}.
|
||||
*
|
||||
|
@ -37,13 +38,13 @@ import org.apache.lucene.search.FieldCache; // for javadocs
|
|||
*/
|
||||
public class CustomScoreProvider {
|
||||
|
||||
protected final IndexReader reader;
|
||||
protected final AtomicReaderContext context;
|
||||
|
||||
/**
|
||||
* Creates a new instance of the provider class for the given {@link IndexReader}.
|
||||
*/
|
||||
public CustomScoreProvider(IndexReader reader) {
|
||||
this.reader = reader;
|
||||
public CustomScoreProvider(AtomicReaderContext context) {
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -175,8 +175,8 @@ public class CustomScoreQuery extends Query {
|
|||
* implementation as specified in the docs of {@link CustomScoreProvider}.
|
||||
* @since 2.9.2
|
||||
*/
|
||||
protected CustomScoreProvider getCustomScoreProvider(IndexReader reader) throws IOException {
|
||||
return new CustomScoreProvider(reader);
|
||||
protected CustomScoreProvider getCustomScoreProvider(AtomicReaderContext context) throws IOException {
|
||||
return new CustomScoreProvider(context);
|
||||
}
|
||||
|
||||
//=========================== W E I G H T ============================
|
||||
|
@ -251,7 +251,7 @@ public class CustomScoreQuery extends Query {
|
|||
for(int i = 0; i < valSrcScorers.length; i++) {
|
||||
valSrcScorers[i] = valSrcWeights[i].scorer(context, scorerContext.scoreDocsInOrder(true));
|
||||
}
|
||||
return new CustomScorer(context.reader, this, subQueryScorer, valSrcScorers);
|
||||
return new CustomScorer(CustomScoreQuery.this.getCustomScoreProvider(context), this, subQueryScorer, valSrcScorers);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -270,7 +270,7 @@ public class CustomScoreQuery extends Query {
|
|||
for(int i = 0; i < valSrcWeights.length; i++) {
|
||||
valSrcExpls[i] = valSrcWeights[i].explain(info, doc);
|
||||
}
|
||||
Explanation customExp = CustomScoreQuery.this.getCustomScoreProvider(info.reader).customExplain(doc,subQueryExpl,valSrcExpls);
|
||||
Explanation customExp = CustomScoreQuery.this.getCustomScoreProvider(info).customExplain(doc,subQueryExpl,valSrcExpls);
|
||||
float sc = getValue() * customExp.getValue();
|
||||
Explanation res = new ComplexExplanation(
|
||||
true, sc, CustomScoreQuery.this.toString() + ", product of:");
|
||||
|
@ -300,14 +300,14 @@ public class CustomScoreQuery extends Query {
|
|||
private float vScores[]; // reused in score() to avoid allocating this array for each doc
|
||||
|
||||
// constructor
|
||||
private CustomScorer(IndexReader reader, CustomWeight w,
|
||||
private CustomScorer(CustomScoreProvider provider, CustomWeight w,
|
||||
Scorer subQueryScorer, Scorer[] valSrcScorers) throws IOException {
|
||||
super(w);
|
||||
this.qWeight = w.getValue();
|
||||
this.subQueryScorer = subQueryScorer;
|
||||
this.valSrcScorers = valSrcScorers;
|
||||
this.vScores = new float[valSrcScorers.length];
|
||||
this.provider = CustomScoreQuery.this.getCustomScoreProvider(reader);
|
||||
this.provider = provider;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -67,6 +67,26 @@ public class RAMOutputStream extends IndexOutput {
|
|||
}
|
||||
}
|
||||
|
||||
/** Copy the current contents of this buffer to output
|
||||
* byte array */
|
||||
public void writeTo(byte[] bytes, int offset) throws IOException {
|
||||
flush();
|
||||
final long end = file.length;
|
||||
long pos = 0;
|
||||
int buffer = 0;
|
||||
int bytesUpto = offset;
|
||||
while (pos < end) {
|
||||
int length = BUFFER_SIZE;
|
||||
long nextPos = pos + length;
|
||||
if (nextPos > end) { // at the last buffer
|
||||
length = (int)(end - pos);
|
||||
}
|
||||
System.arraycopy(file.getBuffer(buffer++), 0, bytes, bytesUpto, length);
|
||||
bytesUpto += length;
|
||||
pos = nextPos;
|
||||
}
|
||||
}
|
||||
|
||||
/** Resets this to an empty file. */
|
||||
public void reset() {
|
||||
currentBuffer = null;
|
||||
|
|
|
@ -21,7 +21,7 @@ import org.apache.lucene.util.ArrayUtil;
|
|||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.fst.FST.INPUT_TYPE;
|
||||
import org.apache.lucene.util.fst.FST.INPUT_TYPE; // javadoc
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
@ -136,6 +136,13 @@ public class Builder<T> {
|
|||
return dedupHash == null ? 0 : fst.nodeCount;
|
||||
}
|
||||
|
||||
/** Pass false to disable the array arc optimization
|
||||
* while building the FST. This is necessary if
|
||||
* encoding a single arc may take more than 255 bytes. */
|
||||
public void setAllowArrayArcs(boolean b) {
|
||||
fst.setAllowArrayArcs(b);
|
||||
}
|
||||
|
||||
private CompiledNode compileNode(UnCompiledNode<T> n) throws IOException {
|
||||
|
||||
final int address;
|
||||
|
|
|
@ -104,6 +104,8 @@ public class FST<T> {
|
|||
// If arc has this label then that arc is final/accepted
|
||||
public static final int END_LABEL = -1;
|
||||
|
||||
private boolean allowArrayArcs = true;
|
||||
|
||||
public final static class Arc<T> {
|
||||
public int label;
|
||||
public T output;
|
||||
|
@ -795,6 +797,10 @@ public class FST<T> {
|
|||
public int getArcWithOutputCount() {
|
||||
return arcWithOutputCount;
|
||||
}
|
||||
|
||||
public void setAllowArrayArcs(boolean v) {
|
||||
allowArrayArcs = v;
|
||||
}
|
||||
|
||||
/**
|
||||
* Nodes will be expanded if their depth (distance from the root node) is
|
||||
|
@ -812,8 +818,9 @@ public class FST<T> {
|
|||
* @see Builder.UnCompiledNode#depth
|
||||
*/
|
||||
private boolean shouldExpand(UnCompiledNode<T> node) {
|
||||
return (node.depth <= FIXED_ARRAY_SHALLOW_DISTANCE && node.numArcs >= FIXED_ARRAY_NUM_ARCS_SHALLOW) ||
|
||||
node.numArcs >= FIXED_ARRAY_NUM_ARCS_DEEP;
|
||||
return allowArrayArcs &&
|
||||
((node.depth <= FIXED_ARRAY_SHALLOW_DISTANCE && node.numArcs >= FIXED_ARRAY_NUM_ARCS_SHALLOW) ||
|
||||
node.numArcs >= FIXED_ARRAY_NUM_ARCS_DEEP);
|
||||
}
|
||||
|
||||
// Non-static: writes to FST's byte[]
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.util.ArrayList;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
|
||||
|
@ -56,12 +56,12 @@ public class MockRandomMergePolicy extends MergePolicy {
|
|||
|
||||
@Override
|
||||
public MergeSpecification findMergesForOptimize(
|
||||
SegmentInfos segmentInfos, int maxSegmentCount, Set<SegmentInfo> segmentsToOptimize)
|
||||
SegmentInfos segmentInfos, int maxSegmentCount, Map<SegmentInfo,Boolean> segmentsToOptimize)
|
||||
throws CorruptIndexException, IOException {
|
||||
|
||||
final List<SegmentInfo> eligibleSegments = new ArrayList<SegmentInfo>();
|
||||
for(SegmentInfo info : segmentInfos) {
|
||||
if (segmentsToOptimize.contains(info)) {
|
||||
if (segmentsToOptimize.containsKey(info)) {
|
||||
eligibleSegments.add(info);
|
||||
}
|
||||
}
|
||||
|
@ -85,7 +85,7 @@ public class MockRandomMergePolicy extends MergePolicy {
|
|||
if (mergeSpec != null) {
|
||||
for(OneMerge merge : mergeSpec.merges) {
|
||||
for(SegmentInfo info : merge.segments) {
|
||||
assert segmentsToOptimize.contains(info);
|
||||
assert segmentsToOptimize.containsKey(info);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -52,6 +52,7 @@ import org.apache.lucene.index.codecs.preflexrw.PreFlexRWCodec;
|
|||
import org.apache.lucene.index.codecs.pulsing.PulsingCodec;
|
||||
import org.apache.lucene.index.codecs.simpletext.SimpleTextCodec;
|
||||
import org.apache.lucene.index.codecs.standard.StandardCodec;
|
||||
import org.apache.lucene.index.codecs.memory.MemoryCodec;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.FieldCache;
|
||||
import org.apache.lucene.search.FieldCache.CacheEntry;
|
||||
|
@ -1452,6 +1453,7 @@ public abstract class LuceneTestCase extends Assert {
|
|||
register(new PreFlexCodec());
|
||||
register(new PulsingCodec(1));
|
||||
register(new SimpleTextCodec());
|
||||
register(new MemoryCodec());
|
||||
Collections.shuffle(knownCodecs, random);
|
||||
}
|
||||
|
||||
|
|
|
@ -1013,6 +1013,7 @@ public class TestIndexWriterReader extends LuceneTestCase {
|
|||
HashSet<String> illegalCodecs = new HashSet<String>();
|
||||
illegalCodecs.add("PreFlex");
|
||||
illegalCodecs.add("SimpleText");
|
||||
illegalCodecs.add("Memory");
|
||||
|
||||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT,
|
||||
new MockAnalyzer(random)).setReaderTermsIndexDivisor(-1);
|
||||
|
|
|
@ -133,6 +133,7 @@ public class TestLazyProxSkipping extends LuceneTestCase {
|
|||
|
||||
public void testLazySkipping() throws IOException {
|
||||
assumeFalse("This test cannot run with SimpleText codec", CodecProvider.getDefault().getFieldCodec(this.field).equals("SimpleText"));
|
||||
assumeFalse("This test cannot run with Memory codec", CodecProvider.getDefault().getFieldCodec(this.field).equals("Memory"));
|
||||
// test whether only the minimum amount of seeks()
|
||||
// are performed
|
||||
performTest(5);
|
||||
|
|
|
@ -17,7 +17,6 @@ package org.apache.lucene.index;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.StringReader;
|
||||
|
||||
|
@ -65,6 +64,7 @@ public class TestLongPostings extends LuceneTestCase {
|
|||
|
||||
public void testLongPostings() throws Exception {
|
||||
assumeFalse("Too slow with SimpleText codec", CodecProvider.getDefault().getFieldCodec("field").equals("SimpleText"));
|
||||
assumeFalse("Too slow with Memory codec", CodecProvider.getDefault().getFieldCodec("field").equals("Memory"));
|
||||
|
||||
// Don't use _TestUtil.getTempDir so that we own the
|
||||
// randomness (ie same seed will point to same dir):
|
||||
|
|
|
@ -96,7 +96,8 @@ public class TestNRTThreads extends LuceneTestCase {
|
|||
|
||||
final long t0 = System.currentTimeMillis();
|
||||
|
||||
if (CodecProvider.getDefault().getDefaultFieldCodec().equals("SimpleText")) {
|
||||
final String defaultCodec = CodecProvider.getDefault().getDefaultFieldCodec();
|
||||
if (defaultCodec.equals("SimpleText") || defaultCodec.equals("Memory")) {
|
||||
// no
|
||||
CodecProvider.getDefault().setDefaultFieldCodec("Standard");
|
||||
}
|
||||
|
|
|
@ -21,7 +21,7 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.index.TermsEnum.SeekStatus;
|
||||
|
@ -271,7 +271,7 @@ public class TestPerSegmentDeletes extends LuceneTestCase {
|
|||
|
||||
@Override
|
||||
public MergeSpecification findMergesForOptimize(SegmentInfos segmentInfos,
|
||||
int maxSegmentCount, Set<SegmentInfo> segmentsToOptimize)
|
||||
int maxSegmentCount, Map<SegmentInfo,Boolean> segmentsToOptimize)
|
||||
throws CorruptIndexException, IOException {
|
||||
return null;
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.lucene.index;
|
|||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.*;
|
||||
import org.apache.lucene.document.Field.Index;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.store.*;
|
||||
import org.apache.lucene.util.*;
|
||||
import org.junit.Test;
|
||||
|
@ -31,12 +32,18 @@ public class TestRollingUpdates extends LuceneTestCase {
|
|||
|
||||
@Test
|
||||
public void testRollingUpdates() throws Exception {
|
||||
final Directory dir = newDirectory();
|
||||
|
||||
final MockDirectoryWrapper dir = newDirectory();
|
||||
dir.setCheckIndexOnClose(false); // we use a custom codec provider
|
||||
final LineFileDocs docs = new LineFileDocs(random);
|
||||
|
||||
final IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)));
|
||||
final int SIZE = atLeast(20);
|
||||
CodecProvider provider = CodecProvider.getDefault();
|
||||
//provider.register(new MemoryCodec());
|
||||
if (random.nextBoolean()) {
|
||||
provider.setFieldCodec("docid", "Memory");
|
||||
}
|
||||
|
||||
final IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setCodecProvider(provider));
|
||||
final int SIZE = atLeast(TEST_NIGHTLY ? 100 : 20);
|
||||
int id = 0;
|
||||
IndexReader r = null;
|
||||
final int numUpdates = (int) (SIZE * (2+random.nextDouble()));
|
||||
|
@ -71,6 +78,7 @@ public class TestRollingUpdates extends LuceneTestCase {
|
|||
w.close();
|
||||
docs.close();
|
||||
|
||||
_TestUtil.checkIndex(dir);
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
|
|
@ -26,7 +26,7 @@ import java.io.IOException;
|
|||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
||||
import org.apache.lucene.index.Term;
|
||||
|
||||
/**
|
||||
|
@ -95,8 +95,8 @@ public class TestCustomScoreQuery extends FunctionTestSetup {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected CustomScoreProvider getCustomScoreProvider(IndexReader reader) {
|
||||
return new CustomScoreProvider(reader) {
|
||||
protected CustomScoreProvider getCustomScoreProvider(AtomicReaderContext context) {
|
||||
return new CustomScoreProvider(context) {
|
||||
@Override
|
||||
public float customScore(int doc, float subQueryScore, float valSrcScore) {
|
||||
return subQueryScore + valSrcScore;
|
||||
|
@ -130,8 +130,8 @@ public class TestCustomScoreQuery extends FunctionTestSetup {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected CustomScoreProvider getCustomScoreProvider(IndexReader reader) {
|
||||
return new CustomScoreProvider(reader) {
|
||||
protected CustomScoreProvider getCustomScoreProvider(AtomicReaderContext context) {
|
||||
return new CustomScoreProvider(context) {
|
||||
@Override
|
||||
public float customScore(int doc, float subQueryScore, float valSrcScores[]) {
|
||||
if (valSrcScores.length == 0) {
|
||||
|
@ -169,12 +169,12 @@ public class TestCustomScoreQuery extends FunctionTestSetup {
|
|||
private final class CustomExternalQuery extends CustomScoreQuery {
|
||||
|
||||
@Override
|
||||
protected CustomScoreProvider getCustomScoreProvider(IndexReader reader) throws IOException {
|
||||
final int[] values = FieldCache.DEFAULT.getInts(reader, INT_FIELD);
|
||||
return new CustomScoreProvider(reader) {
|
||||
protected CustomScoreProvider getCustomScoreProvider(AtomicReaderContext context) throws IOException {
|
||||
final int[] values = FieldCache.DEFAULT.getInts(context.reader, INT_FIELD);
|
||||
return new CustomScoreProvider(context) {
|
||||
@Override
|
||||
public float customScore(int doc, float subScore, float valSrcScore) throws IOException {
|
||||
assertTrue(doc <= reader.maxDoc());
|
||||
assertTrue(doc <= context.reader.maxDoc());
|
||||
return values[doc];
|
||||
}
|
||||
};
|
||||
|
|
|
@ -980,7 +980,8 @@ public class TestFSTs extends LuceneTestCase {
|
|||
// file, up until a time limit
|
||||
public void testRealTerms() throws Exception {
|
||||
|
||||
if (CodecProvider.getDefault().getDefaultFieldCodec().equals("SimpleText")) {
|
||||
final String defaultCodec = CodecProvider.getDefault().getDefaultFieldCodec();
|
||||
if (defaultCodec.equals("SimpleText") || defaultCodec.equals("Memory")) {
|
||||
// no
|
||||
CodecProvider.getDefault().setDefaultFieldCodec("Standard");
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue