Skip to content

Commit

Permalink
Merge commit '67ba6b24233bab9839b97fd8de58df64d14176e9' into trunk
Browse files Browse the repository at this point in the history
  • Loading branch information
markrmiller committed Nov 29, 2010
2 parents c64971a + 67ba6b2 commit dd136b9
Show file tree
Hide file tree
Showing 25 changed files with 194 additions and 62 deletions.
3 changes: 3 additions & 0 deletions lucene/CHANGES.txt
Expand Up @@ -161,6 +161,9 @@ API Changes
* LUCENE-2566: QueryParser: Unary operators +,-,! will not be treated as
operators if they are followed by whitespace. (yonik)

* LUCENE-2778: RAMDirectory now exposes newRAMFile() which allows to override
and return a different RAMFile implementation. (Shai Erera)

New features

* LUCENE-2604: Added RegexpQuery support to QueryParser. Regular expressions
Expand Down
39 changes: 27 additions & 12 deletions lucene/src/java/org/apache/lucene/index/IndexWriter.java
Expand Up @@ -1776,7 +1776,7 @@ private void closeInternal(boolean waitForMerges) throws CorruptIndexException,
message("now call final commit()");

if (!hitOOM) {
commit(0);
commitInternal(null);
}

if (infoStream != null)
Expand Down Expand Up @@ -3133,19 +3133,12 @@ public final void prepareCommit(Map<String,String> commitUserData) throws Corrup

flush(true, true, true);

startCommit(0, commitUserData);
startCommit(commitUserData);
}

// Used only by commit, below; lock order is commitLock -> IW
private final Object commitLock = new Object();

private void commit(long sizeInBytes) throws IOException {
synchronized(commitLock) {
startCommit(sizeInBytes, null);
finishCommit();
}
}

/**
* <p>Commits all pending changes (added & deleted
* documents, optimizations, segment merges, added
Expand Down Expand Up @@ -3193,6 +3186,11 @@ public final void commit(Map<String,String> commitUserData) throws CorruptIndexE

ensureOpen();

commitInternal(commitUserData);
}

private final void commitInternal(Map<String,String> commitUserData) throws CorruptIndexException, IOException {

if (infoStream != null) {
message("commit: start");
}
Expand Down Expand Up @@ -4367,7 +4365,7 @@ private boolean filesExist(SegmentInfos toSync) throws IOException {
* 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(long sizeInBytes, Map<String,String> commitUserData) throws IOException {
private void startCommit(Map<String,String> commitUserData) throws IOException {

assert testPoint("startStartCommit");
assert pendingCommit == null;
Expand All @@ -4379,14 +4377,15 @@ private void startCommit(long sizeInBytes, Map<String,String> commitUserData) th
try {

if (infoStream != null)
message("startCommit(): start sizeInBytes=" + sizeInBytes);
message("startCommit(): start");

final SegmentInfos toSync;
final long myChangeCount;

synchronized(this) {

assert lastCommitChangeCount <= changeCount;
myChangeCount = changeCount;

if (changeCount == lastCommitChangeCount) {
if (infoStream != null)
Expand All @@ -4403,7 +4402,24 @@ private void startCommit(long sizeInBytes, Map<String,String> commitUserData) th

readerPool.commit();

// It's possible another flush (that did not close
// the open do stores) snuck in after the flush we
// just did, so we remove any tail segments
// referencing the open doc store from the
// SegmentInfos we are about to sync (the main
// SegmentInfos will keep them):
toSync = (SegmentInfos) segmentInfos.clone();
final String dss = docWriter.getDocStoreSegment();
if (dss != null) {
while(true) {
final String dss2 = toSync.info(toSync.size()-1).getDocStoreSegment();
if (dss2 == null || !dss2.equals(dss)) {
break;
}
toSync.remove(toSync.size()-1);
changeCount++;
}
}
assert filesExist(toSync);

if (commitUserData != null)
Expand All @@ -4415,7 +4431,6 @@ private void startCommit(long sizeInBytes, Map<String,String> commitUserData) th
// merge completes which would otherwise have
// removed the files we are now syncing.
deleter.incRef(toSync, false);
myChangeCount = changeCount;
}

assert testPoint("midStartCommit");
Expand Down
4 changes: 2 additions & 2 deletions lucene/src/java/org/apache/lucene/index/LogMergePolicy.java
Expand Up @@ -56,7 +56,7 @@ public abstract class LogMergePolicy extends MergePolicy {

/** Default noCFSRatio. If a merge's size is >= 10% of
* the index, then we disable compound file for it.
* @see setNoCFSRatio */
* @see #setNoCFSRatio */
public static final double DEFAULT_NO_CFS_RATIO = 0.1;

