You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2010/12/20 11:53:33 UTC
svn commit: r1051056 [4/8] - in /lucene/dev/branches/docvalues: ./ lucene/
lucene/contrib/ant/src/java/org/apache/lucene/ant/
lucene/contrib/ant/src/test/org/apache/lucene/ant/
lucene/contrib/benchmark/src/test/org/apache/lucene/benchmark/quality/
luce...
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Mon Dec 20 10:53:27 2010
@@ -32,7 +32,6 @@ import org.apache.lucene.index.codecs.Co
import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.MergeState;
import org.apache.lucene.index.codecs.FieldsConsumer;
-import org.apache.lucene.index.values.Type;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
@@ -58,18 +57,12 @@ final class SegmentMerger {
private int termIndexInterval = IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL;
private List<IndexReader> readers = new ArrayList<IndexReader>();
- private FieldInfos fieldInfos;
+ private final FieldInfos fieldInfos;
private int mergedDocs;
private final CheckAbort checkAbort;
- // Whether we should merge doc stores (stored fields and
- // vectors files). When all segments we are merging
- // already share the same doc store files, we don't need
- // to merge the doc stores.
- private boolean mergeDocStores;
-
/** Maximum number of contiguous documents to bulk-copy
when merging stored fields */
private final static int MAX_RAW_MERGE_DOCS = 4192;
@@ -80,10 +73,11 @@ final class SegmentMerger {
private PayloadProcessorProvider payloadProcessorProvider;
- SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, CodecProvider codecs, PayloadProcessorProvider payloadProcessorProvider) {
+ SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, CodecProvider codecs, PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos) {
this.payloadProcessorProvider = payloadProcessorProvider;
directory = dir;
this.codecs = codecs;
+ this.fieldInfos = fieldInfos;
segment = name;
if (merge != null) {
checkAbort = new CheckAbort(merge, directory);
@@ -97,9 +91,9 @@ final class SegmentMerger {
}
this.termIndexInterval = termIndexInterval;
}
-
- boolean hasProx() {
- return fieldInfos.hasProx();
+
+ public FieldInfos fieldInfos() {
+ return fieldInfos;
}
/**
@@ -117,22 +111,6 @@ final class SegmentMerger {
* @throws IOException if there is a low-level IO error
*/
final int merge() throws CorruptIndexException, IOException {
- return merge(true);
- }
-
- /**
- * Merges the readers specified by the {@link #add} method
- * into the directory passed to the constructor.
- * @param mergeDocStores if false, we will not merge the
- * stored fields nor vectors files
- * @return The number of documents that were merged
- * @throws CorruptIndexException if the index is corrupt
- * @throws IOException if there is a low-level IO error
- */
- final int merge(boolean mergeDocStores) throws CorruptIndexException, IOException {
-
- this.mergeDocStores = mergeDocStores;
-
// NOTE: it's important to add calls to
// checkAbort.work(...) if you make any changes to this
// method that will spend alot of time. The frequency
@@ -144,7 +122,7 @@ final class SegmentMerger {
mergeTerms();
mergeNorms();
- if (mergeDocStores && fieldInfos.hasVectors())
+ if (fieldInfos.hasVectors())
mergeVectors();
return mergedDocs;
}
@@ -154,9 +132,7 @@ final class SegmentMerger {
// Basic files
for (String ext : IndexFileNames.COMPOUND_EXTENSIONS_NOT_CODEC) {
- if (mergeDocStores || (!ext.equals(IndexFileNames.FIELDS_EXTENSION) &&
- !ext.equals(IndexFileNames.FIELDS_INDEX_EXTENSION)))
- fileSet.add(IndexFileNames.segmentFileName(segment, "", ext));
+ fileSet.add(IndexFileNames.segmentFileName(segment, "", ext));
}
segmentWriteState.segmentCodecs.files(directory, info, fileSet);
@@ -171,7 +147,7 @@ final class SegmentMerger {
}
// Vector files
- if (fieldInfos.hasVectors() && mergeDocStores) {
+ if (fieldInfos.hasVectors()) {
for (String ext : IndexFileNames.VECTOR_EXTENSIONS) {
fileSet.add(IndexFileNames.segmentFileName(segment, "", ext));
}
@@ -204,13 +180,18 @@ final class SegmentMerger {
for (String field : names) {
fInfos.add(field, true, storeTermVectors,
storePositionWithTermVector, storeOffsetWithTermVector, !reader
- .hasNorms(field), storePayloads, omitTFAndPositions);
+ .hasNorms(field), storePayloads, omitTFAndPositions, null);
}
}
private SegmentReader[] matchingSegmentReaders;
private int[] rawDocLengths;
private int[] rawDocLengths2;
+ private int matchedCount;
+
+ public int getMatchedSubReaderCount() {
+ return matchedCount;
+ }
private void setMatchingSegmentReaders() {
// If the i'th reader is a SegmentReader and has
@@ -235,6 +216,7 @@ final class SegmentMerger {
}
if (same) {
matchingSegmentReaders[i] = segmentReader;
+ matchedCount++;
}
}
}
@@ -250,18 +232,7 @@ final class SegmentMerger {
* @throws CorruptIndexException if the index is corrupt
* @throws IOException if there is a low-level IO error
*/
- private final int mergeFields() throws CorruptIndexException, IOException {
-
- if (!mergeDocStores) {
- // When we are not merging by doc stores, their field
- // name -> number mapping are the same. So, we start
- // with the fieldInfos of the last segment in this
- // case, to keep that numbering.
- final SegmentReader sr = (SegmentReader) readers.get(readers.size()-1);
- fieldInfos = (FieldInfos) sr.core.fieldInfos.clone();
- } else {
- fieldInfos = new FieldInfos();// merge field names
- }
+ private int mergeFields() throws CorruptIndexException, IOException {
for (IndexReader reader : readers) {
if (reader instanceof SegmentReader) {
@@ -269,19 +240,7 @@ final class SegmentMerger {
FieldInfos readerFieldInfos = segmentReader.fieldInfos();
int numReaderFieldInfos = readerFieldInfos.size();
for (int j = 0; j < numReaderFieldInfos; j++) {
- FieldInfo fi = readerFieldInfos.fieldInfo(j);
- FieldInfo merged = fieldInfos.add(fi.name, fi.isIndexed, fi.storeTermVector,
- fi.storePositionWithTermVector, fi.storeOffsetWithTermVector,
- !reader.hasNorms(fi.name), fi.storePayloads,
- fi.omitTermFreqAndPositions);
- final Type fiIndexValues = fi.docValues;
- final Type mergedDocValues = merged.docValues;
- if (mergedDocValues == null) {
- merged.setDocValues(fiIndexValues);
- } else if (mergedDocValues != fiIndexValues) {
- // TODO -- can we recover from this?
- throw new IllegalStateException("cannot merge field " + fi.name + " indexValues changed from " + mergedDocValues + " to " + fiIndexValues);
- }
+ fieldInfos.add(readerFieldInfos.fieldInfo(j));
}
} else {
addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION_OFFSET), true, true, true, false, false);
@@ -302,54 +261,43 @@ final class SegmentMerger {
setMatchingSegmentReaders();
- if (mergeDocStores) {
- // merge field values
- final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, fieldInfos);
-
- try {
- int idx = 0;
- for (IndexReader reader : readers) {
- final SegmentReader matchingSegmentReader = matchingSegmentReaders[idx++];
- FieldsReader matchingFieldsReader = null;
- if (matchingSegmentReader != null) {
- final FieldsReader fieldsReader = matchingSegmentReader.getFieldsReader();
- if (fieldsReader != null) {
- matchingFieldsReader = fieldsReader;
- }
+ final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, fieldInfos);
+
+ try {
+ int idx = 0;
+ for (IndexReader reader : readers) {
+ final SegmentReader matchingSegmentReader = matchingSegmentReaders[idx++];
+ FieldsReader matchingFieldsReader = null;
+ if (matchingSegmentReader != null) {
+ final FieldsReader fieldsReader = matchingSegmentReader.getFieldsReader();
+ if (fieldsReader != null) {
+ matchingFieldsReader = fieldsReader;
}
- if (reader.hasDeletions()) {
- docCount += copyFieldsWithDeletions(fieldsWriter,
- reader, matchingFieldsReader);
- } else {
- docCount += copyFieldsNoDeletions(fieldsWriter,
+ }
+ if (reader.hasDeletions()) {
+ docCount += copyFieldsWithDeletions(fieldsWriter,
reader, matchingFieldsReader);
- }
+ } else {
+ docCount += copyFieldsNoDeletions(fieldsWriter,
+ reader, matchingFieldsReader);
}
- } finally {
- fieldsWriter.close();
}
+ } finally {
+ fieldsWriter.close();
+ }
- final String fileName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
- final long fdxFileLength = directory.fileLength(fileName);
-
- if (4+((long) docCount)*8 != fdxFileLength)
- // This is most likely a bug in Sun JRE 1.6.0_04/_05;
- // we detect that the bug has struck, here, and
- // throw an exception to prevent the corruption from
- // entering the index. See LUCENE-1282 for
- // details.
- throw new RuntimeException("mergeFields produced an invalid result: docCount is " + docCount + " but fdx file size is " + fdxFileLength + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
+ final String fileName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
+ final long fdxFileLength = directory.fileLength(fileName);
- } else {
- // If we are skipping the doc stores, that means there
- // are no deletions in any of these segments, so we
- // just sum numDocs() of each segment to get total docCount
- for (final IndexReader reader : readers) {
- docCount += reader.numDocs();
- }
- }
+ if (4+((long) docCount)*8 != fdxFileLength)
+ // This is most likely a bug in Sun JRE 1.6.0_04/_05;
+ // we detect that the bug has struck, here, and
+ // throw an exception to prevent the corruption from
+ // entering the index. See LUCENE-1282 for
+ // details.
+ throw new RuntimeException("mergeFields produced an invalid result: docCount is " + docCount + " but fdx file size is " + fdxFileLength + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
- segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, null, docCount, 0, termIndexInterval, codecInfo, new AtomicLong(0));
+ segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codecInfo, new AtomicLong(0));
return docCount;
}
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentReader.java Mon Dec 20 10:53:27 2010
@@ -248,7 +248,7 @@ public class SegmentReader extends Index
throw new CorruptIndexException("doc counts differ for segment " + segment + ": fieldsReader shows " + fieldsReaderOrig.size() + " but segmentInfo shows " + si.docCount);
}
- if (fieldInfos.hasVectors()) { // open term vector files only as needed
+ if (si.getHasVectors()) { // open term vector files only as needed
termVectorsReaderOrig = new TermVectorsReader(storeDir, storesSegment, fieldInfos, readBufferSize, si.getDocStoreOffset(), si.docCount);
}
}
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java Mon Dec 20 10:53:27 2010
@@ -32,9 +32,8 @@ public class SegmentWriteState {
public final Directory directory;
public final String segmentName;
public final FieldInfos fieldInfos;
- public final String docStoreSegmentName;
public final int numDocs;
- public int numDocsInStore;
+ public boolean hasVectors;
public final Collection<String> flushedFiles;
public final AtomicLong bytesUsed;
@@ -63,15 +62,12 @@ public class SegmentWriteState {
public SegmentWriteState(PrintStream infoStream, Directory directory, String segmentName, FieldInfos fieldInfos,
- String docStoreSegmentName, int numDocs,
- int numDocsInStore, int termIndexInterval, SegmentCodecs segmentCodecs, AtomicLong bytesUsed) {
+ int numDocs, int termIndexInterval, SegmentCodecs segmentCodecs, AtomicLong bytesUsed) {
this.infoStream = infoStream;
this.directory = directory;
this.segmentName = segmentName;
this.fieldInfos = fieldInfos;
- this.docStoreSegmentName = docStoreSegmentName;
this.numDocs = numDocs;
- this.numDocsInStore = numDocsInStore;
this.termIndexInterval = termIndexInterval;
this.segmentCodecs = segmentCodecs;
flushedFiles = new HashSet<String>();
@@ -87,9 +83,7 @@ public class SegmentWriteState {
directory = state.directory;
segmentName = state.segmentName;
fieldInfos = state.fieldInfos;
- docStoreSegmentName = state.docStoreSegmentName;
numDocs = state.numDocs;
- numDocsInStore = state.numDocsInStore;
termIndexInterval = state.termIndexInterval;
segmentCodecs = state.segmentCodecs;
flushedFiles = state.flushedFiles;
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java Mon Dec 20 10:53:27 2010
@@ -29,7 +29,6 @@ final class StoredFieldsWriter {
final DocumentsWriter docWriter;
final FieldInfos fieldInfos;
int lastDocID;
- private String docStoreSegment;
PerDoc[] docFreeList = new PerDoc[1];
int freeCount;
@@ -44,60 +43,31 @@ final class StoredFieldsWriter {
}
synchronized public void flush(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 FieldsWriter:
- initFieldsWriter();
-
- // Fill fdx file to include any final docs that we
- // skipped because they hit non-aborting exceptions
- fill(state.numDocsInStore - docWriter.getDocStoreOffset());
- }
-
- if (fieldsWriter != null)
- fieldsWriter.flush();
- }
-
- private synchronized void initFieldsWriter() throws IOException {
- if (fieldsWriter == null) {
- docStoreSegment = docWriter.getDocStoreSegment();
- if (docStoreSegment != null) {
- fieldsWriter = new FieldsWriter(docWriter.directory,
- docStoreSegment,
- fieldInfos);
- docWriter.addOpenFile(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_EXTENSION));
- docWriter.addOpenFile(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
- lastDocID = 0;
- }
- }
- }
-
- synchronized public void closeDocStore(SegmentWriteState state) throws IOException {
- final int inc = state.numDocsInStore - lastDocID;
- if (inc > 0) {
+ if (state.numDocs > lastDocID) {
initFieldsWriter();
- fill(state.numDocsInStore - docWriter.getDocStoreOffset());
+ fill(state.numDocs);
}
if (fieldsWriter != null) {
fieldsWriter.close();
fieldsWriter = null;
- assert docStoreSegment != null;
- assert state.docStoreSegmentName != null;
- assert docStoreSegment.equals(state.docStoreSegmentName): "fieldsWriter wrote to segment=" + docStoreSegment + " vs SegmentWriteState segment=" + state.docStoreSegmentName;
lastDocID = 0;
- String fieldsName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.FIELDS_EXTENSION);
- String fieldsIdxName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
+
+ String fieldsName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.FIELDS_EXTENSION);
+ String fieldsIdxName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
state.flushedFiles.add(fieldsName);
state.flushedFiles.add(fieldsIdxName);
- docWriter.removeOpenFile(fieldsName);
- docWriter.removeOpenFile(fieldsIdxName);
+ if (4 + ((long) state.numDocs) * 8 != state.directory.fileLength(fieldsIdxName)) {
+ throw new RuntimeException("after flush: fdx size mismatch: " + state.numDocs + " docs vs " + state.directory.fileLength(fieldsIdxName) + " length in bytes of " + fieldsIdxName + " file exists?=" + state.directory.fileExists(fieldsIdxName));
+ }
+ }
+ }
- if (4+((long) state.numDocsInStore)*8 != state.directory.fileLength(fieldsIdxName))
- throw new RuntimeException("after flush: fdx size mismatch: " + state.numDocsInStore + " docs vs " + state.directory.fileLength(fieldsIdxName) + " length in bytes of " + fieldsIdxName + " file exists?=" + state.directory.fileExists(fieldsIdxName));
+ private synchronized void initFieldsWriter() throws IOException {
+ if (fieldsWriter == null) {
+ fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment(), fieldInfos);
+ lastDocID = 0;
}
}
@@ -114,16 +84,14 @@ final class StoredFieldsWriter {
docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
}
return new PerDoc();
- } else
+ } else {
return docFreeList[--freeCount];
+ }
}
synchronized void abort() {
if (fieldsWriter != null) {
- try {
- fieldsWriter.close();
- } catch (Throwable t) {
- }
+ fieldsWriter.abort();
fieldsWriter = null;
lastDocID = 0;
}
@@ -131,12 +99,9 @@ final class StoredFieldsWriter {
/** Fills in any hole in the docIDs */
void fill(int docID) throws IOException {
- final int docStoreOffset = docWriter.getDocStoreOffset();
-
// We must "catch up" for all docs before us
// that had no stored fields:
- final int end = docID+docStoreOffset;
- while(lastDocID < end) {
+ while(lastDocID < docID) {
fieldsWriter.skipDocument();
lastDocID++;
}
@@ -156,10 +121,6 @@ final class StoredFieldsWriter {
assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument end");
}
- public boolean freeRAM() {
- return false;
- }
-
synchronized void free(PerDoc perDoc) {
assert freeCount < docFreeList.length;
assert 0 == perDoc.numStoredFields;
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java Mon Dec 20 10:53:27 2010
@@ -77,38 +77,30 @@ class TermVectorsReader implements Clone
try {
String idxName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
- if (d.fileExists(idxName)) {
- tvx = d.openInput(idxName, readBufferSize);
- format = checkValidFormat(tvx, idxName);
- String fn = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
- tvd = d.openInput(fn, readBufferSize);
- final int tvdFormat = checkValidFormat(tvd, fn);
- fn = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
- tvf = d.openInput(fn, readBufferSize);
- final int tvfFormat = checkValidFormat(tvf, fn);
-
- assert format == tvdFormat;
- assert format == tvfFormat;
-
- numTotalDocs = (int) (tvx.length() >> 4);
-
- if (-1 == docStoreOffset) {
- this.docStoreOffset = 0;
- this.size = numTotalDocs;
- assert size == 0 || numTotalDocs == size;
- } else {
- this.docStoreOffset = docStoreOffset;
- this.size = size;
- // Verify the file is long enough to hold all of our
- // docs
- assert numTotalDocs >= size + docStoreOffset: "numTotalDocs=" + numTotalDocs + " size=" + size + " docStoreOffset=" + docStoreOffset;
- }
+ tvx = d.openInput(idxName, readBufferSize);
+ format = checkValidFormat(tvx, idxName);
+ String fn = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
+ tvd = d.openInput(fn, readBufferSize);
+ final int tvdFormat = checkValidFormat(tvd, fn);
+ fn = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
+ tvf = d.openInput(fn, readBufferSize);
+ final int tvfFormat = checkValidFormat(tvf, fn);
+
+ assert format == tvdFormat;
+ assert format == tvfFormat;
+
+ numTotalDocs = (int) (tvx.length() >> 4);
+
+ if (-1 == docStoreOffset) {
+ this.docStoreOffset = 0;
+ this.size = numTotalDocs;
+ assert size == 0 || numTotalDocs == size;
} else {
- // 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;
+ this.docStoreOffset = docStoreOffset;
+ this.size = size;
+ // Verify the file is long enough to hold all of our
+ // docs
+ assert numTotalDocs >= size + docStoreOffset: "numTotalDocs=" + numTotalDocs + " size=" + size + " docStoreOffset=" + docStoreOffset;
}
this.fieldInfos = fieldInfos;
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java Mon Dec 20 10:53:27 2010
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import java.io.IOException;
@@ -30,13 +31,13 @@ import java.util.Map;
final class TermVectorsTermsWriter extends TermsHashConsumer {
final DocumentsWriter docWriter;
- TermVectorsWriter termVectorsWriter;
PerDoc[] docFreeList = new PerDoc[1];
int freeCount;
IndexOutput tvx;
IndexOutput tvd;
IndexOutput tvf;
int lastDocID;
+ boolean hasVectors;
public TermVectorsTermsWriter(DocumentsWriter docWriter) {
this.docWriter = docWriter;
@@ -49,25 +50,29 @@ final class TermVectorsTermsWriter exten
@Override
synchronized void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
+ if (tvx != null) {
+ // At least one doc in this run had term vectors enabled
+ fill(state.numDocs);
+ tvx.close();
+ tvf.close();
+ tvd.close();
+ tvx = tvd = tvf = null;
+ assert state.segmentName != null;
+ String idxName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
+ String fldName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
+ String docName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
- // 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 (4 + ((long) state.numDocs) * 16 != state.directory.fileLength(idxName)) {
+ throw new RuntimeException("after flush: tvx size mismatch: " + state.numDocs + " docs vs " + state.directory.fileLength(idxName) + " length in bytes of " + idxName + " file exists?=" + state.directory.fileExists(idxName));
+ }
- if (tvx != null) {
+ state.flushedFiles.add(idxName);
+ state.flushedFiles.add(fldName);
+ state.flushedFiles.add(docName);
- if (state.numDocsInStore > 0)
- // In case there are some final documents that we
- // didn't see (because they hit a non-aborting exception):
- fill(state.numDocsInStore - docWriter.getDocStoreOffset());
-
- tvx.flush();
- tvd.flush();
- tvf.flush();
+ lastDocID = 0;
+ state.hasVectors = hasVectors;
+ hasVectors = false;
}
for (Map.Entry<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> entry : threadsAndFields.entrySet()) {
@@ -82,35 +87,6 @@ final class TermVectorsTermsWriter exten
}
}
- @Override
- synchronized void closeDocStore(final SegmentWriteState state) throws IOException {
- if (tvx != null) {
- // At least one doc in this run had term vectors
- // enabled
- fill(state.numDocsInStore - docWriter.getDocStoreOffset());
- tvx.close();
- tvf.close();
- tvd.close();
- tvx = null;
- assert state.docStoreSegmentName != null;
- String idxName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
- if (4+((long) state.numDocsInStore)*16 != state.directory.fileLength(idxName))
- throw new RuntimeException("after flush: tvx size mismatch: " + state.numDocsInStore + " docs vs " + state.directory.fileLength(idxName) + " length in bytes of " + idxName + " file exists?=" + state.directory.fileExists(idxName));
-
- String fldName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
- String docName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
- state.flushedFiles.add(idxName);
- state.flushedFiles.add(fldName);
- state.flushedFiles.add(docName);
-
- docWriter.removeOpenFile(idxName);
- docWriter.removeOpenFile(fldName);
- docWriter.removeOpenFile(docName);
-
- lastDocID = 0;
- }
- }
-
int allocCount;
synchronized PerDoc getPerDoc() {
@@ -124,18 +100,17 @@ final class TermVectorsTermsWriter exten
docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
}
return new PerDoc();
- } else
+ } else {
return docFreeList[--freeCount];
+ }
}
/** Fills in no-term-vectors for all docs we haven't seen
* since the last doc that had term vectors. */
void fill(int docID) throws IOException {
- final int docStoreOffset = docWriter.getDocStoreOffset();
- final int end = docID+docStoreOffset;
- if (lastDocID < end) {
+ if (lastDocID < docID) {
final long tvfPosition = tvf.getFilePointer();
- while(lastDocID < end) {
+ while(lastDocID < docID) {
tvx.writeLong(tvd.getFilePointer());
tvd.writeVInt(0);
tvx.writeLong(tvfPosition);
@@ -146,31 +121,20 @@ final class TermVectorsTermsWriter exten
synchronized void initTermVectorsWriter() throws IOException {
if (tvx == null) {
-
- final String docStoreSegment = docWriter.getDocStoreSegment();
-
- if (docStoreSegment == null)
- return;
// If we hit an exception while init'ing the term
// vector output files, we must abort this segment
// because those files will be in an unknown
// state:
- String idxName = IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
- String docName = IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
- String fldName = IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
- tvx = docWriter.directory.createOutput(idxName);
- tvd = docWriter.directory.createOutput(docName);
- tvf = docWriter.directory.createOutput(fldName);
+ hasVectors = true;
+ tvx = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION));
+ tvd = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
+ tvf = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
- docWriter.addOpenFile(idxName);
- docWriter.addOpenFile(fldName);
- docWriter.addOpenFile(docName);
-
lastDocID = 0;
}
}
@@ -188,8 +152,9 @@ final class TermVectorsTermsWriter exten
tvx.writeLong(tvf.getFilePointer());
tvd.writeVInt(perDoc.numVectorFields);
if (perDoc.numVectorFields > 0) {
- for(int i=0;i<perDoc.numVectorFields;i++)
+ for(int i=0;i<perDoc.numVectorFields;i++) {
tvd.writeVInt(perDoc.fieldNumbers[i]);
+ }
assert 0 == perDoc.fieldPointers[0];
long lastPos = perDoc.fieldPointers[0];
for(int i=1;i<perDoc.numVectorFields;i++) {
@@ -201,7 +166,7 @@ final class TermVectorsTermsWriter exten
perDoc.numVectorFields = 0;
}
- assert lastDocID == perDoc.docID + docWriter.getDocStoreOffset();
+ assert lastDocID == perDoc.docID;
lastDocID++;
@@ -210,35 +175,26 @@ final class TermVectorsTermsWriter exten
assert docWriter.writer.testPoint("TermVectorsTermsWriter.finishDocument end");
}
- public boolean freeRAM() {
- // We don't hold any state beyond one doc, so we don't
- // free persistent RAM here
- return false;
- }
-
@Override
public void abort() {
- if (tvx != null) {
- try {
- tvx.close();
- } catch (Throwable t) {
- }
- tvx = null;
- }
- if (tvd != null) {
- try {
- tvd.close();
- } catch (Throwable t) {
- }
- tvd = null;
- }
- if (tvf != null) {
- try {
- tvf.close();
- } catch (Throwable t) {
- }
- tvf = null;
+ hasVectors = false;
+ try {
+ IOUtils.closeSafely(tvx, tvd, tvf);
+ } catch (IOException ignored) {
+ }
+ try {
+ docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION));
+ } catch (IOException ignored) {
+ }
+ try {
+ docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
+ } catch (IOException ignored) {
+ }
+ try {
+ docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
+ } catch (IOException ignored) {
}
+ tvx = tvd = tvf = null;
lastDocID = 0;
}
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java Mon Dec 20 10:53:27 2010
@@ -24,6 +24,7 @@ import org.apache.lucene.document.Fielda
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.RamUsageEstimator;
final class TermVectorsTermsWriterPerField extends TermsHashConsumerPerField {
@@ -298,7 +299,7 @@ final class TermVectorsTermsWriterPerFie
@Override
int bytesPerPosting() {
- return super.bytesPerPosting() + 3 * DocumentsWriter.INT_NUM_BYTE;
+ return super.bytesPerPosting() + 3 * RamUsageEstimator.NUM_BYTES_INT;
}
}
}
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHash.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHash.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHash.java Mon Dec 20 10:53:27 2010
@@ -70,13 +70,6 @@ final class TermsHash extends InvertedDo
}
@Override
- synchronized void closeDocStore(SegmentWriteState state) throws IOException {
- consumer.closeDocStore(state);
- if (nextTermsHash != null)
- nextTermsHash.closeDocStore(state);
- }
-
- @Override
synchronized void flush(Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> childThreadsAndFields = new HashMap<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>>();
Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> nextThreadsAndFields;
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java Mon Dec 20 10:53:27 2010
@@ -25,7 +25,6 @@ abstract class TermsHashConsumer {
abstract TermsHashConsumerPerThread addThread(TermsHashPerThread perThread);
abstract void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException;
abstract void abort();
- abstract void closeDocStore(SegmentWriteState state) throws IOException;
FieldInfos fieldInfos;
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java Mon Dec 20 10:53:27 2010
@@ -40,7 +40,8 @@ public class DefaultSegmentInfosReader e
IndexInput input = null;
try {
input = openInput(directory, segmentsFileName);
- int format = input.readInt();
+ final int format = input.readInt();
+ infos.setFormat(format);
// check that it is a format we can understand
if (format > DefaultSegmentInfosWriter.FORMAT_MINIMUM)
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java Mon Dec 20 10:53:27 2010
@@ -35,9 +35,12 @@ public class DefaultSegmentInfosWriter e
* diagnostics storage, and switches userData to Map */
public static final int FORMAT_DIAGNOSTICS = -9;
+ /** Each segment records whether it has term vectors */
+ public static final int FORMAT_HAS_VECTORS = -10;
+
/** Each segment records whether its postings are written
* in the new flex format */
- public static final int FORMAT_4_0 = -10;
+ public static final int FORMAT_4_0 = -11;
/** This must always point to the most recent file format.
* whenever you add a new format, make it 1 smaller (negative version logic)! */
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java Mon Dec 20 10:53:27 2010
@@ -20,16 +20,12 @@ package org.apache.lucene.index.codecs;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.FieldsEnum;
-import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.codecs.docvalues.DocValuesConsumer;
import org.apache.lucene.index.values.DocValues;
-import org.apache.lucene.index.values.Writer;
import java.io.IOException;
import java.io.Closeable;
-import java.util.ArrayList;
-import java.util.List;
/** Abstract API that consumes terms, doc, freq, prox and
* payloads postings. Concrete implementations of this
@@ -47,8 +43,6 @@ public abstract class FieldsConsumer imp
public DocValuesConsumer addValuesField(FieldInfo field) throws IOException {
throw new UnsupportedOperationException("docvalues are not supported");
}
-
-
/** Called when we are done adding everything. */
public abstract void close() throws IOException;
@@ -67,9 +61,10 @@ public abstract class FieldsConsumer imp
}
if (mergeState.fieldInfo.hasDocValues()) {
final DocValues docValues = fieldsEnum.docValues();
- // TODO: is this assert values and if so when?
-// assert docValues != null : "DocValues are null for " + mergeState.fieldInfo.getDocValues();
- if(docValues == null) { // for now just continue
+ if(docValues == null) {
+ /* It is actually possible that a fieldInfo has a values type but no values are actually available.
+ * this can happen if there are already segments without values around.
+ */
continue;
}
final DocValuesConsumer docValuesConsumer = addValuesField(mergeState.fieldInfo);
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java Mon Dec 20 10:53:27 2010
@@ -57,7 +57,7 @@ public class SimpleTextCodec extends Cod
static final String POSTINGS_EXTENSION = "pst";
static String getPostingsFileName(String segment, String id) {
- return IndexFileNames.segmentFileName(segment, "", POSTINGS_EXTENSION);
+ return IndexFileNames.segmentFileName(segment, id, POSTINGS_EXTENSION);
}
@Override
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java Mon Dec 20 10:53:27 2010
@@ -32,15 +32,16 @@ import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util.automaton.fst.Builder;
+import org.apache.lucene.util.automaton.fst.BytesRefFSTEnum;
+import org.apache.lucene.util.automaton.fst.FST;
+import org.apache.lucene.util.automaton.fst.PositiveIntOutputs;
+import org.apache.lucene.util.automaton.fst.PairOutputs;
import java.io.IOException;
import java.util.Comparator;
import java.util.Map;
-import java.util.Set;
import java.util.HashMap;
-import java.util.TreeMap;
-import java.util.SortedMap;
-import java.util.Iterator;
class SimpleTextFieldsReader extends FieldsProducer {
@@ -123,73 +124,39 @@ class SimpleTextFieldsReader extends Fie
private class SimpleTextTermsEnum extends TermsEnum {
private final IndexInput in;
private final boolean omitTF;
- private BytesRef current;
private int docFreq;
private long docsStart;
private boolean ended;
- private final TreeMap<BytesRef,TermData> allTerms;
- private Iterator<Map.Entry<BytesRef,TermData>> iter;
+ private final BytesRefFSTEnum<PairOutputs.Pair<Long,Long>> fstEnum;
- public SimpleTextTermsEnum(TreeMap<BytesRef,TermData> allTerms, boolean omitTF) throws IOException {
+ public SimpleTextTermsEnum(FST<PairOutputs.Pair<Long,Long>> fst, boolean omitTF) throws IOException {
this.in = (IndexInput) SimpleTextFieldsReader.this.in.clone();
- this.allTerms = allTerms;
this.omitTF = omitTF;
- iter = allTerms.entrySet().iterator();
+ fstEnum = new BytesRefFSTEnum<PairOutputs.Pair<Long,Long>>(fst);
}
public SeekStatus seek(BytesRef text, boolean useCache /* ignored */) throws IOException {
-
- final SortedMap<BytesRef,TermData> tailMap = allTerms.tailMap(text);
- if (tailMap.isEmpty()) {
- current = null;
+ fstEnum.reset();
+ //System.out.println("seek to text=" + text.utf8ToString());
+ final BytesRefFSTEnum.InputOutput<PairOutputs.Pair<Long,Long>> result = fstEnum.advance(text);
+ if (result == null) {
+ //System.out.println(" end");
return SeekStatus.END;
} else {
- current = tailMap.firstKey();
- final TermData td = tailMap.get(current);
- docsStart = td.docsStart;
- docFreq = td.docFreq;
- iter = tailMap.entrySet().iterator();
- assert iter.hasNext();
- iter.next();
- if (current.equals(text)) {
- return SeekStatus.FOUND;
- } else {
- return SeekStatus.NOT_FOUND;
- }
- }
-
- /*
- if (current != null) {
- final int cmp = current.compareTo(text);
- if (cmp == 0) {
- return SeekStatus.FOUND;
- } else if (cmp > 0) {
- ended = false;
- in.seek(fieldStart);
- }
- } else {
- ended = false;
- in.seek(fieldStart);
- }
+ //System.out.println(" got text=" + term.utf8ToString());
+ PairOutputs.Pair<Long,Long> pair = result.output;
+ docsStart = pair.output1;
+ docFreq = pair.output2.intValue();
- // Naive!! This just scans... would be better to do
- // up-front scan to build in-RAM index
- BytesRef b;
- while((b = next()) != null) {
- final int cmp = b.compareTo(text);
- if (cmp == 0) {
- ended = false;
+ if (result.input.equals(text)) {
+ //System.out.println(" match docsStart=" + docsStart);
return SeekStatus.FOUND;
- } else if (cmp > 0) {
- ended = false;
+ } else {
+ //System.out.println(" not match docsStart=" + docsStart);
return SeekStatus.NOT_FOUND;
}
}
- current = null;
- ended = true;
- return SeekStatus.END;
- */
}
@Override
@@ -199,56 +166,20 @@ class SimpleTextFieldsReader extends Fie
@Override
public BytesRef next() throws IOException {
assert !ended;
-
- if (iter.hasNext()) {
- Map.Entry<BytesRef,TermData> ent = iter.next();
- current = ent.getKey();
- TermData td = ent.getValue();
- docFreq = td.docFreq;
- docsStart = td.docsStart;
- return current;
+ final BytesRefFSTEnum.InputOutput<PairOutputs.Pair<Long,Long>> result = fstEnum.next();
+ if (result != null) {
+ final PairOutputs.Pair<Long,Long> pair = result.output;
+ docsStart = pair.output1;
+ docFreq = pair.output2.intValue();
+ return result.input;
} else {
- current = null;
- return null;
- }
-
- /*
- readLine(in, scratch);
- if (scratch.equals(END) || scratch.startsWith(FIELD)) {
- ended = true;
- current = null;
return null;
- } else {
- assert scratch.startsWith(TERM): "got " + scratch.utf8ToString();
- docsStart = in.getFilePointer();
- final int len = scratch.length - TERM.length;
- if (len > scratch2.length) {
- scratch2.grow(len);
- }
- System.arraycopy(scratch.bytes, TERM.length, scratch2.bytes, 0, len);
- scratch2.length = len;
- current = scratch2;
- docFreq = 0;
- long lineStart = 0;
- while(true) {
- lineStart = in.getFilePointer();
- readLine(in, scratch);
- if (scratch.equals(END) || scratch.startsWith(FIELD) || scratch.startsWith(TERM)) {
- break;
- }
- if (scratch.startsWith(DOC)) {
- docFreq++;
- }
- }
- in.seek(lineStart);
- return current;
}
- */
}
@Override
public BytesRef term() {
- return current;
+ return fstEnum.current().input;
}
@Override
@@ -519,10 +450,7 @@ class SimpleTextFieldsReader extends Fie
private final String field;
private final long termsStart;
private final boolean omitTF;
-
- // NOTE: horribly, horribly RAM consuming, but then
- // SimpleText should never be used in production
- private final TreeMap<BytesRef,TermData> allTerms = new TreeMap<BytesRef,TermData>();
+ private FST<PairOutputs.Pair<Long,Long>> fst;
private final BytesRef scratch = new BytesRef(10);
@@ -534,6 +462,8 @@ class SimpleTextFieldsReader extends Fie
}
private void loadTerms() throws IOException {
+ PositiveIntOutputs posIntOutputs = PositiveIntOutputs.getSingleton(false);
+ Builder<PairOutputs.Pair<Long,Long>> b = new Builder<PairOutputs.Pair<Long,Long>>(FST.INPUT_TYPE.BYTE1, 0, 0, true, new PairOutputs<Long,Long>(posIntOutputs, posIntOutputs));
IndexInput in = (IndexInput) SimpleTextFieldsReader.this.in.clone();
in.seek(termsStart);
final BytesRef lastTerm = new BytesRef(10);
@@ -543,16 +473,14 @@ class SimpleTextFieldsReader extends Fie
readLine(in, scratch);
if (scratch.equals(END) || scratch.startsWith(FIELD)) {
if (lastDocsStart != -1) {
- allTerms.put(new BytesRef(lastTerm),
- new TermData(lastDocsStart, docFreq));
+ b.add(lastTerm, new PairOutputs.Pair<Long,Long>(lastDocsStart, Long.valueOf(docFreq)));
}
break;
} else if (scratch.startsWith(DOC)) {
docFreq++;
} else if (scratch.startsWith(TERM)) {
if (lastDocsStart != -1) {
- allTerms.put(new BytesRef(lastTerm),
- new TermData(lastDocsStart, docFreq));
+ b.add(lastTerm, new PairOutputs.Pair<Long,Long>(lastDocsStart, Long.valueOf(docFreq)));
}
lastDocsStart = in.getFilePointer();
final int len = scratch.length - TERM.length;
@@ -564,11 +492,23 @@ class SimpleTextFieldsReader extends Fie
docFreq = 0;
}
}
+ fst = b.finish();
+ /*
+ PrintStream ps = new PrintStream("out.dot");
+ fst.toDot(ps);
+ ps.close();
+ System.out.println("SAVED out.dot");
+ */
+ //System.out.println("FST " + fst.sizeInBytes());
}
@Override
public TermsEnum iterator() throws IOException {
- return new SimpleTextTermsEnum(allTerms, omitTF);
+ if (fst != null) {
+ return new SimpleTextTermsEnum(fst, omitTF);
+ } else {
+ return TermsEnum.EMPTY;
+ }
}
@Override
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java Mon Dec 20 10:53:27 2010
@@ -82,16 +82,15 @@ public class StandardPostingsReader exte
maxSkipLevels = termsIn.readInt();
}
- private static class DocTermState extends TermState {
+ // Must keep final because we do non-standard clone
+ private final static class DocTermState extends TermState {
long freqOffset;
long proxOffset;
int skipOffset;
public Object clone() {
- DocTermState other = (DocTermState) super.clone();
- other.freqOffset = freqOffset;
- other.proxOffset = proxOffset;
- other.skipOffset = skipOffset;
+ DocTermState other = new DocTermState();
+ other.copy(this);
return other;
}
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java Mon Dec 20 10:53:27 2010
@@ -51,13 +51,14 @@ import org.apache.lucene.util.PagedBytes
* </p>
* @lucene.experimental
*/
-// TODO - add bulk copy where possible
public final class Bytes {
+ //TODO - add bulk copy where possible
+ private Bytes() { /* don't instantiate! */ }
- // don't instantiate!
- private Bytes() {
- }
-
+ /**
+ *
+ *
+ */
public static enum Mode {
STRAIGHT, DEREF, SORTED
};
@@ -292,13 +293,13 @@ public final class Bytes {
initIndexOut();
}
- protected void initDataOut() throws IOException {
+ private void initDataOut() throws IOException {
datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
DATA_EXTENSION));
CodecUtil.writeHeader(datOut, codecName, version);
}
- protected void initIndexOut() throws IOException {
+ private void initIndexOut() throws IOException {
idxOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
INDEX_EXTENSION));
CodecUtil.writeHeader(idxOut, codecName, version);
@@ -373,7 +374,6 @@ public final class Bytes {
datIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
Writer.DATA_EXTENSION));
version = CodecUtil.checkHeader(datIn, codecName, maxVersion, maxVersion);
-
if (doIndex) {
idxIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
Writer.INDEX_EXTENSION));
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java Mon Dec 20 10:53:27 2010
@@ -51,20 +51,23 @@ class FixedDerefBytesImpl {
static class Writer extends BytesWriterBase {
private int size = -1;
private int[] docToID;
- private final BytesRefHash hash = new BytesRefHash(pool, BytesRefHash.DEFAULT_CAPACITY,
- new TrackingDirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
+ private final BytesRefHash hash = new BytesRefHash(pool,
+ BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
+ BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
- public Writer(Directory dir, String id, AtomicLong bytesUsed) throws IOException {
+ public Writer(Directory dir, String id, AtomicLong bytesUsed)
+ throws IOException {
this(dir, id, new DirectAllocator(ByteBlockPool.BYTE_BLOCK_SIZE),
bytesUsed);
}
public Writer(Directory dir, String id, Allocator allocator,
AtomicLong bytesUsed) throws IOException {
- super(dir, id, CODEC_NAME, VERSION_CURRENT, false, false,
+ super(dir, id, CODEC_NAME, VERSION_CURRENT, true, true,
new ByteBlockPool(allocator), bytesUsed);
docToID = new int[1];
- bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT); // TODO BytesRefHash uses bytes too!
+ bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT); // TODO BytesRefHash
+ // uses bytes too!
}
@Override
@@ -73,7 +76,6 @@ class FixedDerefBytesImpl {
return;
if (size == -1) {
size = bytes.length;
- initDataOut();
datOut.writeInt(size);
} else if (bytes.length != size) {
throw new IllegalArgumentException("expected bytes size=" + size
@@ -101,28 +103,31 @@ class FixedDerefBytesImpl {
// some last docs that we didn't see
@Override
synchronized public void finish(int docCount) throws IOException {
- if (datOut == null) // no added data
- return;
- initIndexOut();
- final int count = 1 + hash.size();
- idxOut.writeInt(count - 1);
- // write index
- final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
- PackedInts.bitsRequired(count - 1));
- final int limit = docCount > docToID.length ? docToID.length : docCount;
- for (int i = 0; i < limit; i++) {
- w.add(docToID[i]);
- }
- // fill up remaining doc with zeros
- for (int i = limit; i < docCount; i++) {
- w.add(0);
- }
- w.finish();
- hash.close();
- super.finish(docCount);
- bytesUsed.addAndGet((-docToID.length)
- * RamUsageEstimator.NUM_BYTES_INT);
- docToID = null;
+ try {
+ if (size == -1) {
+ datOut.writeInt(size);
+ }
+ final int count = 1 + hash.size();
+ idxOut.writeInt(count - 1);
+ // write index
+ final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
+ PackedInts.bitsRequired(count - 1));
+ final int limit = docCount > docToID.length ? docToID.length : docCount;
+ for (int i = 0; i < limit; i++) {
+ w.add(docToID[i]);
+ }
+ // fill up remaining doc with zeros
+ for (int i = limit; i < docCount; i++) {
+ w.add(0);
+ }
+ w.finish();
+ } finally {
+ hash.close();
+ super.finish(docCount);
+ bytesUsed
+ .addAndGet((-docToID.length) * RamUsageEstimator.NUM_BYTES_INT);
+ docToID = null;
+ }
}
}
@@ -131,11 +136,7 @@ class FixedDerefBytesImpl {
Reader(Directory dir, String id, int maxDoc) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true);
- try {
- size = datIn.readInt();
- } catch (IOException e) {
- throw e;
- }
+ size = datIn.readInt();
}
@Override
@@ -184,8 +185,7 @@ class FixedDerefBytesImpl {
@Override
public DocValuesEnum getEnum(AttributeSource source) throws IOException {
- return new DerefBytesEnum(source, cloneData(), cloneIndex(),
- size);
+ return new DerefBytesEnum(source, cloneData(), cloneIndex(), size);
}
static class DerefBytesEnum extends DocValuesEnum {
@@ -202,8 +202,7 @@ class FixedDerefBytesImpl {
}
protected DerefBytesEnum(AttributeSource source, IndexInput datIn,
- IndexInput idxIn, int size, Type enumType)
- throws IOException {
+ IndexInput idxIn, int size, Type enumType) throws IOException {
super(source, enumType);
this.datIn = datIn;
this.size = size;
@@ -215,11 +214,10 @@ class FixedDerefBytesImpl {
bytesRef.offset = 0;
valueCount = idx.size();
}
-
protected void copyReferences(DocValuesEnum valuesEnum) {
bytesRef = valuesEnum.bytesRef;
- if(bytesRef.bytes.length < size) {
+ if (bytesRef.bytes.length < size) {
bytesRef.grow(size);
}
bytesRef.length = size;
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java Mon Dec 20 10:53:27 2010
@@ -57,18 +57,19 @@ class FixedSortedBytesImpl {
private int[] docToEntry;
private final Comparator<BytesRef> comp;
- private final BytesRefHash hash = new BytesRefHash(pool, BytesRefHash.DEFAULT_CAPACITY,
- new TrackingDirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
+ private final BytesRefHash hash = new BytesRefHash(pool,
+ BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
+ BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
- public Writer(Directory dir, String id, Comparator<BytesRef> comp, AtomicLong bytesUsed)
- throws IOException {
+ public Writer(Directory dir, String id, Comparator<BytesRef> comp,
+ AtomicLong bytesUsed) throws IOException {
this(dir, id, comp, new DirectAllocator(ByteBlockPool.BYTE_BLOCK_SIZE),
bytesUsed);
}
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
Allocator allocator, AtomicLong bytesUsed) throws IOException {
- super(dir, id, CODEC_NAME, VERSION_CURRENT, false, false,
+ super(dir, id, CODEC_NAME, VERSION_CURRENT, true, true,
new ByteBlockPool(allocator), bytesUsed);
docToEntry = new int[1];
// docToEntry[0] = -1;
@@ -82,18 +83,15 @@ class FixedSortedBytesImpl {
return; // default - skip it
if (size == -1) {
size = bytes.length;
- initDataOut();
datOut.writeInt(size);
} else if (bytes.length != size) {
throw new IllegalArgumentException("expected bytes size=" + size
+ " but got " + bytes.length);
}
if (docID >= docToEntry.length) {
- int[] newArray = new int[ArrayUtil.oversize(1 + docID,
+ final int[] newArray = new int[ArrayUtil.oversize(1 + docID,
RamUsageEstimator.NUM_BYTES_INT)];
System.arraycopy(docToEntry, 0, newArray, 0, docToEntry.length);
- // Arrays.fill(newArray, docToEntry.length, newArray.length, -1);
-
bytesUsed.addAndGet((newArray.length - docToEntry.length)
* RamUsageEstimator.NUM_BYTES_INT);
docToEntry = newArray;
@@ -106,54 +104,56 @@ class FixedSortedBytesImpl {
// some last docs that we didn't see
@Override
synchronized public void finish(int docCount) throws IOException {
- if (datOut == null)// no data added
- return;
- initIndexOut();
- final int[] sortedEntries = hash.sort(comp);
- final int count = hash.size();
- int[] address = new int[count];
- // first dump bytes data, recording address as we go
- for (int i = 0; i < count; i++) {
- final int e = sortedEntries[i];
- final BytesRef bytes = hash.get(e, new BytesRef());
- assert bytes.length == size;
- datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
- address[e] = 1 + i;
- }
-
- idxOut.writeInt(count);
-
- // next write index
- PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount, PackedInts
- .bitsRequired(count));
- final int limit;
- if (docCount > docToEntry.length) {
- limit = docToEntry.length;
- } else {
- limit = docCount;
- }
- for (int i = 0; i < limit; i++) {
- final int e = docToEntry[i];
- if (e == 0) {
- // null is encoded as zero
- w.add(0);
+ try {
+ if (size == -1) {// no data added
+ datOut.writeInt(size);
+ }
+ final int[] sortedEntries = hash.sort(comp);
+ final int count = hash.size();
+ int[] address = new int[count];
+ // first dump bytes data, recording address as we go
+ for (int i = 0; i < count; i++) {
+ final int e = sortedEntries[i];
+ final BytesRef bytes = hash.get(e, new BytesRef());
+ assert bytes.length == size;
+ datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
+ address[e] = 1 + i;
+ }
+
+ idxOut.writeInt(count);
+
+ // next write index
+ PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
+ PackedInts.bitsRequired(count));
+ final int limit;
+ if (docCount > docToEntry.length) {
+ limit = docToEntry.length;
} else {
- assert e > 0 && e <= count : "index must 0 > && <= " + count
- + " was: " + e;
- w.add(address[e - 1]);
+ limit = docCount;
+ }
+ for (int i = 0; i < limit; i++) {
+ final int e = docToEntry[i];
+ if (e == 0) {
+ // null is encoded as zero
+ w.add(0);
+ } else {
+ assert e > 0 && e <= count : "index must 0 > && <= " + count
+ + " was: " + e;
+ w.add(address[e - 1]);
+ }
}
- }
- for (int i = limit; i < docCount; i++) {
- w.add(0);
+ for (int i = limit; i < docCount; i++) {
+ w.add(0);
+ }
+ w.finish();
+ } finally {
+ super.finish(docCount);
+ bytesUsed.addAndGet((-docToEntry.length)
+ * RamUsageEstimator.NUM_BYTES_INT);
+ docToEntry = null;
+ hash.close();
}
- w.finish();
-
- super.finish(docCount);
- bytesUsed.addAndGet((-docToEntry.length)
- * RamUsageEstimator.NUM_BYTES_INT);
- docToEntry = null;
- hash.close();
}
}
@@ -187,9 +187,10 @@ class FixedSortedBytesImpl {
private final int numValue;
private final int size;
- public Source(IndexInput datIn, IndexInput idxIn, int size, int numValues,
- Comparator<BytesRef> comp) throws IOException {
- super(datIn, idxIn, comp, new PagedBytes(PAGED_BYTES_BITS), size*numValues );
+ public Source(IndexInput datIn, IndexInput idxIn, int size,
+ int numValues, Comparator<BytesRef> comp) throws IOException {
+ super(datIn, idxIn, comp, new PagedBytes(PAGED_BYTES_BITS), size
+ * numValues);
this.size = size;
this.numValue = numValues;
index = PackedInts.getReader(idxIn);
@@ -209,9 +210,10 @@ class FixedSortedBytesImpl {
public int getValueCount() {
return numValue;
}
+
@Override
protected BytesRef deref(int ord, BytesRef bytesRef) {
- return data.fillSlice(bytesRef, (ord* size), size);
+ return data.fillSlice(bytesRef, (ord * size), size);
}
@Override
@@ -228,8 +230,7 @@ class FixedSortedBytesImpl {
@Override
public DocValuesEnum getEnum(AttributeSource source) throws IOException {
// do unsorted
- return new DerefBytesEnum(source, cloneData(), cloneIndex(),
- size);
+ return new DerefBytesEnum(source, cloneData(), cloneIndex(), size);
}
@Override
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java Mon Dec 20 10:53:27 2010
@@ -46,46 +46,49 @@ class FixedStraightBytesImpl {
private byte[] oneRecord;
protected Writer(Directory dir, String id) throws IOException {
- super(dir, id, CODEC_NAME, VERSION_CURRENT, false, false, null, null);
+ super(dir, id, CODEC_NAME, VERSION_CURRENT, false, true, null, null);
}
-
+
// TODO - impl bulk copy here!
@Override
synchronized public void add(int docID, BytesRef bytes) throws IOException {
if (size == -1) {
size = bytes.length;
- initDataOut();
datOut.writeInt(size);
oneRecord = new byte[size];
} else if (bytes.length != size) {
- throw new IllegalArgumentException("expected bytes size=" + size + " but got " + bytes.length);
+ throw new IllegalArgumentException("expected bytes size=" + size
+ + " but got " + bytes.length);
}
fill(docID);
assert bytes.bytes.length >= bytes.length;
datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
}
- /* (non-Javadoc)
- * @see org.apache.lucene.index.values.Writer#merge(org.apache.lucene.index.values.Writer.MergeState)
+ /*
+ * (non-Javadoc)
+ *
+ * @see
+ * org.apache.lucene.index.values.Writer#merge(org.apache.lucene.index.values
+ * .Writer.MergeState)
*/
@Override
protected void merge(MergeState state) throws IOException {
- if(state.bits == null && state.reader instanceof Reader){
+ if (state.bits == null && state.reader instanceof Reader) {
Reader reader = (Reader) state.reader;
final int maxDocs = reader.maxDoc;
- if(maxDocs == 0)
+ if (maxDocs == 0)
return;
- if(size == -1) {
+ if (size == -1) {
size = reader.size;
- initDataOut();
datOut.writeInt(size);
oneRecord = new byte[size];
}
- fill(state.docBase);
- // TODO should we add a transfer to API to each reader?
- datOut.copyBytes(reader.cloneData(), size * maxDocs);
- lastDocID += maxDocs-1;
+ fill(state.docBase);
+ // TODO should we add a transfer to API to each reader?
+ datOut.copyBytes(reader.cloneData(), size * maxDocs);
+ lastDocID += maxDocs - 1;
} else
super.merge(state);
}
@@ -93,7 +96,7 @@ class FixedStraightBytesImpl {
// Fills up to but not including this docID
private void fill(int docID) throws IOException {
assert size >= 0;
- for(int i=lastDocID+1;i<docID;i++) {
+ for (int i = lastDocID + 1; i < docID; i++) {
datOut.writeBytes(oneRecord, size);
}
lastDocID = docID;
@@ -101,24 +104,28 @@ class FixedStraightBytesImpl {
@Override
synchronized public void finish(int docCount) throws IOException {
- if(datOut == null) // no data added
- return;
- fill(docCount);
- super.finish(docCount);
+ try {
+ if (size == -1) {// no data added
+ datOut.writeInt(0);
+ } else {
+ fill(docCount);
+ }
+ } finally {
+ super.finish(docCount);
+ }
}
public long ramBytesUsed() {
return 0;
}
-
+
}
public static class Reader extends BytesReaderBase {
private final int size;
private final int maxDoc;
- Reader(Directory dir, String id, int maxDoc)
- throws IOException {
+ Reader(Directory dir, String id, int maxDoc) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, false);
size = datIn.readInt();
this.maxDoc = maxDoc;
@@ -138,15 +145,16 @@ class FixedStraightBytesImpl {
private final int size;
private final int maxDoc;
- public Source(IndexInput datIn, IndexInput idxIn, int size, int maxDoc) throws IOException {
- super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), size*maxDoc);
+ public Source(IndexInput datIn, IndexInput idxIn, int size, int maxDoc)
+ throws IOException {
+ super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), size * maxDoc);
this.size = size;
this.missingValue.bytesValue = new BytesRef(size);
this.maxDoc = maxDoc;
}
-
+
@Override
- public BytesRef getBytes(int docID, BytesRef bytesRef) {
+ public BytesRef getBytes(int docID, BytesRef bytesRef) {
return data.fillSlice(bytesRef, docID * size, size);
}
@@ -170,7 +178,7 @@ class FixedStraightBytesImpl {
public DocValuesEnum getEnum(AttributeSource source) throws IOException {
return new FixedStraightBytesEnum(source, cloneData(), size, maxDoc);
}
-
+
private static final class FixedStraightBytesEnum extends DocValuesEnum {
private final IndexInput datIn;
private final int size;
@@ -178,7 +186,8 @@ class FixedStraightBytesImpl {
private int pos = -1;
private final long fp;
- public FixedStraightBytesEnum(AttributeSource source, IndexInput datIn, int size, int maxDoc) throws IOException{
+ public FixedStraightBytesEnum(AttributeSource source, IndexInput datIn,
+ int size, int maxDoc) throws IOException {
super(source, Type.BYTES_FIXED_STRAIGHT);
this.datIn = datIn;
this.size = size;
@@ -188,45 +197,45 @@ class FixedStraightBytesImpl {
bytesRef.offset = 0;
fp = datIn.getFilePointer();
}
-
+
protected void copyReferences(DocValuesEnum valuesEnum) {
bytesRef = valuesEnum.bytesRef;
- if(bytesRef.bytes.length < size) {
+ if (bytesRef.bytes.length < size) {
bytesRef.grow(size);
}
bytesRef.length = size;
bytesRef.offset = 0;
}
-
+
public void close() throws IOException {
datIn.close();
}
-
+
@Override
public int advance(int target) throws IOException {
- if(target >= maxDoc){
+ if (target >= maxDoc || size == 0) {
return pos = NO_MORE_DOCS;
}
- if((target-1) != pos) // pos inc == 1
+ if ((target - 1) != pos) // pos inc == 1
datIn.seek(fp + target * size);
datIn.readBytes(bytesRef.bytes, 0, size);
return pos = target;
}
-
+
@Override
public int docID() {
return pos;
}
-
+
@Override
public int nextDoc() throws IOException {
- if(pos >= maxDoc){
+ if (pos >= maxDoc) {
return pos = NO_MORE_DOCS;
}
- return advance(pos+1);
+ return advance(pos + 1);
}
}
-
+
@Override
public Type type() {
return Type.BYTES_FIXED_STRAIGHT;
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java Mon Dec 20 10:53:27 2010
@@ -1,4 +1,5 @@
package org.apache.lucene.index.values;
+
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@@ -33,6 +34,7 @@ import org.apache.lucene.util.FloatsRef;
/**
* Exposes writer/reader for floating point values. You can specify 4 (java
* float) or 8 (java double) byte precision.
+ *
* @lucene.experimental
*/
// TODO - add bulk copy where possible
@@ -45,8 +47,8 @@ public class Floats {
private static final long LONG_DEFAULT = Double
.doubleToRawLongBits(Double.NEGATIVE_INFINITY);
- public static Writer getWriter(Directory dir, String id, int precisionBytes, AtomicLong bytesUsed)
- throws IOException {
+ public static Writer getWriter(Directory dir, String id, int precisionBytes,
+ AtomicLong bytesUsed) throws IOException {
if (precisionBytes != 4 && precisionBytes != 8) {
throw new IllegalArgumentException("precisionBytes must be 4 or 8; got "
+ precisionBytes);
@@ -71,15 +73,16 @@ public class Floats {
protected IndexOutput datOut;
private final byte precision;
- protected FloatsWriter(Directory dir, String id, int precision, AtomicLong bytesUsed)
- throws IOException {
+ protected FloatsWriter(Directory dir, String id, int precision,
+ AtomicLong bytesUsed) throws IOException {
super(bytesUsed);
this.dir = dir;
this.id = id;
this.precision = (byte) precision;
+ initDatOut();
}
- protected void initDatOut() throws IOException {
+ private void initDatOut() throws IOException {
datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
Writer.DATA_EXTENSION));
CodecUtil.writeHeader(datOut, CODEC_NAME, VERSION_CURRENT);
@@ -98,7 +101,7 @@ public class Floats {
@Override
public void add(int docID, PerDocFieldValues docValues) throws IOException {
- add(docID, docValues.getFloat());
+ add(docID, docValues.getFloat());
}
@Override
@@ -139,7 +142,8 @@ public class Floats {
// Writes 4 bytes (float) per value
static class Float4Writer extends FloatsWriter {
- protected Float4Writer(Directory dir, String id, AtomicLong bytesUsed) throws IOException {
+ protected Float4Writer(Directory dir, String id, AtomicLong bytesUsed)
+ throws IOException {
super(dir, id, 4, bytesUsed);
}
@@ -148,9 +152,6 @@ public class Floats {
throws IOException {
assert docID > lastDocId : "docID: " + docID
+ " must be greater than the last added doc id: " + lastDocId;
- if (datOut == null) {
- initDatOut();
- }
if (docID - lastDocId > 1) {
// fill with default values
lastDocId += fillDefault(docID - lastDocId - 1);
@@ -162,13 +163,14 @@ public class Floats {
@Override
synchronized public void finish(int docCount) throws IOException {
- if (datOut == null)
- return; // no data added - don't create file!
- if (docCount > lastDocId + 1)
- for (int i = lastDocId; i < docCount; i++) {
- datOut.writeInt(INT_DEFAULT); // default value
- }
- datOut.close();
+ try {
+ if (docCount > lastDocId + 1)
+ for (int i = lastDocId; i < docCount; i++) {
+ datOut.writeInt(INT_DEFAULT); // default value
+ }
+ } finally {
+ datOut.close();
+ }
}
@Override
@@ -183,7 +185,8 @@ public class Floats {
// Writes 8 bytes (double) per value
static class Float8Writer extends FloatsWriter {
- protected Float8Writer(Directory dir, String id, AtomicLong bytesUsed) throws IOException {
+ protected Float8Writer(Directory dir, String id, AtomicLong bytesUsed)
+ throws IOException {
super(dir, id, 8, bytesUsed);
}
@@ -191,9 +194,6 @@ public class Floats {
synchronized public void add(int docID, double v) throws IOException {
assert docID > lastDocId : "docID: " + docID
+ " must be greater than the last added doc id: " + lastDocId;
- if (datOut == null) {
- initDatOut();
- }
if (docID - lastDocId > 1) {
// fill with default values
lastDocId += fillDefault(docID - lastDocId - 1);
@@ -205,13 +205,14 @@ public class Floats {
@Override
synchronized public void finish(int docCount) throws IOException {
- if (datOut == null)
- return; // no data added - don't create file!
- if (docCount > lastDocId + 1)
- for (int i = lastDocId; i < docCount; i++) {
- datOut.writeLong(LONG_DEFAULT); // default value
- }
- datOut.close();
+ try {
+ if (docCount > lastDocId + 1)
+ for (int i = lastDocId; i < docCount; i++) {
+ datOut.writeLong(LONG_DEFAULT); // default value
+ }
+ } finally {
+ datOut.close();
+ }
}
@Override
@@ -288,7 +289,8 @@ public class Floats {
}
@Override
- public DocValuesEnum getEnum(AttributeSource attrSource) throws IOException {
+ public DocValuesEnum getEnum(AttributeSource attrSource)
+ throws IOException {
final MissingValue missing = getMissing();
return new SourceEnum(attrSource, Type.SIMPLE_FLOAT_4BYTE, this, maxDoc) {
@Override
@@ -327,7 +329,8 @@ public class Floats {
}
@Override
- public DocValuesEnum getEnum(AttributeSource attrSource) throws IOException {
+ public DocValuesEnum getEnum(AttributeSource attrSource)
+ throws IOException {
final MissingValue missing = getMissing();
return new SourceEnum(attrSource, type(), this, maxDoc) {
@Override
Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java Mon Dec 20 10:53:27 2010
@@ -32,7 +32,8 @@ import org.apache.lucene.util.OpenBitSet
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.PackedInts;
-/** Stores ints packed with fixed-bit precision.
+/**
+ * Stores ints packed with fixed-bit precision.
*
* @lucene.experimental
* */
@@ -51,17 +52,21 @@ class PackedIntsImpl {
private long minValue;
private long maxValue;
private boolean started;
- private final Directory dir;
private final String id;
private final OpenBitSet defaultValues = new OpenBitSet(1);
private int lastDocId = -1;
+ private IndexOutput datOut;
- protected IntsWriter(Directory dir, String id, AtomicLong bytesUsed) throws IOException {
+ protected IntsWriter(Directory dir, String id, AtomicLong bytesUsed)
+ throws IOException {
super(bytesUsed);
- this.dir = dir;
this.id = id;
docToValue = new long[1];
- bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_LONG); // TODO the bitset needs memory too
+ bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_LONG); // TODO the bitset
+ // needs memory too
+ datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
+ DATA_EXTENSION));
+ CodecUtil.writeHeader(datOut, CODEC_NAME, VERSION_CURRENT);
}
@Override
@@ -81,50 +86,55 @@ class PackedIntsImpl {
lastDocId = docID;
if (docID >= docToValue.length) {
- final long len = docToValue.length ;
+ final long len = docToValue.length;
docToValue = ArrayUtil.grow(docToValue, 1 + docID);
defaultValues.ensureCapacity(docToValue.length);
- bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_LONG * ((docToValue.length) - len));
+ bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_LONG
+ * ((docToValue.length) - len));
}
docToValue[docID] = v;
}
@Override
public synchronized void finish(int docCount) throws IOException {
- if (!started)
- return;
- final IndexOutput datOut = dir.createOutput(IndexFileNames
- .segmentFileName(id, "", DATA_EXTENSION));
- CodecUtil.writeHeader(datOut, CODEC_NAME, VERSION_CURRENT);
+ try {
+ if (!started) {
+ minValue = maxValue = 0;
+ }
+ // TODO -- long can't work right since it's signed
+ datOut.writeLong(minValue);
+ // write a default value to recognize docs without a value for that
+ // field
+ final long defaultValue = ++maxValue - minValue;
+ datOut.writeLong(defaultValue);
+ PackedInts.Writer w = PackedInts.getWriter(datOut, docCount,
+ PackedInts.bitsRequired(maxValue - minValue));
+ final int firstDoc = defaultValues.nextSetBit(0);
+ lastDocId++;
+ if(firstDoc != -1) {
+ for (int i = 0; i < firstDoc; i++) {
+ w.add(defaultValue); // fill with defaults until first bit set
+ }
- // TODO -- long can't work right since it's signed
- datOut.writeLong(minValue);
- // write a default value to recognize docs without a value for that field
- final long defaultValue = ++maxValue - minValue;
- datOut.writeLong(defaultValue);
- PackedInts.Writer w = PackedInts.getWriter(datOut, docCount, PackedInts
- .bitsRequired(maxValue - minValue));
- final int firstDoc = defaultValues.nextSetBit(0);
- assert firstDoc >= 0; // we have at lest one value!
- for (int i = 0; i < firstDoc; i++) {
- w.add(defaultValue); // fill with defaults until first bit set
- }
- lastDocId++;
- for (int i = firstDoc; i < lastDocId;) {
- w.add(docToValue[i] - minValue);
- final int nextValue = defaultValues.nextSetBit(i);
- for (i++; i < nextValue; i++) {
- w.add(defaultValue); // fill all gaps
+ for (int i = firstDoc; i < lastDocId;) {
+ w.add(docToValue[i] - minValue);
+ final int nextValue = defaultValues.nextSetBit(i);
+ for (i++; i < nextValue; i++) {
+ w.add(defaultValue); // fill all gaps
+ }
+ }
}
+ for (int i = lastDocId; i < docCount; i++) {
+ w.add(defaultValue);
+ }
+ w.finish();
+ } finally {
+ datOut.close();
+ bytesUsed
+ .addAndGet(-(RamUsageEstimator.NUM_BYTES_LONG * docToValue.length));
+ docToValue = null;
}
- for (int i = lastDocId; i < docCount; i++) {
- w.add(defaultValue);
- }
- w.finish();
- datOut.close();
- bytesUsed.addAndGet(-(RamUsageEstimator.NUM_BYTES_LONG * docToValue.length ));
- docToValue = null;
-
+
}
@Override
@@ -139,7 +149,7 @@ class PackedIntsImpl {
@Override
public void add(int docID, PerDocFieldValues docValues) throws IOException {
- add(docID, docValues.getInt());
+ add(docID, docValues.getInt());
}
@Override
@@ -192,7 +202,8 @@ class PackedIntsImpl {
}
@Override
- public DocValuesEnum getEnum(AttributeSource attrSource) throws IOException {
+ public DocValuesEnum getEnum(AttributeSource attrSource)
+ throws IOException {
final MissingValue missing = getMissing();
return new SourceEnum(attrSource, type(), this, values.size()) {
@Override