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 2013/10/21 20:58:44 UTC
svn commit: r1534320 [14/39] - in /lucene/dev/branches/lucene4956: ./
dev-tools/ dev-tools/idea/.idea/ dev-tools/idea/lucene/expressions/
dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/
dev-tools/maven/lucene/ dev-tools/maven/lucene/expressions/...
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java Mon Oct 21 18:58:24 2013
@@ -18,19 +18,31 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.NoSuchElementException;
+import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.index.NumericFieldUpdates.UpdatesIterator;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.TrackingDirectoryWrapper;
import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.MutableBits;
// Used by IndexWriter to hold open SegmentReaders (for
-// searching or merging), plus pending deletes,
+// searching or merging), plus pending deletes and updates,
// for a given segment
-class ReadersAndLiveDocs {
+class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
// Not final because we replace (clone) when we need to
// change it and it's been shared:
public final SegmentInfoPerCommit info;
@@ -43,8 +55,8 @@ class ReadersAndLiveDocs {
// Set once (null, and then maybe set, and never set again):
private SegmentReader reader;
- // Holds the current shared (readable and writable
- // liveDocs). This is null when there are no deleted
+ // Holds the current shared (readable and writable)
+ // liveDocs. This is null when there are no deleted
// docs, and it's copy-on-write (cloned whenever we need
// to change it but it's been shared to an external NRT
// reader).
@@ -56,12 +68,22 @@ class ReadersAndLiveDocs {
// True if the current liveDocs is referenced by an
// external NRT reader:
- private boolean shared;
+ private boolean liveDocsShared;
+ // Indicates whether this segment is currently being merged. While a segment
+ // is merging, all field updates are also registered in the
+ // mergingNumericUpdates map. Also, calls to writeFieldUpdates merge the
+ // updates with mergingNumericUpdates.
+ // That way, when the segment is done merging, IndexWriter can apply the
+ // updates on the merged segment too.
+ private boolean isMerging = false;
+
+ private final Map<String,NumericFieldUpdates> mergingNumericUpdates = new HashMap<String,NumericFieldUpdates>();
+
public ReadersAndLiveDocs(IndexWriter writer, SegmentInfoPerCommit info) {
this.info = info;
this.writer = writer;
- shared = true;
+ liveDocsShared = true;
}
public void incRef() {
@@ -83,7 +105,7 @@ class ReadersAndLiveDocs {
public synchronized int getPendingDeleteCount() {
return pendingDeleteCount;
}
-
+
// Call only from assert!
public synchronized boolean verifyDocCounts() {
int count;
@@ -102,25 +124,21 @@ class ReadersAndLiveDocs {
return true;
}
- // Get reader for searching/deleting
- public synchronized SegmentReader getReader(IOContext context) throws IOException {
- //System.out.println(" livedocs=" + rld.liveDocs);
-
+ /** Returns a {@link SegmentReader}. */
+ public SegmentReader getReader(IOContext context) throws IOException {
if (reader == null) {
// We steal returned ref:
reader = new SegmentReader(info, context);
if (liveDocs == null) {
liveDocs = reader.getLiveDocs();
}
- //System.out.println("ADD seg=" + rld.info + " isMerge=" + isMerge + " " + readerMap.size() + " in pool");
- //System.out.println(Thread.currentThread().getName() + ": getReader seg=" + info.name);
}
-
+
// Ref for caller
reader.incRef();
return reader;
}
-
+
public synchronized void release(SegmentReader sr) throws IOException {
assert info == sr.getSegmentInfo();
sr.decRef();
@@ -130,7 +148,7 @@ class ReadersAndLiveDocs {
assert liveDocs != null;
assert Thread.holdsLock(writer);
assert docID >= 0 && docID < liveDocs.length() : "out of bounds: docid=" + docID + " liveDocsLength=" + liveDocs.length() + " seg=" + info.info.name + " docCount=" + info.info.getDocCount();
- assert !shared;
+ assert !liveDocsShared;
final boolean didDelete = liveDocs.get(docID);
if (didDelete) {
((MutableBits) liveDocs).clear(docID);
@@ -157,18 +175,17 @@ class ReadersAndLiveDocs {
}
/**
- * Returns a ref to a clone. NOTE: this clone is not
- * enrolled in the pool, so you should simply close()
- * it when you're done (ie, do not call release()).
+ * Returns a ref to a clone. NOTE: you should decRef() the reader when you're
+ * dont (ie do not call close()).
*/
public synchronized SegmentReader getReadOnlyClone(IOContext context) throws IOException {
if (reader == null) {
getReader(context).decRef();
assert reader != null;
}
- shared = true;
+ liveDocsShared = true;
if (liveDocs != null) {
- return new SegmentReader(reader.getSegmentInfo(), reader.core, liveDocs, info.info.getDocCount() - info.getDelCount() - pendingDeleteCount);
+ return new SegmentReader(reader.getSegmentInfo(), reader, liveDocs, info.info.getDocCount() - info.getDelCount() - pendingDeleteCount);
} else {
assert reader.getLiveDocs() == liveDocs;
reader.incRef();
@@ -180,7 +197,7 @@ class ReadersAndLiveDocs {
assert Thread.holdsLock(writer);
assert info.info.getDocCount() > 0;
//System.out.println("initWritableLivedocs seg=" + info + " liveDocs=" + liveDocs + " shared=" + shared);
- if (shared) {
+ if (liveDocsShared) {
// Copy on write: this means we've cloned a
// SegmentReader sharing the current liveDocs
// instance; must now make a private clone so we can
@@ -192,9 +209,7 @@ class ReadersAndLiveDocs {
} else {
liveDocs = liveDocsFormat.newLiveDocs(liveDocs);
}
- shared = false;
- } else {
- assert liveDocs != null;
+ liveDocsShared = false;
}
}
@@ -206,7 +221,7 @@ class ReadersAndLiveDocs {
public synchronized Bits getReadOnlyLiveDocs() {
//System.out.println("getROLiveDocs seg=" + info);
assert Thread.holdsLock(writer);
- shared = true;
+ liveDocsShared = true;
//if (liveDocs != null) {
//System.out.println(" liveCount=" + liveDocs.count());
//}
@@ -222,61 +237,288 @@ class ReadersAndLiveDocs {
// deletes onto the newly merged segment, so we can
// discard them on the sub-readers:
pendingDeleteCount = 0;
+ dropMergingUpdates();
}
- // Commit live docs to the directory (writes new
- // _X_N.del files); returns true if it wrote the file
- // and false if there were no new deletes to write:
+ // Commit live docs (writes new _X_N.del files) and field updates (writes new
+ // _X_N updates files) to the directory; returns true if it wrote any file
+ // and false if there were no new deletes or updates to write:
+ // TODO (DVU_RENAME) to writeDeletesAndUpdates
public synchronized boolean writeLiveDocs(Directory dir) throws IOException {
- //System.out.println("rld.writeLiveDocs seg=" + info + " pendingDelCount=" + pendingDeleteCount);
- if (pendingDeleteCount != 0) {
- // We have new deletes
- assert liveDocs.length() == info.info.getDocCount();
-
- // Do this so we can delete any created files on
- // exception; this saves all codecs from having to do
- // it:
- TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
-
- // We can write directly to the actual name (vs to a
- // .tmp & renaming it) because the file is not live
- // until segments file is written:
- boolean success = false;
+ assert Thread.holdsLock(writer);
+ //System.out.println("rld.writeLiveDocs seg=" + info + " pendingDelCount=" + pendingDeleteCount + " numericUpdates=" + numericUpdates);
+ if (pendingDeleteCount == 0) {
+ return false;
+ }
+
+ // We have new deletes
+ assert liveDocs.length() == info.info.getDocCount();
+
+ // Do this so we can delete any created files on
+ // exception; this saves all codecs from having to do
+ // it:
+ TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
+
+ // We can write directly to the actual name (vs to a
+ // .tmp & renaming it) because the file is not live
+ // until segments file is written:
+ boolean success = false;
+ try {
+ Codec codec = info.info.getCodec();
+ codec.liveDocsFormat().writeLiveDocs((MutableBits)liveDocs, trackingDir, info, pendingDeleteCount, IOContext.DEFAULT);
+ success = true;
+ } finally {
+ if (!success) {
+ // Advance only the nextWriteDelGen so that a 2nd
+ // attempt to write will write to a new file
+ info.advanceNextWriteDelGen();
+
+ // Delete any partially created file(s):
+ for (String fileName : trackingDir.getCreatedFiles()) {
+ try {
+ dir.deleteFile(fileName);
+ } catch (Throwable t) {
+ // Ignore so we throw only the first exc
+ }
+ }
+ }
+ }
+
+ // If we hit an exc in the line above (eg disk full)
+ // then info's delGen remains pointing to the previous
+ // (successfully written) del docs:
+ info.advanceDelGen();
+ info.setDelCount(info.getDelCount() + pendingDeleteCount);
+ pendingDeleteCount = 0;
+
+ return true;
+ }
+
+ // Writes field updates (new _X_N updates files) to the directory
+ public synchronized void writeFieldUpdates(Directory dir, Map<String,NumericFieldUpdates> numericFieldUpdates) throws IOException {
+ assert Thread.holdsLock(writer);
+ //System.out.println("rld.writeFieldUpdates: seg=" + info + " numericFieldUpdates=" + numericFieldUpdates);
+
+ assert numericFieldUpdates != null && !numericFieldUpdates.isEmpty();
+
+ // Do this so we can delete any created files on
+ // exception; this saves all codecs from having to do
+ // it:
+ TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
+
+ FieldInfos fieldInfos = null;
+ boolean success = false;
+ try {
+ final Codec codec = info.info.getCodec();
+
+ // reader could be null e.g. for a just merged segment (from
+ // IndexWriter.commitMergedDeletes).
+ final SegmentReader reader = this.reader == null ? new SegmentReader(info, IOContext.READONCE) : this.reader;
try {
- info.info.getCodec().liveDocsFormat().writeLiveDocs((MutableBits)liveDocs, trackingDir, info, pendingDeleteCount, IOContext.DEFAULT);
- success = true;
- } finally {
- if (!success) {
- // Advance only the nextWriteDelGen so that a 2nd
- // attempt to write will write to a new file
- info.advanceNextWriteDelGen();
-
- // Delete any partially created file(s):
- for(String fileName : trackingDir.getCreatedFiles()) {
- try {
- dir.deleteFile(fileName);
- } catch (Throwable t) {
- // Ignore so we throw only the first exc
+ // clone FieldInfos so that we can update their dvGen separately from
+ // the reader's infos and write them to a new fieldInfos_gen file
+ FieldInfos.Builder builder = new FieldInfos.Builder(writer.globalFieldNumberMap);
+ // cannot use builder.add(reader.getFieldInfos()) because it does not
+ // clone FI.attributes as well FI.dvGen
+ for (FieldInfo fi : reader.getFieldInfos()) {
+ FieldInfo clone = builder.add(fi);
+ // copy the stuff FieldInfos.Builder doesn't copy
+ if (fi.attributes() != null) {
+ for (Entry<String,String> e : fi.attributes().entrySet()) {
+ clone.putAttribute(e.getKey(), e.getValue());
}
}
+ clone.setDocValuesGen(fi.getDocValuesGen());
+ }
+ // create new fields or update existing ones to have NumericDV type
+ for (String f : numericFieldUpdates.keySet()) {
+ builder.addOrUpdate(f, NumericDocValuesField.TYPE);
+ }
+
+ fieldInfos = builder.finish();
+ final long nextFieldInfosGen = info.getNextFieldInfosGen();
+ final String segmentSuffix = Long.toString(nextFieldInfosGen, Character.MAX_RADIX);
+ final SegmentWriteState state = new SegmentWriteState(null, trackingDir, info.info, fieldInfos, null, IOContext.DEFAULT, segmentSuffix);
+ final DocValuesFormat docValuesFormat = codec.docValuesFormat();
+ final DocValuesConsumer fieldsConsumer = docValuesFormat.fieldsConsumer(state);
+ boolean fieldsConsumerSuccess = false;
+ try {
+// System.out.println("[" + Thread.currentThread().getName() + "] RLD.writeLiveDocs: applying updates; seg=" + info + " updates=" + numericUpdates);
+ for (Entry<String,NumericFieldUpdates> e : numericFieldUpdates.entrySet()) {
+ final String field = e.getKey();
+ final NumericFieldUpdates fieldUpdates = e.getValue();
+ final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
+ assert fieldInfo != null;
+
+ fieldInfo.setDocValuesGen(nextFieldInfosGen);
+ // write the numeric updates to a new gen'd docvalues file
+ fieldsConsumer.addNumericField(fieldInfo, new Iterable<Number>() {
+ final NumericDocValues currentValues = reader.getNumericDocValues(field);
+ final Bits docsWithField = reader.getDocsWithField(field);
+ final int maxDoc = reader.maxDoc();
+ final UpdatesIterator updatesIter = fieldUpdates.getUpdates();
+ @Override
+ public Iterator<Number> iterator() {
+ updatesIter.reset();
+ return new Iterator<Number>() {
+
+ int curDoc = -1;
+ int updateDoc = updatesIter.nextDoc();
+
+ @Override
+ public boolean hasNext() {
+ return curDoc < maxDoc - 1;
+ }
+
+ @Override
+ public Number next() {
+ if (++curDoc >= maxDoc) {
+ throw new NoSuchElementException("no more documents to return values for");
+ }
+ if (curDoc == updateDoc) { // this document has an updated value
+ Long value = updatesIter.value(); // either null (unset value) or updated value
+ updateDoc = updatesIter.nextDoc(); // prepare for next round
+ return value;
+ } else {
+ // no update for this document
+ assert curDoc < updateDoc;
+ if (currentValues != null && docsWithField.get(curDoc)) {
+ // only read the current value if the document had a value before
+ return currentValues.get(curDoc);
+ } else {
+ return null;
+ }
+ }
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException("this iterator does not support removing elements");
+ }
+ };
+ }
+ });
+ }
+
+ codec.fieldInfosFormat().getFieldInfosWriter().write(trackingDir, info.info.name, segmentSuffix, fieldInfos, IOContext.DEFAULT);
+ fieldsConsumerSuccess = true;
+ } finally {
+ if (fieldsConsumerSuccess) {
+ fieldsConsumer.close();
+ } else {
+ IOUtils.closeWhileHandlingException(fieldsConsumer);
+ }
+ }
+ } finally {
+ if (reader != this.reader) {
+// System.out.println("[" + Thread.currentThread().getName() + "] RLD.writeLiveDocs: closeReader " + reader);
+ reader.close();
}
}
+
+ success = true;
+ } finally {
+ if (!success) {
+ // Advance only the nextWriteDocValuesGen so that a 2nd
+ // attempt to write will write to a new file
+ info.advanceNextWriteFieldInfosGen();
+
+ // Delete any partially created file(s):
+ for (String fileName : trackingDir.getCreatedFiles()) {
+ try {
+ dir.deleteFile(fileName);
+ } catch (Throwable t) {
+ // Ignore so we throw only the first exc
+ }
+ }
+ }
+ }
+
+ info.advanceFieldInfosGen();
+ // copy all the updates to mergingUpdates, so they can later be applied to the merged segment
+ if (isMerging) {
+ for (Entry<String,NumericFieldUpdates> e : numericFieldUpdates.entrySet()) {
+ NumericFieldUpdates fieldUpdates = mergingNumericUpdates.get(e.getKey());
+ if (fieldUpdates == null) {
+ mergingNumericUpdates.put(e.getKey(), e.getValue());
+ } else {
+ fieldUpdates.merge(e.getValue());
+ }
+ }
+ }
+
+ // create a new map, keeping only the gens that are in use
+ Map<Long,Set<String>> genUpdatesFiles = info.getUpdatesFiles();
+ Map<Long,Set<String>> newGenUpdatesFiles = new HashMap<Long,Set<String>>();
+ final long fieldInfosGen = info.getFieldInfosGen();
+ for (FieldInfo fi : fieldInfos) {
+ long dvGen = fi.getDocValuesGen();
+ if (dvGen != -1 && !newGenUpdatesFiles.containsKey(dvGen)) {
+ if (dvGen == fieldInfosGen) {
+ newGenUpdatesFiles.put(fieldInfosGen, trackingDir.getCreatedFiles());
+ } else {
+ newGenUpdatesFiles.put(dvGen, genUpdatesFiles.get(dvGen));
+ }
+ }
+ }
+
+ info.setGenUpdatesFiles(newGenUpdatesFiles);
+
+ // wrote new files, should checkpoint()
+ writer.checkpoint();
- // If we hit an exc in the line above (eg disk full)
- // then info's delGen remains pointing to the previous
- // (successfully written) del docs:
- info.advanceDelGen();
- info.setDelCount(info.getDelCount() + pendingDeleteCount);
-
- pendingDeleteCount = 0;
- return true;
- } else {
- return false;
+ // if there is a reader open, reopen it to reflect the updates
+ if (reader != null) {
+ SegmentReader newReader = new SegmentReader(info, reader, liveDocs, info.info.getDocCount() - info.getDelCount() - pendingDeleteCount);
+ boolean reopened = false;
+ try {
+ reader.decRef();
+ reader = newReader;
+ reopened = true;
+ } finally {
+ if (!reopened) {
+ newReader.decRef();
+ }
+ }
}
}
+ /**
+ * Returns a reader for merge. This method applies field updates if there are
+ * any and marks that this segment is currently merging.
+ */
+ synchronized SegmentReader getReaderForMerge(IOContext context) throws IOException {
+ assert Thread.holdsLock(writer);
+ // must execute these two statements as atomic operation, otherwise we
+ // could lose updates if e.g. another thread calls writeFieldUpdates in
+ // between, or the updates are applied to the obtained reader, but then
+ // re-applied in IW.commitMergedDeletes (unnecessary work and potential
+ // bugs).
+ isMerging = true;
+ return getReader(context);
+ }
+
+ /**
+ * Drops all merging updates. Called from IndexWriter after this segment
+ * finished merging (whether successfully or not).
+ */
+ public synchronized void dropMergingUpdates() {
+ mergingNumericUpdates.clear();
+ isMerging = false;
+ }
+
+ /** Returns updates that came in while this segment was merging. */
+ public synchronized Map<String,NumericFieldUpdates> getMergingFieldUpdates() {
+ return mergingNumericUpdates;
+ }
+
@Override
public String toString() {
- return "ReadersAndLiveDocs(seg=" + info + " pendingDeleteCount=" + pendingDeleteCount + " shared=" + shared + ")";
+ StringBuilder sb = new StringBuilder();
+ sb.append("ReadersAndLiveDocs(seg=").append(info);
+ sb.append(" pendingDeleteCount=").append(pendingDeleteCount);
+ sb.append(" liveDocsShared=").append(liveDocsShared);
+ return sb.toString();
}
+
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java Mon Oct 21 18:58:24 2013
@@ -26,12 +26,11 @@ import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
-import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.SegmentReader.CoreClosedListener;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
@@ -43,29 +42,24 @@ import org.apache.lucene.util.IOUtils;
* SegmentReader is cloned or reopened */
final class SegmentCoreReaders {
- // Counts how many other reader share the core objects
+ // Counts how many other readers share the core objects
// (freqStream, proxStream, tis, etc.) of this reader;
// when coreRef drops to 0, these core objects may be
// closed. A given instance of SegmentReader may be
- // closed, even those it shares core objects with other
+ // closed, even though it shares core objects with other
// SegmentReaders:
private final AtomicInteger ref = new AtomicInteger(1);
- final FieldInfos fieldInfos;
-
final FieldsProducer fields;
- final DocValuesProducer dvProducer;
final DocValuesProducer normsProducer;
- private final SegmentReader owner;
-
final StoredFieldsReader fieldsReaderOrig;
final TermVectorsReader termVectorsReaderOrig;
final CompoundFileDirectory cfsReader;
// TODO: make a single thread local w/ a
// Thingy class holding fieldsReader, termVectorsReader,
- // normsProducer, dvProducer
+ // normsProducer
final CloseableThreadLocal<StoredFieldsReader> fieldsReaderLocal = new CloseableThreadLocal<StoredFieldsReader>() {
@Override
@@ -81,13 +75,6 @@ final class SegmentCoreReaders {
}
};
- final CloseableThreadLocal<Map<String,Object>> docValuesLocal = new CloseableThreadLocal<Map<String,Object>>() {
- @Override
- protected Map<String,Object> initialValue() {
- return new HashMap<String,Object>();
- }
- };
-
final CloseableThreadLocal<Map<String,Object>> normsLocal = new CloseableThreadLocal<Map<String,Object>>() {
@Override
protected Map<String,Object> initialValue() {
@@ -99,7 +86,7 @@ final class SegmentCoreReaders {
Collections.synchronizedSet(new LinkedHashSet<CoreClosedListener>());
SegmentCoreReaders(SegmentReader owner, Directory dir, SegmentInfoPerCommit si, IOContext context) throws IOException {
-
+
final Codec codec = si.info.getCodec();
final Directory cfsDir; // confusing name: if (cfs) its the cfsdir, otherwise its the segment's directory.
@@ -112,10 +99,11 @@ final class SegmentCoreReaders {
cfsReader = null;
cfsDir = dir;
}
- fieldInfos = codec.fieldInfosFormat().getFieldInfosReader().read(cfsDir, si.info.name, IOContext.READONCE);
- final PostingsFormat format = codec.postingsFormat();
+ final FieldInfos fieldInfos = owner.fieldInfos;
+
final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si.info, fieldInfos, context);
+ final PostingsFormat format = codec.postingsFormat();
// Ask codec for its Fields
fields = format.fieldsProducer(segmentReadState);
assert fields != null;
@@ -123,13 +111,6 @@ final class SegmentCoreReaders {
// TODO: since we don't write any norms file if there are no norms,
// kinda jaky to assume the codec handles the case of no norms file at all gracefully?!
- if (fieldInfos.hasDocValues()) {
- dvProducer = codec.docValuesFormat().fieldsProducer(segmentReadState);
- assert dvProducer != null;
- } else {
- dvProducer = null;
- }
-
if (fieldInfos.hasNorms()) {
normsProducer = codec.normsFormat().normsProducer(segmentReadState);
assert normsProducer != null;
@@ -151,148 +132,25 @@ final class SegmentCoreReaders {
decRef();
}
}
-
- // Must assign this at the end -- if we hit an
- // exception above core, we don't want to attempt to
- // purge the FieldCache (will hit NPE because core is
- // not assigned yet).
- this.owner = owner;
}
- void incRef() {
- ref.incrementAndGet();
- }
-
- NumericDocValues getNumericDocValues(String field) throws IOException {
- FieldInfo fi = fieldInfos.fieldInfo(field);
- if (fi == null) {
- // Field does not exist
- return null;
- }
- if (fi.getDocValuesType() == null) {
- // Field was not indexed with doc values
- return null;
- }
- if (fi.getDocValuesType() != DocValuesType.NUMERIC) {
- // DocValues were not numeric
- return null;
- }
-
- assert dvProducer != null;
-
- Map<String,Object> dvFields = docValuesLocal.get();
-
- NumericDocValues dvs = (NumericDocValues) dvFields.get(field);
- if (dvs == null) {
- dvs = dvProducer.getNumeric(fi);
- dvFields.put(field, dvs);
- }
-
- return dvs;
- }
-
- BinaryDocValues getBinaryDocValues(String field) throws IOException {
- FieldInfo fi = fieldInfos.fieldInfo(field);
- if (fi == null) {
- // Field does not exist
- return null;
- }
- if (fi.getDocValuesType() == null) {
- // Field was not indexed with doc values
- return null;
- }
- if (fi.getDocValuesType() != DocValuesType.BINARY) {
- // DocValues were not binary
- return null;
- }
-
- assert dvProducer != null;
-
- Map<String,Object> dvFields = docValuesLocal.get();
-
- BinaryDocValues dvs = (BinaryDocValues) dvFields.get(field);
- if (dvs == null) {
- dvs = dvProducer.getBinary(fi);
- dvFields.put(field, dvs);
- }
-
- return dvs;
- }
-
- SortedDocValues getSortedDocValues(String field) throws IOException {
- FieldInfo fi = fieldInfos.fieldInfo(field);
- if (fi == null) {
- // Field does not exist
- return null;
- }
- if (fi.getDocValuesType() == null) {
- // Field was not indexed with doc values
- return null;
- }
- if (fi.getDocValuesType() != DocValuesType.SORTED) {
- // DocValues were not sorted
- return null;
- }
-
- assert dvProducer != null;
-
- Map<String,Object> dvFields = docValuesLocal.get();
-
- SortedDocValues dvs = (SortedDocValues) dvFields.get(field);
- if (dvs == null) {
- dvs = dvProducer.getSorted(fi);
- dvFields.put(field, dvs);
- }
-
- return dvs;
+ int getRefCount() {
+ return ref.get();
}
- SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
- FieldInfo fi = fieldInfos.fieldInfo(field);
- if (fi == null) {
- // Field does not exist
- return null;
- }
- if (fi.getDocValuesType() == null) {
- // Field was not indexed with doc values
- return null;
- }
- if (fi.getDocValuesType() != DocValuesType.SORTED_SET) {
- // DocValues were not sorted
- return null;
- }
-
- assert dvProducer != null;
-
- Map<String,Object> dvFields = docValuesLocal.get();
-
- SortedSetDocValues dvs = (SortedSetDocValues) dvFields.get(field);
- if (dvs == null) {
- dvs = dvProducer.getSortedSet(fi);
- dvFields.put(field, dvs);
- }
-
- return dvs;
+ void incRef() {
+ ref.incrementAndGet();
}
- NumericDocValues getNormValues(String field) throws IOException {
- FieldInfo fi = fieldInfos.fieldInfo(field);
- if (fi == null) {
- // Field does not exist
- return null;
- }
- if (!fi.hasNorms()) {
- return null;
- }
-
+ NumericDocValues getNormValues(FieldInfo fi) throws IOException {
assert normsProducer != null;
Map<String,Object> normFields = normsLocal.get();
- NumericDocValues norms = (NumericDocValues) normFields.get(field);
+ NumericDocValues norms = (NumericDocValues) normFields.get(fi.name);
if (norms == null) {
norms = normsProducer.getNumeric(fi);
- normFields.put(field, norms);
+ normFields.put(fi.name, norms);
}
return norms;
@@ -300,8 +158,9 @@ final class SegmentCoreReaders {
void decRef() throws IOException {
if (ref.decrementAndGet() == 0) {
- IOUtils.close(termVectorsLocal, fieldsReaderLocal, docValuesLocal, normsLocal, fields, dvProducer,
- termVectorsReaderOrig, fieldsReaderOrig, cfsReader, normsProducer);
+// System.err.println("--- closing core readers");
+ IOUtils.close(termVectorsLocal, fieldsReaderLocal, normsLocal, fields, termVectorsReaderOrig, fieldsReaderOrig,
+ cfsReader, normsProducer);
notifyCoreClosedListeners();
}
}
@@ -309,7 +168,9 @@ final class SegmentCoreReaders {
private void notifyCoreClosedListeners() {
synchronized(coreClosedListeners) {
for (CoreClosedListener listener : coreClosedListeners) {
- listener.onClose(owner);
+ // SegmentReader uses our instance as its
+ // coreCacheKey:
+ listener.onClose(this);
}
}
}
@@ -322,8 +183,11 @@ final class SegmentCoreReaders {
coreClosedListeners.remove(listener);
}
- @Override
- public String toString() {
- return "SegmentCoreReader(owner=" + owner + ")";
+ /** Returns approximate RAM bytes used */
+ public long ramBytesUsed() {
+ return ((normsProducer!=null) ? normsProducer.ramBytesUsed() : 0) +
+ ((fields!=null) ? fields.ramBytesUsed() : 0) +
+ ((fieldsReaderOrig!=null)? fieldsReaderOrig.ramBytesUsed() : 0) +
+ ((termVectorsReaderOrig!=null) ? termVectorsReaderOrig.ramBytesUsed() : 0);
}
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java Mon Oct 21 18:58:24 2013
@@ -20,7 +20,6 @@ package org.apache.lucene.index;
import java.util.Collection;
import java.util.Collections;
-import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import java.util.regex.Matcher;
@@ -60,8 +59,6 @@ public final class SegmentInfo {
private Map<String,String> diagnostics;
- private Map<String,String> attributes;
-
// Tracks the Lucene version this segment was created with, since 3.1. Null
// indicates an older than 3.0 index, and it's used to detect a too old index.
// The format expected is "x.y" - "2.x" for pre-3.0 indexes (or null), and
@@ -85,7 +82,7 @@ public final class SegmentInfo {
* the codecs package.</p>
*/
public SegmentInfo(Directory dir, String version, String name, int docCount,
- boolean isCompoundFile, Codec codec, Map<String,String> diagnostics, Map<String,String> attributes) {
+ boolean isCompoundFile, Codec codec, Map<String,String> diagnostics) {
assert !(dir instanceof TrackingDirectoryWrapper);
this.dir = dir;
this.version = version;
@@ -94,7 +91,6 @@ public final class SegmentInfo {
this.isCompoundFile = isCompoundFile;
this.codec = codec;
this.diagnostics = diagnostics;
- this.attributes = attributes;
}
/**
@@ -119,7 +115,7 @@ public final class SegmentInfo {
public void setCodec(Codec codec) {
assert this.codec == null;
if (codec == null) {
- throw new IllegalArgumentException("segmentCodecs must be non-null");
+ throw new IllegalArgumentException("codec must be non-null");
}
this.codec = codec;
}
@@ -170,7 +166,6 @@ public final class SegmentInfo {
* left off when there are no deletions).</p>
*/
public String toString(Directory dir, int delCount) {
-
StringBuilder s = new StringBuilder();
s.append(name).append('(').append(version == null ? "?" : version).append(')').append(':');
char cfs = getUseCompoundFile() ? 'c' : 'C';
@@ -260,40 +255,4 @@ public final class SegmentInfo {
}
}
- /**
- * Get a codec attribute value, or null if it does not exist
- */
- public String getAttribute(String key) {
- if (attributes == null) {
- return null;
- } else {
- return attributes.get(key);
- }
- }
-
- /**
- * Puts a codec attribute value.
- * <p>
- * This is a key-value mapping for the field that the codec can use
- * to store additional metadata, and will be available to the codec
- * when reading the segment via {@link #getAttribute(String)}
- * <p>
- * If a value already exists for the field, it will be replaced with
- * the new value.
- */
- public String putAttribute(String key, String value) {
- if (attributes == null) {
- attributes = new HashMap<String,String>();
- }
- return attributes.put(key, value);
- }
-
- /**
- * Returns the internal codec attributes map.
- *
- * @return internal codec attributes map. May be null if no mappings exist.
- */
- public Map<String,String> attributes() {
- return attributes;
- }
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java Mon Oct 21 18:58:24 2013
@@ -19,7 +19,12 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
import org.apache.lucene.store.Directory;
@@ -27,9 +32,8 @@ import org.apache.lucene.store.Directory
* fields.
*
* @lucene.experimental */
-
-public class SegmentInfoPerCommit {
-
+public class SegmentInfoPerCommit { // TODO (DVU_RENAME) to SegmentCommitInfo
+
/** The {@link SegmentInfo} that we wrap. */
public final SegmentInfo info;
@@ -44,15 +48,31 @@ public class SegmentInfoPerCommit {
// attempt to write:
private long nextWriteDelGen;
+ // Generation number of the FieldInfos (-1 if there are no updates)
+ private long fieldInfosGen;
+
+ // Normally 1 + fieldInfosGen, unless an exception was hit on last attempt to
+ // write
+ private long nextWriteFieldInfosGen;
+
+ // Track the per-generation updates files
+ private final Map<Long,Set<String>> genUpdatesFiles = new HashMap<Long,Set<String>>();
+
private volatile long sizeInBytes = -1;
- /** Sole constructor.
- * @param info {@link SegmentInfo} that we wrap
- * @param delCount number of deleted documents in this segment
- * @param delGen deletion generation number (used to name
- deletion files)
+ /**
+ * Sole constructor.
+ *
+ * @param info
+ * {@link SegmentInfo} that we wrap
+ * @param delCount
+ * number of deleted documents in this segment
+ * @param delGen
+ * deletion generation number (used to name deletion files)
+ * @param fieldInfosGen
+ * FieldInfos generation number (used to name field-infos files)
**/
- public SegmentInfoPerCommit(SegmentInfo info, int delCount, long delGen) {
+ public SegmentInfoPerCommit(SegmentInfo info, int delCount, long delGen, long fieldInfosGen) {
this.info = info;
this.delCount = delCount;
this.delGen = delGen;
@@ -61,8 +81,26 @@ public class SegmentInfoPerCommit {
} else {
nextWriteDelGen = delGen+1;
}
+
+ this.fieldInfosGen = fieldInfosGen;
+ if (fieldInfosGen == -1) {
+ nextWriteFieldInfosGen = 1;
+ } else {
+ nextWriteFieldInfosGen = fieldInfosGen + 1;
+ }
}
+ /** Returns the per generation updates files. */
+ public Map<Long,Set<String>> getUpdatesFiles() {
+ return Collections.unmodifiableMap(genUpdatesFiles);
+ }
+
+ /** Sets the updates file names per generation. Does not deep clone the map. */
+ public void setGenUpdatesFiles(Map<Long,Set<String>> genUpdatesFiles) {
+ this.genUpdatesFiles.clear();
+ this.genUpdatesFiles.putAll(genUpdatesFiles);
+ }
+
/** Called when we succeed in writing deletes */
void advanceDelGen() {
delGen = nextWriteDelGen;
@@ -76,6 +114,21 @@ public class SegmentInfoPerCommit {
void advanceNextWriteDelGen() {
nextWriteDelGen++;
}
+
+ /** Called when we succeed in writing a new FieldInfos generation. */
+ void advanceFieldInfosGen() {
+ fieldInfosGen = nextWriteFieldInfosGen;
+ nextWriteFieldInfosGen = fieldInfosGen + 1;
+ sizeInBytes = -1;
+ }
+
+ /**
+ * Called if there was an exception while writing a new generation of
+ * FieldInfos, so that we don't try to write to the same file more than once.
+ */
+ void advanceNextWriteFieldInfosGen() {
+ nextWriteFieldInfosGen++;
+ }
/** Returns total size in bytes of all files for this
* segment. */
@@ -96,9 +149,17 @@ public class SegmentInfoPerCommit {
// Start from the wrapped info's files:
Collection<String> files = new HashSet<String>(info.files());
+ // TODO we could rely on TrackingDir.getCreatedFiles() (like we do for
+ // updates) and then maybe even be able to remove LiveDocsFormat.files().
+
// Must separately add any live docs files:
info.getCodec().liveDocsFormat().files(this, files);
+ // Must separately add any field updates files
+ for (Set<String> updateFiles : genUpdatesFiles.values()) {
+ files.addAll(updateFiles);
+ }
+
return files;
}
@@ -115,26 +176,30 @@ public class SegmentInfoPerCommit {
sizeInBytes = -1;
}
- void clearDelGen() {
- delGen = -1;
- sizeInBytes = -1;
- }
-
- /**
- * Sets the generation number of the live docs file.
- * @see #getDelGen()
- */
- public void setDelGen(long delGen) {
- this.delGen = delGen;
- sizeInBytes = -1;
- }
-
/** Returns true if there are any deletions for the
* segment at this commit. */
public boolean hasDeletions() {
return delGen != -1;
}
+ /** Returns true if there are any field updates for the segment in this commit. */
+ public boolean hasFieldUpdates() {
+ return fieldInfosGen != -1;
+ }
+
+ /** Returns the next available generation number of the FieldInfos files. */
+ public long getNextFieldInfosGen() {
+ return nextWriteFieldInfosGen;
+ }
+
+ /**
+ * Returns the generation number of the field infos file or -1 if there are no
+ * field updates yet.
+ */
+ public long getFieldInfosGen() {
+ return fieldInfosGen;
+ }
+
/**
* Returns the next available generation number
* of the live docs file.
@@ -165,26 +230,36 @@ public class SegmentInfoPerCommit {
/** Returns a description of this segment. */
public String toString(Directory dir, int pendingDelCount) {
- return info.toString(dir, delCount + pendingDelCount);
- }
-
- @Override
- public String toString() {
- String s = info.toString(info.dir, delCount);
+ String s = info.toString(dir, delCount + pendingDelCount);
if (delGen != -1) {
s += ":delGen=" + delGen;
}
+ if (fieldInfosGen != -1) {
+ s += ":fieldInfosGen=" + fieldInfosGen;
+ }
return s;
}
@Override
+ public String toString() {
+ return toString(info.dir, 0);
+ }
+
+ @Override
public SegmentInfoPerCommit clone() {
- SegmentInfoPerCommit other = new SegmentInfoPerCommit(info, delCount, delGen);
+ SegmentInfoPerCommit other = new SegmentInfoPerCommit(info, delCount, delGen, fieldInfosGen);
// Not clear that we need to carry over nextWriteDelGen
// (i.e. do we ever clone after a failed write and
// before the next successful write?), but just do it to
// be safe:
other.nextWriteDelGen = nextWriteDelGen;
+ other.nextWriteFieldInfosGen = nextWriteFieldInfosGen;
+
+ // deep clone
+ for (Entry<Long,Set<String>> e : genUpdatesFiles.entrySet()) {
+ other.genUpdatesFiles.put(e.getKey(), new HashSet<String>(e.getValue()));
+ }
+
return other;
}
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java Mon Oct 21 18:58:24 2013
@@ -28,21 +28,22 @@ import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import java.util.Map.Entry;
import java.util.Set;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.ChecksumIndexOutput;
-import org.apache.lucene.store.DataOutput; // javadocs
+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.store.NoSuchDirectoryException;
import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.ThreadInterruptedException;
/**
* A collection of segmentInfo objects with methods for operating on
@@ -70,7 +71,7 @@ import org.apache.lucene.util.ThreadInte
* <ul>
* <li><tt>segments.gen</tt>: GenHeader, Generation, Generation
* <li><tt>segments_N</tt>: Header, Version, NameCounter, SegCount,
- * <SegName, SegCodec, DelGen, DeletionCount><sup>SegCount</sup>,
+ * <SegName, SegCodec, DelGen, DeletionCount, FieldInfosGen, UpdatesFiles><sup>SegCount</sup>,
* CommitUserData, Checksum
* </ul>
* </p>
@@ -79,9 +80,10 @@ import org.apache.lucene.util.ThreadInte
* <ul>
* <li>Header --> {@link CodecUtil#writeHeader CodecHeader}</li>
* <li>GenHeader, NameCounter, SegCount, DeletionCount --> {@link DataOutput#writeInt Int32}</li>
- * <li>Generation, Version, DelGen, Checksum --> {@link DataOutput#writeLong Int64}</li>
+ * <li>Generation, Version, DelGen, Checksum, FieldInfosGen --> {@link DataOutput#writeLong Int64}</li>
* <li>SegName, SegCodec --> {@link DataOutput#writeString String}</li>
* <li>CommitUserData --> {@link DataOutput#writeStringStringMap Map<String,String>}</li>
+ * <li>UpdatesFiles --> {@link DataOutput#writeStringSet(Set) Set<String>}</li>
* </ul>
* </p>
* Field Descriptions:
@@ -104,6 +106,10 @@ import org.apache.lucene.util.ThreadInte
* <li>CommitUserData stores an optional user-supplied opaque
* Map<String,String> that was passed to
* {@link IndexWriter#setCommitData(java.util.Map)}.</li>
+ * <li>FieldInfosGen is the generation count of the fieldInfos file. If this is -1,
+ * there are no updates to the fieldInfos in that segment. Anything above zero
+ * means there are updates to fieldInfos stored by {@link FieldInfosFormat}.</li>
+ * <li>UpdatesFiles stores the list of files that were updated in that segment.</li>
* </ul>
* </p>
*
@@ -111,11 +117,12 @@ import org.apache.lucene.util.ThreadInte
*/
public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCommit> {
- /**
- * The file format version for the segments_N codec header
- */
+ /** The file format version for the segments_N codec header, up to 4.5. */
public static final int VERSION_40 = 0;
+ /** The file format version for the segments_N codec header, since 4.6+. */
+ public static final int VERSION_46 = 1;
+
/** Used for the segments.gen file only!
* Whenever you add a new format, make it 1 smaller (negative version logic)! */
public static final int FORMAT_SEGMENTS_GEN_CURRENT = -2;
@@ -319,14 +326,14 @@ public final class SegmentInfos implemen
throw new IndexFormatTooOldException(input, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
}
// 4.0+
- CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_40, VERSION_40);
+ int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_40, VERSION_46);
version = input.readLong();
counter = input.readInt();
int numSegments = input.readInt();
if (numSegments < 0) {
throw new CorruptIndexException("invalid segment count: " + numSegments + " (resource: " + input + ")");
}
- for(int seg=0;seg<numSegments;seg++) {
+ for (int seg = 0; seg < numSegments; seg++) {
String segName = input.readString();
Codec codec = Codec.forName(input.readString());
//System.out.println("SIS.read seg=" + seg + " codec=" + codec);
@@ -337,7 +344,25 @@ public final class SegmentInfos implemen
if (delCount < 0 || delCount > info.getDocCount()) {
throw new CorruptIndexException("invalid deletion count: " + delCount + " (resource: " + input + ")");
}
- add(new SegmentInfoPerCommit(info, delCount, delGen));
+ long fieldInfosGen = -1;
+ if (format >= VERSION_46) {
+ fieldInfosGen = input.readLong();
+ }
+ SegmentInfoPerCommit siPerCommit = new SegmentInfoPerCommit(info, delCount, delGen, fieldInfosGen);
+ if (format >= VERSION_46) {
+ int numGensUpdatesFiles = input.readInt();
+ final Map<Long,Set<String>> genUpdatesFiles;
+ if (numGensUpdatesFiles == 0) {
+ genUpdatesFiles = Collections.emptyMap();
+ } else {
+ genUpdatesFiles = new HashMap<Long,Set<String>>(numGensUpdatesFiles);
+ for (int i = 0; i < numGensUpdatesFiles; i++) {
+ genUpdatesFiles.put(input.readLong(), input.readStringSet());
+ }
+ }
+ siPerCommit.setGenUpdatesFiles(genUpdatesFiles);
+ }
+ add(siPerCommit);
}
userData = input.readStringStringMap();
@@ -395,7 +420,7 @@ public final class SegmentInfos implemen
try {
segnOutput = new ChecksumIndexOutput(directory.createOutput(segmentFileName, IOContext.DEFAULT));
- CodecUtil.writeHeader(segnOutput, "segments", VERSION_40);
+ CodecUtil.writeHeader(segnOutput, "segments", VERSION_46);
segnOutput.writeLong(version);
segnOutput.writeInt(counter); // write counter
segnOutput.writeInt(size()); // write infos
@@ -405,6 +430,13 @@ public final class SegmentInfos implemen
segnOutput.writeString(si.getCodec().getName());
segnOutput.writeLong(siPerCommit.getDelGen());
segnOutput.writeInt(siPerCommit.getDelCount());
+ segnOutput.writeLong(siPerCommit.getFieldInfosGen());
+ final Map<Long,Set<String>> genUpdatesFiles = siPerCommit.getUpdatesFiles();
+ segnOutput.writeInt(genUpdatesFiles.size());
+ for (Entry<Long,Set<String>> e : genUpdatesFiles.entrySet()) {
+ segnOutput.writeLong(e.getKey());
+ segnOutput.writeStringSet(e.getValue());
+ }
assert si.dir == directory;
assert siPerCommit.getDelCount() <= si.getDocCount();
@@ -804,10 +836,6 @@ public final class SegmentInfos implemen
if (includeSegmentsFile) {
final String segmentFileName = getSegmentsFileName();
if (segmentFileName != null) {
- /*
- * TODO: if lastGen == -1 we get might get null here it seems wrong to
- * add null to the files set
- */
files.add(segmentFileName);
}
}
@@ -819,6 +847,7 @@ public final class SegmentInfos implemen
files.addAll(info.files());
}
}
+
return files;
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java Mon Oct 21 18:58:24 2013
@@ -22,14 +22,14 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.FieldInfosWriter;
-import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.Bits;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.InfoStream;
@@ -52,12 +52,18 @@ final class SegmentMerger {
// note, just like in codec apis Directory 'dir' is NOT the same as segmentInfo.dir!!
SegmentMerger(List<AtomicReader> readers, SegmentInfo segmentInfo, InfoStream infoStream, Directory dir,
- MergeState.CheckAbort checkAbort, FieldInfos.FieldNumbers fieldNumbers, IOContext context) {
+ MergeState.CheckAbort checkAbort, FieldInfos.FieldNumbers fieldNumbers, IOContext context) throws IOException {
mergeState = new MergeState(readers, segmentInfo, infoStream, checkAbort);
directory = dir;
this.codec = segmentInfo.getCodec();
this.context = context;
this.fieldInfosBuilder = new FieldInfos.Builder(fieldNumbers);
+ mergeState.segmentInfo.setDocCount(setDocMaps());
+ }
+
+ /** True if any merging should happen */
+ boolean shouldMerge() {
+ return mergeState.segmentInfo.getDocCount() > 0;
}
/**
@@ -67,14 +73,15 @@ final class SegmentMerger {
* @throws IOException if there is a low-level IO error
*/
MergeState merge() throws IOException {
+ if (!shouldMerge()) {
+ throw new IllegalStateException("Merge would result in 0 document segment");
+ }
// 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
// of this check impacts how long
// IndexWriter.close(false) takes to actually stop the
// threads.
-
- mergeState.segmentInfo.setDocCount(setDocMaps());
mergeFieldInfos();
setMatchingSegmentReaders();
long t0 = 0;
@@ -135,7 +142,7 @@ final class SegmentMerger {
// write the merged infos
FieldInfosWriter fieldInfosWriter = codec.fieldInfosFormat().getFieldInfosWriter();
- fieldInfosWriter.write(directory, mergeState.segmentInfo.name, mergeState.fieldInfos, context);
+ fieldInfosWriter.write(directory, mergeState.segmentInfo.name, "", mergeState.fieldInfos, context);
return mergeState;
}
@@ -149,24 +156,32 @@ final class SegmentMerger {
if (type != null) {
if (type == DocValuesType.NUMERIC) {
List<NumericDocValues> toMerge = new ArrayList<NumericDocValues>();
+ List<Bits> docsWithField = new ArrayList<Bits>();
for (AtomicReader reader : mergeState.readers) {
NumericDocValues values = reader.getNumericDocValues(field.name);
+ Bits bits = reader.getDocsWithField(field.name);
if (values == null) {
values = NumericDocValues.EMPTY;
+ bits = new Bits.MatchNoBits(reader.maxDoc());
}
toMerge.add(values);
+ docsWithField.add(bits);
}
- consumer.mergeNumericField(field, mergeState, toMerge);
+ consumer.mergeNumericField(field, mergeState, toMerge, docsWithField);
} else if (type == DocValuesType.BINARY) {
List<BinaryDocValues> toMerge = new ArrayList<BinaryDocValues>();
+ List<Bits> docsWithField = new ArrayList<Bits>();
for (AtomicReader reader : mergeState.readers) {
BinaryDocValues values = reader.getBinaryDocValues(field.name);
+ Bits bits = reader.getDocsWithField(field.name);
if (values == null) {
values = BinaryDocValues.EMPTY;
+ bits = new Bits.MatchNoBits(reader.maxDoc());
}
toMerge.add(values);
+ docsWithField.add(bits);
}
- consumer.mergeBinaryField(field, mergeState, toMerge);
+ consumer.mergeBinaryField(field, mergeState, toMerge, docsWithField);
} else if (type == DocValuesType.SORTED) {
List<SortedDocValues> toMerge = new ArrayList<SortedDocValues>();
for (AtomicReader reader : mergeState.readers) {
@@ -209,14 +224,16 @@ final class SegmentMerger {
for (FieldInfo field : mergeState.fieldInfos) {
if (field.hasNorms()) {
List<NumericDocValues> toMerge = new ArrayList<NumericDocValues>();
+ List<Bits> docsWithField = new ArrayList<Bits>();
for (AtomicReader reader : mergeState.readers) {
NumericDocValues norms = reader.getNormValues(field.name);
if (norms == null) {
norms = NumericDocValues.EMPTY;
}
toMerge.add(norms);
+ docsWithField.add(new Bits.MatchAllBits(reader.maxDoc()));
}
- consumer.mergeNumericField(field, mergeState, toMerge);
+ consumer.mergeNumericField(field, mergeState, toMerge, docsWithField);
}
}
success = true;
@@ -357,19 +374,10 @@ final class SegmentMerger {
docBase += maxDoc;
}
- final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState);
- boolean success = false;
- try {
- consumer.merge(mergeState,
- new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
- slices.toArray(ReaderSlice.EMPTY_ARRAY)));
- success = true;
- } finally {
- if (success) {
- IOUtils.close(consumer);
- } else {
- IOUtils.closeWhileHandlingException(consumer);
- }
- }
+ Fields mergedFields = new MappedMultiFields(mergeState,
+ new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
+ slices.toArray(ReaderSlice.EMPTY_ARRAY)));
+
+ codec.postingsFormat().fieldsConsumer(segmentWriteState).write(mergedFields);
}
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java Mon Oct 21 18:58:24 2013
@@ -18,13 +18,26 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
-
-import org.apache.lucene.store.Directory;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
-import org.apache.lucene.search.FieldCache; // javadocs
+import org.apache.lucene.index.FieldInfo.DocValuesType;
+import org.apache.lucene.search.FieldCache;
+import org.apache.lucene.store.CompoundFileDirectory;
+import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.CloseableThreadLocal;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.RefCount;
/**
* IndexReader implementation over a single segment.
@@ -35,6 +48,18 @@ import org.apache.lucene.util.Bits;
*/
public final class SegmentReader extends AtomicReader {
+ private static final class DocValuesRefCount extends RefCount<DocValuesProducer> {
+
+ public DocValuesRefCount(DocValuesProducer object) {
+ super(object);
+ }
+
+ @Override
+ protected void release() throws IOException {
+ object.close();
+ }
+ }
+
private final SegmentInfoPerCommit si;
private final Bits liveDocs;
@@ -45,6 +70,25 @@ public final class SegmentReader extends
final SegmentCoreReaders core;
+ final CloseableThreadLocal<Map<String,Object>> docValuesLocal = new CloseableThreadLocal<Map<String,Object>>() {
+ @Override
+ protected Map<String,Object> initialValue() {
+ return new HashMap<String,Object>();
+ }
+ };
+
+ final CloseableThreadLocal<Map<String,Bits>> docsWithFieldLocal = new CloseableThreadLocal<Map<String,Bits>>() {
+ @Override
+ protected Map<String,Bits> initialValue() {
+ return new HashMap<String,Bits>();
+ }
+ };
+
+ final Map<String,DocValuesProducer> dvProducers = new HashMap<String,DocValuesProducer>();
+ final Map<Long,RefCount<DocValuesProducer>> genDVProducers = new HashMap<Long,RefCount<DocValuesProducer>>();
+
+ final FieldInfos fieldInfos;
+
/**
* Constructs a new SegmentReader with a new core.
* @throws CorruptIndexException if the index is corrupt
@@ -53,17 +97,51 @@ public final class SegmentReader extends
// TODO: why is this public?
public SegmentReader(SegmentInfoPerCommit si, IOContext context) throws IOException {
this.si = si;
+ // TODO if the segment uses CFS, we may open the CFS file twice: once for
+ // reading the FieldInfos (if they are not gen'd) and second time by
+ // SegmentCoreReaders. We can open the CFS here and pass to SCR, but then it
+ // results in less readable code (resource not closed where it was opened).
+ // Best if we could somehow read FieldInfos in SCR but not keep it there, but
+ // constructors don't allow returning two things...
+ fieldInfos = readFieldInfos(si);
core = new SegmentCoreReaders(this, si.info.dir, si, context);
+
boolean success = false;
+ final Codec codec = si.info.getCodec();
try {
if (si.hasDeletions()) {
// NOTE: the bitvector is stored using the regular directory, not cfs
- liveDocs = si.info.getCodec().liveDocsFormat().readLiveDocs(directory(), si, new IOContext(IOContext.READ, true));
+ liveDocs = codec.liveDocsFormat().readLiveDocs(directory(), si, IOContext.READONCE);
} else {
assert si.getDelCount() == 0;
liveDocs = null;
}
numDocs = si.info.getDocCount() - si.getDelCount();
+
+ if (fieldInfos.hasDocValues()) {
+ final Directory dir = core.cfsReader != null ? core.cfsReader : si.info.dir;
+ final DocValuesFormat dvFormat = codec.docValuesFormat();
+ // initialize the per generation numericDVProducers and put the correct
+ // DVProducer for each field
+ final Map<Long,List<FieldInfo>> genInfos = getGenInfos();
+
+// System.out.println("[" + Thread.currentThread().getName() + "] SR.init: new reader: " + si + "; gens=" + genInfos.keySet());
+
+ for (Entry<Long,List<FieldInfo>> e : genInfos.entrySet()) {
+ Long gen = e.getKey();
+ List<FieldInfo> infos = e.getValue();
+ RefCount<DocValuesProducer> dvp = genDVProducers.get(gen);
+ if (dvp == null) {
+ dvp = newDocValuesProducer(si, context, dir, dvFormat, gen, infos);
+ assert dvp != null;
+ genDVProducers.put(gen, dvp);
+ }
+ for (FieldInfo fi : infos) {
+ dvProducers.put(fi.name, dvp.get());
+ }
+ }
+ }
+
success = true;
} finally {
// With lock-less commits, it's entirely possible (and
@@ -72,7 +150,7 @@ public final class SegmentReader extends
// of things that were opened so that we don't have to
// wait for a GC to do so.
if (!success) {
- core.decRef();
+ doClose();
}
}
}
@@ -80,9 +158,9 @@ public final class SegmentReader extends
/** Create new SegmentReader sharing core from a previous
* SegmentReader and loading new live docs from a new
* deletes file. Used by openIfChanged. */
- SegmentReader(SegmentInfoPerCommit si, SegmentCoreReaders core, IOContext context) throws IOException {
- this(si, core,
- si.info.getCodec().liveDocsFormat().readLiveDocs(si.info.dir, si, context),
+ SegmentReader(SegmentInfoPerCommit si, SegmentReader sr) throws IOException {
+ this(si, sr,
+ si.info.getCodec().liveDocsFormat().readLiveDocs(si.info.dir, si, IOContext.READONCE),
si.info.getDocCount() - si.getDelCount());
}
@@ -90,15 +168,124 @@ public final class SegmentReader extends
* SegmentReader and using the provided in-memory
* liveDocs. Used by IndexWriter to provide a new NRT
* reader */
- SegmentReader(SegmentInfoPerCommit si, SegmentCoreReaders core, Bits liveDocs, int numDocs) {
+ SegmentReader(SegmentInfoPerCommit si, SegmentReader sr, Bits liveDocs, int numDocs) throws IOException {
this.si = si;
- this.core = core;
+ this.liveDocs = liveDocs;
+ this.numDocs = numDocs;
+ this.core = sr.core;
core.incRef();
+
+// System.out.println("[" + Thread.currentThread().getName() + "] SR.init: sharing reader: " + sr + " for gens=" + sr.genDVProducers.keySet());
+
+ // increment refCount of DocValuesProducers that are used by this reader
+ boolean success = false;
+ try {
+ final Codec codec = si.info.getCodec();
+ if (si.getFieldInfosGen() == -1) {
+ fieldInfos = sr.fieldInfos;
+ } else {
+ fieldInfos = readFieldInfos(si);
+ }
+
+ if (fieldInfos.hasDocValues()) {
+ final Directory dir = core.cfsReader != null ? core.cfsReader : si.info.dir;
+
+ final DocValuesFormat dvFormat = codec.docValuesFormat();
+ final Map<Long,List<FieldInfo>> genInfos = getGenInfos();
+
+ for (Entry<Long,List<FieldInfo>> e : genInfos.entrySet()) {
+ Long gen = e.getKey();
+ List<FieldInfo> infos = e.getValue();
+ RefCount<DocValuesProducer> dvp = genDVProducers.get(gen);
+ if (dvp == null) {
+ // check if this DVP gen is used by the given reader
+ dvp = sr.genDVProducers.get(gen);
+ if (dvp != null) {
+ // gen used by given reader, incRef its DVP
+ dvp.incRef();
+// System.out.println("[" + Thread.currentThread().getName() + "] SR.init: sharing DVP for gen=" + gen + " refCount=" + dvp.getRefCount());
+ } else {
+ // this gen is not used by given reader, initialize a new one
+ dvp = newDocValuesProducer(si, IOContext.READ, dir, dvFormat, gen, infos);
+// System.out.println("[" + Thread.currentThread().getName() + "] SR.init: new DVP for gen=" + gen + " refCount=" + dvp.getRefCount());
+ }
+ assert dvp != null;
+ genDVProducers.put(gen, dvp);
+ }
+ for (FieldInfo fi : infos) {
+ dvProducers.put(fi.name, dvp.get());
+ }
+ }
+ }
+ success = true;
+ } finally {
+ if (!success) {
+ doClose();
+ }
+ }
+ }
- assert liveDocs != null;
- this.liveDocs = liveDocs;
+ /**
+ * Reads the most recent {@link FieldInfos} of the given segment info.
+ *
+ * @lucene.internal
+ */
+ static FieldInfos readFieldInfos(SegmentInfoPerCommit info) throws IOException {
+ final Directory dir;
+ final boolean closeDir;
+ if (info.getFieldInfosGen() == -1 && info.info.getUseCompoundFile()) {
+ // no fieldInfos gen and segment uses a compound file
+ dir = new CompoundFileDirectory(info.info.dir,
+ IndexFileNames.segmentFileName(info.info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION),
+ IOContext.READONCE,
+ false);
+ closeDir = true;
+ } else {
+ // gen'd FIS are read outside CFS, or the segment doesn't use a compound file
+ dir = info.info.dir;
+ closeDir = false;
+ }
+
+ try {
+ final String segmentSuffix = info.getFieldInfosGen() == -1 ? "" : Long.toString(info.getFieldInfosGen(), Character.MAX_RADIX);
+ return info.info.getCodec().fieldInfosFormat().getFieldInfosReader().read(dir, info.info.name, segmentSuffix, IOContext.READONCE);
+ } finally {
+ if (closeDir) {
+ dir.close();
+ }
+ }
+ }
+
+ // returns a gen->List<FieldInfo> mapping. Fields without DV updates have gen=-1
+ private Map<Long,List<FieldInfo>> getGenInfos() {
+ final Map<Long,List<FieldInfo>> genInfos = new HashMap<Long,List<FieldInfo>>();
+ for (FieldInfo fi : fieldInfos) {
+ if (fi.getDocValuesType() == null) {
+ continue;
+ }
+ long gen = fi.getDocValuesGen();
+ List<FieldInfo> infos = genInfos.get(gen);
+ if (infos == null) {
+ infos = new ArrayList<FieldInfo>();
+ genInfos.put(gen, infos);
+ }
+ infos.add(fi);
+ }
+ return genInfos;
+ }
- this.numDocs = numDocs;
+ private RefCount<DocValuesProducer> newDocValuesProducer(SegmentInfoPerCommit si, IOContext context, Directory dir,
+ DocValuesFormat dvFormat, Long gen, List<FieldInfo> infos) throws IOException {
+ Directory dvDir = dir;
+ String segmentSuffix = "";
+ if (gen.longValue() != -1) {
+ dvDir = si.info.dir; // gen'd files are written outside CFS, so use SegInfo directory
+ segmentSuffix = Long.toString(gen.longValue(), Character.MAX_RADIX);
+ }
+
+ // set SegmentReadState to list only the fields that are relevant to that gen
+ SegmentReadState srs = new SegmentReadState(dvDir, si.info, new FieldInfos(infos.toArray(new FieldInfo[infos.size()])), context, segmentSuffix);
+ return new DocValuesRefCount(dvFormat.fieldsProducer(srs));
}
@Override
@@ -110,13 +297,31 @@ public final class SegmentReader extends
@Override
protected void doClose() throws IOException {
//System.out.println("SR.close seg=" + si);
- core.decRef();
+ try {
+ core.decRef();
+ } finally {
+ Throwable t = null;
+ for (RefCount<DocValuesProducer> dvp : genDVProducers.values()) {
+ try {
+ dvp.decRef();
+ } catch (Throwable th) {
+ if (t != null) {
+ t = th;
+ }
+ }
+ }
+ genDVProducers.clear();
+ dvProducers.clear();
+ docValuesLocal.close();
+ docsWithFieldLocal.close();
+ IOUtils.reThrow(t);
+ }
}
@Override
public FieldInfos getFieldInfos() {
ensureOpen();
- return core.fieldInfos;
+ return fieldInfos;
}
/** Expert: retrieve thread-private {@link
@@ -209,6 +414,9 @@ public final class SegmentReader extends
// same entry in the FieldCache. See LUCENE-1579.
@Override
public Object getCoreCacheKey() {
+ // NOTE: if this ever changes, be sure to fix
+ // SegmentCoreReader.notifyCoreClosedListeners to match!
+ // Today it passes "this" as its coreCacheKey:
return core;
}
@@ -217,34 +425,151 @@ public final class SegmentReader extends
return this;
}
+ // returns the FieldInfo that corresponds to the given field and type, or
+ // null if the field does not exist, or not indexed as the requested
+ // DovDocValuesType.
+ private FieldInfo getDVField(String field, DocValuesType type) {
+ FieldInfo fi = fieldInfos.fieldInfo(field);
+ if (fi == null) {
+ // Field does not exist
+ return null;
+ }
+ if (fi.getDocValuesType() == null) {
+ // Field was not indexed with doc values
+ return null;
+ }
+ if (fi.getDocValuesType() != type) {
+ // Field DocValues are different than requested type
+ return null;
+ }
+
+ return fi;
+ }
+
@Override
public NumericDocValues getNumericDocValues(String field) throws IOException {
ensureOpen();
- return core.getNumericDocValues(field);
+ FieldInfo fi = getDVField(field, DocValuesType.NUMERIC);
+ if (fi == null) {
+ return null;
+ }
+
+ DocValuesProducer dvProducer = dvProducers.get(field);
+ assert dvProducer != null;
+
+ Map<String,Object> dvFields = docValuesLocal.get();
+
+ NumericDocValues dvs = (NumericDocValues) dvFields.get(field);
+ if (dvs == null) {
+ dvs = dvProducer.getNumeric(fi);
+ dvFields.put(field, dvs);
+ }
+
+ return dvs;
+ }
+
+ @Override
+ public Bits getDocsWithField(String field) throws IOException {
+ ensureOpen();
+ FieldInfo fi = fieldInfos.fieldInfo(field);
+ if (fi == null) {
+ // Field does not exist
+ return null;
+ }
+ if (fi.getDocValuesType() == null) {
+ // Field was not indexed with doc values
+ return null;
+ }
+
+ DocValuesProducer dvProducer = dvProducers.get(field);
+ assert dvProducer != null;
+
+ Map<String,Bits> dvFields = docsWithFieldLocal.get();
+
+ Bits dvs = dvFields.get(field);
+ if (dvs == null) {
+ dvs = dvProducer.getDocsWithField(fi);
+ dvFields.put(field, dvs);
+ }
+
+ return dvs;
}
@Override
public BinaryDocValues getBinaryDocValues(String field) throws IOException {
ensureOpen();
- return core.getBinaryDocValues(field);
+ FieldInfo fi = getDVField(field, DocValuesType.BINARY);
+ if (fi == null) {
+ return null;
+ }
+
+ DocValuesProducer dvProducer = dvProducers.get(field);
+ assert dvProducer != null;
+
+ Map<String,Object> dvFields = docValuesLocal.get();
+
+ BinaryDocValues dvs = (BinaryDocValues) dvFields.get(field);
+ if (dvs == null) {
+ dvs = dvProducer.getBinary(fi);
+ dvFields.put(field, dvs);
+ }
+
+ return dvs;
}
@Override
public SortedDocValues getSortedDocValues(String field) throws IOException {
ensureOpen();
- return core.getSortedDocValues(field);
+ FieldInfo fi = getDVField(field, DocValuesType.SORTED);
+ if (fi == null) {
+ return null;
+ }
+
+ DocValuesProducer dvProducer = dvProducers.get(field);
+ assert dvProducer != null;
+
+ Map<String,Object> dvFields = docValuesLocal.get();
+
+ SortedDocValues dvs = (SortedDocValues) dvFields.get(field);
+ if (dvs == null) {
+ dvs = dvProducer.getSorted(fi);
+ dvFields.put(field, dvs);
+ }
+
+ return dvs;
}
@Override
public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
ensureOpen();
- return core.getSortedSetDocValues(field);
+ FieldInfo fi = getDVField(field, DocValuesType.SORTED_SET);
+ if (fi == null) {
+ return null;
+ }
+
+ DocValuesProducer dvProducer = dvProducers.get(field);
+ assert dvProducer != null;
+
+ Map<String,Object> dvFields = docValuesLocal.get();
+
+ SortedSetDocValues dvs = (SortedSetDocValues) dvFields.get(field);
+ if (dvs == null) {
+ dvs = dvProducer.getSortedSet(fi);
+ dvFields.put(field, dvs);
+ }
+
+ return dvs;
}
@Override
public NumericDocValues getNormValues(String field) throws IOException {
ensureOpen();
- return core.getNormValues(field);
+ FieldInfo fi = fieldInfos.fieldInfo(field);
+ if (fi == null || !fi.hasNorms()) {
+ // Field does not exist or does not index norms
+ return null;
+ }
+ return core.getNormValues(fi);
}
/**
@@ -261,9 +586,9 @@ public final class SegmentReader extends
* @lucene.experimental
*/
public static interface CoreClosedListener {
- /** Invoked when the shared core of the provided {@link
+ /** Invoked when the shared core of the original {@code
* SegmentReader} has closed. */
- public void onClose(SegmentReader owner);
+ public void onClose(Object ownerCoreCacheKey);
}
/** Expert: adds a CoreClosedListener to this reader's shared core */
@@ -277,4 +602,22 @@ public final class SegmentReader extends
ensureOpen();
core.removeCoreClosedListener(listener);
}
+
+ private long dvRamBytesUsed() {
+ long ramBytesUsed = 0;
+ for (RefCount<DocValuesProducer> dvp : genDVProducers.values()) {
+ ramBytesUsed += dvp.get().ramBytesUsed();
+ }
+ return ramBytesUsed;
+ }
+
+ /** Returns approximate RAM Bytes used */
+ public long ramBytesUsed() {
+ ensureOpen();
+ long ramBytesUsed = dvRamBytesUsed();
+ if (core != null) {
+ ramBytesUsed += core.ramBytesUsed();
+ }
+ return ramBytesUsed;
+ }
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java Mon Oct 21 18:58:24 2013
@@ -74,12 +74,23 @@ public class SegmentWriteState {
/** Sole constructor. */
public SegmentWriteState(InfoStream infoStream, Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos,
BufferedDeletes segDeletes, IOContext context) {
+ this(infoStream, directory, segmentInfo, fieldInfos, segDeletes, context, "");
+ }
+
+ /**
+ * Constructor which takes segment suffix.
+ *
+ * @see #SegmentWriteState(InfoStream, Directory, SegmentInfo, FieldInfos,
+ * BufferedDeletes, IOContext)
+ */
+ public SegmentWriteState(InfoStream infoStream, Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos,
+ BufferedDeletes segDeletes, IOContext context, String segmentSuffix) {
this.infoStream = infoStream;
this.segDeletes = segDeletes;
this.directory = directory;
this.segmentInfo = segmentInfo;
this.fieldInfos = fieldInfos;
- segmentSuffix = "";
+ this.segmentSuffix = segmentSuffix;
this.context = context;
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java Mon Oct 21 18:58:24 2013
@@ -64,9 +64,7 @@ public final class SlowCompositeReaderWr
}
}
- /** Sole constructor, wrapping the provided {@link
- * CompositeReader}. */
- public SlowCompositeReaderWrapper(CompositeReader reader) throws IOException {
+ private SlowCompositeReaderWrapper(CompositeReader reader) throws IOException {
super();
in = reader;
fields = MultiFields.getFields(in);
@@ -92,6 +90,12 @@ public final class SlowCompositeReaderWr
}
@Override
+ public Bits getDocsWithField(String field) throws IOException {
+ ensureOpen();
+ return MultiDocValues.getDocsWithField(in, field);
+ }
+
+ @Override
public BinaryDocValues getBinaryDocValues(String field) throws IOException {
ensureOpen();
return MultiDocValues.getBinaryValues(in, field);
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java Mon Oct 21 18:58:24 2013
@@ -37,12 +37,13 @@ public abstract class SortedDocValues ex
* Returns the ordinal for the specified docID.
* @param docID document ID to lookup
* @return ordinal for the document: this is dense, starts at 0, then
- * increments by 1 for the next value in sorted order.
+ * increments by 1 for the next value in sorted order. Note that
+ * missing values are indicated by -1.
*/
public abstract int getOrd(int docID);
/** Retrieves the value for the specified ordinal.
- * @param ord ordinal to lookup
+ * @param ord ordinal to lookup (must be >= 0 and < {@link #getValueCount()})
* @param result will be populated with the ordinal's value
* @see #getOrd(int)
*/
@@ -59,7 +60,7 @@ public abstract class SortedDocValues ex
public void get(int docID, BytesRef result) {
int ord = getOrd(docID);
if (ord == -1) {
- result.bytes = MISSING;
+ result.bytes = BytesRef.EMPTY_BYTES;
result.length = 0;
result.offset = 0;
} else {
@@ -67,23 +68,23 @@ public abstract class SortedDocValues ex
}
}
- /** An empty SortedDocValues which returns {@link #MISSING} for every document */
+ /** An empty SortedDocValues which returns {@link BytesRef#EMPTY_BYTES} for every document */
public static final SortedDocValues EMPTY = new SortedDocValues() {
@Override
public int getOrd(int docID) {
- return 0;
+ return -1;
}
@Override
public void lookupOrd(int ord, BytesRef result) {
- result.bytes = MISSING;
+ result.bytes = BytesRef.EMPTY_BYTES;
result.offset = 0;
result.length = 0;
}
@Override
public int getValueCount() {
- return 1;
+ return 0;
}
};
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java Mon Oct 21 18:58:24 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
-import java.util.Comparator;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@@ -125,11 +124,6 @@ class SortedDocValuesTermsEnum extends T
}
@Override
- public Comparator<BytesRef> getComparator() {
- return BytesRef.getUTF8SortedAsUnicodeComparator();
- }
-
- @Override
public void seekExact(BytesRef term, TermState state) throws IOException {
assert state != null && state instanceof OrdTermState;
this.seekExact(((OrdTermState)state).ord);
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java Mon Oct 21 18:58:24 2013
@@ -30,19 +30,19 @@ import org.apache.lucene.util.BytesRefHa
import org.apache.lucene.util.BytesRefHash;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
+import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
import org.apache.lucene.util.packed.PackedInts;
/** Buffers up pending byte[] per doc, deref and sorting via
* int ord, then flushes when segment flushes. */
class SortedDocValuesWriter extends DocValuesWriter {
final BytesRefHash hash;
- private AppendingPackedLongBuffer pending;
+ private AppendingDeltaPackedLongBuffer pending;
private final Counter iwBytesUsed;
private long bytesUsed; // this currently only tracks differences in 'pending'
private final FieldInfo fieldInfo;
- private static final BytesRef EMPTY = new BytesRef(BytesRef.EMPTY_BYTES);
+ private static final int EMPTY_ORD = -1;
public SortedDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
this.fieldInfo = fieldInfo;
@@ -52,7 +52,7 @@ class SortedDocValuesWriter extends DocV
new ByteBlockPool.DirectTrackingAllocator(iwBytesUsed)),
BytesRefHash.DEFAULT_CAPACITY,
new DirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, iwBytesUsed));
- pending = new AppendingPackedLongBuffer(PackedInts.COMPACT);
+ pending = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
bytesUsed = pending.ramBytesUsed();
iwBytesUsed.addAndGet(bytesUsed);
}
@@ -70,7 +70,7 @@ class SortedDocValuesWriter extends DocV
// Fill in any holes:
while(pending.size() < docID) {
- addOneValue(EMPTY);
+ pending.add(EMPTY_ORD);
}
addOneValue(value);
@@ -79,8 +79,9 @@ class SortedDocValuesWriter extends DocV
@Override
public void finish(int maxDoc) {
while(pending.size() < maxDoc) {
- addOneValue(EMPTY);
+ pending.add(EMPTY_ORD);
}
+ updateBytesUsed();
}
private void addOneValue(BytesRef value) {
@@ -177,7 +178,7 @@ class SortedDocValuesWriter extends DocV
// iterates over the ords for each doc we have in ram
private class OrdsIterator implements Iterator<Number> {
- final AppendingPackedLongBuffer.Iterator iter = pending.iterator();
+ final AppendingDeltaPackedLongBuffer.Iterator iter = pending.iterator();
final int ordMap[];
final int maxDoc;
int docUpto;
@@ -200,8 +201,7 @@ class SortedDocValuesWriter extends DocV
}
int ord = (int) iter.next();
docUpto++;
- // TODO: make reusable Number
- return ordMap[ord];
+ return ord == -1 ? ord : ordMap[ord];
}
@Override
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java Mon Oct 21 18:58:24 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
-import java.util.Comparator;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@@ -125,11 +124,6 @@ class SortedSetDocValuesTermsEnum extend
}
@Override
- public Comparator<BytesRef> getComparator() {
- return BytesRef.getUTF8SortedAsUnicodeComparator();
- }
-
- @Override
public void seekExact(BytesRef term, TermState state) throws IOException {
assert state != null && state instanceof OrdTermState;
this.seekExact(((OrdTermState)state).ord);