You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2011/10/01 05:05:07 UTC
svn commit: r1177888 [3/16] - in /lucene/dev/branches/lucene2621: ./
dev-tools/eclipse/ dev-tools/idea/lucene/contrib/ dev-tools/maven/ lucene/
lucene/contrib/ lucene/contrib/demo/src/java/org/apache/lucene/demo/
lucene/contrib/demo/src/java/org/apache...
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/Field.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/Field.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/Field.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/Field.java Sat Oct 1 03:04:53 2011
@@ -60,7 +60,7 @@ public class Field implements IndexableF
this.type = type;
}
- public Field(String name, IndexableFieldType type, Reader reader) {
+ public Field(String name, Reader reader, IndexableFieldType type) {
if (name == null) {
throw new NullPointerException("name cannot be null");
}
@@ -76,7 +76,7 @@ public class Field implements IndexableF
this.type = type;
}
- public Field(String name, IndexableFieldType type, TokenStream tokenStream) {
+ public Field(String name, TokenStream tokenStream, IndexableFieldType type) {
if (name == null) {
throw new NullPointerException("name cannot be null");
}
@@ -93,15 +93,15 @@ public class Field implements IndexableF
this.type = type;
}
- public Field(String name, IndexableFieldType type, byte[] value) {
- this(name, type, value, 0, value.length);
+ public Field(String name, byte[] value, IndexableFieldType type) {
+ this(name, value, 0, value.length, type);
}
- public Field(String name, IndexableFieldType type, byte[] value, int offset, int length) {
- this(name, type, new BytesRef(value, offset, length));
+ public Field(String name, byte[] value, int offset, int length, IndexableFieldType type) {
+ this(name, new BytesRef(value, offset, length), type);
}
- public Field(String name, IndexableFieldType type, BytesRef bytes) {
+ public Field(String name, BytesRef bytes, IndexableFieldType type) {
if (type.indexed() && !type.tokenized()) {
throw new IllegalArgumentException("Non-tokenized fields must use String values");
}
@@ -111,7 +111,7 @@ public class Field implements IndexableF
this.name = name;
}
- public Field(String name, IndexableFieldType type, String value) {
+ public Field(String name, String value, IndexableFieldType type) {
if (name == null) {
throw new IllegalArgumentException("name cannot be null");
}
@@ -350,9 +350,9 @@ public class Field implements IndexableF
if (tokenStream != null) {
return tokenStream;
} else if (readerValue() != null) {
- return analyzer.reusableTokenStream(name(), readerValue());
+ return analyzer.tokenStream(name(), readerValue());
} else if (stringValue() != null) {
- return analyzer.reusableTokenStream(name(), new StringReader(stringValue()));
+ return analyzer.tokenStream(name(), new StringReader(stringValue()));
}
throw new IllegalArgumentException("Field must have either TokenStream, String or Reader value");
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/StringField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/StringField.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/StringField.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/StringField.java Sat Oct 1 03:04:53 2011
@@ -54,7 +54,7 @@ public final class StringField extends F
/** Creates a new un-stored StringField */
public StringField(String name, String value) {
- super(name, TYPE_UNSTORED, value);
+ super(name, value, TYPE_UNSTORED);
}
@Override
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/TextField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/TextField.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/TextField.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/document/TextField.java Sat Oct 1 03:04:53 2011
@@ -50,16 +50,16 @@ public final class TextField extends Fie
/** Creates a new un-stored TextField */
public TextField(String name, Reader reader) {
- super(name, TextField.TYPE_UNSTORED, reader);
+ super(name, reader, TextField.TYPE_UNSTORED);
}
/** Creates a new un-stored TextField */
public TextField(String name, String value) {
- super(name, TextField.TYPE_UNSTORED, value);
+ super(name, value, TextField.TYPE_UNSTORED);
}
/** Creates a new un-stored TextField */
public TextField(String name, TokenStream stream) {
- super(name, TextField.TYPE_UNSTORED, stream);
+ super(name, stream, TextField.TYPE_UNSTORED);
}
}
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java Sat Oct 1 03:04:53 2011
@@ -83,9 +83,9 @@ final class DocFieldProcessor extends Do
// FieldInfo.storePayload.
final String fileName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.FIELD_INFOS_EXTENSION);
state.fieldInfos.write(state.directory, fileName);
- for (DocValuesConsumer consumers : docValues.values()) {
- consumers.finish(state.numDocs);
- };
+ for (DocValuesConsumerAndDocID consumers : docValues.values()) {
+ consumers.docValuesConsumer.finish(state.numDocs);
+ }
// close perDocConsumer during flush to ensure all files are flushed due to PerCodec CFS
IOUtils.close(perDocConsumers.values());
}
@@ -297,14 +297,28 @@ final class DocFieldProcessor extends Do
}
}
- final private Map<String, DocValuesConsumer> docValues = new HashMap<String, DocValuesConsumer>();
+ private static class DocValuesConsumerAndDocID {
+ public int docID;
+ final DocValuesConsumer docValuesConsumer;
+
+ public DocValuesConsumerAndDocID(DocValuesConsumer docValuesConsumer) {
+ this.docValuesConsumer = docValuesConsumer;
+ }
+ }
+
+ final private Map<String, DocValuesConsumerAndDocID> docValues = new HashMap<String, DocValuesConsumerAndDocID>();
final private Map<Integer, PerDocConsumer> perDocConsumers = new HashMap<Integer, PerDocConsumer>();
DocValuesConsumer docValuesConsumer(DocState docState, FieldInfo fieldInfo)
throws IOException {
- DocValuesConsumer docValuesConsumer = docValues.get(fieldInfo.name);
- if (docValuesConsumer != null) {
- return docValuesConsumer;
+ DocValuesConsumerAndDocID docValuesConsumerAndDocID = docValues.get(fieldInfo.name);
+ if (docValuesConsumerAndDocID != null) {
+ if (docState.docID == docValuesConsumerAndDocID.docID) {
+ throw new IllegalArgumentException("IndexDocValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed, per field)");
+ }
+ assert docValuesConsumerAndDocID.docID < docState.docID;
+ docValuesConsumerAndDocID.docID = docState.docID;
+ return docValuesConsumerAndDocID.docValuesConsumer;
}
PerDocConsumer perDocConsumer = perDocConsumers.get(fieldInfo.getCodecId());
if (perDocConsumer == null) {
@@ -316,6 +330,7 @@ final class DocFieldProcessor extends Do
perDocConsumers.put(Integer.valueOf(fieldInfo.getCodecId()), perDocConsumer);
}
boolean success = false;
+ DocValuesConsumer docValuesConsumer = null;
try {
docValuesConsumer = perDocConsumer.addValuesField(fieldInfo);
fieldInfo.commitDocValues();
@@ -325,7 +340,10 @@ final class DocFieldProcessor extends Do
fieldInfo.revertUncommitted();
}
}
- docValues.put(fieldInfo.name, docValuesConsumer);
+
+ docValuesConsumerAndDocID = new DocValuesConsumerAndDocID(docValuesConsumer);
+ docValuesConsumerAndDocID.docID = docState.docID;
+ docValues.put(fieldInfo.name, docValuesConsumerAndDocID);
return docValuesConsumer;
}
}
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexReader.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexReader.java Sat Oct 1 03:04:53 2011
@@ -27,6 +27,7 @@ import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DocumentStoredFieldVisitor;
import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.codecs.PerDocValues;
@@ -199,11 +200,45 @@ public abstract class IndexReader implem
* references.
*
* @see #decRef
+ * @see #tryIncRef
*/
public void incRef() {
ensureOpen();
refCount.incrementAndGet();
}
+
+ /**
+ * Expert: increments the refCount of this IndexReader
+ * instance only if the IndexReader has not been closed yet
+ * and returns <code>true</code> iff the refCount was
+ * successfully incremented, otherwise <code>false</code>.
+ * If this method returns <code>false</code> the reader is either
+ * already closed or is currently been closed. Either way this
+ * reader instance shouldn't be used by an application unless
+ * <code>true</code> is returned.
+ * <p>
+ * RefCounts are used to determine when a
+ * reader can be closed safely, i.e. as soon as there are
+ * no more references. Be sure to always call a
+ * corresponding {@link #decRef}, in a finally clause;
+ * otherwise the reader may never be closed. Note that
+ * {@link #close} simply calls decRef(), which means that
+ * the IndexReader will not really be closed until {@link
+ * #decRef} has been called for all outstanding
+ * references.
+ *
+ * @see #decRef
+ * @see #incRef
+ */
+ public boolean tryIncRef() {
+ int count;
+ while ((count = refCount.get()) > 0) {
+ if(refCount.compareAndSet(count, count+1)) {
+ return true;
+ }
+ }
+ return false;
+ }
/** {@inheritDoc} */
@Override
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexWriter.java Sat Oct 1 03:04:53 2011
@@ -1631,7 +1631,7 @@ public class IndexWriter implements Clos
* default merge policy, but individual merge policies may implement
* optimize in different ways.
*
- * <p> Optimize is a fairly costly operation, so you
+ * <p> Optimize is a very costly operation, so you
* should only do it if your search performance really
* requires it. Many search applications do fine never
* calling optimize. </p>
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java Sat Oct 1 03:04:53 2011
@@ -188,12 +188,12 @@ public class PersistentSnapshotDeletionP
Document d = new Document();
FieldType ft = new FieldType();
ft.setStored(true);
- d.add(new Field(SNAPSHOTS_ID, ft, ""));
+ d.add(new Field(SNAPSHOTS_ID, "", ft));
for (Entry<String, String> e : super.getSnapshots().entrySet()) {
- d.add(new Field(e.getKey(), ft, e.getValue()));
+ d.add(new Field(e.getKey(), e.getValue(), ft));
}
if (id != null) {
- d.add(new Field(id, ft, segment));
+ d.add(new Field(id, segment, ft));
}
writer.addDocument(d);
writer.commit();
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/StoredFieldVisitor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/StoredFieldVisitor.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/StoredFieldVisitor.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/StoredFieldVisitor.java Sat Oct 1 03:04:53 2011
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DocumentStoredFieldVisitor;
import org.apache.lucene.store.IndexInput;
/**
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/Codec.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/Codec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/Codec.java Sat Oct 1 03:04:53 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.index.codecs;
*/
import java.io.IOException;
-import java.util.Comparator;
import java.util.Set;
import org.apache.lucene.index.PerDocWriteState;
@@ -26,7 +25,6 @@ import org.apache.lucene.index.SegmentIn
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
/** @lucene.experimental */
public abstract class Codec {
@@ -34,22 +32,9 @@ public abstract class Codec {
/** Unique name that's used to retrieve this codec when
* reading the index */
public final String name;
- protected final boolean dvUseCompoundFile;
- protected final Comparator<BytesRef> docValuesSortComparator;
protected Codec(String name) {
- this(name, true);
- }
-
- protected Codec(String name, boolean docValuesUseCompoundFile) {
- this(name, docValuesUseCompoundFile, BytesRef.getUTF8SortedAsUnicodeComparator());
- }
-
- protected Codec(String name, boolean docValuesUseCompoundFile,
- Comparator<BytesRef> docValuesSortComparator) {
this.name = name;
- this.dvUseCompoundFile = docValuesUseCompoundFile;
- this.docValuesSortComparator = docValuesSortComparator;
}
/** Writes a new segment */
@@ -87,38 +72,6 @@ public abstract class Codec {
/** Records all file extensions this codec uses */
public abstract void getExtensions(Set<String> extensions);
-
- /**
- * Returns <code>true</code> iff compound file should be used for
- * IndexDocValues, otherwise <code>false</code>. The default is
- * <code>true</code>.
- * <p>
- * NOTE: To change the default value you need to subclass a {@link Codec} with
- * a distinct name since this value is final and should not be changed to
- * prevent the risk of a index corruption. This setting is private to a
- * {@link Codec}. If you intend to change this value on an existing
- * {@link Codec} re-indexing is required.
- *
- * @return <code>true</code> iff compound file should be used for
- * IndexDocValues, otherwise <code>false</code>.
- */
- public boolean getDocValuesUseCFS() {
- return dvUseCompoundFile;
- }
-
- /**
- * Returns the {@link BytesRef} comparator for sorted IndexDocValue variants.
- * The default is {@link BytesRef#getUTF8SortedAsUnicodeComparator()}.
- * <p>
- * NOTE: To change the default value you need to subclass a {@link Codec} with
- * a distinct name since this value is final and should not be changed to
- * prevent the risk of a index corruption. This setting is private to a
- * {@link Codec}. If you intend to change this value on an existing
- * {@link Codec} re-indexing is required.
- */
- public Comparator<BytesRef> getDocValuesSortComparator() {
- return docValuesSortComparator;
- }
@Override
public String toString() {
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java Sat Oct 1 03:04:53 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.index.codecs;
*/
import java.io.IOException;
-import java.util.Comparator;
import java.util.Set;
import org.apache.lucene.index.FieldInfo;
@@ -26,113 +25,51 @@ import org.apache.lucene.index.FieldInfo
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.values.Writer;
+import org.apache.lucene.index.codecs.DocValuesWriterBase;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.Counter;
/**
- *
+ * Default PerDocConsumer implementation that uses compound file.
* @lucene.experimental
*/
-public class DefaultDocValuesConsumer extends PerDocConsumer {
- private final String segmentName;
- private final int codecId;
+public class DefaultDocValuesConsumer extends DocValuesWriterBase {
private final Directory directory;
- private final Counter bytesUsed;
- private final Comparator<BytesRef> comparator;
- private boolean useCompoundFile;
- private final IOContext context;
- public DefaultDocValuesConsumer(PerDocWriteState state, Comparator<BytesRef> comparator, boolean useCompoundFile) throws IOException {
- this.segmentName = state.segmentName;
- this.codecId = state.codecId;
- this.bytesUsed = state.bytesUsed;
- this.context = state.context;
+ public DefaultDocValuesConsumer(PerDocWriteState state) throws IOException {
+ super(state);
//TODO maybe we should enable a global CFS that all codecs can pull on demand to further reduce the number of files?
- this.directory = useCompoundFile ? new CompoundFileDirectory(state.directory,
- IndexFileNames.segmentFileName(segmentName, codecId,
- IndexFileNames.COMPOUND_FILE_EXTENSION), context, true) : state.directory;
- this.comparator = comparator;
- this.useCompoundFile = useCompoundFile;
+ this.directory = new CompoundFileDirectory(state.directory,
+ IndexFileNames.segmentFileName(state.segmentName, state.codecId,
+ IndexFileNames.COMPOUND_FILE_EXTENSION), state.context, true);
}
-
- public void close() throws IOException {
- if (useCompoundFile) {
- this.directory.close();
- }
+
+ @Override
+ protected Directory getDirectory() {
+ return directory;
}
@Override
- public DocValuesConsumer addValuesField(FieldInfo field) throws IOException {
- return Writer.create(field.getDocValues(),
- docValuesId(segmentName, codecId, field.number),
- directory, comparator, bytesUsed, context);
+ public void close() throws IOException {
+ this.directory.close();
}
-
+
@SuppressWarnings("fallthrough")
- public static void files(Directory dir, SegmentInfo segmentInfo, int codecId,
- Set<String> files, boolean useCompoundFile) throws IOException {
+ public static void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files) throws IOException {
FieldInfos fieldInfos = segmentInfo.getFieldInfos();
for (FieldInfo fieldInfo : fieldInfos) {
if (fieldInfo.getCodecId() == codecId && fieldInfo.hasDocValues()) {
- String filename = docValuesId(segmentInfo.name, codecId,
- fieldInfo.number);
- if (useCompoundFile) {
- files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION));
- files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
- assert dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
- assert dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION));
- return;
- } else {
- switch (fieldInfo.getDocValues()) {
- case BYTES_FIXED_DEREF:
- case BYTES_VAR_DEREF:
- case BYTES_VAR_SORTED:
- case BYTES_FIXED_SORTED:
- case BYTES_VAR_STRAIGHT:
- files.add(IndexFileNames.segmentFileName(filename, "",
- Writer.INDEX_EXTENSION));
- assert dir.fileExists(IndexFileNames.segmentFileName(filename, "",
- Writer.INDEX_EXTENSION));
- // until here all types use an index
- case BYTES_FIXED_STRAIGHT:
- case FLOAT_32:
- case FLOAT_64:
- case VAR_INTS:
- case FIXED_INTS_16:
- case FIXED_INTS_32:
- case FIXED_INTS_64:
- case FIXED_INTS_8:
- files.add(IndexFileNames.segmentFileName(filename, "",
- Writer.DATA_EXTENSION));
- assert dir.fileExists(IndexFileNames.segmentFileName(filename, "",
- Writer.DATA_EXTENSION));
- break;
-
- default:
- assert false;
- }
- }
+ files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION));
+ files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
+ assert dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
+ assert dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION));
+ return;
}
}
}
-
- static String docValuesId(String segmentsName, int codecID, int fieldId) {
- return segmentsName + "_" + codecID + "-" + fieldId;
- }
-
- public static void getDocValuesExtensions(Set<String> extensions, boolean useCompoundFile) {
- if (useCompoundFile) {
- extensions.add(IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
- extensions.add(IndexFileNames.COMPOUND_FILE_EXTENSION);
- } else {
- extensions.add(Writer.DATA_EXTENSION);
- extensions.add(Writer.INDEX_EXTENSION);
- }
+ public static void getExtensions(Set<String> extensions) {
+ extensions.add(IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
+ extensions.add(IndexFileNames.COMPOUND_FILE_EXTENSION);
}
-
}
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java Sat Oct 1 03:04:53 2011
@@ -16,184 +16,50 @@ package org.apache.lucene.index.codecs;
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
-import java.util.Comparator;
+import java.util.Map;
import java.util.TreeMap;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.values.Bytes;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.codecs.DocValuesReaderBase;
import org.apache.lucene.index.values.IndexDocValues;
-import org.apache.lucene.index.values.Floats;
-import org.apache.lucene.index.values.Ints;
-import org.apache.lucene.index.values.ValueType;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
-import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
/**
- * Abstract base class for FieldsProducer implementations supporting
- * {@link IndexDocValues}.
- *
+ * Default PerDocValues implementation that uses compound file.
* @lucene.experimental
*/
-public class DefaultDocValuesProducer extends PerDocValues {
-
+public class DefaultDocValuesProducer extends DocValuesReaderBase {
protected final TreeMap<String, IndexDocValues> docValues;
- private final boolean useCompoundFile;
- private final Closeable cfs;
- private final Comparator<BytesRef> sortComparator;
+ private final Directory cfs;
/**
- *
* Creates a new {@link DefaultDocValuesProducer} instance and loads all
* {@link IndexDocValues} instances for this segment and codec.
- *
- * @param si
- * the segment info to load the {@link IndexDocValues} for.
- * @param dir
- * the directory to load the {@link IndexDocValues} from.
- * @param fieldInfo
- * the {@link FieldInfos}
- * @param codecId
- * the codec ID
- * @param useCompoundFile
- * if <code>true</code> this producer opens a compound file to read
- * IndexDocValues fields, otherwise each field defines its own set of
- * files.
- * @param sortComparator
- * defines the sort order for sorted IndexDocValues variants
- * @throws IOException
- * if an {@link IOException} occurs
*/
- public DefaultDocValuesProducer(SegmentInfo si, Directory dir,
- FieldInfos fieldInfo, int codecId, boolean useCompoundFile, Comparator<BytesRef> sortComparator, IOContext context) throws IOException {
- this.useCompoundFile = useCompoundFile;
- this.sortComparator = sortComparator;
- final Directory directory;
- if (useCompoundFile) {
- cfs = directory = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(si.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
- } else {
- cfs = null;
- directory = dir;
- }
- docValues = load(fieldInfo, si.name, si.docCount, directory, codecId, context);
- }
-
- /**
- * Returns a {@link IndexDocValues} instance for the given field name or
- * <code>null</code> if this field has no {@link IndexDocValues}.
- */
- @Override
- public IndexDocValues docValues(String field) throws IOException {
- return docValues.get(field);
- }
-
- // Only opens files... doesn't actually load any values
- protected TreeMap<String, IndexDocValues> load(FieldInfos fieldInfos,
- String segment, int docCount, Directory dir, int codecId, IOContext context)
- throws IOException {
- TreeMap<String, IndexDocValues> values = new TreeMap<String, IndexDocValues>();
- boolean success = false;
- try {
-
- for (FieldInfo fieldInfo : fieldInfos) {
- if (codecId == fieldInfo.getCodecId() && fieldInfo.hasDocValues()) {
- final String field = fieldInfo.name;
- // TODO can we have a compound file per segment and codec for
- // docvalues?
- final String id = DefaultDocValuesConsumer.docValuesId(segment,
- codecId, fieldInfo.number);
- values.put(field,
- loadDocValues(docCount, dir, id, fieldInfo.getDocValues(), sortComparator, context));
- }
- }
- success = true;
- } finally {
- if (!success) {
- // if we fail we must close all opened resources if there are any
- closeInternal(values.values());
- }
- }
- return values;
+ public DefaultDocValuesProducer(SegmentReadState state) throws IOException {
+ cfs = new CompoundFileDirectory(state.dir,
+ IndexFileNames.segmentFileName(state.segmentInfo.name, state.codecId, IndexFileNames.COMPOUND_FILE_EXTENSION),
+ state.context, false);
+ docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.docCount, cfs, state.codecId, state.context);
}
-
- /**
- * Loads a {@link IndexDocValues} instance depending on the given {@link ValueType}.
- * Codecs that use different implementations for a certain {@link ValueType} can
- * simply override this method and return their custom implementations.
- *
- * @param docCount
- * number of documents in the segment
- * @param dir
- * the {@link Directory} to load the {@link IndexDocValues} from
- * @param id
- * the unique file ID within the segment
- * @param type
- * the type to load
- * @param sortComparator byte comparator used by sorted variants
- * @return a {@link IndexDocValues} instance for the given type
- * @throws IOException
- * if an {@link IOException} occurs
- * @throws IllegalArgumentException
- * if the given {@link ValueType} is not supported
- */
- protected IndexDocValues loadDocValues(int docCount, Directory dir, String id,
- ValueType type, Comparator<BytesRef> sortComparator, IOContext context) throws IOException {
- switch (type) {
- case FIXED_INTS_16:
- case FIXED_INTS_32:
- case FIXED_INTS_64:
- case FIXED_INTS_8:
- case VAR_INTS:
- return Ints.getValues(dir, id, docCount, context);
- case FLOAT_32:
- return Floats.getValues(dir, id, docCount, context);
- case FLOAT_64:
- return Floats.getValues(dir, id, docCount, context);
- case BYTES_FIXED_STRAIGHT:
- return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount, sortComparator, context);
- case BYTES_FIXED_DEREF:
- return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount, sortComparator, context);
- case BYTES_FIXED_SORTED:
- return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount, sortComparator, context);
- case BYTES_VAR_STRAIGHT:
- return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount, sortComparator, context);
- case BYTES_VAR_DEREF:
- return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount, sortComparator, context);
- case BYTES_VAR_SORTED:
- return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount, sortComparator, context);
- default:
- throw new IllegalStateException("unrecognized index values mode " + type);
- }
- }
-
- public void close() throws IOException {
- closeInternal(docValues.values());
- }
-
- private void closeInternal(Collection<? extends Closeable> closeables) throws IOException {
- final Collection<? extends Closeable> toClose;
- if (useCompoundFile) {
- final ArrayList<Closeable> list = new ArrayList<Closeable>(closeables);
- list.add(cfs);
- toClose = list;
- } else {
- toClose = closeables;
- }
- IOUtils.close(toClose);
+ @Override
+ protected Map<String,IndexDocValues> docValues() {
+ return docValues;
}
@Override
- public Collection<String> fields() {
- return docValues.keySet();
+ protected void closeInternal(Collection<? extends Closeable> closeables) throws IOException {
+ final ArrayList<Closeable> list = new ArrayList<Closeable>(closeables);
+ list.add(cfs);
+ IOUtils.close(list);
}
}
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java Sat Oct 1 03:04:53 2011
@@ -51,5 +51,4 @@ public abstract class PerDocValues imple
* Returns all fields this {@link PerDocValues} contains values for.
*/
public abstract Collection<String> fields();
-
}
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java Sat Oct 1 03:04:53 2011
@@ -787,22 +787,22 @@ public class MemoryCodec extends Codec {
@Override
public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, EXTENSION));
- DefaultDocValuesConsumer.files(dir, segmentInfo, id, files, getDocValuesUseCFS());
+ DefaultDocValuesConsumer.files(dir, segmentInfo, id, files);
}
@Override
public void getExtensions(Set<String> extensions) {
extensions.add(EXTENSION);
- DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());
+ DefaultDocValuesConsumer.getExtensions(extensions);
}
@Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
- return new DefaultDocValuesConsumer(state, getDocValuesSortComparator(), getDocValuesUseCFS());
+ return new DefaultDocValuesConsumer(state);
}
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
- return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), IOContext.READONCE);
+ return new DefaultDocValuesProducer(state);
}
}
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java Sat Oct 1 03:04:53 2011
@@ -134,22 +134,22 @@ public class PulsingCodec extends Codec
public void files(Directory dir, SegmentInfo segmentInfo, int codecID, Set<String> files) throws IOException {
StandardPostingsReader.files(dir, segmentInfo, codecID, files);
BlockTreeTermsReader.files(dir, segmentInfo, codecID, files);
- DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files, getDocValuesUseCFS());
+ DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files);
}
@Override
public void getExtensions(Set<String> extensions) {
StandardCodec.getStandardExtensions(extensions);
- DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());
+ DefaultDocValuesConsumer.getExtensions(extensions);
}
@Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
- return new DefaultDocValuesConsumer(state, getDocValuesSortComparator(), getDocValuesUseCFS());
+ return new DefaultDocValuesConsumer(state);
}
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
- return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
+ return new DefaultDocValuesProducer(state);
}
}
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java Sat Oct 1 03:04:53 2011
@@ -69,23 +69,23 @@ public class SimpleTextCodec extends Cod
@Override
public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
files.add(getPostingsFileName(segmentInfo.name, id));
- DefaultDocValuesConsumer.files(dir, segmentInfo, id, files, getDocValuesUseCFS());
+ DefaultDocValuesConsumer.files(dir, segmentInfo, id, files);
}
@Override
public void getExtensions(Set<String> extensions) {
extensions.add(POSTINGS_EXTENSION);
- DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());
+ DefaultDocValuesConsumer.getExtensions(extensions);
}
// TODO: would be great if these used a plain text impl
@Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
- return new DefaultDocValuesConsumer(state, getDocValuesSortComparator(), getDocValuesUseCFS());
+ return new DefaultDocValuesConsumer(state);
}
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
- return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
+ return new DefaultDocValuesProducer(state);
}
}
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java Sat Oct 1 03:04:53 2011
@@ -110,19 +110,19 @@ public class StandardCodec extends Codec
public void files(Directory dir, SegmentInfo segmentInfo, int codecID, Set<String> files) throws IOException {
StandardPostingsReader.files(dir, segmentInfo, codecID, files);
BlockTreeTermsReader.files(dir, segmentInfo, codecID, files);
- DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files, getDocValuesUseCFS());
+ DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files);
}
@Override
public void getExtensions(Set<String> extensions) {
getStandardExtensions(extensions);
- DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());
}
public static void getStandardExtensions(Set<String> extensions) {
extensions.add(FREQ_EXTENSION);
extensions.add(PROX_EXTENSION);
BlockTreeTermsReader.getExtensions(extensions);
+ DefaultDocValuesConsumer.getExtensions(extensions);
}
@Override
@@ -132,11 +132,11 @@ public class StandardCodec extends Codec
@Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
- return new DefaultDocValuesConsumer(state, getDocValuesSortComparator(), getDocValuesUseCFS());
+ return new DefaultDocValuesConsumer(state);
}
@Override
public PerDocValues docsProducer(SegmentReadState state) throws IOException {
- return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
+ return new DefaultDocValuesProducer(state);
}
}
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Bytes.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Bytes.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Bytes.java Sat Oct 1 03:04:53 2011
@@ -27,16 +27,25 @@ import org.apache.lucene.index.IndexFile
import org.apache.lucene.index.values.IndexDocValues.SortedSource;
import org.apache.lucene.index.values.IndexDocValues.Source;
import org.apache.lucene.index.values.IndexDocValues.SourceEnum;
+import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefHash;
import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.ByteBlockPool.Allocator;
+import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
+import org.apache.lucene.util.BytesRefHash.TrackingDirectBytesStartArray;
+import org.apache.lucene.util.packed.PackedInts;
/**
* Provides concrete Writer/Reader implementations for <tt>byte[]</tt> value per
@@ -185,16 +194,18 @@ public final class Bytes {
}
// TODO open up this API?
- static abstract class BytesBaseSource extends Source {
+ static abstract class BytesSourceBase extends Source {
private final PagedBytes pagedBytes;
+ private final ValueType type;
protected final IndexInput datIn;
protected final IndexInput idxIn;
protected final static int PAGED_BYTES_BITS = 15;
protected final PagedBytes.Reader data;
protected final long totalLengthInBytes;
+
- protected BytesBaseSource(IndexInput datIn, IndexInput idxIn,
- PagedBytes pagedBytes, long bytesToRead) throws IOException {
+ protected BytesSourceBase(IndexInput datIn, IndexInput idxIn,
+ PagedBytes pagedBytes, long bytesToRead, ValueType type) throws IOException {
assert bytesToRead <= datIn.length() : " file size is less than the expected size diff: "
+ (bytesToRead - datIn.length()) + " pos: " + datIn.getFilePointer();
this.datIn = datIn;
@@ -203,6 +214,7 @@ public final class Bytes {
this.pagedBytes.copy(datIn, bytesToRead);
data = pagedBytes.freeze(true);
this.idxIn = idxIn;
+ this.type = type;
}
public void close() throws IOException {
@@ -220,6 +232,17 @@ public final class Bytes {
}
}
}
+
+ @Override
+ public ValueType type() {
+ return type;
+ }
+
+
+ @Override
+ public int getValueCount() {
+ throw new UnsupportedOperationException();
+ }
/**
* Returns one greater than the largest possible document number.
@@ -245,18 +268,46 @@ public final class Bytes {
}
}
+
+ static abstract class DerefBytesSourceBase extends BytesSourceBase {
+ protected final PackedInts.Reader addresses;
+ public DerefBytesSourceBase(IndexInput datIn, IndexInput idxIn, long bytesToRead, ValueType type) throws IOException {
+ super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), bytesToRead, type);
+ addresses = PackedInts.getReader(idxIn);
+ }
+
+ @Override
+ public int getValueCount() {
+ return addresses.size();
+ }
+
+ @Override
+ protected int maxDoc() {
+ return addresses.size();
+ }
+
+ }
- static abstract class BytesBaseSortedSource extends SortedSource {
+ static abstract class BytesSortedSourceBase extends SortedSource {
+ private final PagedBytes pagedBytes;
+ private final Comparator<BytesRef> comp;
+ protected final PackedInts.Reader docToOrdIndex;
+ private final ValueType type;
+
protected final IndexInput datIn;
protected final IndexInput idxIn;
protected final BytesRef defaultValue = new BytesRef();
protected final static int PAGED_BYTES_BITS = 15;
- private final PagedBytes pagedBytes;
protected final PagedBytes.Reader data;
- private final Comparator<BytesRef> comp;
+
- protected BytesBaseSortedSource(IndexInput datIn, IndexInput idxIn,
- Comparator<BytesRef> comp, PagedBytes pagedBytes, long bytesToRead)
+ protected BytesSortedSourceBase(IndexInput datIn, IndexInput idxIn,
+ Comparator<BytesRef> comp, long bytesToRead, ValueType type) throws IOException {
+ this(datIn, idxIn, comp, new PagedBytes(PAGED_BYTES_BITS), bytesToRead, type);
+ }
+
+ protected BytesSortedSourceBase(IndexInput datIn, IndexInput idxIn,
+ Comparator<BytesRef> comp, PagedBytes pagedBytes, long bytesToRead,ValueType type)
throws IOException {
assert bytesToRead <= datIn.length() : " file size is less than the expected size diff: "
+ (bytesToRead - datIn.length()) + " pos: " + datIn.getFilePointer();
@@ -267,8 +318,15 @@ public final class Bytes {
this.idxIn = idxIn;
this.comp = comp == null ? BytesRef.getUTF8SortedAsUnicodeComparator()
: comp;
+ docToOrdIndex = PackedInts.getReader(idxIn);
+ this.type = type;
}
+
+ @Override
+ public int ord(int docID) {
+ return (int) docToOrdIndex.get(docID) -1;
+ }
@Override
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
@@ -277,22 +335,15 @@ public final class Bytes {
}
protected void closeIndexInput() throws IOException {
- try {
- if (datIn != null) {
- datIn.close();
- }
- } finally {
- if (idxIn != null) {// if straight
- idxIn.close();
- }
- }
+ IOUtils.close(datIn, idxIn);
}
-
+
/**
* Returns the largest doc id + 1 in this doc values source
*/
- protected abstract int maxDoc();
-
+ public int maxDoc() {
+ return docToOrdIndex.size();
+ }
/**
* Copies the value for the given ord to the given {@link BytesRef} and
* returns it.
@@ -336,6 +387,11 @@ public final class Bytes {
}
};
}
+
+ @Override
+ public ValueType type() {
+ return type;
+ }
}
// TODO: open up this API?!
@@ -359,7 +415,7 @@ public final class Bytes {
this.context = context;
}
- protected IndexOutput getDataOut() throws IOException {
+ protected IndexOutput getOrCreateDataOut() throws IOException {
if (datOut == null) {
boolean success = false;
try {
@@ -375,8 +431,16 @@ public final class Bytes {
}
return datOut;
}
+
+ protected IndexOutput getIndexOut() {
+ return idxOut;
+ }
+
+ protected IndexOutput getDataOut() {
+ return datOut;
+ }
- protected IndexOutput getIndexOut() throws IOException {
+ protected IndexOutput getOrCreateIndexOut() throws IOException {
boolean success = false;
try {
if (idxOut == null) {
@@ -503,5 +567,223 @@ public final class Bytes {
}
}
}
+
+ static abstract class DerefBytesWriterBase extends BytesWriterBase {
+ protected int size = -1;
+ protected int[] docToEntry;
+ protected final BytesRefHash hash;
+
+ protected DerefBytesWriterBase(Directory dir, String id, String codecName,
+ int codecVersion, Counter bytesUsed, IOContext context)
+ throws IOException {
+ this(dir, id, codecName, codecVersion, new DirectTrackingAllocator(
+ ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed), bytesUsed, context);
+ }
+
+ protected DerefBytesWriterBase(Directory dir, String id, String codecName, int codecVersion, Allocator allocator,
+ Counter bytesUsed, IOContext context) throws IOException {
+ super(dir, id, codecName, codecVersion, bytesUsed, context);
+ hash = new BytesRefHash(new ByteBlockPool(allocator),
+ BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
+ BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
+ docToEntry = new int[1];
+ bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
+ }
+
+ protected static int writePrefixLength(DataOutput datOut, BytesRef bytes)
+ throws IOException {
+ if (bytes.length < 128) {
+ datOut.writeByte((byte) bytes.length);
+ return 1;
+ } else {
+ datOut.writeByte((byte) (0x80 | (bytes.length >> 8)));
+ datOut.writeByte((byte) (bytes.length & 0xff));
+ return 2;
+ }
+ }
+
+ @Override
+ public void add(int docID, BytesRef bytes) throws IOException {
+ if (bytes.length == 0) { // default value - skip it
+ return;
+ }
+ checkSize(bytes);
+ int ord = hash.add(bytes);
+ if (ord < 0) {
+ ord = (-ord) - 1;
+ }
+ if (docID >= docToEntry.length) {
+ final int size = docToEntry.length;
+ docToEntry = ArrayUtil.grow(docToEntry, 1 + docID);
+ bytesUsed.addAndGet((docToEntry.length - size)
+ * RamUsageEstimator.NUM_BYTES_INT);
+ }
+ docToEntry[docID] = 1 + ord;
+ }
+
+ protected void checkSize(BytesRef bytes) {
+ if (size == -1) {
+ size = bytes.length;
+ } else if (bytes.length != size) {
+ throw new IllegalArgumentException("expected bytes size=" + size
+ + " but got " + bytes.length);
+ }
+ }
+
+ // Important that we get docCount, in case there were
+ // some last docs that we didn't see
+ @Override
+ public void finish(int docCount) throws IOException {
+ boolean success = false;
+ try {
+ finishInternal(docCount);
+ success = true;
+ } finally {
+ releaseResources();
+ if (success) {
+ IOUtils.close(getIndexOut(), getDataOut());
+ } else {
+ IOUtils.closeWhileHandlingException(getIndexOut(), getDataOut());
+ }
+
+ }
+ }
+
+ protected abstract void finishInternal(int docCount) throws IOException;
+
+ protected void releaseResources() {
+ hash.close();
+ bytesUsed
+ .addAndGet((-docToEntry.length) * RamUsageEstimator.NUM_BYTES_INT);
+ docToEntry = null;
+ }
+
+ protected void writeIndex(IndexOutput idxOut, int docCount,
+ long maxValue, int[] toEntry) throws IOException {
+ writeIndex(idxOut, docCount, maxValue, (int[])null, toEntry);
+ }
+
+ protected void writeIndex(IndexOutput idxOut, int docCount,
+ long maxValue, int[] addresses, int[] toEntry) throws IOException {
+ final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
+ PackedInts.bitsRequired(maxValue));
+ final int limit = docCount > docToEntry.length ? docToEntry.length
+ : docCount;
+ assert toEntry.length >= limit -1;
+ if (addresses != null) {
+ for (int i = 0; i < limit; i++) {
+ assert addresses[toEntry[i]] >= 0;
+ w.add(addresses[toEntry[i]]);
+ }
+ } else {
+ for (int i = 0; i < limit; i++) {
+ assert toEntry[i] >= 0;
+ w.add(toEntry[i]);
+ }
+ }
+ for (int i = limit; i < docCount; i++) {
+ w.add(0);
+ }
+ w.finish();
+ }
+
+ protected void writeIndex(IndexOutput idxOut, int docCount,
+ long maxValue, long[] addresses, int[] toEntry) throws IOException {
+ final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
+ PackedInts.bitsRequired(maxValue));
+ final int limit = docCount > docToEntry.length ? docToEntry.length
+ : docCount;
+ assert toEntry.length >= limit -1;
+ if (addresses != null) {
+ for (int i = 0; i < limit; i++) {
+ assert addresses[toEntry[i]] >= 0;
+ w.add(addresses[toEntry[i]]);
+ }
+ } else {
+ for (int i = 0; i < limit; i++) {
+ assert toEntry[i] >= 0;
+ w.add(toEntry[i]);
+ }
+ }
+ for (int i = limit; i < docCount; i++) {
+ w.add(0);
+ }
+ w.finish();
+ }
+
+ }
+
+ abstract static class DerefBytesEnumBase extends ValuesEnum {
+ private final PackedInts.ReaderIterator idx;
+ private final int valueCount;
+ private int pos = -1;
+ protected final IndexInput datIn;
+ protected final long fp;
+ protected final int size;
+
+ protected DerefBytesEnumBase(AttributeSource source, IndexInput datIn,
+ IndexInput idxIn, int size, ValueType enumType) throws IOException {
+ super(source, enumType);
+ this.datIn = datIn;
+ this.size = size;
+ idx = PackedInts.getReaderIterator(idxIn);
+ fp = datIn.getFilePointer();
+ if (size > 0) {
+ bytesRef.grow(this.size);
+ bytesRef.length = this.size;
+ }
+ bytesRef.offset = 0;
+ valueCount = idx.size();
+ }
+
+ protected void copyFrom(ValuesEnum valuesEnum) {
+ bytesRef = valuesEnum.bytesRef;
+ if (bytesRef.bytes.length < size) {
+ bytesRef.grow(size);
+ }
+ bytesRef.length = size;
+ bytesRef.offset = 0;
+ }
+
+ @Override
+ public int advance(int target) throws IOException {
+ if (target < valueCount) {
+ long address;
+ while ((address = idx.advance(target)) == 0) {
+ if (++target >= valueCount) {
+ return pos = NO_MORE_DOCS;
+ }
+ }
+ pos = idx.ord();
+ fill(address, bytesRef);
+ return pos;
+ }
+ return pos = NO_MORE_DOCS;
+ }
+
+ @Override
+ public int nextDoc() throws IOException {
+ if (pos >= valueCount) {
+ return pos = NO_MORE_DOCS;
+ }
+ return advance(pos + 1);
+ }
+
+ public void close() throws IOException {
+ try {
+ datIn.close();
+ } finally {
+ idx.close();
+ }
+ }
+
+ protected abstract void fill(long address, BytesRef ref) throws IOException;
+
+ @Override
+ public int docID() {
+ return pos;
+ }
+
+ }
}
\ No newline at end of file
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java Sat Oct 1 03:04:53 2011
@@ -19,26 +19,17 @@ package org.apache.lucene.index.values;
import java.io.IOException;
-import org.apache.lucene.index.values.Bytes.BytesBaseSource;
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
-import org.apache.lucene.index.values.Bytes.BytesWriterBase;
+import org.apache.lucene.index.values.Bytes.DerefBytesSourceBase;
+import org.apache.lucene.index.values.Bytes.DerefBytesEnumBase;
+import org.apache.lucene.index.values.Bytes.DerefBytesWriterBase;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefHash;
import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.PagedBytes;
-import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.ByteBlockPool.Allocator;
-import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
-import org.apache.lucene.util.BytesRefHash.TrackingDirectBytesStartArray;
-import org.apache.lucene.util.packed.PackedInts;
// Stores fixed-length byte[] by deref, ie when two docs
// have the same value, they store only 1 byte[]
@@ -51,135 +42,55 @@ class FixedDerefBytesImpl {
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
- static class Writer extends BytesWriterBase {
- private int size = -1;
- private int[] docToID;
- private final BytesRefHash hash;
+ public static class Writer extends DerefBytesWriterBase {
public Writer(Directory dir, String id, Counter bytesUsed, IOContext context)
throws IOException {
- this(dir, id, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
- bytesUsed, context);
- }
-
- public Writer(Directory dir, String id, Allocator allocator,
- Counter bytesUsed, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
- hash = new BytesRefHash(new ByteBlockPool(allocator),
- BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
- BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
- docToID = new int[1];
- bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
- }
-
- @Override
- public void add(int docID, BytesRef bytes) throws IOException {
- if (bytes.length == 0) // default value - skip it
- return;
- if (size == -1) {
- size = bytes.length;
- } else if (bytes.length != size) {
- throw new IllegalArgumentException("expected bytes size=" + size
- + " but got " + bytes.length);
- }
- int ord = hash.add(bytes);
- if (ord < 0) {
- ord = (-ord) - 1;
- }
- if (docID >= docToID.length) {
- final int size = docToID.length;
- docToID = ArrayUtil.grow(docToID, 1 + docID);
- bytesUsed.addAndGet((docToID.length - size)
- * RamUsageEstimator.NUM_BYTES_INT);
- }
- docToID[docID] = 1 + ord;
}
- // Important that we get docCount, in case there were
- // some last docs that we didn't see
@Override
- public void finish(int docCount) throws IOException {
- boolean success = false;
+ protected void finishInternal(int docCount) throws IOException {
final int numValues = hash.size();
- final IndexOutput datOut = getDataOut();
- try {
- datOut.writeInt(size);
- if (size != -1) {
- final BytesRef bytesRef = new BytesRef(size);
- for (int i = 0; i < numValues; i++) {
- hash.get(i, bytesRef);
- datOut.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
- }
- }
- success = true;
- } finally {
- if (success) {
- IOUtils.close(datOut);
- } else {
- IOUtils.closeWhileHandlingException(datOut);
+ final IndexOutput datOut = getOrCreateDataOut();
+ datOut.writeInt(size);
+ if (size != -1) {
+ final BytesRef bytesRef = new BytesRef(size);
+ for (int i = 0; i < numValues; i++) {
+ hash.get(i, bytesRef);
+ datOut.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
}
- hash.close();
- }
- success = false;
- final IndexOutput idxOut = getIndexOut();
- try {
- final int count = 1 + numValues;
- 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();
- success = true;
- } finally {
- if (success) {
- IOUtils.close(idxOut);
- } else {
- IOUtils.closeWhileHandlingException(idxOut);
- }
- bytesUsed
- .addAndGet((-docToID.length) * RamUsageEstimator.NUM_BYTES_INT);
- docToID = null;
}
+ final IndexOutput idxOut = getOrCreateIndexOut();
+ idxOut.writeInt(numValues);
+ writeIndex(idxOut, docCount, numValues, docToEntry);
}
}
public static class Reader extends BytesReaderBase {
private final int size;
-
+ private final int numValuesStored;
Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true, context);
size = datIn.readInt();
+ numValuesStored = idxIn.readInt();
}
@Override
public Source load() throws IOException {
- final IndexInput index = cloneIndex();
- return new Source(cloneData(), index, size, index.readInt());
+ return new Source(cloneData(), cloneIndex(), size, numValuesStored);
}
- private static class Source extends BytesBaseSource {
- private final PackedInts.Reader index;
+ private static final class Source extends DerefBytesSourceBase {
private final int size;
- private final int numValues;
- protected Source(IndexInput datIn, IndexInput idxIn, int size,
- int numValues) throws IOException {
- super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), size * numValues);
+ protected Source(IndexInput datIn, IndexInput idxIn, int size, long numValues) throws IOException {
+ super(datIn, idxIn, size * numValues, ValueType.BYTES_FIXED_DEREF);
this.size = size;
- this.numValues = numValues;
- index = PackedInts.getReader(idxIn);
}
@Override
public BytesRef getBytes(int docID, BytesRef bytesRef) {
- final int id = (int) index.get(docID);
+ final int id = (int) addresses.get(docID);
if (id == 0) {
bytesRef.length = 0;
return bytesRef;
@@ -187,95 +98,18 @@ class FixedDerefBytesImpl {
return data.fillSlice(bytesRef, ((id - 1) * size), size);
}
- @Override
- public int getValueCount() {
- return numValues;
- }
-
- @Override
- public ValueType type() {
- return ValueType.BYTES_FIXED_DEREF;
- }
-
- @Override
- protected int maxDoc() {
- return index.size();
- }
}
@Override
public ValuesEnum getEnum(AttributeSource source) throws IOException {
- return new DerefBytesEnum(source, cloneData(), cloneIndex(), size);
+ return new DerefBytesEnum(source, cloneData(), cloneIndex(), size);
}
- static class DerefBytesEnum extends ValuesEnum {
- protected final IndexInput datIn;
- private final PackedInts.ReaderIterator idx;
- protected final long fp;
- private final int size;
- private final int valueCount;
- private int pos = -1;
+ final static class DerefBytesEnum extends DerefBytesEnumBase {
public DerefBytesEnum(AttributeSource source, IndexInput datIn,
IndexInput idxIn, int size) throws IOException {
- this(source, datIn, idxIn, size, ValueType.BYTES_FIXED_DEREF);
- }
-
- protected DerefBytesEnum(AttributeSource source, IndexInput datIn,
- IndexInput idxIn, int size, ValueType enumType) throws IOException {
- super(source, enumType);
- this.datIn = datIn;
- this.size = size;
- idxIn.readInt();// read valueCount
- idx = PackedInts.getReaderIterator(idxIn);
- fp = datIn.getFilePointer();
- if (size > 0) {
- bytesRef.grow(this.size);
- bytesRef.length = this.size;
- }
- bytesRef.offset = 0;
- valueCount = idx.size();
- }
-
- protected void copyFrom(ValuesEnum valuesEnum) {
- bytesRef = valuesEnum.bytesRef;
- if (bytesRef.bytes.length < size) {
- bytesRef.grow(size);
- }
- bytesRef.length = size;
- bytesRef.offset = 0;
- }
-
- @Override
- public int advance(int target) throws IOException {
- if (target < valueCount) {
- long address;
- while ((address = idx.advance(target)) == 0) {
- if (++target >= valueCount) {
- return pos = NO_MORE_DOCS;
- }
- }
- pos = idx.ord();
- fill(address, bytesRef);
- return pos;
- }
- return pos = NO_MORE_DOCS;
- }
-
- @Override
- public int nextDoc() throws IOException {
- if (pos >= valueCount) {
- return pos = NO_MORE_DOCS;
- }
- return advance(pos + 1);
- }
-
- public void close() throws IOException {
- try {
- datIn.close();
- } finally {
- idx.close();
- }
+ super(source, datIn, idxIn, size, ValueType.BYTES_FIXED_DEREF);
}
protected void fill(long address, BytesRef ref) throws IOException {
@@ -284,12 +118,6 @@ class FixedDerefBytesImpl {
ref.length = size;
ref.offset = 0;
}
-
- @Override
- public int docID() {
- return pos;
- }
-
}
@Override
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java Sat Oct 1 03:04:53 2011
@@ -20,28 +20,17 @@ package org.apache.lucene.index.values;
import java.io.IOException;
import java.util.Comparator;
-import org.apache.lucene.index.values.Bytes.BytesBaseSortedSource;
+import org.apache.lucene.index.values.Bytes.BytesSortedSourceBase;
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
-import org.apache.lucene.index.values.Bytes.BytesWriterBase;
+import org.apache.lucene.index.values.Bytes.DerefBytesWriterBase;
import org.apache.lucene.index.values.FixedDerefBytesImpl.Reader.DerefBytesEnum;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefHash;
-import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.PagedBytes;
-import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.ByteBlockPool.Allocator;
-import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
-import org.apache.lucene.util.BytesRefHash.TrackingDirectBytesStartArray;
-import org.apache.lucene.util.packed.PackedInts;
// Stores fixed-length byte[] by deref, ie when two docs
// have the same value, they store only 1 byte[]
@@ -55,132 +44,49 @@ class FixedSortedBytesImpl {
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
- static class Writer extends BytesWriterBase {
- private int size = -1;
- private int[] docToEntry;
+ static class Writer extends DerefBytesWriterBase {
private final Comparator<BytesRef> comp;
- private final BytesRefHash hash;
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
Counter bytesUsed, IOContext context) throws IOException {
- this(dir, id, comp, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
- bytesUsed, context);
- }
-
- public Writer(Directory dir, String id, Comparator<BytesRef> comp,
- Allocator allocator, Counter bytesUsed, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
- ByteBlockPool pool = new ByteBlockPool(allocator);
- hash = new BytesRefHash(pool, BytesRefHash.DEFAULT_CAPACITY,
- new TrackingDirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY,
- bytesUsed));
- docToEntry = new int[1];
- bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
this.comp = comp;
}
- @Override
- public void add(int docID, BytesRef bytes) throws IOException {
- if (bytes.length == 0)
- return; // default - skip it
- if (size == -1) {
- size = bytes.length;
- } else if (bytes.length != size) {
- throw new IllegalArgumentException("expected bytes size=" + size
- + " but got " + bytes.length);
- }
- if (docID >= docToEntry.length) {
- final int[] newArray = new int[ArrayUtil.oversize(1 + docID,
- RamUsageEstimator.NUM_BYTES_INT)];
- System.arraycopy(docToEntry, 0, newArray, 0, docToEntry.length);
- bytesUsed.addAndGet((newArray.length - docToEntry.length)
- * RamUsageEstimator.NUM_BYTES_INT);
- docToEntry = newArray;
- }
- int e = hash.add(bytes);
- docToEntry[docID] = 1 + (e < 0 ? (-e) - 1 : e);
- }
-
// Important that we get docCount, in case there were
// some last docs that we didn't see
@Override
- public void finish(int docCount) throws IOException {
- final IndexOutput datOut = getDataOut();
- boolean success = false;
+ public void finishInternal(int docCount) throws IOException {
+ final IndexOutput datOut = getOrCreateDataOut();
final int count = hash.size();
- final int[] address = new int[count];
-
- try {
- datOut.writeInt(size);
- if (size != -1) {
- final int[] sortedEntries = hash.sort(comp);
- // first dump bytes data, recording address as we go
- final BytesRef bytesRef = new BytesRef(size);
- for (int i = 0; i < count; i++) {
- final int e = sortedEntries[i];
- final BytesRef bytes = hash.get(e, bytesRef);
- assert bytes.length == size;
- datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
- address[e] = 1 + i;
- }
- }
- success = true;
- } finally {
- if (success) {
- IOUtils.close(datOut);
- } else {
- IOUtils.closeWhileHandlingException(datOut);
- }
- hash.close();
- }
- final IndexOutput idxOut = getIndexOut();
- success = false;
- try {
- idxOut.writeInt(count);
- // next write index
- final 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);
- } 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);
- }
- w.finish();
- } finally {
- if (success) {
- IOUtils.close(idxOut);
- } else {
- IOUtils.closeWhileHandlingException(idxOut);
- }
- bytesUsed.addAndGet((-docToEntry.length)
- * RamUsageEstimator.NUM_BYTES_INT);
- docToEntry = null;
- }
+ final int[] address = new int[count+1]; // addr 0 is default values
+ datOut.writeInt(size);
+ if (size != -1) {
+ final int[] sortedEntries = hash.sort(comp);
+ // first dump bytes data, recording address as we go
+ final BytesRef bytesRef = new BytesRef(size);
+ for (int i = 0; i < count; i++) {
+ final int e = sortedEntries[i];
+ final BytesRef bytes = hash.get(e, bytesRef);
+ assert bytes.length == size;
+ datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
+ address[e + 1] = 1 + i;
+ }
+ }
+ final IndexOutput idxOut = getOrCreateIndexOut();
+ idxOut.writeInt(count);
+ writeIndex(idxOut, docCount, count, address, docToEntry);
}
}
public static class Reader extends BytesReaderBase {
private final int size;
+ private final int numValuesStored;
public Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME, VERSION_START, true, context);
size = datIn.readInt();
+ numValuesStored = idxIn.readInt();
}
@Override
@@ -192,58 +98,35 @@ class FixedSortedBytesImpl {
@Override
public SortedSource loadSorted(Comparator<BytesRef> comp)
throws IOException {
- final IndexInput idxInput = cloneIndex();
- final IndexInput datInput = cloneData();
- datInput.seek(CodecUtil.headerLength(CODEC_NAME) + 4);
- idxInput.seek(CodecUtil.headerLength(CODEC_NAME));
- return new Source(datInput, idxInput, size, idxInput.readInt(), comp);
+ return new Source(cloneData(), cloneIndex(), size, numValuesStored, comp);
}
- private static class Source extends BytesBaseSortedSource {
-
- private final PackedInts.Reader index;
- private final int numValue;
+ private static class Source extends BytesSortedSourceBase {
+ private final int valueCount;
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);
+ super(datIn, idxIn, comp, size * numValues, ValueType.BYTES_FIXED_SORTED);
this.size = size;
- this.numValue = numValues;
- index = PackedInts.getReader(idxIn);
+ this.valueCount = numValues;
closeIndexInput();
}
@Override
- public int ord(int docID) {
- return (int) index.get(docID) -1;
- }
-
- @Override
public int getByValue(BytesRef bytes, BytesRef tmpRef) {
- return binarySearch(bytes, tmpRef, 0, numValue - 1);
+ return binarySearch(bytes, tmpRef, 0, valueCount - 1);
}
@Override
public int getValueCount() {
- return numValue;
+ return valueCount;
}
@Override
protected BytesRef deref(int ord, BytesRef bytesRef) {
return data.fillSlice(bytesRef, (ord * size), size);
}
-
- @Override
- public ValueType type() {
- return ValueType.BYTES_FIXED_SORTED;
- }
-
- @Override
- protected int maxDoc() {
- return index.size();
- }
}
@Override
Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java Sat Oct 1 03:04:53 2011
@@ -21,7 +21,7 @@ import static org.apache.lucene.util.Byt
import java.io.IOException;
-import org.apache.lucene.index.values.Bytes.BytesBaseSource;
+import org.apache.lucene.index.values.Bytes.BytesSourceBase;
import org.apache.lucene.index.values.Bytes.BytesReaderBase;
import org.apache.lucene.index.values.Bytes.BytesWriterBase;
import org.apache.lucene.store.Directory;
@@ -46,26 +46,24 @@ class FixedStraightBytesImpl {
static final String CODEC_NAME = "FixedStraightBytes";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
-
- static class Writer extends BytesWriterBase {
- private int size = -1;
+
+ static abstract class FixedBytesWriterBase extends BytesWriterBase {
+ protected int lastDocID = -1;
// start at -1 if the first added value is > 0
- private int lastDocID = -1;
+ protected int size = -1;
+ private final int byteBlockSize = BYTE_BLOCK_SIZE;
private final ByteBlockPool pool;
- private boolean merge;
- private final int byteBlockSize;
- private IndexOutput datOut;
- public Writer(Directory dir, String id, Counter bytesUsed, IOContext context) throws IOException {
- super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
+ protected FixedBytesWriterBase(Directory dir, String id, String codecName,
+ int version, Counter bytesUsed, IOContext context) throws IOException {
+ super(dir, id, codecName, version, bytesUsed, context);
pool = new ByteBlockPool(new DirectTrackingAllocator(bytesUsed));
- byteBlockSize = BYTE_BLOCK_SIZE;
}
-
+
@Override
public void add(int docID, BytesRef bytes) throws IOException {
assert lastDocID < docID;
- assert !merge;
+
if (size == -1) {
if (bytes.length > BYTE_BLOCK_SIZE) {
throw new IllegalArgumentException("bytes arrays > " + Short.MAX_VALUE + " are not supported");
@@ -84,7 +82,6 @@ class FixedStraightBytesImpl {
}
private final void advancePool(int docID) {
- assert !merge;
long numBytes = (docID - (lastDocID+1))*size;
while(numBytes > 0) {
if (numBytes + pool.byteUpto < byteBlockSize) {
@@ -97,14 +94,50 @@ class FixedStraightBytesImpl {
}
assert numBytes == 0;
}
+
+ protected void set(BytesRef ref, int docId) {
+ assert BYTE_BLOCK_SIZE % size == 0 : "BYTE_BLOCK_SIZE ("+ BYTE_BLOCK_SIZE + ") must be a multiple of the size: " + size;
+ ref.offset = docId*size;
+ ref.length = size;
+ pool.deref(ref);
+ }
+
+ protected void resetPool() {
+ pool.dropBuffersAndReset();
+ }
+
+ protected void writeData(IndexOutput out) throws IOException {
+ pool.writePool(out);
+ }
+
+ protected void writeZeros(int num, IndexOutput out) throws IOException {
+ final byte[] zeros = new byte[size];
+ for (int i = 0; i < num; i++) {
+ out.writeBytes(zeros, zeros.length);
+ }
+ }
+ }
+
+ static class Writer extends FixedBytesWriterBase {
+ private boolean merge;
+ private IndexOutput datOut;
+
+ public Writer(Directory dir, String id, Counter bytesUsed, IOContext context) throws IOException {
+ super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
+ }
+
+ public Writer(Directory dir, String id, String codecName, int version, Counter bytesUsed, IOContext context) throws IOException {
+ super(dir, id, codecName, version, bytesUsed, context);
+ }
+
@Override
protected void merge(MergeState state) throws IOException {
merge = true;
- datOut = getDataOut();
+ datOut = getOrCreateDataOut();
boolean success = false;
try {
- if (state.liveDocs == null && state.reader instanceof Reader) {
+ if (state.liveDocs == null && state.reader instanceof Reader ) {
Reader reader = (Reader) state.reader;
final int maxDocs = reader.maxDoc;
if (maxDocs == 0) {
@@ -113,7 +146,10 @@ class FixedStraightBytesImpl {
if (size == -1) {
size = reader.size;
datOut.writeInt(size);
- }
+ } else if (size != reader.size) {
+ throw new IllegalArgumentException("expected bytes size=" + size
+ + " but got " + reader.size);
+ }
if (lastDocID+1 < state.docBase) {
fill(datOut, state.docBase);
lastDocID = state.docBase-1;
@@ -137,7 +173,7 @@ class FixedStraightBytesImpl {
}
}
}
-
+
@Override
protected void mergeDoc(int docID) throws IOException {
assert lastDocID < docID;
@@ -158,11 +194,7 @@ class FixedStraightBytesImpl {
// Fills up to but not including this docID
private void fill(IndexOutput datOut, int docID) throws IOException {
assert size >= 0;
- final long numBytes = (docID - (lastDocID+1))*size;
- final byte zero = 0;
- for (long i = 0; i < numBytes; i++) {
- datOut.writeByte(zero);
- }
+ writeZeros((docID - (lastDocID+1)), datOut);
}
@Override
@@ -172,12 +204,12 @@ class FixedStraightBytesImpl {
if (!merge) {
// indexing path - no disk IO until here
assert datOut == null;
- datOut = getDataOut();
+ datOut = getOrCreateDataOut();
if (size == -1) {
datOut.writeInt(0);
} else {
datOut.writeInt(size);
- pool.writePool(datOut);
+ writeData(datOut);
}
if (lastDocID + 1 < docCount) {
fill(datOut, docCount);
@@ -193,7 +225,7 @@ class FixedStraightBytesImpl {
}
success = true;
} finally {
- pool.dropBuffersAndReset();
+ resetPool();
if (success) {
IOUtils.close(datOut);
} else {
@@ -201,14 +233,19 @@ class FixedStraightBytesImpl {
}
}
}
+
}
public static class Reader extends BytesReaderBase {
- private final int size;
- private final int maxDoc;
-
+ protected final int size;
+ protected final int maxDoc;
+
Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
- super(dir, id, CODEC_NAME, VERSION_START, false, context);
+ this(dir, id, CODEC_NAME, VERSION_CURRENT, maxDoc, context);
+ }
+
+ protected Reader(Directory dir, String id, String codec, int version, int maxDoc, IOContext context) throws IOException {
+ super(dir, id, codec, version, false, context);
size = datIn.readInt();
this.maxDoc = maxDoc;
}
@@ -271,13 +308,13 @@ class FixedStraightBytesImpl {
}
- private static class StraightBytesSource extends BytesBaseSource {
+ private final static class StraightBytesSource extends BytesSourceBase {
private final int size;
private final int maxDoc;
public StraightBytesSource(IndexInput datIn, int size, int maxDoc)
throws IOException {
- super(datIn, null, new PagedBytes(PAGED_BYTES_BITS), size * maxDoc);
+ super(datIn, null, new PagedBytes(PAGED_BYTES_BITS), size * maxDoc, ValueType.BYTES_FIXED_STRAIGHT);
this.size = size;
this.maxDoc = maxDoc;
}
@@ -293,11 +330,6 @@ class FixedStraightBytesImpl {
}
@Override
- public ValueType type() {
- return ValueType.BYTES_FIXED_STRAIGHT;
- }
-
- @Override
protected int maxDoc() {
return maxDoc;
}
@@ -308,66 +340,68 @@ class FixedStraightBytesImpl {
return new FixedStraightBytesEnum(source, cloneData(), size, maxDoc);
}
- private static final class FixedStraightBytesEnum extends ValuesEnum {
- private final IndexInput datIn;
- private final int size;
- private final int maxDoc;
- private int pos = -1;
- private final long fp;
+
- public FixedStraightBytesEnum(AttributeSource source, IndexInput datIn,
- int size, int maxDoc) throws IOException {
- super(source, ValueType.BYTES_FIXED_STRAIGHT);
- this.datIn = datIn;
- this.size = size;
- this.maxDoc = maxDoc;
- bytesRef.grow(size);
- bytesRef.length = size;
- bytesRef.offset = 0;
- fp = datIn.getFilePointer();
- }
+ @Override
+ public ValueType type() {
+ return ValueType.BYTES_FIXED_STRAIGHT;
+ }
+ }
+
+ static class FixedStraightBytesEnum extends ValuesEnum {
+ private final IndexInput datIn;
+ private final int size;
+ private final int maxDoc;
+ private int pos = -1;
+ private final long fp;
- protected void copyFrom(ValuesEnum valuesEnum) {
- bytesRef = valuesEnum.bytesRef;
- if (bytesRef.bytes.length < size) {
- bytesRef.grow(size);
- }
- bytesRef.length = size;
- bytesRef.offset = 0;
- }
+ public FixedStraightBytesEnum(AttributeSource source, IndexInput datIn,
+ int size, int maxDoc) throws IOException {
+ super(source, ValueType.BYTES_FIXED_STRAIGHT);
+ this.datIn = datIn;
+ this.size = size;
+ this.maxDoc = maxDoc;
+ bytesRef.grow(size);
+ bytesRef.length = size;
+ bytesRef.offset = 0;
+ fp = datIn.getFilePointer();
+ }
- public void close() throws IOException {
- datIn.close();
+ protected void copyFrom(ValuesEnum valuesEnum) {
+ super.copyFrom(valuesEnum);
+ if (bytesRef.bytes.length < size) {
+ bytesRef.grow(size);
}
+ bytesRef.length = size;
+ bytesRef.offset = 0;
+ }
- @Override
- public int advance(int target) throws IOException {
- if (target >= maxDoc || size == 0) {
- return pos = NO_MORE_DOCS;
- }
- if ((target - 1) != pos) // pos inc == 1
- datIn.seek(fp + target * size);
- datIn.readBytes(bytesRef.bytes, 0, size);
- return pos = target;
- }
+ public void close() throws IOException {
+ datIn.close();
+ }
- @Override
- public int docID() {
- return pos;
- }
+ @Override
+ public int advance(int target) throws IOException {
+ if (target >= maxDoc || size == 0) {
+ return pos = NO_MORE_DOCS;
+ }
+ if ((target - 1) != pos) // pos inc == 1
+ datIn.seek(fp + target * size);
+ datIn.readBytes(bytesRef.bytes, 0, size);
+ return pos = target;
+ }
- @Override
- public int nextDoc() throws IOException {
- if (pos >= maxDoc) {
- return pos = NO_MORE_DOCS;
- }
- return advance(pos + 1);
- }
+ @Override
+ public int docID() {
+ return pos;
}
@Override
- public ValueType type() {
- return ValueType.BYTES_FIXED_STRAIGHT;
+ public int nextDoc() throws IOException {
+ if (pos >= maxDoc) {
+ return pos = NO_MORE_DOCS;
+ }
+ return advance(pos + 1);
}
}
}