You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2013/01/18 19:31:23 UTC
svn commit: r1435287 [8/41] - in /lucene/dev/branches/LUCENE-2878: ./
dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/
dev-tools/idea/lucene/analysis/icu/ dev-tools/maven/
dev-tools/maven/lucene/benchmark/ dev-tools/maven/solr/ dev-tools/m...
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Fri Jan 18 18:30:54 2013
@@ -22,6 +22,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
+import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
@@ -336,7 +337,7 @@ public class IndexWriter implements Clos
// obtained during this flush are pooled, the first time
// this method is called:
poolReaders = true;
- final DirectoryReader r;
+ DirectoryReader r = null;
doBeforeFlush();
boolean anySegmentFlushed = false;
/*
@@ -346,46 +347,54 @@ public class IndexWriter implements Clos
* We release the two stage full flush after we are done opening the
* directory reader!
*/
- synchronized (fullFlushLock) {
- boolean success = false;
- try {
- anySegmentFlushed = docWriter.flushAllThreads();
- if (!anySegmentFlushed) {
- // prevent double increment since docWriter#doFlush increments the flushcount
- // if we flushed anything.
- flushCount.incrementAndGet();
- }
- success = true;
- // Prevent segmentInfos from changing while opening the
- // reader; in theory we could do similar retry logic,
- // just like we do when loading segments_N
- synchronized(this) {
- maybeApplyDeletes(applyAllDeletes);
- r = StandardDirectoryReader.open(this, segmentInfos, applyAllDeletes);
- if (infoStream.isEnabled("IW")) {
- infoStream.message("IW", "return reader version=" + r.getVersion() + " reader=" + r);
+ boolean success2 = false;
+ try {
+ synchronized (fullFlushLock) {
+ boolean success = false;
+ try {
+ anySegmentFlushed = docWriter.flushAllThreads();
+ if (!anySegmentFlushed) {
+ // prevent double increment since docWriter#doFlush increments the flushcount
+ // if we flushed anything.
+ flushCount.incrementAndGet();
}
- }
- } catch (OutOfMemoryError oom) {
- handleOOM(oom, "getReader");
- // never reached but javac disagrees:
- return null;
- } finally {
- if (!success) {
- if (infoStream.isEnabled("IW")) {
- infoStream.message("IW", "hit exception during NRT reader");
+ success = true;
+ // Prevent segmentInfos from changing while opening the
+ // reader; in theory we could do similar retry logic,
+ // just like we do when loading segments_N
+ synchronized(this) {
+ maybeApplyDeletes(applyAllDeletes);
+ r = StandardDirectoryReader.open(this, segmentInfos, applyAllDeletes);
+ if (infoStream.isEnabled("IW")) {
+ infoStream.message("IW", "return reader version=" + r.getVersion() + " reader=" + r);
+ }
}
+ } catch (OutOfMemoryError oom) {
+ handleOOM(oom, "getReader");
+ // never reached but javac disagrees:
+ return null;
+ } finally {
+ if (!success) {
+ if (infoStream.isEnabled("IW")) {
+ infoStream.message("IW", "hit exception during NRT reader");
+ }
+ }
+ // Done: finish the full flush!
+ docWriter.finishFullFlush(success);
+ doAfterFlush();
}
- // Done: finish the full flush!
- docWriter.finishFullFlush(success);
- doAfterFlush();
}
- }
- if (anySegmentFlushed) {
- maybeMerge(MergeTrigger.FULL_FLUSH, UNBOUNDED_MAX_MERGE_SEGMENTS);
- }
- if (infoStream.isEnabled("IW")) {
- infoStream.message("IW", "getReader took " + (System.currentTimeMillis() - tStart) + " msec");
+ if (anySegmentFlushed) {
+ maybeMerge(MergeTrigger.FULL_FLUSH, UNBOUNDED_MAX_MERGE_SEGMENTS);
+ }
+ if (infoStream.isEnabled("IW")) {
+ infoStream.message("IW", "getReader took " + (System.currentTimeMillis() - tStart) + " msec");
+ }
+ success2 = true;
+ } finally {
+ if (!success2) {
+ IOUtils.closeWhileHandlingException(r);
+ }
}
return r;
}
@@ -445,15 +454,23 @@ public class IndexWriter implements Clos
/** Remove all our references to readers, and commits
* any pending changes. */
synchronized void dropAll(boolean doSave) throws IOException {
+ Throwable priorE = null;
final Iterator<Map.Entry<SegmentInfoPerCommit,ReadersAndLiveDocs>> it = readerMap.entrySet().iterator();
while(it.hasNext()) {
final ReadersAndLiveDocs rld = it.next().getValue();
- if (doSave && rld.writeLiveDocs(directory)) {
- // Make sure we only write del docs for a live segment:
- assert infoIsLive(rld.info);
- // Must checkpoint w/ deleter, because we just
- // created created new _X_N.del file.
- deleter.checkpoint(segmentInfos, false);
+
+ try {
+ if (doSave && rld.writeLiveDocs(directory)) {
+ // Make sure we only write del docs for a live segment:
+ assert infoIsLive(rld.info);
+ // Must checkpoint w/ deleter, because we just
+ // created created new _X_N.del file.
+ deleter.checkpoint(segmentInfos, false);
+ }
+ } catch (Throwable t) {
+ if (priorE != null) {
+ priorE = t;
+ }
}
// Important to remove as-we-go, not with .clear()
@@ -466,9 +483,18 @@ public class IndexWriter implements Clos
// actually close the SRs; this happens when a
// near real-time reader is kept open after the
// IndexWriter instance is closed:
- rld.dropReaders();
+ try {
+ rld.dropReaders();
+ } catch (Throwable t) {
+ if (priorE != null) {
+ priorE = t;
+ }
+ }
}
assert readerMap.size() == 0;
+ if (priorE != null) {
+ throw new RuntimeException(priorE);
+ }
}
/**
@@ -540,13 +566,18 @@ public class IndexWriter implements Clos
}
/**
- * Used internally to throw an {@link
- * AlreadyClosedException} if this IndexWriter has been
- * closed.
- * @throws AlreadyClosedException if this IndexWriter is closed
+ * Used internally to throw an {@link AlreadyClosedException} if this
+ * IndexWriter has been closed or is in the process of closing.
+ *
+ * @param failIfClosing
+ * if true, also fail when {@code IndexWriter} is in the process of
+ * closing ({@code closing=true}) but not yet done closing (
+ * {@code closed=false})
+ * @throws AlreadyClosedException
+ * if this IndexWriter is closed or in the process of closing
*/
- protected final void ensureOpen(boolean includePendingClose) throws AlreadyClosedException {
- if (closed || (includePendingClose && closing)) {
+ protected final void ensureOpen(boolean failIfClosing) throws AlreadyClosedException {
+ if (closed || (failIfClosing && closing)) {
throw new AlreadyClosedException("this IndexWriter is closed");
}
}
@@ -554,7 +585,8 @@ public class IndexWriter implements Clos
/**
* Used internally to throw an {@link
* AlreadyClosedException} if this IndexWriter has been
- * closed.
+ * closed ({@code closed=true}) or is in the process of
+ * closing ({@code closing=true}).
* <p>
* Calls {@link #ensureOpen(boolean) ensureOpen(true)}.
* @throws AlreadyClosedException if this IndexWriter is closed
@@ -817,6 +849,7 @@ public class IndexWriter implements Clos
*
* @throws IOException if there is a low-level IO error
*/
+ @Override
public void close() throws IOException {
close(true);
}
@@ -954,7 +987,7 @@ public class IndexWriter implements Clos
}
if (doFlush) {
- commitInternal(null);
+ commitInternal();
}
if (infoStream.isEnabled("IW")) {
@@ -1418,7 +1451,7 @@ public class IndexWriter implements Clos
*/
public void updateDocument(Term term, IndexDocument doc) throws IOException {
ensureOpen();
- updateDocument(term, doc, getAnalyzer());
+ updateDocument(term, doc, analyzer);
}
/**
@@ -1910,6 +1943,7 @@ public class IndexWriter implements Clos
* call to {@link #prepareCommit}.
* @throws IOException if there is a low-level IO error
*/
+ @Override
public void rollback() throws IOException {
ensureOpen();
@@ -2553,20 +2587,7 @@ public class IndexWriter implements Clos
*/
protected void doBeforeFlush() throws IOException {}
- /** Expert: prepare for commit.
- *
- * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
- * you should immediately close the writer. See <a
- * href="#OOME">above</a> for details.</p>
- *
- * @see #prepareCommit(Map) */
- public final void prepareCommit() throws IOException {
- ensureOpen();
- prepareCommit(null);
- }
-
- /** <p>Expert: prepare for commit, specifying
- * commitUserData Map (String -> String). This does the
+ /** <p>Expert: prepare for commit. This does the
* first phase of 2-phase commit. This method does all
* steps necessary to commit changes since this writer
* was opened: flushes pending added and deleted docs,
@@ -2576,29 +2597,23 @@ public class IndexWriter implements Clos
* #rollback()} to revert the commit and undo all changes
* done since the writer was opened.</p>
*
- * <p>You can also just call {@link #commit(Map)} directly
+ * <p>You can also just call {@link #commit()} directly
* without prepareCommit first in which case that method
* will internally call prepareCommit.
*
* <p><b>NOTE</b>: if this method hits an OutOfMemoryError
* you should immediately close the writer. See <a
* href="#OOME">above</a> for details.</p>
- *
- * @param commitUserData Opaque Map (String->String)
- * that's recorded into the segments file in the index,
- * and retrievable by {@link
- * IndexCommit#getUserData}. Note that when
- * IndexWriter commits itself during {@link #close}, the
- * commitUserData is unchanged (just carried over from
- * the prior commit). If this is null then the previous
- * commitUserData is kept. Also, the commitUserData will
- * only "stick" if there are actually changes in the
- * index to commit.
*/
- public final void prepareCommit(Map<String,String> commitUserData) throws IOException {
- ensureOpen(false);
+ @Override
+ public final void prepareCommit() throws IOException {
+ ensureOpen();
+ prepareCommitInternal();
+ }
+ private void prepareCommitInternal() throws IOException {
synchronized(commitLock) {
+ ensureOpen(false);
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "prepareCommit: flush");
infoStream.message("IW", " index before flush " + segString());
@@ -2688,10 +2703,33 @@ public class IndexWriter implements Clos
}
}
- startCommit(toCommit, commitUserData);
+ startCommit(toCommit);
}
}
-
+
+ /**
+ * Sets the commit user data map. That method is considered a transaction by
+ * {@link IndexWriter} and will be {@link #commit() committed} even if no other
+ * changes were made to the writer instance. Note that you must call this method
+ * before {@link #prepareCommit()}, or otherwise it won't be included in the
+ * follow-on {@link #commit()}.
+ * <p>
+ * <b>NOTE:</b> the map is cloned internally, therefore altering the map's
+ * contents after calling this method has no effect.
+ */
+ public final synchronized void setCommitData(Map<String,String> commitUserData) {
+ segmentInfos.setUserData(new HashMap<String,String>(commitUserData));
+ ++changeCount;
+ }
+
+ /**
+ * Returns the commit user data map that was last committed, or the one that
+ * was set on {@link #setCommitData(Map)}.
+ */
+ public final synchronized Map<String,String> getCommitData() {
+ return segmentInfos.getUserData();
+ }
+
// Used only by commit and prepareCommit, below; lock
// order is commitLock -> IW
private final Object commitLock = new Object();
@@ -2724,29 +2762,14 @@ public class IndexWriter implements Clos
* href="#OOME">above</a> for details.</p>
*
* @see #prepareCommit
- * @see #commit(Map)
*/
+ @Override
public final void commit() throws IOException {
- commit(null);
- }
-
- /** Commits all changes to the index, specifying a
- * commitUserData Map (String -> String). This just
- * calls {@link #prepareCommit(Map)} (if you didn't
- * already call it) and then {@link #commit}.
- *
- * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
- * you should immediately close the writer. See <a
- * href="#OOME">above</a> for details.</p>
- */
- public final void commit(Map<String,String> commitUserData) throws IOException {
-
ensureOpen();
-
- commitInternal(commitUserData);
+ commitInternal();
}
- private final void commitInternal(Map<String,String> commitUserData) throws IOException {
+ private final void commitInternal() throws IOException {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "commit: start");
@@ -2763,7 +2786,7 @@ public class IndexWriter implements Clos
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "commit: now prepare");
}
- prepareCommit(commitUserData);
+ prepareCommitInternal();
} else {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "commit: already prepared");
@@ -2787,7 +2810,6 @@ public class IndexWriter implements Clos
}
lastCommitChangeCount = pendingCommitChangeCount;
segmentInfos.updateGeneration(pendingCommit);
- segmentInfos.setUserData(pendingCommit.getUserData());
rollbackSegments = pendingCommit.createBackupSegmentInfos();
deleter.checkpoint(pendingCommit, true);
} finally {
@@ -3061,13 +3083,6 @@ public class IndexWriter implements Clos
}
}
- // If new deletes were applied while we were merging
- // (which happens if eg commit() or getReader() is
- // called during our merge), then it better be the case
- // that the delGen has increased for all our merged
- // segments:
- assert mergedDeletes == null || minGen > merge.info.getBufferedDeletesGen();
-
merge.info.setBufferedDeletesGen(minGen);
return mergedDeletes;
@@ -3468,6 +3483,7 @@ public class IndexWriter implements Clos
diagnostics.put("os.version", Constants.OS_VERSION);
diagnostics.put("java.version", Constants.JAVA_VERSION);
diagnostics.put("java.vendor", Constants.JAVA_VENDOR);
+ diagnostics.put("timestamp", Long.toString(new Date().getTime()));
if (details != null) {
diagnostics.putAll(details);
}
@@ -3906,7 +3922,7 @@ public class IndexWriter implements Clos
* if it wasn't already. If that succeeds, then we
* prepare a new segments_N file but do not fully commit
* it. */
- private void startCommit(final SegmentInfos toSync, final Map<String,String> commitUserData) throws IOException {
+ private void startCommit(final SegmentInfos toSync) throws IOException {
assert testPoint("startStartCommit");
assert pendingCommit == null;
@@ -3939,10 +3955,6 @@ public class IndexWriter implements Clos
}
assert filesExist(toSync);
-
- if (commitUserData != null) {
- toSync.setUserData(commitUserData);
- }
}
assert testPoint("midStartCommit");
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java Fri Jan 18 18:30:54 2013
@@ -35,6 +35,7 @@ public final class KeepOnlyLastCommitDel
/**
* Deletes all commits except the most recent one.
*/
+ @Override
public void onInit(List<? extends IndexCommit> commits) {
// Note that commits.size() should normally be 1:
onCommit(commits);
@@ -43,6 +44,7 @@ public final class KeepOnlyLastCommitDel
/**
* Deletes all commits except the most recent one.
*/
+ @Override
public void onCommit(List<? extends IndexCommit> commits) {
// Note that commits.size() should normally be 2 (if not
// called by onInit above):
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java Fri Jan 18 18:30:54 2013
@@ -543,6 +543,7 @@ public abstract class LogMergePolicy ext
}
// Sorts largest to smallest
+ @Override
public int compareTo(SegmentInfoAndLevel other) {
if (level < other.level) {
return 1;
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java Fri Jan 18 18:30:54 2013
@@ -379,6 +379,7 @@ public abstract class MergePolicy implem
/**
* Release all resources for the policy.
*/
+ @Override
public abstract void close();
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java Fri Jan 18 18:30:54 2013
@@ -38,5 +38,6 @@ public abstract class MergeScheduler imp
public abstract void merge(IndexWriter writer) throws IOException;
/** Close this MergeScheduler. */
+ @Override
public abstract void close() throws IOException;
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiBits.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiBits.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiBits.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiBits.java Fri Jan 18 18:30:54 2013
@@ -48,6 +48,7 @@ final class MultiBits implements Bits {
return true;
}
+ @Override
public boolean get(int doc) {
final int reader = ReaderUtil.subIndex(doc, starts);
assert reader != -1;
@@ -109,6 +110,7 @@ final class MultiBits implements Bits {
return subResult;
}
+ @Override
public int length() {
return starts[starts.length-1];
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java Fri Jan 18 18:30:54 2013
@@ -47,10 +47,12 @@ class MultiDocValues extends DocValues {
private static DocValuesPuller DEFAULT_PULLER = new DocValuesPuller();
private static final DocValuesPuller NORMS_PULLER = new DocValuesPuller() {
+ @Override
public DocValues pull(AtomicReader reader, String field) throws IOException {
return reader.normValues(field);
}
+ @Override
public boolean stopLoadingOnNull(AtomicReader reader, String field) {
// for norms we drop all norms if one leaf reader has no norms and the field is present
FieldInfos fieldInfos = reader.getFieldInfos();
@@ -263,6 +265,7 @@ class MultiDocValues extends DocValues {
this.direct = direct;
}
+ @Override
public long getInt(int docID) {
final int doc = ensureSource(docID);
return current.getInt(doc);
@@ -293,11 +296,13 @@ class MultiDocValues extends DocValues {
}
}
+ @Override
public double getFloat(int docID) {
final int doc = ensureSource(docID);
return current.getFloat(doc);
}
+ @Override
public BytesRef getBytes(int docID, BytesRef bytesRef) {
final int doc = ensureSource(docID);
return current.getBytes(doc, bytesRef);
@@ -357,6 +362,7 @@ class MultiDocValues extends DocValues {
return docBases;
}
+ @Override
public boolean hasArray() {
boolean oneRealSource = false;
for (DocValuesSlice slice : slices) {
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java Fri Jan 18 18:30:54 2013
@@ -479,7 +479,7 @@ public final class MultiTermsEnum extend
subDocsAndPositions[upto].slice = entry.subSlice;
upto++;
} else {
- if (entry.terms.docs(b, null, 0) != null) {
+ if (entry.terms.docs(b, null, DocsEnum.FLAG_NONE) != null) {
// At least one of our subs does not store
// offsets or positions -- we can't correctly
// produce a MultiDocsAndPositions enum
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/NoDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/NoDeletionPolicy.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/NoDeletionPolicy.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/NoDeletionPolicy.java Fri Jan 18 18:30:54 2013
@@ -33,8 +33,10 @@ public final class NoDeletionPolicy impl
// keep private to avoid instantiation
}
+ @Override
public void onCommit(List<? extends IndexCommit> commits) {}
+ @Override
public void onInit(List<? extends IndexCommit> commits) {}
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Norm.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Norm.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Norm.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Norm.java Fri Jan 18 18:30:54 2013
@@ -115,15 +115,6 @@ public final class Norm {
setType(Type.FIXED_INTS_64);
this.field.setLongValue(norm);
}
-
- /**
- * Sets a packed long norm value.
- * @lucene.experimental
- */
- public void setPackedLong(long norm) {
- setType(Type.VAR_INTS);
- this.field.setLongValue(norm);
- }
/**
* Sets a byte norm value
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java Fri Jan 18 18:30:54 2013
@@ -43,6 +43,7 @@ class PrefixCodedTerms implements Iterab
}
/** @return iterator over the bytes */
+ @Override
public Iterator<Term> iterator() {
return new PrefixCodedTermsIterator();
}
@@ -61,10 +62,12 @@ class PrefixCodedTerms implements Iterab
}
}
+ @Override
public boolean hasNext() {
return input.getFilePointer() < input.length();
}
+ @Override
public Term next() {
assert hasNext();
try {
@@ -85,6 +88,7 @@ class PrefixCodedTerms implements Iterab
}
}
+ @Override
public void remove() {
throw new UnsupportedOperationException();
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java Fri Jan 18 18:30:54 2013
@@ -23,6 +23,7 @@ import java.util.concurrent.atomic.Atomi
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.TrackingDirectoryWrapper;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.MutableBits;
@@ -182,16 +183,28 @@ class ReadersAndLiveDocs {
// NOTE: removes callers ref
public synchronized void dropReaders() throws IOException {
- if (reader != null) {
- //System.out.println(" pool.drop info=" + info + " rc=" + reader.getRefCount());
- reader.decRef();
- reader = null;
- }
- if (mergeReader != null) {
- //System.out.println(" pool.drop info=" + info + " merge rc=" + mergeReader.getRefCount());
- mergeReader.decRef();
- mergeReader = null;
+ // TODO: can we somehow use IOUtils here...? problem is
+ // we are calling .decRef not .close)...
+ try {
+ if (reader != null) {
+ //System.out.println(" pool.drop info=" + info + " rc=" + reader.getRefCount());
+ try {
+ reader.decRef();
+ } finally {
+ reader = null;
+ }
+ }
+ } finally {
+ if (mergeReader != null) {
+ //System.out.println(" pool.drop info=" + info + " merge rc=" + mergeReader.getRefCount());
+ try {
+ mergeReader.decRef();
+ } finally {
+ mergeReader = null;
+ }
+ }
}
+
decRef();
}
@@ -272,13 +285,37 @@ class ReadersAndLiveDocs {
// We have new deletes
assert liveDocs.length() == info.info.getDocCount();
+ // Do this so we can delete any created files on
+ // exception; this saves all codecs from having to do
+ // it:
+ TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
+
// We can write directly to the actual name (vs to a
// .tmp & renaming it) because the file is not live
// until segments file is written:
- info.info.getCodec().liveDocsFormat().writeLiveDocs((MutableBits)liveDocs, dir, info, pendingDeleteCount, IOContext.DEFAULT);
+ boolean success = false;
+ try {
+ info.info.getCodec().liveDocsFormat().writeLiveDocs((MutableBits)liveDocs, trackingDir, info, pendingDeleteCount, IOContext.DEFAULT);
+ success = true;
+ } finally {
+ if (!success) {
+ // Advance only the nextWriteDelGen so that a 2nd
+ // attempt to write will write to a new file
+ info.advanceNextWriteDelGen();
+
+ // Delete any partially created file(s):
+ for(String fileName : trackingDir.getCreatedFiles()) {
+ try {
+ dir.deleteFile(fileName);
+ } catch (Throwable t) {
+ // Ignore so we throw only the first exc
+ }
+ }
+ }
+ }
// If we hit an exc in the line above (eg disk full)
- // then info remains pointing to the previous
+ // then info's delGen remains pointing to the previous
// (successfully written) del docs:
info.advanceDelGen();
info.setDelCount(info.getDelCount() + pendingDeleteCount);
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java Fri Jan 18 18:30:54 2013
@@ -40,6 +40,10 @@ public class SegmentInfoPerCommit {
// are no deletes yet):
private long delGen;
+ // Normally 1+delGen, unless an exception was hit on last
+ // attempt to write:
+ private long nextWriteDelGen;
+
private volatile long sizeInBytes = -1;
/** Sole constructor.
@@ -52,17 +56,27 @@ public class SegmentInfoPerCommit {
this.info = info;
this.delCount = delCount;
this.delGen = delGen;
- }
-
- void advanceDelGen() {
if (delGen == -1) {
- delGen = 1;
+ nextWriteDelGen = 1;
} else {
- delGen++;
+ nextWriteDelGen = delGen+1;
}
+ }
+
+ /** Called when we succeed in writing deletes */
+ void advanceDelGen() {
+ delGen = nextWriteDelGen;
+ nextWriteDelGen = delGen+1;
sizeInBytes = -1;
}
+ /** Called if there was an exception while writing
+ * deletes, so that we don't try to write to the same
+ * file more than once. */
+ void advanceNextWriteDelGen() {
+ nextWriteDelGen++;
+ }
+
/** Returns total size in bytes of all files for this
* segment. */
public long sizeInBytes() throws IOException {
@@ -126,11 +140,7 @@ public class SegmentInfoPerCommit {
* of the live docs file.
*/
public long getNextDelGen() {
- if (delGen == -1) {
- return 1;
- } else {
- return delGen + 1;
- }
+ return nextWriteDelGen;
}
/**
@@ -169,6 +179,12 @@ public class SegmentInfoPerCommit {
@Override
public SegmentInfoPerCommit clone() {
- return new SegmentInfoPerCommit(info, delCount, delGen);
+ SegmentInfoPerCommit other = new SegmentInfoPerCommit(info, delCount, delGen);
+ // Not clear that we need to carry over nextWriteDelGen
+ // (i.e. do we ever clone after a failed write and
+ // before the next successful write?), but just do it to
+ // be safe:
+ other.nextWriteDelGen = nextWriteDelGen;
+ return other;
}
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java Fri Jan 18 18:30:54 2013
@@ -103,8 +103,8 @@ import org.apache.lucene.util.ThreadInte
* <li>SegCodec is the {@link Codec#getName() name} of the Codec that encoded
* this segment.</li>
* <li>CommitUserData stores an optional user-supplied opaque
- * Map<String,String> that was passed to {@link IndexWriter#commit(java.util.Map)}
- * or {@link IndexWriter#prepareCommit(java.util.Map)}.</li>
+ * Map<String,String> that was passed to
+ * {@link IndexWriter#setCommitData(java.util.Map)}.</li>
* </ul>
* </p>
*
@@ -868,9 +868,6 @@ public final class SegmentInfos implemen
// Ignore; this file is only used in a retry
// fallback on init.
}
- if (t instanceof ThreadInterruptedException) {
- throw (ThreadInterruptedException) t;
- }
}
}
@@ -903,7 +900,7 @@ public final class SegmentInfos implemen
/** Return {@code userData} saved with this commit.
*
- * @see IndexWriter#commit(Map)
+ * @see IndexWriter#commit()
*/
public Map<String,String> getUserData() {
return userData;
@@ -991,6 +988,7 @@ public final class SegmentInfos implemen
/** Returns an <b>unmodifiable</b> {@link Iterator} of contained segments in order. */
// @Override (comment out until Java 6)
+ @Override
public Iterator<SegmentInfoPerCommit> iterator() {
return asList().iterator();
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java Fri Jan 18 18:30:54 2013
@@ -31,7 +31,6 @@ import org.apache.lucene.codecs.StoredFi
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
-import org.apache.lucene.util.Bits;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.InfoStream;
@@ -99,20 +98,57 @@ final class SegmentMerger {
mergeState.segmentInfo.setDocCount(setDocMaps());
mergeDocValuesAndNormsFieldInfos();
setMatchingSegmentReaders();
+ long t0 = 0;
+ if (mergeState.infoStream.isEnabled("SM")) {
+ t0 = System.nanoTime();
+ }
int numMerged = mergeFields();
+ if (mergeState.infoStream.isEnabled("SM")) {
+ long t1 = System.nanoTime();
+ mergeState.infoStream.message("SM", ((t1-t0)/1000000) + " msec to merge stored fields [" + numMerged + " docs]");
+ }
assert numMerged == mergeState.segmentInfo.getDocCount();
final SegmentWriteState segmentWriteState = new SegmentWriteState(mergeState.infoStream, directory, mergeState.segmentInfo,
mergeState.fieldInfos, termIndexInterval, null, context);
+ if (mergeState.infoStream.isEnabled("SM")) {
+ t0 = System.nanoTime();
+ }
mergeTerms(segmentWriteState);
+ if (mergeState.infoStream.isEnabled("SM")) {
+ long t1 = System.nanoTime();
+ mergeState.infoStream.message("SM", ((t1-t0)/1000000) + " msec to merge postings [" + numMerged + " docs]");
+ }
+
+ if (mergeState.infoStream.isEnabled("SM")) {
+ t0 = System.nanoTime();
+ }
mergePerDoc(segmentWriteState);
+ if (mergeState.infoStream.isEnabled("SM")) {
+ long t1 = System.nanoTime();
+ mergeState.infoStream.message("SM", ((t1-t0)/1000000) + " msec to merge doc values [" + numMerged + " docs]");
+ }
if (mergeState.fieldInfos.hasNorms()) {
+ if (mergeState.infoStream.isEnabled("SM")) {
+ t0 = System.nanoTime();
+ }
mergeNorms(segmentWriteState);
+ if (mergeState.infoStream.isEnabled("SM")) {
+ long t1 = System.nanoTime();
+ mergeState.infoStream.message("SM", ((t1-t0)/1000000) + " msec to merge norms [" + numMerged + " docs]");
+ }
}
if (mergeState.fieldInfos.hasVectors()) {
+ if (mergeState.infoStream.isEnabled("SM")) {
+ t0 = System.nanoTime();
+ }
numMerged = mergeVectors();
+ if (mergeState.infoStream.isEnabled("SM")) {
+ long t1 = System.nanoTime();
+ mergeState.infoStream.message("SM", ((t1-t0)/1000000) + " msec to merge vectors [" + numMerged + " docs]");
+ }
assert numMerged == mergeState.segmentInfo.getDocCount();
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SnapshotDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SnapshotDeletionPolicy.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SnapshotDeletionPolicy.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SnapshotDeletionPolicy.java Fri Jan 18 18:30:54 2013
@@ -261,12 +261,14 @@ public class SnapshotDeletionPolicy impl
return idToSnapshot.containsKey(id);
}
+ @Override
public synchronized void onCommit(List<? extends IndexCommit> commits)
throws IOException {
primary.onCommit(wrapCommits(commits));
lastCommit = commits.get(commits.size() - 1);
}
+ @Override
public synchronized void onInit(List<? extends IndexCommit> commits)
throws IOException {
primary.onInit(wrapCommits(commits));
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java Fri Jan 18 18:30:54 2013
@@ -105,6 +105,7 @@ final class StoredFieldsConsumer {
while(lastDocID < docID) {
fieldsWriter.startDocument(0);
lastDocID++;
+ fieldsWriter.finishDocument();
}
}
@@ -119,6 +120,7 @@ final class StoredFieldsConsumer {
for (int i = 0; i < numStoredFields; i++) {
fieldsWriter.writeField(fieldInfos[i], storedFields[i]);
}
+ fieldsWriter.finishDocument();
lastDocID++;
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Term.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Term.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Term.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Term.java Fri Jan 18 18:30:54 2013
@@ -110,6 +110,7 @@ public final class Term implements Compa
argument, and a positive integer if this term belongs after the argument.
The ordering of terms is first by field, then by text.*/
+ @Override
public final int compareTo(Term other) {
if (field.equals(other.field)) {
return bytes.compareTo(other.bytes);
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java Fri Jan 18 18:30:54 2013
@@ -78,6 +78,7 @@ final class TermVectorsConsumer extends
void fill(int docID) throws IOException {
while(lastDocID < docID) {
writer.startDocument(0);
+ writer.finishDocument();
lastDocID++;
}
}
@@ -108,6 +109,7 @@ final class TermVectorsConsumer extends
for (int i = 0; i < numVectorFields; i++) {
perFields[i].finishDocument();
}
+ writer.finishDocument();
assert lastDocID == docState.docID: "lastDocID=" + lastDocID + " docState.docID=" + docState.docID;
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java Fri Jan 18 18:30:54 2013
@@ -182,7 +182,9 @@ final class TermVectorsConsumerPerField
}
tv.addProx(freq, posReader, offReader);
}
+ tv.finishTerm();
}
+ tv.finishField();
termsHashPerField.reset();
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Terms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Terms.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Terms.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Terms.java Fri Jan 18 18:30:54 2013
@@ -80,7 +80,7 @@ public abstract class Terms {
* if there are no terms. This method may be invoked
* many times; it's best to cache a single instance &
* reuse it. */
- public abstract Comparator<BytesRef> getComparator() throws IOException;
+ public abstract Comparator<BytesRef> getComparator();
/** Returns the number of terms for this field, or -1 if this
* measure isn't stored by the codec. Note that, just like
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java Fri Jan 18 18:30:54 2013
@@ -269,6 +269,7 @@ public class TieredMergePolicy extends M
}
private class SegmentByteSizeDescending implements Comparator<SegmentInfoPerCommit> {
+ @Override
public int compare(SegmentInfoPerCommit o1, SegmentInfoPerCommit o2) {
try {
final long sz1 = size(o1);
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TwoPhaseCommit.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TwoPhaseCommit.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TwoPhaseCommit.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TwoPhaseCommit.java Fri Jan 18 18:30:54 2013
@@ -1,7 +1,6 @@
package org.apache.lucene.index;
import java.io.IOException;
-import java.util.Map;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -38,18 +37,6 @@ public interface TwoPhaseCommit {
public void prepareCommit() throws IOException;
/**
- * Like {@link #commit()}, but takes an additional commit data to be included
- * w/ the commit.
- * <p>
- * <b>NOTE:</b> some implementations may not support any custom data to be
- * included w/ the commit and may discard it altogether. Consult the actual
- * implementation documentation for verifying if this is supported.
- *
- * @see #prepareCommit()
- */
- public void prepareCommit(Map<String, String> commitData) throws IOException;
-
- /**
* The second phase of a 2-phase commit. Implementations should ideally do
* very little work in this method (following {@link #prepareCommit()}, and
* after it returns, the caller can assume that the changes were successfully
@@ -58,15 +45,6 @@ public interface TwoPhaseCommit {
public void commit() throws IOException;
/**
- * Like {@link #commit()}, but takes an additional commit data to be included
- * w/ the commit.
- *
- * @see #commit()
- * @see #prepareCommit(Map)
- */
- public void commit(Map<String, String> commitData) throws IOException;
-
- /**
* Discards any changes that have occurred since the last commit. In a 2-phase
* commit algorithm, where one of the objects failed to {@link #commit()} or
* {@link #prepareCommit()}, this method is used to roll all other objects
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TwoPhaseCommitTool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TwoPhaseCommitTool.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TwoPhaseCommitTool.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TwoPhaseCommitTool.java Fri Jan 18 18:30:54 2013
@@ -1,7 +1,6 @@
package org.apache.lucene.index;
import java.io.IOException;
-import java.util.Map;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -32,44 +31,6 @@ public final class TwoPhaseCommitTool {
private TwoPhaseCommitTool() {}
/**
- * A wrapper of a {@link TwoPhaseCommit}, which delegates all calls to the
- * wrapped object, passing the specified commitData. This object is useful for
- * use with {@link TwoPhaseCommitTool#execute(TwoPhaseCommit...)} if one would
- * like to store commitData as part of the commit.
- */
- public static final class TwoPhaseCommitWrapper implements TwoPhaseCommit {
-
- private final TwoPhaseCommit tpc;
- private final Map<String, String> commitData;
-
- /** Sole constructor. */
- public TwoPhaseCommitWrapper(TwoPhaseCommit tpc, Map<String, String> commitData) {
- this.tpc = tpc;
- this.commitData = commitData;
- }
-
- public void prepareCommit() throws IOException {
- prepareCommit(commitData);
- }
-
- public void prepareCommit(Map<String, String> commitData) throws IOException {
- tpc.prepareCommit(this.commitData);
- }
-
- public void commit() throws IOException {
- commit(commitData);
- }
-
- public void commit(Map<String, String> commitData) throws IOException {
- tpc.commit(this.commitData);
- }
-
- public void rollback() throws IOException {
- tpc.rollback();
- }
- }
-
- /**
* Thrown by {@link TwoPhaseCommitTool#execute(TwoPhaseCommit...)} when an
* object fails to prepareCommit().
*/
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java Fri Jan 18 18:30:54 2013
@@ -158,6 +158,7 @@ public class BooleanQuery extends Query
* make it possible to do:
* <pre class="prettyprint">for (BooleanClause clause : booleanQuery) {}</pre>
*/
+ @Override
public final Iterator<BooleanClause> iterator() { return clauses().iterator(); }
/**
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BoostAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BoostAttributeImpl.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BoostAttributeImpl.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BoostAttributeImpl.java Fri Jan 18 18:30:54 2013
@@ -25,10 +25,12 @@ import org.apache.lucene.util.AttributeI
public final class BoostAttributeImpl extends AttributeImpl implements BoostAttribute {
private float boost = 1.0f;
+ @Override
public void setBoost(float boost) {
this.boost = boost;
}
+ @Override
public float getBoost() {
return boost;
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java Fri Jan 18 18:30:54 2013
@@ -39,36 +39,12 @@ public class CachingWrapperFilter extend
// level of the readers hierarchy it should be cached.
private final Filter filter;
private final Map<Object,DocIdSet> cache = Collections.synchronizedMap(new WeakHashMap<Object,DocIdSet>());
- private final boolean recacheDeletes;
/** Wraps another filter's result and caches it.
- * Deletions are not cached and AND'd in on the fly, see
- * {@link #CachingWrapperFilter(Filter,boolean)} for an explanation.
- * This constructor is recommended for often changing indexes.
* @param filter Filter to cache results of
- * @see #CachingWrapperFilter(Filter,boolean)
*/
public CachingWrapperFilter(Filter filter) {
- this(filter, false);
- }
-
- /** Wraps another filter's result and caches it. If
- * {@code recacheDeletes} is {@code true}, then new deletes (for example
- * after {@link DirectoryReader#openIfChanged}) will cause the filter
- * {@link DocIdSet} to be recached.
- *
- * <p>If your index changes seldom, it is recommended to use {@code recacheDeletes=true},
- * as recaching will only occur when the index is reopened.
- * For near-real-time indexes or indexes that are often
- * reopened with (e.g., {@link DirectoryReader#openIfChanged} is used), you should
- * pass {@code recacheDeletes=false}. This will cache the filter results omitting
- * deletions and will AND them in while scoring.
- * @param filter Filter to cache results of
- * @param recacheDeletes if deletions on the underlying index should recache
- */
- public CachingWrapperFilter(Filter filter, boolean recacheDeletes) {
this.filter = filter;
- this.recacheDeletes = recacheDeletes;
}
/** Provide the DocIdSet to be cached, using the DocIdSet provided
@@ -104,54 +80,34 @@ public class CachingWrapperFilter extend
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context, final Bits acceptDocs) throws IOException {
final AtomicReader reader = context.reader();
-
- // Only cache if incoming acceptDocs is == live docs;
- // if Lucene passes in more interesting acceptDocs in
- // the future (@UweSays: it already does when you chain FilteredQuery) we don't want to over-cache:
- final Bits liveDocs = reader.getLiveDocs();
- final boolean doCacheAcceptDocs = (recacheDeletes && acceptDocs == liveDocs);
-
- final Object key;
- final Bits cacheAcceptDocs;
- if (doCacheAcceptDocs) {
- assert acceptDocs == liveDocs;
- key = reader.getCombinedCoreAndDeletesKey();
- cacheAcceptDocs = acceptDocs;
- } else {
- key = reader.getCoreCacheKey();
- cacheAcceptDocs = null;
- }
+ final Object key = reader.getCoreCacheKey();
DocIdSet docIdSet = cache.get(key);
if (docIdSet != null) {
hitCount++;
} else {
missCount++;
- docIdSet = docIdSetToCache(filter.getDocIdSet(context, cacheAcceptDocs), reader);
+ docIdSet = docIdSetToCache(filter.getDocIdSet(context, null), reader);
cache.put(key, docIdSet);
}
- if (doCacheAcceptDocs) {
- return docIdSet;
- } else {
- return BitsFilteredDocIdSet.wrap(docIdSet, acceptDocs);
- }
+ return BitsFilteredDocIdSet.wrap(docIdSet, acceptDocs);
}
@Override
public String toString() {
- return "CachingWrapperFilter("+filter+",recacheDeletes=" + recacheDeletes + ")";
+ return "CachingWrapperFilter("+filter+")";
}
@Override
public boolean equals(Object o) {
if (!(o instanceof CachingWrapperFilter)) return false;
final CachingWrapperFilter other = (CachingWrapperFilter) o;
- return this.filter.equals(other.filter) && this.recacheDeletes == other.recacheDeletes;
+ return this.filter.equals(other.filter);
}
@Override
public int hashCode() {
- return (filter.hashCode() ^ 0x1117BF25) + (recacheDeletes ? 0 : 1);
+ return (filter.hashCode() ^ 0x1117BF25);
}
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java Fri Jan 18 18:30:54 2013
@@ -60,6 +60,7 @@ class ConjunctionScorer extends Scorer {
// Also we use mergeSort here to be stable (so order of Scoreres that
// match on first document keeps preserved):
ArrayUtil.mergeSort(scorers, new Comparator<Scorer>() { // sort the array
+ @Override
public int compare(Scorer o1, Scorer o2) {
return o1.docID() - o2.docID();
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java Fri Jan 18 18:30:54 2013
@@ -47,6 +47,7 @@ class ConjunctionTermScorer extends Scor
// Sort the array the first time to allow the least frequent DocsEnum to
// lead the matching.
ArrayUtil.mergeSort(docsAndFreqs, new Comparator<DocsAndFreqs>() {
+ @Override
public int compare(DocsAndFreqs o1, DocsAndFreqs o2) {
return o1.docFreq - o2.docFreq;
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java Fri Jan 18 18:30:54 2013
@@ -26,6 +26,8 @@ import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ToStringUtils;
import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
import java.util.Set;
/**
@@ -265,6 +267,14 @@ public class ConstantScoreQuery extends
throw new UnsupportedOperationException("positions are only supported on Scorer subclasses");
}
}
+
+ @Override
+ public Collection<ChildScorer> getChildren() {
+ if (docIdSetIterator instanceof Scorer)
+ return Collections.singletonList(new ChildScorer((Scorer) docIdSetIterator, "constant"));
+ else
+ return Collections.emptyList();
+ }
}
@Override
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java Fri Jan 18 18:30:54 2013
@@ -85,6 +85,7 @@ public class DisjunctionMaxQuery extends
}
/** @return An {@code Iterator<Query>} over the disjuncts */
+ @Override
public Iterator<Query> iterator() {
return disjuncts.iterator();
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCache.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCache.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCache.java Fri Jan 18 18:30:54 2013
@@ -122,6 +122,7 @@ public interface FieldCache {
/** The default parser for byte values, which are encoded by {@link Byte#toString(byte)} */
public static final ByteParser DEFAULT_BYTE_PARSER = new ByteParser() {
+ @Override
public byte parseByte(BytesRef term) {
// TODO: would be far better to directly parse from
// UTF8 bytes... but really users should use
@@ -137,6 +138,7 @@ public interface FieldCache {
/** The default parser for short values, which are encoded by {@link Short#toString(short)} */
public static final ShortParser DEFAULT_SHORT_PARSER = new ShortParser() {
+ @Override
public short parseShort(BytesRef term) {
// TODO: would be far better to directly parse from
// UTF8 bytes... but really users should use
@@ -152,6 +154,7 @@ public interface FieldCache {
/** The default parser for int values, which are encoded by {@link Integer#toString(int)} */
public static final IntParser DEFAULT_INT_PARSER = new IntParser() {
+ @Override
public int parseInt(BytesRef term) {
// TODO: would be far better to directly parse from
// UTF8 bytes... but really users should use
@@ -167,6 +170,7 @@ public interface FieldCache {
/** The default parser for float values, which are encoded by {@link Float#toString(float)} */
public static final FloatParser DEFAULT_FLOAT_PARSER = new FloatParser() {
+ @Override
public float parseFloat(BytesRef term) {
// TODO: would be far better to directly parse from
// UTF8 bytes... but really users should use
@@ -182,6 +186,7 @@ public interface FieldCache {
/** The default parser for long values, which are encoded by {@link Long#toString(long)} */
public static final LongParser DEFAULT_LONG_PARSER = new LongParser() {
+ @Override
public long parseLong(BytesRef term) {
// TODO: would be far better to directly parse from
// UTF8 bytes... but really users should use
@@ -197,6 +202,7 @@ public interface FieldCache {
/** The default parser for double values, which are encoded by {@link Double#toString(double)} */
public static final DoubleParser DEFAULT_DOUBLE_PARSER = new DoubleParser() {
+ @Override
public double parseDouble(BytesRef term) {
// TODO: would be far better to directly parse from
// UTF8 bytes... but really users should use
@@ -215,6 +221,7 @@ public interface FieldCache {
* via {@link IntField}/{@link NumericTokenStream}.
*/
public static final IntParser NUMERIC_UTILS_INT_PARSER=new IntParser(){
+ @Override
public int parseInt(BytesRef term) {
if (NumericUtils.getPrefixCodedIntShift(term) > 0)
throw new StopFillCacheException();
@@ -231,6 +238,7 @@ public interface FieldCache {
* via {@link FloatField}/{@link NumericTokenStream}.
*/
public static final FloatParser NUMERIC_UTILS_FLOAT_PARSER=new FloatParser(){
+ @Override
public float parseFloat(BytesRef term) {
if (NumericUtils.getPrefixCodedIntShift(term) > 0)
throw new StopFillCacheException();
@@ -247,6 +255,7 @@ public interface FieldCache {
* via {@link LongField}/{@link NumericTokenStream}.
*/
public static final LongParser NUMERIC_UTILS_LONG_PARSER = new LongParser(){
+ @Override
public long parseLong(BytesRef term) {
if (NumericUtils.getPrefixCodedLongShift(term) > 0)
throw new StopFillCacheException();
@@ -263,6 +272,7 @@ public interface FieldCache {
* via {@link DoubleField}/{@link NumericTokenStream}.
*/
public static final DoubleParser NUMERIC_UTILS_DOUBLE_PARSER = new DoubleParser(){
+ @Override
public double parseDouble(BytesRef term) {
if (NumericUtils.getPrefixCodedLongShift(term) > 0)
throw new StopFillCacheException();
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCacheDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCacheDocIdSet.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCacheDocIdSet.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCacheDocIdSet.java Fri Jan 18 18:30:54 2013
@@ -57,18 +57,22 @@ public abstract class FieldCacheDocIdSet
@Override
public final Bits bits() {
return (acceptDocs == null) ? new Bits() {
+ @Override
public boolean get(int docid) {
return matchDoc(docid);
}
+ @Override
public int length() {
return maxDoc;
}
} : new Bits() {
+ @Override
public boolean get(int docid) {
return matchDoc(docid) && acceptDocs.get(docid);
}
+ @Override
public int length() {
return maxDoc;
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java Fri Jan 18 18:30:54 2013
@@ -71,16 +71,19 @@ class FieldCacheImpl implements FieldCac
caches.put(DocsWithFieldCache.class, new DocsWithFieldCache(this));
}
+ @Override
public synchronized void purgeAllCaches() {
init();
}
+ @Override
public synchronized void purge(AtomicReader r) {
for(Cache c : caches.values()) {
c.purge(r);
}
}
+ @Override
public synchronized CacheEntry[] getCacheEntries() {
List<CacheEntry> result = new ArrayList<CacheEntry>(17);
for(final Map.Entry<Class<?>,Cache> cacheEntry: caches.entrySet()) {
@@ -319,11 +322,13 @@ class FieldCacheImpl implements FieldCac
}
// inherit javadocs
+ @Override
public byte[] getBytes (AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
return getBytes(reader, field, null, setDocsWithField);
}
// inherit javadocs
+ @Override
public byte[] getBytes(AtomicReader reader, String field, ByteParser parser, boolean setDocsWithField)
throws IOException {
return (byte[]) caches.get(Byte.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
@@ -364,7 +369,7 @@ class FieldCacheImpl implements FieldCac
break;
}
final byte termval = parser.parseByte(term);
- docs = termsEnum.docs(null, docs, 0);
+ docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
while (true) {
final int docID = docs.nextDoc();
if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -391,11 +396,13 @@ class FieldCacheImpl implements FieldCac
}
// inherit javadocs
+ @Override
public short[] getShorts (AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
return getShorts(reader, field, null, setDocsWithField);
}
// inherit javadocs
+ @Override
public short[] getShorts(AtomicReader reader, String field, ShortParser parser, boolean setDocsWithField)
throws IOException {
return (short[]) caches.get(Short.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
@@ -437,7 +444,7 @@ class FieldCacheImpl implements FieldCac
break;
}
final short termval = parser.parseShort(term);
- docs = termsEnum.docs(null, docs, 0);
+ docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
while (true) {
final int docID = docs.nextDoc();
if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -485,11 +492,13 @@ class FieldCacheImpl implements FieldCac
}
// inherit javadocs
+ @Override
public int[] getInts (AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
return getInts(reader, field, null, setDocsWithField);
}
// inherit javadocs
+ @Override
public int[] getInts(AtomicReader reader, String field, IntParser parser, boolean setDocsWithField)
throws IOException {
return (int[]) caches.get(Integer.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
@@ -541,7 +550,7 @@ class FieldCacheImpl implements FieldCac
retArray = new int[maxDoc];
}
- docs = termsEnum.docs(null, docs, 0);
+ docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
while (true) {
final int docID = docs.nextDoc();
if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -572,6 +581,7 @@ class FieldCacheImpl implements FieldCac
}
}
+ @Override
public Bits getDocsWithField(AtomicReader reader, String field)
throws IOException {
return (Bits) caches.get(DocsWithFieldCache.class).get(reader, new Entry(field, null), false);
@@ -608,7 +618,7 @@ class FieldCacheImpl implements FieldCac
res = new FixedBitSet(maxDoc);
}
- docs = termsEnum.docs(null, docs, 0);
+ docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
// TODO: use bulk API
while (true) {
final int docID = docs.nextDoc();
@@ -633,12 +643,14 @@ class FieldCacheImpl implements FieldCac
}
// inherit javadocs
+ @Override
public float[] getFloats (AtomicReader reader, String field, boolean setDocsWithField)
throws IOException {
return getFloats(reader, field, null, setDocsWithField);
}
// inherit javadocs
+ @Override
public float[] getFloats(AtomicReader reader, String field, FloatParser parser, boolean setDocsWithField)
throws IOException {
@@ -691,7 +703,7 @@ class FieldCacheImpl implements FieldCac
retArray = new float[maxDoc];
}
- docs = termsEnum.docs(null, docs, 0);
+ docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
while (true) {
final int docID = docs.nextDoc();
if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -723,11 +735,13 @@ class FieldCacheImpl implements FieldCac
}
+ @Override
public long[] getLongs(AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
return getLongs(reader, field, null, setDocsWithField);
}
// inherit javadocs
+ @Override
public long[] getLongs(AtomicReader reader, String field, FieldCache.LongParser parser, boolean setDocsWithField)
throws IOException {
return (long[]) caches.get(Long.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
@@ -779,7 +793,7 @@ class FieldCacheImpl implements FieldCac
retArray = new long[maxDoc];
}
- docs = termsEnum.docs(null, docs, 0);
+ docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
while (true) {
final int docID = docs.nextDoc();
if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -811,12 +825,14 @@ class FieldCacheImpl implements FieldCac
}
// inherit javadocs
+ @Override
public double[] getDoubles(AtomicReader reader, String field, boolean setDocsWithField)
throws IOException {
return getDoubles(reader, field, null, setDocsWithField);
}
// inherit javadocs
+ @Override
public double[] getDoubles(AtomicReader reader, String field, FieldCache.DoubleParser parser, boolean setDocsWithField)
throws IOException {
return (double[]) caches.get(Double.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
@@ -868,7 +884,7 @@ class FieldCacheImpl implements FieldCac
retArray = new double[maxDoc];
}
- docs = termsEnum.docs(null, docs, 0);
+ docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
while (true) {
final int docID = docs.nextDoc();
if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -984,6 +1000,7 @@ class FieldCacheImpl implements FieldCac
}
}
+ @Override
public void seekExact(long ord) throws IOException {
assert(ord >= 0 && ord <= numOrd);
// TODO: if gap is small, could iterate from current position? Or let user decide that?
@@ -1071,10 +1088,12 @@ class FieldCacheImpl implements FieldCac
}
}
+ @Override
public DocTermsIndex getTermsIndex(AtomicReader reader, String field) throws IOException {
return getTermsIndex(reader, field, PackedInts.FAST);
}
+ @Override
public DocTermsIndex getTermsIndex(AtomicReader reader, String field, float acceptableOverheadRatio) throws IOException {
return (DocTermsIndex) caches.get(DocTermsIndex.class).get(reader, new Entry(field, acceptableOverheadRatio), false);
}
@@ -1161,7 +1180,7 @@ class FieldCacheImpl implements FieldCac
termOrdToBytesOffset = termOrdToBytesOffset.resize(ArrayUtil.oversize(1+termOrd, 1));
}
termOrdToBytesOffset.set(termOrd, bytes.copyUsingLengthPrefix(term));
- docs = termsEnum.docs(null, docs, 0);
+ docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
while (true) {
final int docID = docs.nextDoc();
if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -1210,10 +1229,12 @@ class FieldCacheImpl implements FieldCac
// TODO: this if DocTermsIndex was already created, we
// should share it...
+ @Override
public DocTerms getTerms(AtomicReader reader, String field) throws IOException {
return getTerms(reader, field, PackedInts.FAST);
}
+ @Override
public DocTerms getTerms(AtomicReader reader, String field, float acceptableOverheadRatio) throws IOException {
return (DocTerms) caches.get(DocTerms.class).get(reader, new Entry(field, acceptableOverheadRatio), false);
}
@@ -1277,7 +1298,7 @@ class FieldCacheImpl implements FieldCac
break;
}
final long pointer = bytes.copyUsingLengthPrefix(term);
- docs = termsEnum.docs(null, docs, 0);
+ docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
while (true) {
final int docID = docs.nextDoc();
if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -1293,6 +1314,7 @@ class FieldCacheImpl implements FieldCac
}
}
+ @Override
public DocTermOrds getDocTermOrds(AtomicReader reader, String field) throws IOException {
return (DocTermOrds) caches.get(DocTermOrds.class).get(reader, new Entry(field, null), false);
}
@@ -1311,10 +1333,12 @@ class FieldCacheImpl implements FieldCac
private volatile PrintStream infoStream;
+ @Override
public void setInfoStream(PrintStream stream) {
infoStream = stream;
}
+ @Override
public PrintStream getInfoStream() {
return infoStream;
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSet.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSet.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSet.java Fri Jan 18 18:30:54 2013
@@ -60,10 +60,12 @@ public abstract class FilteredDocIdSet e
public Bits bits() throws IOException {
final Bits bits = _innerSet.bits();
return (bits == null) ? null : new Bits() {
+ @Override
public boolean get(int docid) {
return bits.get(docid) && FilteredDocIdSet.this.match(docid);
}
+ @Override
public int length() {
return bits.length();
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FuzzyQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FuzzyQuery.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FuzzyQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FuzzyQuery.java Fri Jan 18 18:30:54 2013
@@ -96,7 +96,7 @@ public class FuzzyQuery extends MultiTer
/**
* Calls {@link #FuzzyQuery(Term, int, int, int, boolean)
- * FuzzyQuery(term, minimumSimilarity, prefixLength, defaultMaxExpansions, defaultTranspositions)}.
+ * FuzzyQuery(term, maxEdits, prefixLength, defaultMaxExpansions, defaultTranspositions)}.
*/
public FuzzyQuery(Term term, int maxEdits, int prefixLength) {
this(term, maxEdits, prefixLength, defaultMaxExpansions, defaultTranspositions);
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Fri Jan 18 18:30:54 2013
@@ -412,6 +412,7 @@ public class FuzzyTermsEnum extends Term
public static final class LevenshteinAutomataAttributeImpl extends AttributeImpl implements LevenshteinAutomataAttribute {
private final List<CompiledAutomaton> automata = new ArrayList<CompiledAutomaton>();
+ @Override
public List<CompiledAutomaton> automata() {
return automata;
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java Fri Jan 18 18:30:54 2013
@@ -182,18 +182,27 @@ public class IndexSearcher {
return reader;
}
- /** Sugar for <code>.getIndexReader().document(docID)</code> */
+ /**
+ * Sugar for <code>.getIndexReader().document(docID)</code>
+ * @see IndexReader#document(int)
+ */
public StoredDocument doc(int docID) throws IOException {
return reader.document(docID);
}
- /** Sugar for <code>.getIndexReader().document(docID, fieldVisitor)</code> */
+ /**
+ * Sugar for <code>.getIndexReader().document(docID, fieldVisitor)</code>
+ * @see IndexReader#document(int, StoredFieldVisitor)
+ */
public void doc(int docID, StoredFieldVisitor fieldVisitor) throws IOException {
reader.document(docID, fieldVisitor);
}
- /** Sugar for <code>.getIndexReader().document(docID, fieldsToLoad)</code> */
- public final StoredDocument document(int docID, Set<String> fieldsToLoad) throws IOException {
+ /**
+ * Sugar for <code>.getIndexReader().document(docID, fieldsToLoad)</code>
+ * @see IndexReader#document(int, Set)
+ */
+ public StoredDocument doc(int docID, Set<String> fieldsToLoad) throws IOException {
return reader.document(docID, fieldsToLoad);
}
@@ -689,6 +698,7 @@ public class IndexSearcher {
this.slice = slice;
}
+ @Override
public TopDocs call() throws IOException {
final TopDocs docs = searcher.search(Arrays.asList(slice.leaves), weight, after, nDocs);
final ScoreDoc[] scoreDocs = docs.scoreDocs;
@@ -781,6 +791,7 @@ public class IndexSearcher {
private final FakeScorer fakeScorer = new FakeScorer();
+ @Override
public TopFieldDocs call() throws IOException {
assert slice.leaves.length == 1;
final TopFieldDocs docs = searcher.search(Arrays.asList(slice.leaves),
@@ -823,6 +834,7 @@ public class IndexSearcher {
this.service = new ExecutorCompletionService<T>(executor);
}
+ @Override
public boolean hasNext() {
return numTasks > 0;
}
@@ -832,6 +844,7 @@ public class IndexSearcher {
++numTasks;
}
+ @Override
public T next() {
if(!this.hasNext())
throw new NoSuchElementException("next() is called but hasNext() returned false");
@@ -846,10 +859,12 @@ public class IndexSearcher {
}
}
+ @Override
public void remove() {
throw new UnsupportedOperationException();
}
+ @Override
public Iterator<T> iterator() {
// use the shortcut here - this is only used in a private context
return this;
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttributeImpl.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttributeImpl.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttributeImpl.java Fri Jan 18 18:30:54 2013
@@ -27,18 +27,22 @@ public final class MaxNonCompetitiveBoos
private float maxNonCompetitiveBoost = Float.NEGATIVE_INFINITY;
private BytesRef competitiveTerm = null;
+ @Override
public void setMaxNonCompetitiveBoost(final float maxNonCompetitiveBoost) {
this.maxNonCompetitiveBoost = maxNonCompetitiveBoost;
}
+ @Override
public float getMaxNonCompetitiveBoost() {
return maxNonCompetitiveBoost;
}
+ @Override
public void setCompetitiveTerm(final BytesRef competitiveTerm) {
this.competitiveTerm = competitiveTerm;
}
+ @Override
public BytesRef getCompetitiveTerm() {
return competitiveTerm;
}
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java Fri Jan 18 18:30:54 2013
@@ -28,6 +28,25 @@ import org.apache.lucene.util.PriorityQu
import java.io.IOException;
import java.util.*;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermContext;
+import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.similarities.Similarity.SloppySimScorer;
+import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.PriorityQueue;
+import org.apache.lucene.util.ToStringUtils;
+
/**
* MultiPhraseQuery is a generalized version of PhraseQuery, with an added
* method {@link #add(Term[])}.
@@ -215,11 +234,11 @@ public class MultiPhraseQuery extends Qu
return null;
}
termsEnum.seekExact(term.bytes(), termState);
- postingsEnum = termsEnum.docsAndPositions(liveDocs, null, 0);
+ postingsEnum = termsEnum.docsAndPositions(liveDocs, null, DocsEnum.FLAG_NONE);
if (postingsEnum == null) {
// term does exist, but has no positions
- assert termsEnum.docs(liveDocs, null, 0) != null: "termstate found but no term exists in reader";
+ assert termsEnum.docs(liveDocs, null, DocsEnum.FLAG_NONE) != null: "termstate found but no term exists in reader";
throw new IllegalStateException("field \"" + term.field() + "\" was indexed without position data; cannot run PhraseQuery (term=" + term.text() + ")");
}
@@ -548,7 +567,7 @@ class UnionDocsAndPositionsEnum extends
continue;
}
termsEnum.seekExact(term.bytes(), termState);
- DocsAndPositionsEnum postings = termsEnum.docsAndPositions(liveDocs, null, flags.docsAndPositionsFlags());
+ DocsAndPositionsEnum postings = termsEnum.docsAndPositions(liveDocs, null, DocsEnum.FLAG_NONE);
if (postings == null) {
// term does exist, but has no positions
throw new IllegalStateException("field \"" + term.field() + "\" was indexed without position data; cannot run PhraseQuery (term=" + term.text() + ")");
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java Fri Jan 18 18:30:54 2013
@@ -106,7 +106,7 @@ public class MultiTermQueryWrapperFilter
do {
// System.out.println(" iter termCount=" + termCount + " term=" +
// enumerator.term().toBytesString());
- docsEnum = termsEnum.docs(acceptDocs, docsEnum, 0);
+ docsEnum = termsEnum.docs(acceptDocs, docsEnum, DocsEnum.FLAG_NONE);
int docid;
while ((docid = docsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
bitSet.set(docid);