Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseQuery.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseQuery.java Thu Jul 22 19:34:35 2010
@@ -20,6 +20,7 @@ package org.apache.lucene.search;
import java.io.IOException;
import java.util.Set;
import java.util.ArrayList;
+import java.util.Arrays;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.BytesRef;
@@ -120,6 +121,22 @@ public class PhraseQuery extends Query {
return super.rewrite(reader);
}
+ static class PostingsAndFreq implements Comparable<PostingsAndFreq> {
+ final DocsAndPositionsEnum postings;
+ final int docFreq;
+ final int position;
+
+ public PostingsAndFreq(DocsAndPositionsEnum postings, int docFreq, int position) {
+ this.postings = postings;
+ this.docFreq = docFreq;
+ this.position = position;
+ }
+
+ public int compareTo(PostingsAndFreq other) {
+ return docFreq - other.docFreq;
+ }
+ }
+
private class PhraseWeight extends Weight {
private final Similarity similarity;
private float value;
@@ -163,19 +180,18 @@ public class PhraseQuery extends Query {
if (terms.size() == 0) // optimize zero-term case
return null;
- DocsAndPositionsEnum[] postings = new DocsAndPositionsEnum[terms.size()];
+ PostingsAndFreq[] postingsFreqs = new PostingsAndFreq[terms.size()];
final Bits delDocs = MultiFields.getDeletedDocs(reader);
for (int i = 0; i < terms.size(); i++) {
final Term t = terms.get(i);
- final BytesRef text = new BytesRef(t.text());
DocsAndPositionsEnum postingsEnum = MultiFields.getTermPositionsEnum(reader,
delDocs,
t.field(),
- text);
+ t.bytes());
// PhraseQuery on a field that did not index
// positions.
if (postingsEnum == null) {
- if (MultiFields.getTermDocsEnum(reader, delDocs, t.field(), text) != null) {
+ if (MultiFields.getTermDocsEnum(reader, delDocs, t.field(), t.bytes()) != null) {
// term does exist, but has no positions
throw new IllegalStateException("field \"" + t.field() + "\" was indexed with Field.omitTermFreqAndPositions=true; cannot run PhraseQuery (term=" + t.text() + ")");
} else {
@@ -183,17 +199,27 @@ public class PhraseQuery extends Query {
return null;
}
}
- postings[i] = postingsEnum;
+ postingsFreqs[i] = new PostingsAndFreq(postingsEnum, reader.docFreq(t.field(), t.bytes()), positions.get(i).intValue());
+ }
+
+ // sort by increasing docFreq order
+ if (slop == 0) {
+ Arrays.sort(postingsFreqs);
}
- if (slop == 0) // optimize exact case
- return new ExactPhraseScorer(this, postings, getPositions(), similarity,
- reader.norms(field));
- else
+ if (slop == 0) { // optimize exact case
+ ExactPhraseScorer s = new ExactPhraseScorer(this, postingsFreqs, similarity,
+ reader.norms(field));
+ if (s.noDocs) {
+ return null;
+ } else {
+ return s;
+ }
+ } else {
return
- new SloppyPhraseScorer(this, postings, getPositions(), similarity, slop,
+ new SloppyPhraseScorer(this, postingsFreqs, similarity, slop,
reader.norms(field));
-
+ }
}
@Override
@@ -244,13 +270,23 @@ public class PhraseQuery extends Query {
fieldExpl.setDescription("fieldWeight("+field+":"+query+" in "+doc+
"), product of:");
- PhraseScorer scorer = (PhraseScorer) scorer(reader, true, false);
+ Scorer scorer = (Scorer) scorer(reader, true, false);
if (scorer == null) {
return new Explanation(0.0f, "no matching docs");
}
Explanation tfExplanation = new Explanation();
int d = scorer.advance(doc);
- float phraseFreq = (d == doc) ? scorer.currentFreq() : 0.0f;
+ float phraseFreq;
+ if (d == doc) {
+ if (slop == 0) {
+ phraseFreq = ((ExactPhraseScorer) scorer).currentFreq();
+ } else {
+ phraseFreq = ((SloppyPhraseScorer) scorer).currentFreq();
+ }
+ } else {
+ phraseFreq = 0.0f;
+ }
+
tfExplanation.setValue(similarity.tf(phraseFreq));
tfExplanation.setDescription("tf(phraseFreq=" + phraseFreq + ")");
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseScorer.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseScorer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseScorer.java Thu Jul 22 19:34:35 2010
@@ -19,8 +19,6 @@ package org.apache.lucene.search;
import java.io.IOException;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-
/** Expert: Scoring functionality for phrase queries.
* <br>A document is considered matching if it contains the phrase-query terms
* at "valid" positions. What "valid positions" are
@@ -43,7 +41,7 @@ abstract class PhraseScorer extends Scor
private float freq; //phrase frequency in current doc as computed by phraseFreq().
- PhraseScorer(Weight weight, DocsAndPositionsEnum[] postings, int[] offsets,
+ PhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
Similarity similarity, byte[] norms) {
super(similarity);
this.norms = norms;
@@ -56,7 +54,7 @@ abstract class PhraseScorer extends Scor
// this allows to easily identify a matching (exact) phrase
// when all PhrasePositions have exactly the same position.
for (int i = 0; i < postings.length; i++) {
- PhrasePositions pp = new PhrasePositions(postings[i], offsets[i]);
+ PhrasePositions pp = new PhrasePositions(postings[i].postings, postings[i].position);
if (last != null) { // add next to end of list
last.next = pp;
} else {
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PrefixQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PrefixQuery.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PrefixQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PrefixQuery.java Thu Jul 22 19:34:35 2010
@@ -44,14 +44,9 @@ public class PrefixQuery extends MultiTe
/** Returns the prefix of this query. */
public Term getPrefix() { return prefix; }
- @Override @Deprecated
- protected FilteredTermEnum getEnum(IndexReader reader) throws IOException {
- return new PrefixTermEnum(reader, prefix);
- }
-
@Override
protected TermsEnum getTermsEnum(IndexReader reader) throws IOException {
- if (prefix.text().length() == 0) {
+ if (prefix.bytes().length == 0) {
// no prefix -- match all terms for this field:
final Terms terms = MultiFields.getTerms(reader, getField());
return (terms != null) ? terms.iterator() : TermsEnum.EMPTY;
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java Thu Jul 22 19:34:35 2010
@@ -36,7 +36,7 @@ public class PrefixTermsEnum extends Fil
public PrefixTermsEnum(IndexReader reader, Term prefix) throws IOException {
super(reader, prefix.field());
- setInitialSeekTerm(prefixRef = new BytesRef(prefix.text()));
+ setInitialSeekTerm(prefixRef = prefix.bytes());
}
@Override
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/QueryTermVector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/QueryTermVector.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/QueryTermVector.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/QueryTermVector.java Thu Jul 22 19:34:35 2010
@@ -29,14 +29,16 @@ import java.util.Map;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
import org.apache.lucene.index.TermFreqVector;
+import org.apache.lucene.util.BytesRef;
/**
*
*
**/
public class QueryTermVector implements TermFreqVector {
- private String [] terms = new String[0];
+ private BytesRef [] terms = new BytesRef[0];
private int [] termFreqs = new int[0];
public String getField() { return null; }
@@ -45,7 +47,7 @@ public class QueryTermVector implements
*
* @param queryTerms The original list of terms from the query, can contain duplicates
*/
- public QueryTermVector(String [] queryTerms) {
+ public QueryTermVector(BytesRef [] queryTerms) {
processTerms(queryTerms);
}
@@ -56,35 +58,37 @@ public class QueryTermVector implements
TokenStream stream = analyzer.tokenStream("", new StringReader(queryString));
if (stream != null)
{
- List<String> terms = new ArrayList<String>();
+ List<BytesRef> terms = new ArrayList<BytesRef>();
try {
boolean hasMoreTokens = false;
stream.reset();
- final CharTermAttribute termAtt = stream.addAttribute(CharTermAttribute.class);
+ final TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
hasMoreTokens = stream.incrementToken();
while (hasMoreTokens) {
- terms.add(termAtt.toString());
+ BytesRef bytes = new BytesRef();
+ termAtt.toBytesRef(bytes);
+ terms.add(bytes);
hasMoreTokens = stream.incrementToken();
}
- processTerms(terms.toArray(new String[terms.size()]));
+ processTerms(terms.toArray(new BytesRef[terms.size()]));
} catch (IOException e) {
}
}
}
}
- private void processTerms(String[] queryTerms) {
+ private void processTerms(BytesRef[] queryTerms) {
if (queryTerms != null) {
Arrays.sort(queryTerms);
- Map<String,Integer> tmpSet = new HashMap<String,Integer>(queryTerms.length);
+ Map<BytesRef,Integer> tmpSet = new HashMap<BytesRef,Integer>(queryTerms.length);
//filter out duplicates
- List<String> tmpList = new ArrayList<String>(queryTerms.length);
+ List<BytesRef> tmpList = new ArrayList<BytesRef>(queryTerms.length);
List<Integer> tmpFreqs = new ArrayList<Integer>(queryTerms.length);
int j = 0;
for (int i = 0; i < queryTerms.length; i++) {
- String term = queryTerms[i];
+ BytesRef term = queryTerms[i];
Integer position = tmpSet.get(term);
if (position == null) {
tmpSet.put(term, Integer.valueOf(j++));
@@ -112,7 +116,7 @@ public class QueryTermVector implements
sb.append('{');
for (int i=0; i<terms.length; i++) {
if (i>0) sb.append(", ");
- sb.append(terms[i]).append('/').append(termFreqs[i]);
+ sb.append(terms[i].utf8ToString()).append('/').append(termFreqs[i]);
}
sb.append('}');
return sb.toString();
@@ -123,7 +127,7 @@ public class QueryTermVector implements
return terms.length;
}
- public String[] getTerms() {
+ public BytesRef[] getTerms() {
return terms;
}
@@ -131,12 +135,12 @@ public class QueryTermVector implements
return termFreqs;
}
- public int indexOf(String term) {
+ public int indexOf(BytesRef term) {
int res = Arrays.binarySearch(terms, term);
return res >= 0 ? res : -1;
}
- public int[] indexesOf(String[] terms, int start, int len) {
+ public int[] indexesOf(BytesRef[] terms, int start, int len) {
int res[] = new int[len];
for (int i=0; i < len; i++) {
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SingleTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SingleTermsEnum.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SingleTermsEnum.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SingleTermsEnum.java Thu Jul 22 19:34:35 2010
@@ -41,7 +41,7 @@ public final class SingleTermsEnum exten
*/
public SingleTermsEnum(IndexReader reader, Term singleTerm) throws IOException {
super(reader, singleTerm.field());
- singleRef = new BytesRef(singleTerm.text());
+ singleRef = singleTerm.bytes();
setInitialSeekTerm(singleRef);
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java Thu Jul 22 19:34:35 2010
@@ -17,8 +17,6 @@ package org.apache.lucene.search;
* limitations under the License.
*/
-import org.apache.lucene.index.DocsAndPositionsEnum;
-
import java.io.IOException;
import java.util.HashMap;
@@ -28,9 +26,9 @@ final class SloppyPhraseScorer extends P
private PhrasePositions tmpPos[]; // for flipping repeating pps.
private boolean checkedRepeats;
- SloppyPhraseScorer(Weight weight, DocsAndPositionsEnum[] postings, int[] offsets, Similarity similarity,
+ SloppyPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings, Similarity similarity,
int slop, byte[] norms) {
- super(weight, postings, offsets, similarity, norms);
+ super(weight, postings, similarity, norms);
this.slop = slop;
}
@@ -53,7 +51,7 @@ final class SloppyPhraseScorer extends P
* We may want to fix this in the future (currently not, for performance reasons).
*/
@Override
- protected final float phraseFreq() throws IOException {
+ protected float phraseFreq() throws IOException {
int end = initPhrasePositions();
float freq = 0.0f;
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermQuery.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermQuery.java Thu Jul 22 19:34:35 2010
@@ -75,7 +75,7 @@ public class TermQuery extends Query {
public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
// NOTE: debateably, the caller should never pass in a
// multi reader...
- DocsEnum docs = MultiFields.getTermDocsEnum(reader, MultiFields.getDeletedDocs(reader), term.field(), new BytesRef(term.text()));
+ DocsEnum docs = MultiFields.getTermDocsEnum(reader, MultiFields.getDeletedDocs(reader), term.field(), term.bytes());
if (docs == null) {
return null;
}
@@ -118,7 +118,7 @@ public class TermQuery extends Query {
Explanation tfExplanation = new Explanation();
int tf = 0;
- DocsEnum docs = reader.termDocsEnum(MultiFields.getDeletedDocs(reader), term.field(), new BytesRef(term.text()));
+ DocsEnum docs = reader.termDocsEnum(MultiFields.getDeletedDocs(reader), term.field(), term.bytes());
if (docs != null) {
int newDoc = docs.advance(doc);
if (newDoc == doc) {
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java Thu Jul 22 19:34:35 2010
@@ -129,12 +129,6 @@ public class TermRangeQuery extends Mult
/** Returns the collator used to determine range inclusion, if any. */
public Collator getCollator() { return collator; }
- @Override @Deprecated
- protected FilteredTermEnum getEnum(IndexReader reader) throws IOException {
- return new TermRangeTermEnum(reader, field, lowerTerm,
- upperTerm, includeLower, includeUpper, collator);
- }
-
@Override
protected TermsEnum getTermsEnum(IndexReader reader) throws IOException {
if (collator == null && lowerTerm != null && upperTerm != null && lowerTerm.compareTo(upperTerm) > 0) {
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TimeLimitingCollector.java Thu Jul 22 19:34:35 2010
@@ -111,7 +111,7 @@ public class TimeLimitingCollector exten
public long getTimeElapsed() {
return timeElapsed;
}
- /** Returns last doc that was collected when the search time exceeded. */
+ /** Returns last doc (absolute doc id) that was collected when the search time exceeded. */
public int getLastDocCollected() {
return lastDocCollected;
}
@@ -129,6 +129,8 @@ public class TimeLimitingCollector exten
private final long t0;
private final long timeout;
private final Collector collector;
+
+ private int docBase;
/**
* Create a TimeLimitedCollector wrapper over another {@link Collector} with a specified timeout.
@@ -200,19 +202,20 @@ public class TimeLimitingCollector exten
long time = TIMER_THREAD.getMilliseconds();
if (timeout < time) {
if (greedy) {
- //System.out.println(this+" greedy: before failing, collecting doc: "+doc+" "+(time-t0));
+ //System.out.println(this+" greedy: before failing, collecting doc: "+(docBase + doc)+" "+(time-t0));
collector.collect(doc);
}
- //System.out.println(this+" failing on: "+doc+" "+(time-t0));
- throw new TimeExceededException( timeout-t0, time-t0, doc );
+ //System.out.println(this+" failing on: "+(docBase + doc)+" "+(time-t0));
+ throw new TimeExceededException( timeout-t0, time-t0, docBase + doc );
}
- //System.out.println(this+" collecting: "+doc+" "+(time-t0));
+ //System.out.println(this+" collecting: "+(docBase + doc)+" "+(time-t0));
collector.collect(doc);
}
@Override
public void setNextReader(IndexReader reader, int base) throws IOException {
collector.setNextReader(reader, base);
+ this.docBase = base;
}
@Override
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TopDocsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TopDocsCollector.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TopDocsCollector.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TopDocsCollector.java Thu Jul 22 19:34:35 2010
@@ -25,8 +25,11 @@ import org.apache.lucene.util.PriorityQu
* collector allows easy extension by providing a single constructor which
* accepts a {@link PriorityQueue} as well as protected members for that
* priority queue and a counter of the number of total hits.<br>
- * Extending classes can override {@link #topDocs(int, int)} and
- * {@link #getTotalHits()} in order to provide their own implementation.
+ * Extending classes can override any of the methods to provide their own
+ * implementation, as well as avoid the use of the priority queue entirely by
+ * passing null to {@link #TopDocsCollector(PriorityQueue)}. In that case
+ * however, you might want to consider overriding all methods, in order to avoid
+ * a NullPointerException.
*/
public abstract class TopDocsCollector<T extends ScoreDoc> extends Collector {
@@ -50,7 +53,7 @@ public abstract class TopDocsCollector<T
}
/**
- * Populates the results array with the ScoreDoc instaces. This can be
+ * Populates the results array with the ScoreDoc instances. This can be
* overridden in case a different ScoreDoc type should be returned.
*/
protected void populateResults(ScoreDoc[] results, int howMany) {
@@ -75,7 +78,7 @@ public abstract class TopDocsCollector<T
}
/** Returns the top docs that were collected by this collector. */
- public final TopDocs topDocs() {
+ public TopDocs topDocs() {
// In case pq was populated with sentinel values, there might be less
// results than pq.size(). Therefore return all results until either
// pq.size() or totalHits.
@@ -94,7 +97,7 @@ public abstract class TopDocsCollector<T
* with the returned {@link TopDocs} object, which will contain all the
* results this search execution collected.
*/
- public final TopDocs topDocs(int start) {
+ public TopDocs topDocs(int start) {
// In case pq was populated with sentinel values, there might be less
// results than pq.size(). Therefore return all results until either
// pq.size() or totalHits.
@@ -115,7 +118,7 @@ public abstract class TopDocsCollector<T
* returned {@link TopDocs} object, which will contain all the results this
* search execution collected.
*/
- public final TopDocs topDocs(int start, int howMany) {
+ public TopDocs topDocs(int start, int howMany) {
// In case pq was populated with sentinel values, there might be less
// results than pq.size(). Therefore return all results until either
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/WildcardQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/WildcardQuery.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/WildcardQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/WildcardQuery.java Thu Jul 22 19:34:35 2010
@@ -17,15 +17,12 @@ package org.apache.lucene.search;
* limitations under the License.
*/
-import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.ToStringUtils;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.BasicAutomata;
import org.apache.lucene.util.automaton.BasicOperations;
-import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
@@ -81,17 +78,6 @@ public class WildcardQuery extends Autom
return BasicOperations.concatenate(automata);
}
- @Override @Deprecated
- protected FilteredTermEnum getEnum(IndexReader reader) throws IOException {
- return new WildcardTermEnum(reader, term);
- }
-
- // we override this method, else backwards layer in MTQ will prefer getEnum!
- @Override
- protected TermsEnum getTermsEnum(IndexReader reader) throws IOException {
- return super.getTermsEnum(reader);
- }
-
/**
* Returns the pattern term.
*/
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanTermQuery.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanTermQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanTermQuery.java Thu Jul 22 19:34:35 2010
@@ -85,16 +85,15 @@ public class SpanTermQuery extends SpanQ
public Spans getSpans(final IndexReader reader) throws IOException {
// NOTE: debateably, the caller should never pass in a
// multi reader...
- final BytesRef textBytes = new BytesRef(term.text());
final DocsAndPositionsEnum postings = MultiFields.getTermPositionsEnum(reader,
MultiFields.getDeletedDocs(reader),
term.field(),
- textBytes);
+ term.bytes());
if (postings != null) {
return new TermSpans(postings, term);
} else {
- if (MultiFields.getTermDocsEnum(reader, MultiFields.getDeletedDocs(reader), term.field(), textBytes) != null) {
+ if (MultiFields.getTermDocsEnum(reader, MultiFields.getDeletedDocs(reader), term.field(), term.bytes()) != null) {
// term does exist, but has no positions
throw new IllegalStateException("field \"" + term.field() + "\" was indexed with Field.omitTermFreqAndPositions=true; cannot run SpanTermQuery (term=" + term.text() + ")");
} else {
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java Thu Jul 22 19:34:35 2010
@@ -24,6 +24,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Set;
+import java.util.HashSet;
/**
* Expert: A Directory instance that switches files between
@@ -76,12 +77,14 @@ public class FileSwitchDirectory extends
@Override
public String[] listAll() throws IOException {
- String[] primaryFiles = primaryDir.listAll();
- String[] secondaryFiles = secondaryDir.listAll();
- String[] files = new String[primaryFiles.length + secondaryFiles.length];
- System.arraycopy(primaryFiles, 0, files, 0, primaryFiles.length);
- System.arraycopy(secondaryFiles, 0, files, primaryFiles.length, secondaryFiles.length);
- return files;
+ Set<String> files = new HashSet<String>();
+ for(String f : primaryDir.listAll()) {
+ files.add(f);
+ }
+ for(String f : secondaryDir.listAll()) {
+ files.add(f);
+ }
+ return files.toArray(new String[files.size()]);
}
/** Utility method to return a file's extension. */
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/NativeFSLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/NativeFSLockFactory.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/NativeFSLockFactory.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/NativeFSLockFactory.java Thu Jul 22 19:34:35 2010
@@ -85,7 +85,7 @@ public class NativeFSLockFactory extends
// Also, remove any non-alphanumeric characters, so that the lock file will
// be created for sure on all systems.
String randomLockName = "lucene-"
- + ManagementFactory.getRuntimeMXBean().getName().replaceAll("[^a..zA..Z0..9]+","") + "-"
+ + ManagementFactory.getRuntimeMXBean().getName().replaceAll("[^a-zA-Z0-9]+","") + "-"
+ Long.toString(new Random().nextInt(), Character.MAX_RADIX)
+ "-test.lock";
@@ -152,13 +152,20 @@ public class NativeFSLockFactory extends
// they are locked, but, still do this in case people
// really want to see the files go away:
if (lockDir.exists()) {
+
+ // Try to release the lock first - if it's held by another process, this
+ // method should not silently fail.
+ // NOTE: makeLock fixes the lock name by prefixing it w/ lockPrefix.
+ // Therefore it should be called before the code block next which prefixes
+ // the given name.
+ makeLock(lockName).release();
+
if (lockPrefix != null) {
lockName = lockPrefix + "-" + lockName;
}
- File lockFile = new File(lockDir, lockName);
- if (lockFile.exists() && !lockFile.delete()) {
- throw new IOException("Cannot delete " + lockFile);
- }
+
+ // As mentioned above, we don't care if the deletion of the file failed.
+ new File(lockDir, lockName).delete();
}
}
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/ArrayUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/ArrayUtil.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/ArrayUtil.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/ArrayUtil.java Thu Jul 22 19:34:35 2010
@@ -327,6 +327,29 @@ public final class ArrayUtil {
return array;
}
+ public static boolean[] grow(boolean[] array, int minSize) {
+ if (array.length < minSize) {
+ boolean[] newArray = new boolean[oversize(minSize, 1)];
+ System.arraycopy(array, 0, newArray, 0, array.length);
+ return newArray;
+ } else
+ return array;
+ }
+
+ public static boolean[] grow(boolean[] array) {
+ return grow(array, 1 + array.length);
+ }
+
+ public static boolean[] shrink(boolean[] array, int targetSize) {
+ final int newSize = getShrinkSize(array.length, targetSize, 1);
+ if (newSize != array.length) {
+ boolean[] newArray = new boolean[newSize];
+ System.arraycopy(array, 0, newArray, 0, newSize);
+ return newArray;
+ } else
+ return array;
+ }
+
public static char[] grow(char[] array, int minSize) {
if (array.length < minSize) {
char[] newArray = new char[oversize(minSize, RamUsageEstimator.NUM_BYTES_CHAR)];
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRef.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRef.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRef.java Thu Jul 22 19:34:35 2010
@@ -77,6 +77,16 @@ public final class BytesRef implements C
this();
copy(text);
}
+
+ /**
+ * @param text Initialize the byte[] from the UTF8 bytes
+ * for the provided array. This must be well-formed
+ * unicode text, with no unpaired surrogates or U+FFFF.
+ */
+ public BytesRef(char text[], int offset, int length) {
+ this(length * 4);
+ copy(text, offset, length);
+ }
public BytesRef(BytesRef other) {
this();
@@ -106,6 +116,15 @@ public final class BytesRef implements C
UnicodeUtil.UTF16toUTF8(text, 0, text.length(), this);
}
+ /**
+ * Copies the UTF8 bytes for this string.
+ *
+ * @param text Must be well-formed unicode text, with no
+ * unpaired surrogates or invalid UTF16 code units.
+ */
+ public void copy(char text[], int offset, int length) {
+ UnicodeUtil.UTF16toUTF8(text, offset, length, this);
+ }
public boolean bytesEquals(BytesRef other) {
if (length == other.length) {
int otherUpto = other.offset;
@@ -217,14 +236,7 @@ public final class BytesRef implements C
bytes = ArrayUtil.grow(bytes, newLength);
}
- private final static Comparator<BytesRef> utf8SortedAsUTF16SortOrder = new UTF8SortedAsUTF16Comparator();
-
- public static Comparator<BytesRef> getUTF8SortedAsUTF16Comparator() {
- return utf8SortedAsUTF16SortOrder;
- }
-
/** Unsigned byte order comparison */
- /*
public int compareTo(BytesRef other) {
if (this == other) return 0;
@@ -245,44 +257,49 @@ public final class BytesRef implements C
// One is a prefix of the other, or, they are equal:
return this.length - other.length;
}
- */
- /** Lucene default index order. Currently the same as String.compareTo() (UTF16) but will change
- * in the future to unsigned byte comparison. */
- public int compareTo(BytesRef other) {
- if (this == other) return 0;
+ private final static Comparator<BytesRef> utf8SortedAsUnicodeSortOrder = new UTF8SortedAsUnicodeComparator();
- final byte[] aBytes = this.bytes;
- int aUpto = this.offset;
- final byte[] bBytes = other.bytes;
- int bUpto = other.offset;
+ public static Comparator<BytesRef> getUTF8SortedAsUnicodeComparator() {
+ return utf8SortedAsUnicodeSortOrder;
+ }
- final int aStop = aUpto + Math.min(this.length, other.length);
+ private static class UTF8SortedAsUnicodeComparator implements Comparator<BytesRef> {
+ // Only singleton
+ private UTF8SortedAsUnicodeComparator() {};
- while(aUpto < aStop) {
- int aByte = aBytes[aUpto++] & 0xff;
- int bByte = bBytes[bUpto++] & 0xff;
- if (aByte != bByte) {
+ public int compare(BytesRef a, BytesRef b) {
+ final byte[] aBytes = a.bytes;
+ int aUpto = a.offset;
+ final byte[] bBytes = b.bytes;
+ int bUpto = b.offset;
+
+ final int aStop;
+ if (a.length < b.length) {
+ aStop = aUpto + a.length;
+ } else {
+ aStop = aUpto + b.length;
+ }
- // See http://icu-project.org/docs/papers/utf16_code_point_order.html#utf-8-in-utf-16-order
+ while(aUpto < aStop) {
+ int aByte = aBytes[aUpto++] & 0xff;
+ int bByte = bBytes[bUpto++] & 0xff;
- // We know the terms are not equal, but, we may
- // have to carefully fixup the bytes at the
- // difference to match UTF16's sort order:
- if (aByte >= 0xee && bByte >= 0xee) {
- if ((aByte & 0xfe) == 0xee) {
- aByte += 0x10;
- }
- if ((bByte&0xfe) == 0xee) {
- bByte += 0x10;
- }
+ int diff = aByte - bByte;
+ if (diff != 0) {
+ return diff;
}
- return aByte - bByte;
}
- }
- // One is a prefix of the other, or, they are equal:
- return this.length - other.length;
+ // One is a prefix of the other, or, they are equal:
+ return a.length - b.length;
+ }
+ }
+
+ private final static Comparator<BytesRef> utf8SortedAsUTF16SortOrder = new UTF8SortedAsUTF16Comparator();
+
+ public static Comparator<BytesRef> getUTF8SortedAsUTF16Comparator() {
+ return utf8SortedAsUTF16SortOrder;
}
private static class UTF8SortedAsUTF16Comparator implements Comparator<BytesRef> {
@@ -352,7 +369,7 @@ public final class BytesRef implements C
bytes = new byte[length];
in.read(bytes, 0, length);
} else {
- bytes = null;
+ bytes = EMPTY_BYTES;
}
}
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/CodecUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/CodecUtil.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/CodecUtil.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/CodecUtil.java Thu Jul 22 19:34:35 2010
@@ -21,6 +21,8 @@ package org.apache.lucene.util;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.IndexFormatTooNewException;
+import org.apache.lucene.index.IndexFormatTooOldException;
import java.io.IOException;
@@ -48,7 +50,7 @@ public final class CodecUtil {
return 9+codec.length();
}
- public static int checkHeader(IndexInput in, String codec, int maxVersion)
+ public static int checkHeader(IndexInput in, String codec, int minVersion, int maxVersion)
throws IOException {
// Safety to guard against reading a bogus string:
@@ -63,8 +65,11 @@ public final class CodecUtil {
}
final int actualVersion = in.readInt();
+ if (actualVersion < minVersion) {
+ throw new IndexFormatTooOldException(null, actualVersion, minVersion, maxVersion);
+ }
if (actualVersion > maxVersion) {
- throw new CorruptIndexException("version " + actualVersion + " is too new (expected <= version " + maxVersion + ")");
+ throw new IndexFormatTooNewException(null, actualVersion, minVersion, maxVersion);
}
return actualVersion;
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/NumericUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/NumericUtils.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/NumericUtils.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/NumericUtils.java Thu Jul 22 19:34:35 2010
@@ -446,8 +446,11 @@ public final class NumericUtils {
final long
nextMinBound = (hasLower ? (minBound + diff) : minBound) & ~mask,
nextMaxBound = (hasUpper ? (maxBound - diff) : maxBound) & ~mask;
-
- if (shift+precisionStep>=valSize || nextMinBound>nextMaxBound) {
+ final boolean
+ lowerWrapped = nextMinBound < minBound,
+ upperWrapped = nextMaxBound > maxBound;
+
+ if (shift+precisionStep>=valSize || nextMinBound>nextMaxBound || lowerWrapped || upperWrapped) {
// We are in the lowest precision or the next precision is not available.
addRange(builder, valSize, minBound, maxBound, shift);
// exit the split recursion loop
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/PagedBytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/PagedBytes.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/PagedBytes.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/PagedBytes.java Thu Jul 22 19:34:35 2010
@@ -32,6 +32,7 @@ import java.io.IOException;
* <p>@lucene.internal</p>*/
public final class PagedBytes {
private final List<byte[]> blocks = new ArrayList<byte[]>();
+ private final List<Integer> blockEnd = new ArrayList<Integer>();
private final int blockSize;
private final int blockBits;
private final int blockMask;
@@ -42,6 +43,7 @@ public final class PagedBytes {
public final static class Reader implements Closeable {
private final byte[][] blocks;
+ private final int[] blockEnds;
private final int blockBits;
private final int blockMask;
private final int blockSize;
@@ -52,6 +54,10 @@ public final class PagedBytes {
for(int i=0;i<blocks.length;i++) {
blocks[i] = pagedBytes.blocks.get(i);
}
+ blockEnds = new int[blocks.length];
+ for(int i=0;i< blockEnds.length;i++) {
+ blockEnds[i] = pagedBytes.blockEnd.get(i);
+ }
blockBits = pagedBytes.blockBits;
blockMask = pagedBytes.blockMask;
blockSize = pagedBytes.blockSize;
@@ -102,6 +108,54 @@ public final class PagedBytes {
return b;
}
+ /** @lucene.internal Reads length as 1 or 2 byte vInt prefix, starting @ start. Returns the block number of the term. */
+ public int fillUsingLengthPrefix2(BytesRef b, long start) {
+ final int index = (int) (start >> blockBits);
+ final int offset = (int) (start & blockMask);
+ final byte[] block = b.bytes = blocks[index];
+
+ if ((block[offset] & 128) == 0) {
+ b.length = block[offset];
+ b.offset = offset+1;
+ } else {
+ b.length = (((int) (block[offset] & 0x7f)) << 8) | (block[1+offset] & 0xff);
+ b.offset = offset+2;
+ assert b.length > 0;
+ }
+ return index;
+ }
+
+ /** @lucene.internal Reads length as 1 or 2 byte vInt prefix, starting @ start.
+ * Returns the start offset of the next part, suitable as start parameter on next call
+ * to sequentially read all BytesRefs. */
+ public long fillUsingLengthPrefix3(BytesRef b, long start) {
+ final int index = (int) (start >> blockBits);
+ final int offset = (int) (start & blockMask);
+ final byte[] block = b.bytes = blocks[index];
+
+ if ((block[offset] & 128) == 0) {
+ b.length = block[offset];
+ b.offset = offset+1;
+ start += 1L + b.length;
+ } else {
+ b.length = (((int) (block[offset] & 0x7f)) << 8) | (block[1+offset] & 0xff);
+ b.offset = offset+2;
+ start += 2L + b.length;
+ assert b.length > 0;
+ }
+ return start;
+ }
+
+ /** @lucene.internal */
+ public byte[][] getBlocks() {
+ return blocks;
+ }
+
+ /** @lucene.internal */
+ public int[] getBlockEnds() {
+ return blockEnds;
+ }
+
public void close() {
threadBuffers.close();
}
@@ -123,6 +177,7 @@ public final class PagedBytes {
if (left == 0) {
if (currentBlock != null) {
blocks.add(currentBlock);
+ blockEnd.add(upto);
}
currentBlock = new byte[blockSize];
upto = 0;
@@ -149,6 +204,7 @@ public final class PagedBytes {
if (left == 0) {
if (currentBlock != null) {
blocks.add(currentBlock);
+ blockEnd.add(upto);
}
currentBlock = new byte[blockSize];
upto = 0;
@@ -167,9 +223,34 @@ public final class PagedBytes {
}
}
- /** Commits final byte[], trimming it if necessary. */
- public Reader freeze() {
- if (upto < blockSize) {
+ /** Copy BytesRef in, setting BytesRef out to the result.
+ * Do not use this if you will use freeze(true).
+ * This only supports bytes.length <= blockSize */
+ public void copy(BytesRef bytes, BytesRef out) throws IOException {
+ int left = blockSize - upto;
+ if (bytes.length > left) {
+ if (currentBlock != null) {
+ blocks.add(currentBlock);
+ blockEnd.add(upto);
+ }
+ currentBlock = new byte[blockSize];
+ upto = 0;
+ left = blockSize;
+ assert bytes.length <= blockSize;
+ // TODO: we could also support variable block sizes
+ }
+
+ out.bytes = currentBlock;
+ out.offset = upto;
+ out.length = bytes.length;
+
+ System.arraycopy(bytes.bytes, bytes.offset, currentBlock, upto, bytes.length);
+ upto += bytes.length;
+ }
+
+ /** Commits final byte[], trimming it if necessary and if trim=true */
+ public Reader freeze(boolean trim) {
+ if (trim && upto < blockSize) {
final byte[] newBlock = new byte[upto];
System.arraycopy(currentBlock, 0, newBlock, 0, upto);
currentBlock = newBlock;
@@ -178,6 +259,7 @@ public final class PagedBytes {
currentBlock = EMPTY_BYTES;
}
blocks.add(currentBlock);
+ blockEnd.add(upto);
currentBlock = null;
return new Reader(this);
}
@@ -200,6 +282,7 @@ public final class PagedBytes {
}
if (currentBlock != null) {
blocks.add(currentBlock);
+ blockEnd.add(upto);
}
currentBlock = new byte[blockSize];
upto = 0;
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java Thu Jul 22 19:34:35 2010
@@ -358,7 +358,6 @@ final public class UnicodeUtil {
out[outUpto++] = (char) ((chHalf & HALF_MASK) + UNI_SUR_LOW_START);
}
}
-
offsets[upto] = outUpto;
result.length = outUpto;
}
@@ -483,7 +482,7 @@ final public class UnicodeUtil {
}
}
*/
- public static final boolean validUTF16String(CharSequence s) {
+ public static boolean validUTF16String(CharSequence s) {
final int size = s.length();
for(int i=0;i<size;i++) {
char ch = s.charAt(i);
@@ -507,7 +506,7 @@ final public class UnicodeUtil {
return true;
}
- public static final boolean validUTF16String(char[] s, int size) {
+ public static boolean validUTF16String(char[] s, int size) {
for(int i=0;i<size;i++) {
char ch = s[i];
if (ch >= UNI_SUR_HIGH_START && ch <= UNI_SUR_HIGH_END) {
@@ -559,7 +558,7 @@ final public class UnicodeUtil {
/** Returns the number of code points in this utf8
* sequence. Behavior is undefined if the utf8 sequence
* is invalid.*/
- public static final int codePointCount(BytesRef utf8) {
+ public static int codePointCount(BytesRef utf8) {
int upto = utf8.offset;
final int limit = utf8.offset + utf8.length;
final byte[] bytes = utf8.bytes;
@@ -673,4 +672,33 @@ final public class UnicodeUtil {
}
return new String(chars, 0, w);
}
+
+ // for debugging
+ public static String toHexString(String s) {
+ StringBuilder sb = new StringBuilder();
+ for(int i=0;i<s.length();i++) {
+ char ch = s.charAt(i);
+ if (i > 0) {
+ sb.append(' ');
+ }
+ if (ch < 128) {
+ sb.append(ch);
+ } else {
+ if (ch >= UNI_SUR_HIGH_START && ch <= UNI_SUR_HIGH_END) {
+ sb.append("H:");
+ } else if (ch >= UNI_SUR_LOW_START && ch <= UNI_SUR_LOW_END) {
+ sb.append("L:");
+ } else if (ch > UNI_SUR_LOW_END) {
+ if (ch == 0xffff) {
+ sb.append("F:");
+ } else {
+ sb.append("E:");
+ }
+ }
+
+ sb.append("0x" + Integer.toHexString(ch));
+ }
+ }
+ return sb.toString();
+ }
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java Thu Jul 22 19:34:35 2010
@@ -38,12 +38,10 @@ import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
-import java.util.IdentityHashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
-import java.util.Random;
/**
* Basic automata operations.
@@ -848,143 +846,4 @@ final public class BasicOperations {
return accept;
}
}
-
- // picks a random int code point that this transition
- // accepts, avoiding the surrogates range since they are
- // "defined" in UTF32. Don't call this on a transition
- // that only accepts UTF16 surrogate values!!
- private static int getRandomCodePoint(final Random r, final Transition t) {
- return t.min+r.nextInt(t.max-t.min+1);
- }
-
- public static class RandomAcceptedStrings {
-
- private final Map<Transition,Boolean> leadsToAccept;
- private final Automaton a;
-
- private static class ArrivingTransition {
- final State from;
- final Transition t;
- public ArrivingTransition(State from, Transition t) {
- this.from = from;
- this.t = t;
- }
- }
-
- public RandomAcceptedStrings(Automaton a) {
- this.a = a;
- if (a.isSingleton()) {
- leadsToAccept = null;
- return;
- }
-
- // must use IdentityHashmap because two Transitions w/
- // different start nodes can be considered the same
- leadsToAccept = new IdentityHashMap<Transition,Boolean>();
- final Map<State,List<ArrivingTransition>> allArriving = new HashMap<State,List<ArrivingTransition>>();
-
- final LinkedList<State> q = new LinkedList<State>();
- final Set<State> seen = new HashSet<State>();
-
- // reverse map the transitions, so we can quickly look
- // up all arriving transitions to a given state
- for(State s: a.getNumberedStates()) {
- for(int i=0;i<s.numTransitions;i++) {
- final Transition t = s.transitionsArray[i];
- List<ArrivingTransition> tl = allArriving.get(t.to);
- if (tl == null) {
- tl = new ArrayList<ArrivingTransition>();
- allArriving.put(t.to, tl);
- }
- tl.add(new ArrivingTransition(s, t));
- }
- if (s.accept) {
- q.add(s);
- seen.add(s);
- }
- }
-
- // Breadth-first search, from accept states,
- // backwards:
- while(!q.isEmpty()) {
- final State s = q.removeFirst();
- List<ArrivingTransition> arriving = allArriving.get(s);
- if (arriving != null) {
- for(ArrivingTransition at : arriving) {
- final State from = at.from;
- if (!seen.contains(from)) {
- q.add(from);
- seen.add(from);
- leadsToAccept.put(at.t, Boolean.TRUE);
- }
- }
- }
- }
- }
-
- public int[] getRandomAcceptedString(Random r) {
-
- final List<Integer> soFar = new ArrayList<Integer>();
- if (a.isSingleton()) {
- // accepts only one
- final String s = a.singleton;
-
- int charUpto = 0;
- while(charUpto < s.length()) {
- final int cp = s.codePointAt(charUpto);
- charUpto += Character.charCount(cp);
- soFar.add(cp);
- }
- } else {
-
- State s = a.initial;
-
- while(true) {
-
- if (s.accept) {
- if (s.numTransitions == 0) {
- // stop now
- break;
- } else {
- if (r.nextBoolean()) {
- break;
- }
- }
- }
-
- if (s.numTransitions == 0) {
- throw new RuntimeException("this automaton has dead states");
- }
-
- boolean cheat = r.nextBoolean();
-
- final Transition t;
- if (cheat) {
- // pick a transition that we know is the fastest
- // path to an accept state
- List<Transition> toAccept = new ArrayList<Transition>();
- for(int i=0;i<s.numTransitions;i++) {
- final Transition t0 = s.transitionsArray[i];
- if (leadsToAccept.containsKey(t0)) {
- toAccept.add(t0);
- }
- }
- if (toAccept.size() == 0) {
- // this is OK -- it means we jumped into a cycle
- t = s.transitionsArray[r.nextInt(s.numTransitions)];
- } else {
- t = toAccept.get(r.nextInt(toAccept.size()));
- }
- } else {
- t = s.transitionsArray[r.nextInt(s.numTransitions)];
- }
-
- soFar.add(getRandomCodePoint(r, t));
- s = t.to;
- }
- }
-
- return ArrayUtil.toIntArray(soFar);
- }
- }
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/Transition.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/Transition.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/Transition.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/Transition.java Thu Jul 22 19:34:35 2010
@@ -210,64 +210,4 @@ public class Transition implements Seria
}
public static final Comparator<Transition> CompareByMinMaxThenDest = new CompareByMinMaxThenDestSingle();
-
- private static class UTF8InUTF16Order {
- protected int compareCodePoint(int aByte, int bByte) {
- if (aByte != bByte) {
- // See http://icu-project.org/docs/papers/utf16_code_point_order.html#utf-8-in-utf-16-order
-
- // We know the terms are not equal, but, we may
- // have to carefully fixup the bytes at the
- // difference to match UTF16's sort order:
- if (aByte >= 0xee && bByte >= 0xee) {
- if ((aByte & 0xfe) == 0xee) {
- aByte += 0x10;
- }
- if ((bByte&0xfe) == 0xee) {
- bByte += 0x10;
- }
- }
- return aByte - bByte;
- }
- return 0;
- }
- }
-
- private static final class CompareByDestThenMinMaxUTF8InUTF16OrderSingle extends UTF8InUTF16Order implements Comparator<Transition> {
- public int compare(Transition t1, Transition t2) {
- if (t1.to != t2.to) {
- if (t1.to == null) return -1;
- else if (t2.to == null) return 1;
- else if (t1.to.number < t2.to.number) return -1;
- else if (t1.to.number > t2.to.number) return 1;
- }
- int minComp = compareCodePoint(t1.min, t2.min);
- if (minComp != 0) return minComp;
- int maxComp = compareCodePoint(t1.max, t2.max);
- if (maxComp != 0) return maxComp;
- return 0;
- }
- }
-
- public static final Comparator<Transition> CompareByDestThenMinMaxUTF8InUTF16Order = new CompareByDestThenMinMaxUTF8InUTF16OrderSingle();
-
- private static final class CompareByMinMaxThenDestUTF8InUTF16OrderSingle extends UTF8InUTF16Order implements Comparator<Transition> {
- public int compare(Transition t1, Transition t2) {
- int minComp = compareCodePoint(t1.min, t2.min);
- if (minComp != 0) return minComp;
- int maxComp = compareCodePoint(t1.max, t2.max);
- if (maxComp != 0) return maxComp;
- if (t1.to != t2.to) {
- if (t1.to == null) return -1;
- else if (t2.to == null) return 1;
- else if (t1.to.number < t2.to.number) return -1;
- else if (t1.to.number > t2.to.number) return 1;
- }
- return 0;
- }
- }
-
- public static final Comparator<Transition> CompareByMinMaxThenDestUTF8InUTF16Order = new CompareByMinMaxThenDestUTF8InUTF16OrderSingle();
-
-
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/Direct16.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/Direct16.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/Direct16.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/Direct16.java Thu Jul 22 19:34:35 2010
@@ -25,9 +25,10 @@ import java.util.Arrays;
/**
* Direct wrapping of 16 bit values to a backing array of shorts.
+ * @lucene.internal
*/
-class Direct16 extends PackedInts.ReaderImpl
+public class Direct16 extends PackedInts.ReaderImpl
implements PackedInts.Mutable {
private short[] values;
private static final int BITS_PER_VALUE = 16;
@@ -67,6 +68,10 @@ class Direct16 extends PackedInts.Reader
this.values = values;
}
+ public short[] getArray() {
+ return values;
+ }
+
public long get(final int index) {
return 0xFFFFL & values[index];
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/Direct32.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/Direct32.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/Direct32.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/Direct32.java Thu Jul 22 19:34:35 2010
@@ -25,9 +25,10 @@ import java.util.Arrays;
/**
* Direct wrapping of 32 bit values to a backing array of ints.
+ * @lucene.internal
*/
-class Direct32 extends PackedInts.ReaderImpl
+public class Direct32 extends PackedInts.ReaderImpl
implements PackedInts.Mutable {
private int[] values;
private static final int BITS_PER_VALUE = 32;
@@ -63,6 +64,10 @@ class Direct32 extends PackedInts.Reader
this.values = values;
}
+ public int[] getArray() {
+ return values;
+ }
+
public long get(final int index) {
return 0xFFFFFFFFL & values[index];
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/Direct64.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/Direct64.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/Direct64.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/Direct64.java Thu Jul 22 19:34:35 2010
@@ -25,9 +25,10 @@ import java.util.Arrays;
/**
* Direct wrapping of 32 bit values to a backing array of ints.
+ * @lucene.internal
*/
-class Direct64 extends PackedInts.ReaderImpl
+public class Direct64 extends PackedInts.ReaderImpl
implements PackedInts.Mutable {
private long[] values;
private static final int BITS_PER_VALUE = 64;
@@ -47,6 +48,9 @@ class Direct64 extends PackedInts.Reader
this.values = values;
}
+ public long[] getArray() {
+ return values;
+ }
/**
* Creates an array backed by the given values.
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/Direct8.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/Direct8.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/Direct8.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/Direct8.java Thu Jul 22 19:34:35 2010
@@ -25,9 +25,10 @@ import java.util.Arrays;
/**
* Direct wrapping of 8 bit values to a backing array of bytes.
+ * @lucene.internal
*/
-class Direct8 extends PackedInts.ReaderImpl
+public class Direct8 extends PackedInts.ReaderImpl
implements PackedInts.Mutable {
private byte[] values;
private static final int BITS_PER_VALUE = 8;
@@ -68,6 +69,10 @@ class Direct8 extends PackedInts.ReaderI
this.values = values;
}
+ public byte[] getArray() {
+ return values;
+ }
+
public long get(final int index) {
return 0xFFL & values[index];
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/PackedInts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/PackedInts.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/PackedInts.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/packed/PackedInts.java Thu Jul 22 19:34:35 2010
@@ -157,7 +157,7 @@ public class PackedInts {
* @lucene.internal
*/
public static Reader getReader(IndexInput in) throws IOException {
- CodecUtil.checkHeader(in, CODEC_NAME, VERSION_START);
+ CodecUtil.checkHeader(in, CODEC_NAME, VERSION_START, VERSION_START);
final int bitsPerValue = in.readVInt();
assert bitsPerValue > 0 && bitsPerValue <= 64: "bitsPerValue=" + bitsPerValue;
final int valueCount = in.readVInt();
@@ -188,7 +188,7 @@ public class PackedInts {
* @lucene.internal
*/
public static ReaderIterator getReaderIterator(IndexInput in) throws IOException {
- CodecUtil.checkHeader(in, CODEC_NAME, VERSION_START);
+ CodecUtil.checkHeader(in, CODEC_NAME, VERSION_START, VERSION_START);
final int bitsPerValue = in.readVInt();
assert bitsPerValue > 0 && bitsPerValue <= 64: "bitsPerValue=" + bitsPerValue;
final int valueCount = in.readVInt();
Modified: lucene/dev/branches/realtime_search/lucene/src/site/changes/changes2html.pl
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/site/changes/changes2html.pl?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/site/changes/changes2html.pl (original)
+++ lucene/dev/branches/realtime_search/lucene/src/site/changes/changes2html.pl Thu Jul 22 19:34:35 2010
@@ -62,8 +62,8 @@ for (my $line_num = 0 ; $line_num <= $#l
if (/\s*===+\s*(.*?)\s*===+\s*/) { # New-style release headings
$release = $1;
- $release =~ s/^release\s*//i; # Trim "Release " prefix
- ($release, $relinfo) = ($release =~ /^(\d+(?:\.\d+)*|Trunk)\s*(.*)/i);
+ $release =~ s/^(?:release|lucene)\s*//i; # Trim "Release " or "Lucene " prefix
+ ($release, $relinfo) = ($release =~ /^(\d+(?:\.(?:\d+|[xyz]))*|Trunk)\s*(.*)/i);
$relinfo =~ s/\s*:\s*$//; # Trim trailing colon
$relinfo =~ s/^\s*,\s*//; # Trim leading comma
($reldate, $relinfo) = get_release_date($release, $relinfo);
@@ -164,6 +164,14 @@ for (my $line_num = 0 ; $line_num <= $#l
}
}
+# Recognize IDs of top level nodes of the most recent two releases,
+# escaping JavaScript regex metacharacters, e.g.: "^(?:trunk|2\\\\.4\\\\.0)"
+my $first_relid_regex = $first_relid;
+$first_relid_regex =~ s!([.+*?{}()|^$/\[\]\\])!\\\\\\\\$1!g;
+my $second_relid_regex = $second_relid;
+$second_relid_regex =~ s!([.+*?{}()|^$/\[\]\\])!\\\\\\\\$1!g;
+my $newer_version_regex = "^(?:$first_relid_regex|$second_relid_regex)";
+
#
# Print HTML-ified version to STDOUT
#
@@ -258,7 +266,7 @@ print<<"__HTML_HEADER__";
}
- var newerRegex = new RegExp("^(?:trunk|2\\\\.4\\\\.0)");
+ var newerRegex = new RegExp("$newer_version_regex");
function isOlder(listId) {
return ! newerRegex.test(listId);
}
@@ -388,16 +396,21 @@ for my $rel (@releases) {
for my $itemnum (1..$#{$items}) {
my $item = $items->[$itemnum];
- $item =~ s:&:&:g; # Escape HTML metachars,
- $item =~ s:<(?!/?code>):<:gi; # but leave <code> tags intact
- $item =~ s:(?<!code)>:>:gi; # and add <pre> tags so that
- $item =~ s:<code>:<code><pre>:gi; # whitespace is preserved in the
- $item =~ s:\s*</code>:</pre></code>:gi; # output.
+ $item =~ s:&:&:g; # Escape HTML metachars, but leave
+ $item =~ s:<(?!/?code>):<:gi; # <code> tags intact and add <pre>
+ $item =~ s:(?<!code)>:>:gi; # wrappers for non-inline sections
+ $item =~ s{((?:^|.*\n)\s*)<code>(?!</code>.+)(.+)</code>(?![ \t]*\S)}
+ {
+ my $prefix = $1;
+ my $code = $2;
+ $code =~ s/\s+$//;
+ "$prefix<code><pre>$code</pre></code>"
+ }gise;
# Put attributions on their own lines.
# Check for trailing parenthesized attribution with no following period.
# Exclude things like "(see #3 above)" and "(use the bug number instead of xxxx)"
- unless ($item =~ s:\s*(\((?!see #|use the bug number)[^)"]+?\))\s*$:\n<br /><span class="attrib">$1</span>:) {
+ unless ($item =~ s:\s*(\((?!see #|use the bug number)[^()"]+?\))\s*$:\n<br /><span class="attrib">$1</span>:) {
# If attribution is not found, then look for attribution with a
# trailing period, but try not to include trailing parenthesized things
# that are not attributions.
@@ -405,9 +418,9 @@ for my $rel (@releases) {
# Rule of thumb: if a trailing parenthesized expression with a following
# period does not contain "LUCENE-XXX", and it either has three or
# fewer words or it includes the word "via" or the phrase "updates from",
- # then it is considered to be an attribution.
+ # then it is considered to be an attribution.
- $item =~ s{(\s*(\((?!see \#|use the bug number)[^)"]+?\)))
+ $item =~ s{(\s*(\((?!see \#|use the bug number)[^()"]+?\)))
((?:\.|(?i:\.?\s*Issue\s+\d{3,}|LUCENE-\d+)\.?)\s*)$}
{
my $subst = $1; # default: no change
Modified: lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/demo2.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/demo2.xml?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/demo2.xml (original)
+++ lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/demo2.xml Thu Jul 22 19:34:35 2010
@@ -74,7 +74,7 @@ words such as articles (a, an, the, etc.
(e.g. <b>'s</b>) . It should be noted that there are different rules for every language, and you
should use the proper analyzer for each. Lucene currently provides Analyzers for a number of
different languages (see the <code>*Analyzer.java</code> sources under <a
-href="http://svn.apache.org/repos/asf/lucene/dev/trunk/lucene/contrib/analyzers/common/src/java/org/apache/lucene/analysis/">contrib/analyzers/src/java/org/apache/lucene/analysis</a>).
+href="http://svn.apache.org/repos/asf/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/">modules/analysis/common/src/java/org/apache/lucene/analysis</a>).
</p>
<p>
Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestExternalCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestExternalCodecs.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestExternalCodecs.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestExternalCodecs.java Thu Jul 22 19:34:35 2010
@@ -179,7 +179,7 @@ public class TestExternalCodecs extends
@Override
public Comparator<BytesRef> getComparator() {
- return BytesRef.getUTF8SortedAsUTF16Comparator();
+ return BytesRef.getUTF8SortedAsUnicodeComparator();
}
@Override
@@ -263,7 +263,7 @@ public class TestExternalCodecs extends
@Override
public Comparator<BytesRef> getComparator() {
- return BytesRef.getUTF8SortedAsUTF16Comparator();
+ return BytesRef.getUTF8SortedAsUnicodeComparator();
}
@Override
Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/TestCachingTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/TestCachingTokenFilter.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/TestCachingTokenFilter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/TestCachingTokenFilter.java Thu Jul 22 19:34:35 2010
@@ -27,11 +27,12 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.document.Field.TermVector;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TermPositions;
+import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.RAMDirectory;
+import org.apache.lucene.util.BytesRef;
public class TestCachingTokenFilter extends BaseTokenStreamTestCase {
private String[] tokens = new String[] {"term1", "term2", "term3", "term2"};
@@ -75,19 +76,28 @@ public class TestCachingTokenFilter exte
writer.close();
IndexReader reader = IndexReader.open(dir, true);
- TermPositions termPositions = reader.termPositions(new Term("preanalyzed", "term1"));
- assertTrue(termPositions.next());
+ DocsAndPositionsEnum termPositions = MultiFields.getTermPositionsEnum(reader,
+ MultiFields.getDeletedDocs(reader),
+ "preanalyzed",
+ new BytesRef("term1"));
+ assertTrue(termPositions.nextDoc() != termPositions.NO_MORE_DOCS);
assertEquals(1, termPositions.freq());
assertEquals(0, termPositions.nextPosition());
- termPositions.seek(new Term("preanalyzed", "term2"));
- assertTrue(termPositions.next());
+ termPositions = MultiFields.getTermPositionsEnum(reader,
+ MultiFields.getDeletedDocs(reader),
+ "preanalyzed",
+ new BytesRef("term2"));
+ assertTrue(termPositions.nextDoc() != termPositions.NO_MORE_DOCS);
assertEquals(2, termPositions.freq());
assertEquals(1, termPositions.nextPosition());
assertEquals(3, termPositions.nextPosition());
- termPositions.seek(new Term("preanalyzed", "term3"));
- assertTrue(termPositions.next());
+ termPositions = MultiFields.getTermPositionsEnum(reader,
+ MultiFields.getDeletedDocs(reader),
+ "preanalyzed",
+ new BytesRef("term3"));
+ assertTrue(termPositions.nextDoc() != termPositions.NO_MORE_DOCS);
assertEquals(1, termPositions.freq());
assertEquals(2, termPositions.nextPosition());
reader.close();
Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestBinaryDocument.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestBinaryDocument.java?rev=966819&r1=966818&r2=966819&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestBinaryDocument.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestBinaryDocument.java Thu Jul 22 19:34:35 2010
@@ -4,8 +4,8 @@ import org.apache.lucene.util.LuceneTest
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.store.MockRAMDirectory;
/**
@@ -58,13 +58,12 @@ public class TestBinaryDocument extends
/** add the doc to a ram index */
MockRAMDirectory dir = new MockRAMDirectory();
- IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(
- TEST_VERSION_CURRENT, new MockAnalyzer()));
+ RandomIndexWriter writer = new RandomIndexWriter(newRandom(), dir,
+ new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()));
writer.addDocument(doc);
- writer.close();
/** open a reader and fetch the document */
- IndexReader reader = IndexReader.open(dir, false);
+ IndexReader reader = writer.getReader();
Document docFromReader = reader.document(0);
assertTrue(docFromReader != null);
@@ -76,6 +75,10 @@ public class TestBinaryDocument extends
String stringFldStoredTest = docFromReader.get("stringStored");
assertTrue(stringFldStoredTest.equals(binaryValStored));
+ writer.close();
+ reader.close();
+
+ reader = IndexReader.open(dir, false);
/** delete the document from index */
reader.deleteDocument(0);
assertEquals(0, reader.numDocs());
@@ -95,13 +98,12 @@ public class TestBinaryDocument extends
/** add the doc to a ram index */
MockRAMDirectory dir = new MockRAMDirectory();
- IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(
- TEST_VERSION_CURRENT, new MockAnalyzer()));
+ RandomIndexWriter writer = new RandomIndexWriter(newRandom(), dir,
+ new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()));
writer.addDocument(doc);
- writer.close();
/** open a reader and fetch the document */
- IndexReader reader = IndexReader.open(dir, false);
+ IndexReader reader = writer.getReader();
Document docFromReader = reader.document(0);
assertTrue(docFromReader != null);
@@ -110,6 +112,7 @@ public class TestBinaryDocument extends
assertTrue(binaryFldCompressedTest.equals(binaryValCompressed));
assertTrue(CompressionTools.decompressString(docFromReader.getBinaryValue("stringCompressed")).equals(binaryValCompressed));
+ writer.close();
reader.close();
dir.close();
}
Propchange: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestDateTools.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Jul 22 19:34:35 2010
@@ -1,4 +1,5 @@
-/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/document/TestDateTools.java:943137,949730
+/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/document/TestDateTools.java:943137,949730,957490,960490,961612
+/lucene/dev/trunk/lucene/src/test/org/apache/lucene/document/TestDateTools.java:953476-966816
/lucene/java/branches/flex_1458/src/test/org/apache/lucene/document/TestDateTools.java:824912-931101
/lucene/java/branches/lucene_2_4/src/test/org/apache/lucene/document/TestDateTools.java:748824
/lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/document/TestDateTools.java:829134,829881,831036,896850,909334,948516
|