protected int mergeFactor = DEFAULT_MERGE_FACTOR;
Expand All @@ -81,7 +81,7 @@ protected boolean verbose() {
return w != null && w.verbose();
}

/** @see setNoCFSRatio */
/** @see #setNoCFSRatio */
public double getNoCFSRatio() {
return noCFSRatio;
}
Expand Down
3 changes: 2 additions & 1 deletion lucene/src/java/org/apache/lucene/index/SegmentCodecs.java
Expand Up @@ -18,6 +18,7 @@
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.IdentityHashMap;
import java.util.Map;
import java.util.Set;
Expand Down Expand Up @@ -128,6 +129,6 @@ void files(Directory dir, SegmentInfo info, Set<String> files)

@Override
public String toString() {
return "CodecInfo [codecs=" + codecs + ", provider=" + provider + "]";
return "SegmentCodecs [codecs=" + Arrays.toString(codecs) + ", provider=" + provider + "]";
}
}
4 changes: 2 additions & 2 deletions lucene/src/java/org/apache/lucene/index/SegmentInfos.java
Expand Up @@ -78,7 +78,7 @@ public final class SegmentInfos extends Vector<SegmentInfo> {
* If non-null, information about loading segments_N files
* will be printed here. @see #setInfoStream.
*/
private static PrintStream infoStream;
private static PrintStream infoStream = null;

public SegmentInfos() {
this(CodecProvider.getDefault());
Expand Down Expand Up @@ -621,7 +621,7 @@ public Object run(IndexCommit commit) throws CorruptIndexException, IOException

try {
Object v = doBody(segmentFileName);
if (exc != null && infoStream != null) {
if (infoStream != null) {
message("success on " + segmentFileName);
}
return v;
Expand Down
Expand Up @@ -104,9 +104,10 @@ class TermVectorsReader implements Cloneable {
assert numTotalDocs >= size + docStoreOffset: "numTotalDocs=" + numTotalDocs + " size=" + size + " docStoreOffset=" + docStoreOffset;
}
} else {
// TODO: understand why FieldInfos.hasVectors() can
// return true yet the term vectors files don't
// exist...
// If all documents flushed in a segment had hit
// non-aborting exceptions, it's possible that
// FieldInfos.hasVectors returns true yet the term
// vector files don't exist.
format = 0;
}

Expand Down
Expand Up @@ -50,12 +50,13 @@ public TermsHashConsumerPerThread addThread(TermsHashPerThread termsHashPerThrea
@Override
synchronized void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {

if (state.numDocsInStore > 0) {
// It's possible that all documents seen in this segment
// hit non-aborting exceptions, in which case we will
// not have yet init'd the TermVectorsWriter:
initTermVectorsWriter();
}
// NOTE: it's possible that all documents seen in this segment
// hit non-aborting exceptions, in which case we will
// not have yet init'd the TermVectorsWriter. This is
// actually OK (unlike in the stored fields case)
// because, although IieldInfos.hasVectors() will return
// true, the TermVectorsReader gracefully handles
// non-existence of the term vectors files.

if (tvx != null) {

Expand Down
Expand Up @@ -358,7 +358,7 @@ public boolean nextIndexTerm(long ord, TermsIndexResult result) throws IOExcepti
private void fillResult(int idx, TermsIndexResult result) {
final long offset = termOffsets.get(idx);
final int length = (int) (termOffsets.get(1+idx) - offset);
termBytesReader.fill(result.term, termBytesStart + offset, length);
termBytesReader.fillSlice(result.term, termBytesStart + offset, length);
result.position = idx * totalIndexInterval;
result.offset = termsStart + termsDictOffsets.get(idx);
}
Expand All @@ -373,7 +373,7 @@ public void getIndexOffset(BytesRef term, TermsIndexResult result) throws IOExce

final long offset = termOffsets.get(mid);
final int length = (int) (termOffsets.get(1+mid) - offset);
termBytesReader.fill(result.term, termBytesStart + offset, length);
termBytesReader.fillSlice(result.term, termBytesStart + offset, length);

int delta = termComp.compare(term, result.term);
if (delta < 0) {
Expand All @@ -394,7 +394,7 @@ public void getIndexOffset(BytesRef term, TermsIndexResult result) throws IOExce

final long offset = termOffsets.get(hi);
final int length = (int) (termOffsets.get(1+hi) - offset);
termBytesReader.fill(result.term, termBytesStart + offset, length);
termBytesReader.fillSlice(result.term, termBytesStart + offset, length);

result.position = hi*totalIndexInterval;
result.offset = termsStart + termsDictOffsets.get(hi);
Expand Down
Expand Up @@ -165,7 +165,7 @@ public boolean exists(int docID) {
@Override
public BytesRef getTerm(int docID, BytesRef ret) {
final long pointer = docToOffset.get(docID);
return bytes.fillUsingLengthPrefix(ret, pointer);
return bytes.fill(ret, pointer);
}
}
}
Expand Up @@ -213,7 +213,7 @@ public int size() {

@Override
public BytesRef lookup(int ord, BytesRef ret) {
return bytes.fillUsingLengthPrefix(ret, termOrdToBytesOffset.get(ord));
return bytes.fill(ret, termOrdToBytesOffset.get(ord));
}

@Override
Expand All @@ -235,7 +235,7 @@ public DocTermsIndexEnum() {
currentBlockNumber = 0;
blocks = bytes.getBlocks();
blockEnds = bytes.getBlockEnds();
currentBlockNumber = bytes.fillUsingLengthPrefix2(term, termOrdToBytesOffset.get(0));
currentBlockNumber = bytes.fillAndGetIndex(term, termOrdToBytesOffset.get(0));
end = blockEnds[currentBlockNumber];
}

Expand All @@ -249,7 +249,7 @@ public SeekStatus seek(BytesRef text, boolean useCache) throws IOException {
public SeekStatus seek(long ord) throws IOException {
assert(ord >= 0 && ord <= numOrd);
// TODO: if gap is small, could iterate from current position? Or let user decide that?
currentBlockNumber = bytes.fillUsingLengthPrefix2(term, termOrdToBytesOffset.get((int)ord));
currentBlockNumber = bytes.fillAndGetIndex(term, termOrdToBytesOffset.get((int)ord));
end = blockEnds[currentBlockNumber];
currentOrd = (int)ord;
return SeekStatus.FOUND;
Expand Down
18 changes: 13 additions & 5 deletions lucene/src/java/org/apache/lucene/store/RAMDirectory.java
Expand Up @@ -35,8 +35,8 @@ public class RAMDirectory extends Directory implements Serializable {

private static final long serialVersionUID = 1l;

HashMap<String,RAMFile> fileMap = new HashMap<String,RAMFile>();
final AtomicLong sizeInBytes = new AtomicLong();
protected HashMap<String,RAMFile> fileMap = new HashMap<String,RAMFile>();
protected final AtomicLong sizeInBytes = new AtomicLong();

// *****
// Lock acquisition sequence: RAMDirectory, then RAMFile
Expand Down Expand Up @@ -169,9 +169,8 @@ public synchronized final long sizeInBytes() {
@Override
public synchronized void deleteFile(String name) throws IOException {
ensureOpen();
RAMFile file = fileMap.get(name);
RAMFile file = fileMap.remove(name);
if (file!=null) {
fileMap.remove(name);
file.directory = null;
sizeInBytes.addAndGet(-file.sizeInBytes);
} else
Expand All @@ -182,7 +181,7 @@ public synchronized void deleteFile(String name) throws IOException {
@Override
public IndexOutput createOutput(String name) throws IOException {
ensureOpen();
RAMFile file = new RAMFile(this);
RAMFile file = newRAMFile();
synchronized (this) {
RAMFile existing = fileMap.get(name);
if (existing!=null) {
Expand All @@ -194,6 +193,15 @@ public IndexOutput createOutput(String name) throws IOException {
return new RAMOutputStream(file);
}

/**
* Returns a new {@link RAMFile} for storing data. This method can be
* overridden to return different {@link RAMFile} impls, that e.g. override
* {@link RAMFile#newBuffer(int)}.
*/
protected RAMFile newRAMFile() {
return new RAMFile(this);
}

/** Returns a stream reading an existing file. */
@Override
public IndexInput openInput(String name) throws IOException {
Expand Down

0 comments on commit dd136b9

Please sign in to comment.