LUCENE-5189: add NumericDocValues updates

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1523461 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Shai Erera 2013-09-15 17:11:52 +00:00
parent 11a2ca8959
commit 1120e46961
30 changed files with 2558 additions and 451 deletions

View File

@ -17,6 +17,16 @@ package org.apache.lucene.codecs.simpletext;
* limitations under the License.
*/
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.END;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.FIELD;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.LENGTH;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.MAXLENGTH;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.MINVALUE;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.NUMVALUES;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.ORDPATTERN;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.PATTERN;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.TYPE;
import java.io.IOException;
import java.math.BigDecimal;
import java.math.BigInteger;
@ -31,27 +41,17 @@ import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.StringHelper;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.END;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.FIELD;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.LENGTH;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.MAXLENGTH;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.MINVALUE;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.NUMVALUES;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.ORDPATTERN;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.PATTERN;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.TYPE;
class SimpleTextDocValuesReader extends DocValuesProducer {
static class OneField {
@ -62,8 +62,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
boolean fixedLength;
long minValue;
long numValues;
};
}
final int maxDoc;
final IndexInput data;
@ -71,7 +70,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
final Map<String,OneField> fields = new HashMap<String,OneField>();
public SimpleTextDocValuesReader(SegmentReadState state, String ext) throws IOException {
//System.out.println("dir=" + state.directory + " seg=" + state.segmentInfo.name + " ext=" + ext);
// System.out.println("dir=" + state.directory + " seg=" + state.segmentInfo.name + " file=" + IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext));
data = state.directory.openInput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext), state.context);
maxDoc = state.segmentInfo.getDocCount();
while(true) {
@ -83,8 +82,6 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
assert startsWith(FIELD) : scratch.utf8ToString();
String fieldName = stripPrefix(FIELD);
//System.out.println(" field=" + fieldName);
FieldInfo fieldInfo = state.fieldInfos.fieldInfo(fieldName);
assert fieldInfo != null;
OneField field = new OneField();
fields.put(fieldName, field);

View File

@ -55,7 +55,7 @@ class SimpleTextDocValuesWriter extends DocValuesConsumer {
private final Set<String> fieldsSeen = new HashSet<String>(); // for asserting
public SimpleTextDocValuesWriter(SegmentWriteState state, String ext) throws IOException {
//System.out.println("WRITE: " + IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext) + " " + state.segmentInfo.getDocCount() + " docs");
// System.out.println("WRITE: " + IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext) + " " + state.segmentInfo.getDocCount() + " docs");
data = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext), state.context);
numDocs = state.segmentInfo.getDocCount();
}

View File

@ -22,13 +22,13 @@ import java.io.IOException;
import java.util.HashMap;
import java.util.IdentityHashMap;
import java.util.Map;
import java.util.ServiceLoader; // javadocs
import java.util.ServiceLoader;
import java.util.TreeMap;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.NumericDocValues;
@ -76,11 +76,10 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
}
@Override
public final DocValuesConsumer fieldsConsumer(SegmentWriteState state)
throws IOException {
public final DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return new FieldsWriter(state);
}
static class ConsumerAndSuffix implements Closeable {
DocValuesConsumer consumer;
int suffix;
@ -97,7 +96,7 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
private final Map<String,Integer> suffixes = new HashMap<String,Integer>();
private final SegmentWriteState segmentWriteState;
public FieldsWriter(SegmentWriteState state) {
segmentWriteState = state;
}
@ -123,32 +122,44 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
}
private DocValuesConsumer getInstance(FieldInfo field) throws IOException {
final DocValuesFormat format = getDocValuesFormatForField(field.name);
final DocValuesFormat format;
if (segmentWriteState.isFieldUpdate) {
final String formatName = field.getAttribute(PER_FIELD_FORMAT_KEY);
assert formatName != null : "invalid null FORMAT_KEY for field=\"" + field.name + "\" (field updates)";
format = DocValuesFormat.forName(formatName);
} else {
format = getDocValuesFormatForField(field.name);
}
if (format == null) {
throw new IllegalStateException("invalid null DocValuesFormat for field=\"" + field.name + "\"");
}
final String formatName = format.getName();
String previousValue = field.putAttribute(PER_FIELD_FORMAT_KEY, formatName);
assert previousValue == null: "formatName=" + formatName + " prevValue=" + previousValue;
assert segmentWriteState.isFieldUpdate || previousValue == null: "formatName=" + formatName + " prevValue=" + previousValue;
Integer suffix;
ConsumerAndSuffix consumer = formats.get(format);
if (consumer == null) {
// First time we are seeing this format; create a new instance
// bump the suffix
suffix = suffixes.get(formatName);
if (suffix == null) {
suffix = 0;
if (segmentWriteState.isFieldUpdate) {
final String suffixAtt = field.getAttribute(PER_FIELD_SUFFIX_KEY);
assert suffixAtt != null : "invalid numm SUFFIX_KEY for field=\"" + field.name + "\" (field updates)";
suffix = Integer.valueOf(suffixAtt);
} else {
suffix = suffix + 1;
// bump the suffix
suffix = suffixes.get(formatName);
if (suffix == null) {
suffix = 0;
} else {
suffix = suffix + 1;
}
}
suffixes.put(formatName, suffix);
final String segmentSuffix = getFullSegmentSuffix(field.name,
segmentWriteState.segmentSuffix,
final String segmentSuffix = getFullSegmentSuffix(segmentWriteState.segmentSuffix,
getSuffix(formatName, Integer.toString(suffix)));
consumer = new ConsumerAndSuffix();
consumer.consumer = format.fieldsConsumer(new SegmentWriteState(segmentWriteState, segmentSuffix));
@ -161,10 +172,10 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
}
previousValue = field.putAttribute(PER_FIELD_SUFFIX_KEY, Integer.toString(suffix));
assert previousValue == null;
assert segmentWriteState.isFieldUpdate || previousValue == null : "suffix=" + Integer.toString(suffix) + " prevValue=" + previousValue;
// TODO: we should only provide the "slice" of FIS
// that this PF actually sees ...
// that this DVF actually sees ...
return consumer.consumer;
}
@ -179,14 +190,11 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
return formatName + "_" + suffix;
}
static String getFullSegmentSuffix(String fieldName, String outerSegmentSuffix, String segmentSuffix) {
static String getFullSegmentSuffix(String outerSegmentSuffix, String segmentSuffix) {
if (outerSegmentSuffix.length() == 0) {
return segmentSuffix;
} else {
// TODO: support embedding; I think it should work but
// we need a test confirm to confirm
// return outerSegmentSuffix + "_" + segmentSuffix;
throw new IllegalStateException("cannot embed PerFieldPostingsFormat inside itself (field \"" + fieldName + "\" returned PerFieldPostingsFormat)");
return outerSegmentSuffix + "_" + segmentSuffix;
}
}
@ -210,7 +218,7 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
final String suffix = fi.getAttribute(PER_FIELD_SUFFIX_KEY);
assert suffix != null;
DocValuesFormat format = DocValuesFormat.forName(formatName);
String segmentSuffix = getSuffix(formatName, suffix);
String segmentSuffix = getFullSegmentSuffix(readState.segmentSuffix, getSuffix(formatName, suffix));
if (!formats.containsKey(segmentSuffix)) {
formats.put(segmentSuffix, format.fieldsProducer(new SegmentReadState(readState, segmentSuffix)));
}

View File

@ -19,25 +19,26 @@ package org.apache.lucene.index;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.RamUsageEstimator;
/* Holds buffered deletes, by docID, term or query for a
/* Holds buffered deletes and updates, by docID, term or query for a
* single segment. This is used to hold buffered pending
* deletes against the to-be-flushed segment. Once the
* deletes are pushed (on flush in DocumentsWriter), these
* deletes are converted to a FrozenDeletes instance. */
* deletes and updates against the to-be-flushed segment. Once the
* deletes and updates are pushed (on flush in DocumentsWriter), they
* are converted to a FrozenDeletes instance. */
// NOTE: instances of this class are accessed either via a private
// instance on DocumentWriterPerThread, or via sync'd code by
// DocumentsWriterDeleteQueue
class BufferedDeletes {
class BufferedDeletes { // TODO (DVU_RENAME) BufferedUpdates?
/* Rough logic: HashMap has an array[Entry] w/ varying
load factor (say 2 * POINTER). Entry is object w/ Term
@ -63,11 +64,45 @@ class BufferedDeletes {
undercount (say 24 bytes). Integer is OBJ_HEADER + INT. */
final static int BYTES_PER_DEL_QUERY = 5*RamUsageEstimator.NUM_BYTES_OBJECT_REF + 2*RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 2*RamUsageEstimator.NUM_BYTES_INT + 24;
/* Rough logic: NumericUpdate calculates its actual size,
* including the update Term and DV field (String). The
* per-term map holds a reference to the update Term, and
* therefore we only account for the object reference and
* map space itself. This is incremented when we first see
* an update Term.
* LinkedHashMap has an array[Entry] w/ varying load factor
* (say 2*POINTER). Entry is an object w/ Term key, Map val,
* int hash, Entry next, Entry before, Entry after (OBJ_HEADER + 5*POINTER + INT).
* Term (key) is counted only as POINTER.
* Map (val) is counted as OBJ_HEADER, array[Entry] ref + header, 4*INT, 1*FLOAT,
* Set (entrySet) (2*OBJ_HEADER + ARRAY_HEADER + 2*POINTER + 4*INT + FLOAT)
*/
final static int BYTES_PER_NUMERIC_UPDATE_TERM_ENTRY =
9*RamUsageEstimator.NUM_BYTES_OBJECT_REF + 3*RamUsageEstimator.NUM_BYTES_OBJECT_HEADER +
RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + 5*RamUsageEstimator.NUM_BYTES_INT + RamUsageEstimator.NUM_BYTES_FLOAT;
/* Rough logic: Incremented when we see another field for an already updated
* Term.
* HashMap has an array[Entry] w/ varying load
* factor (say 2*POINTER). Entry is an object w/ String key,
* NumericUpdate val, int hash, Entry next (OBJ_HEADER + 3*POINTER + INT).
* NumericUpdate returns its own size, and therefore isn't accounted for here.
*/
final static int BYTES_PER_NUMERIC_UPDATE_ENTRY = 5*RamUsageEstimator.NUM_BYTES_OBJECT_REF + RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + RamUsageEstimator.NUM_BYTES_INT;
final AtomicInteger numTermDeletes = new AtomicInteger();
final AtomicInteger numNumericUpdates = new AtomicInteger();
final Map<Term,Integer> terms = new HashMap<Term,Integer>();
final Map<Query,Integer> queries = new HashMap<Query,Integer>();
final List<Integer> docIDs = new ArrayList<Integer>();
// Map<updateTerm,Map<dvField,NumericUpdate>>
// LinkedHashMap because we need to preserve the order of the updates. That
// is, if two terms update the same document and same DV field, whoever came
// in last should win. LHM guarantees we iterate on the map in insertion
// order.
final Map<Term,Map<String,NumericUpdate>> numericUpdates = new LinkedHashMap<Term,Map<String,NumericUpdate>>();
public static final Integer MAX_INT = Integer.valueOf(Integer.MAX_VALUE);
final AtomicLong bytesUsed;
@ -75,21 +110,17 @@ class BufferedDeletes {
private final static boolean VERBOSE_DELETES = false;
long gen;
public BufferedDeletes() {
this(new AtomicLong());
}
BufferedDeletes(AtomicLong bytesUsed) {
assert bytesUsed != null;
this.bytesUsed = bytesUsed;
this.bytesUsed = new AtomicLong();
}
@Override
public String toString() {
if (VERBOSE_DELETES) {
return "gen=" + gen + " numTerms=" + numTermDeletes + ", terms=" + terms
+ ", queries=" + queries + ", docIDs=" + docIDs + ", bytesUsed="
+ bytesUsed;
+ ", queries=" + queries + ", docIDs=" + docIDs + ", numericUpdates=" + numericUpdates
+ ", bytesUsed=" + bytesUsed;
} else {
String s = "gen=" + gen;
if (numTermDeletes.get() != 0) {
@ -101,6 +132,9 @@ class BufferedDeletes {
if (docIDs.size() != 0) {
s += " " + docIDs.size() + " deleted docIDs";
}
if (numNumericUpdates.get() != 0) {
s += " " + numNumericUpdates.get() + " numeric updates (unique count=" + numericUpdates.size() + ")";
}
if (bytesUsed.get() != 0) {
s += " bytesUsed=" + bytesUsed.get();
}
@ -145,20 +179,41 @@ class BufferedDeletes {
}
}
public void addNumericUpdate(NumericUpdate update, int docIDUpto) {
Map<String,NumericUpdate> termUpdates = numericUpdates.get(update.term);
if (termUpdates == null) {
termUpdates = new HashMap<String,NumericUpdate>();
numericUpdates.put(update.term, termUpdates);
bytesUsed.addAndGet(BYTES_PER_NUMERIC_UPDATE_TERM_ENTRY);
}
final NumericUpdate current = termUpdates.get(update.field);
if (current != null && docIDUpto < current.docIDUpto) {
// Only record the new number if it's greater than or equal to the current
// one. This is important because if multiple threads are replacing the
// same doc at nearly the same time, it's possible that one thread that
// got a higher docID is scheduled before the other threads.
return;
}
update.docIDUpto = docIDUpto;
termUpdates.put(update.field, update);
numNumericUpdates.incrementAndGet();
if (current == null) {
bytesUsed.addAndGet(BYTES_PER_NUMERIC_UPDATE_ENTRY + update.sizeInBytes());
}
}
void clear() {
terms.clear();
queries.clear();
docIDs.clear();
numericUpdates.clear();
numTermDeletes.set(0);
numNumericUpdates.set(0);
bytesUsed.set(0);
}
void clearDocIDs() {
bytesUsed.addAndGet(-docIDs.size()*BYTES_PER_DEL_DOCID);
docIDs.clear();
}
boolean any() {
return terms.size() > 0 || docIDs.size() > 0 || queries.size() > 0;
return terms.size() > 0 || docIDs.size() > 0 || queries.size() > 0 || numericUpdates.size() > 0;
}
}

View File

@ -18,10 +18,11 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.List;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
@ -35,12 +36,12 @@ import org.apache.lucene.util.InfoStream;
/* Tracks the stream of {@link BufferedDeletes}.
* When DocumentsWriterPerThread flushes, its buffered
* deletes are appended to this stream. We later
* apply these deletes (resolve them to the actual
* deletes and updates are appended to this stream. We later
* apply them (resolve them to the actual
* docIDs, per segment) when a merge is started
* (only to the to-be-merged segments). We
* also apply to all segments when NRT reader is pulled,
* commit/close is called, or when too many deletes are
* commit/close is called, or when too many deletes or updates are
* buffered and must be flushed (by RAM usage or by count).
*
* Each packet is assigned a generation, and each flushed or
@ -48,7 +49,7 @@ import org.apache.lucene.util.InfoStream;
* track which BufferedDeletes packets to apply to any given
* segment. */
class BufferedDeletesStream {
class BufferedDeletesStream { // TODO (DVU_RENAME) BufferedUpdatesStream
// TODO: maybe linked list?
private final List<FrozenBufferedDeletes> deletes = new ArrayList<FrozenBufferedDeletes>();
@ -114,6 +115,7 @@ class BufferedDeletesStream {
}
public static class ApplyDeletesResult {
// True if any actual deletes took place:
public final boolean anyDeletes;
@ -123,10 +125,14 @@ class BufferedDeletesStream {
// If non-null, contains segments that are 100% deleted
public final List<SegmentInfoPerCommit> allDeleted;
ApplyDeletesResult(boolean anyDeletes, long gen, List<SegmentInfoPerCommit> allDeleted) {
// True if any actual numeric docvalues updates took place
public final boolean anyNumericDVUpdates;
ApplyDeletesResult(boolean anyDeletes, long gen, List<SegmentInfoPerCommit> allDeleted, boolean anyNumericDVUpdates) {
this.anyDeletes = anyDeletes;
this.gen = gen;
this.allDeleted = allDeleted;
this.anyNumericDVUpdates = anyNumericDVUpdates;
}
}
@ -145,7 +151,7 @@ class BufferedDeletesStream {
final long t0 = System.currentTimeMillis();
if (infos.size() == 0) {
return new ApplyDeletesResult(false, nextGen++, null);
return new ApplyDeletesResult(false, nextGen++, null, false);
}
assert checkDeleteStats();
@ -154,7 +160,7 @@ class BufferedDeletesStream {
if (infoStream.isEnabled("BD")) {
infoStream.message("BD", "applyDeletes: no deletes; skipping");
}
return new ApplyDeletesResult(false, nextGen++, null);
return new ApplyDeletesResult(false, nextGen++, null, false);
}
if (infoStream.isEnabled("BD")) {
@ -169,6 +175,7 @@ class BufferedDeletesStream {
CoalescedDeletes coalescedDeletes = null;
boolean anyNewDeletes = false;
boolean anyNewUpdates = false;
int infosIDX = infos2.size()-1;
int delIDX = deletes.size()-1;
@ -183,7 +190,7 @@ class BufferedDeletesStream {
final long segGen = info.getBufferedDeletesGen();
if (packet != null && segGen < packet.delGen()) {
//System.out.println(" coalesce");
// System.out.println(" coalesce");
if (coalescedDeletes == null) {
coalescedDeletes = new CoalescedDeletes();
}
@ -206,7 +213,7 @@ class BufferedDeletesStream {
// Lock order: IW -> BD -> RP
assert readerPool.infoIsLive(info);
final ReadersAndLiveDocs rld = readerPool.get(info, true);
final SegmentReader reader = rld.getReader(IOContext.READ);
final SegmentReader reader = rld.getReader(false, IOContext.READ); // don't apply deletes, as we're about to add more!
int delCount = 0;
final boolean segAllDeletes;
try {
@ -214,11 +221,13 @@ class BufferedDeletesStream {
//System.out.println(" del coalesced");
delCount += applyTermDeletes(coalescedDeletes.termsIterable(), rld, reader);
delCount += applyQueryDeletes(coalescedDeletes.queriesIterable(), rld, reader);
anyNewUpdates |= applyNumericDocValueUpdates(coalescedDeletes.numericDVUpdates, rld, reader);
}
//System.out.println(" del exact");
// Don't delete by Term here; DocumentsWriterPerThread
// already did that on flush:
delCount += applyQueryDeletes(packet.queriesIterable(), rld, reader);
anyNewUpdates |= applyNumericDocValueUpdates(Arrays.asList(packet.updates), rld, reader);
final int fullDelCount = rld.info.getDelCount() + rld.getPendingDeleteCount();
assert fullDelCount <= rld.info.info.getDocCount();
segAllDeletes = fullDelCount == rld.info.info.getDocCount();
@ -259,12 +268,13 @@ class BufferedDeletesStream {
// Lock order: IW -> BD -> RP
assert readerPool.infoIsLive(info);
final ReadersAndLiveDocs rld = readerPool.get(info, true);
final SegmentReader reader = rld.getReader(IOContext.READ);
final SegmentReader reader = rld.getReader(false, IOContext.READ); // don't apply deletes, as we're about to add more!
int delCount = 0;
final boolean segAllDeletes;
try {
delCount += applyTermDeletes(coalescedDeletes.termsIterable(), rld, reader);
delCount += applyQueryDeletes(coalescedDeletes.queriesIterable(), rld, reader);
anyNewUpdates |= applyNumericDocValueUpdates(coalescedDeletes.numericDVUpdates, rld, reader);
final int fullDelCount = rld.info.getDelCount() + rld.getPendingDeleteCount();
assert fullDelCount <= rld.info.info.getDocCount();
segAllDeletes = fullDelCount == rld.info.info.getDocCount();
@ -297,7 +307,7 @@ class BufferedDeletesStream {
}
// assert infos != segmentInfos || !any() : "infos=" + infos + " segmentInfos=" + segmentInfos + " any=" + any;
return new ApplyDeletesResult(anyNewDeletes, gen, allDeleted);
return new ApplyDeletesResult(anyNewDeletes, gen, allDeleted, anyNewUpdates);
}
synchronized long getNextGen() {
@ -402,15 +412,15 @@ class BufferedDeletesStream {
if (docID == DocIdSetIterator.NO_MORE_DOCS) {
break;
}
if (!any) {
rld.initWritableLiveDocs();
any = true;
}
// NOTE: there is no limit check on the docID
// when deleting by Term (unlike by Query)
// because on flush we apply all Term deletes to
// each segment. So all Term deleting here is
// against prior segments:
if (!any) {
rld.initWritableLiveDocs();
any = true;
}
if (rld.delete(docID)) {
delCount++;
}
@ -422,6 +432,65 @@ class BufferedDeletesStream {
return delCount;
}
// NumericDocValue Updates
private synchronized boolean applyNumericDocValueUpdates(Iterable<NumericUpdate> updates, ReadersAndLiveDocs rld, SegmentReader reader) throws IOException {
Fields fields = reader.fields();
if (fields == null) {
// This reader has no postings
return false;
}
TermsEnum termsEnum = null;
DocsEnum docs = null;
boolean any = false;
//System.out.println(Thread.currentThread().getName() + " numericDVUpdate reader=" + reader);
for (NumericUpdate update : updates) {
Term term = update.term;
int limit = update.docIDUpto;
// TODO: we rely on the map being ordered by updates order, not by terms order.
// we need that so that if two terms update the same document, the one that came
// last wins.
// alternatively, we could keep a map from doc->lastUpto and apply the update
// in terms order, where an update is applied only if its docIDUpto is greater
// than lastUpto.
// but, since app can send two updates, in order, which will have same upto, we
// cannot rely solely on docIDUpto, and need to have our own gen, which is
// incremented with every update.
// Unlike applyTermDeletes, we visit terms in update order, not term order.
// Therefore we cannot assume we can only seek forwards and must ask for a
// new TermsEnum
Terms terms = fields.terms(term.field);
if (terms == null) { // no terms in that field
termsEnum = null;
continue;
}
termsEnum = terms.iterator(termsEnum);
// System.out.println(" term=" + term);
if (termsEnum.seekExact(term.bytes())) {
// we don't need term frequencies for this
DocsEnum docsEnum = termsEnum.docs(rld.getLiveDocs(), docs, DocsEnum.FLAG_NONE);
//System.out.println("BDS: got docsEnum=" + docsEnum);
int doc;
while ((doc = docsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
//System.out.println(Thread.currentThread().getName() + " numericDVUpdate term=" + term + " doc=" + docID);
if (doc >= limit) {
break; // no more docs that can be updated for this term
}
rld.updateNumericDocValue(update.field, doc, update.value);
any = true;
}
}
}
return any;
}
public static class QueryAndLimit {
public final Query query;
public final int limit;

View File

@ -29,20 +29,27 @@ import org.apache.lucene.index.BufferedDeletesStream.QueryAndLimit;
class CoalescedDeletes {
final Map<Query,Integer> queries = new HashMap<Query,Integer>();
final List<Iterable<Term>> iterables = new ArrayList<Iterable<Term>>();
final List<NumericUpdate> numericDVUpdates = new ArrayList<NumericUpdate>();
@Override
public String toString() {
// note: we could add/collect more debugging information
return "CoalescedDeletes(termSets=" + iterables.size() + ",queries=" + queries.size() + ")";
return "CoalescedDeletes(termSets=" + iterables.size() + ",queries=" + queries.size() + ",numericUpdates=" + numericDVUpdates.size() + ")";
}
void update(FrozenBufferedDeletes in) {
iterables.add(in.termsIterable());
for(int queryIdx=0;queryIdx<in.queries.length;queryIdx++) {
for (int queryIdx = 0; queryIdx < in.queries.length; queryIdx++) {
final Query query = in.queries[queryIdx];
queries.put(query, BufferedDeletes.MAX_INT);
}
for (NumericUpdate nu : in.updates) {
NumericUpdate clone = new NumericUpdate(nu.term, nu.field, nu.value);
clone.docIDUpto = Integer.MAX_VALUE;
numericDVUpdates.add(clone);
}
}
public Iterable<Term> termsIterable() {

View File

@ -158,6 +158,15 @@ final class DocumentsWriter {
return applyAllDeletes( deleteQueue);
}
synchronized void updateNumericDocValue(Term term, String field, Long value) throws IOException {
final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue;
deleteQueue.addNumericUpdate(new NumericUpdate(term, field, value));
flushControl.doOnDelete();
if (flushControl.doApplyAllDeletes()) {
applyAllDeletes(deleteQueue);
}
}
DocumentsWriterDeleteQueue currentDeleteSession() {
return deleteQueue;
}

View File

@ -107,6 +107,11 @@ final class DocumentsWriterDeleteQueue {
tryApplyGlobalSlice();
}
void addNumericUpdate(NumericUpdate update) {
add(new NumericUpdateNode(update));
tryApplyGlobalSlice();
}
/**
* invariant for document update
*/
@ -380,6 +385,22 @@ final class DocumentsWriterDeleteQueue {
}
}
private static final class NumericUpdateNode extends Node<NumericUpdate> {
NumericUpdateNode(NumericUpdate update) {
super(update);
}
@Override
void apply(BufferedDeletes bufferedDeletes, int docIDUpto) {
bufferedDeletes.addNumericUpdate(item, docIDUpto);
}
@Override
public String toString() {
return "update=" + item;
}
}
private boolean forceApplyGlobalSlice() {
globalBufferLock.lock();

View File

@ -17,9 +17,11 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK;
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
import java.io.IOException;
import java.text.NumberFormat;
import java.util.Collection;
import java.util.HashSet;
import java.util.Locale;
import java.util.Set;
@ -41,9 +43,6 @@ import org.apache.lucene.util.IntBlockPool;
import org.apache.lucene.util.MutableBits;
import org.apache.lucene.util.RamUsageEstimator;
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK;
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
class DocumentsWriterPerThread {
/**
@ -174,8 +173,9 @@ class DocumentsWriterPerThread {
final DocConsumer consumer;
final Counter bytesUsed;
//Deletes for our still-in-RAM (to be flushed next) segment
final BufferedDeletes pendingDeletes;
SegmentWriteState flushState;
// Deletes for our still-in-RAM (to be flushed next) segment
final BufferedDeletes pendingDeletes;
private final SegmentInfo segmentInfo; // Current segment we are working on
boolean aborting = false; // True if an abort is pending
boolean hasAborted = false; // True if the last exception throws by #updateDocument was aborting
@ -467,7 +467,7 @@ class DocumentsWriterPerThread {
pendingDeletes.terms.clear();
segmentInfo.setFiles(new HashSet<String>(directory.getCreatedFiles()));
final SegmentInfoPerCommit segmentInfoPerCommit = new SegmentInfoPerCommit(segmentInfo, 0, -1L);
final SegmentInfoPerCommit segmentInfoPerCommit = new SegmentInfoPerCommit(segmentInfo, 0, -1L, -1L);
if (infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : (flushState.segmentInfo.getDocCount() - flushState.delCountOnFlush)) + " deleted docs");
infoStream.message("DWPT", "new segment has " +
@ -481,7 +481,8 @@ class DocumentsWriterPerThread {
}
final BufferedDeletes segmentDeletes;
if (pendingDeletes.queries.isEmpty()) {
if (pendingDeletes.queries.isEmpty() && pendingDeletes.numericUpdates.isEmpty()) {
pendingDeletes.clear();
segmentDeletes = null;
} else {
segmentDeletes = pendingDeletes;

View File

@ -79,7 +79,7 @@ public final class FieldInfo {
* Character offsets are encoded alongside the positions.
*/
DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS,
};
}
/**
* DocValues types.
@ -110,7 +110,7 @@ public final class FieldInfo {
* ordinal and by-value. Values must be <= 32766 bytes.
*/
SORTED_SET
};
}
/**
* Sole Constructor.

View File

@ -223,6 +223,20 @@ public class FieldInfos implements Iterable<FieldInfo> {
(dvType == null || docValuesType.get(name) == null || dvType == docValuesType.get(name));
}
/**
* Returns true if the {@code fieldName} exists in the map and is of the
* same {@code dvType}.
*/
synchronized boolean contains(String fieldName, DocValuesType dvType) {
// used by IndexWriter.updateNumericDocValue
if (!nameToNumber.containsKey(fieldName)) {
return false;
} else {
// only return true if the field has the same dvType as the requested one
return dvType == docValuesType.get(fieldName);
}
}
synchronized void clear() {
numberToName.clear();
nameToNumber.clear();

View File

@ -17,32 +17,38 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.lucene.index.BufferedDeletesStream.QueryAndLimit;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.index.BufferedDeletesStream.QueryAndLimit;
/** Holds buffered deletes by term or query, once pushed.
* Pushed deletes are write-once, so we shift to more
* memory efficient data structure to hold them. We don't
* hold docIDs because these are applied on flush. */
class FrozenBufferedDeletes {
/**
* Holds buffered deletes and updates by term or query, once pushed. Pushed
* deletes/updates are write-once, so we shift to more memory efficient data
* structure to hold them. We don't hold docIDs because these are applied on
* flush.
*/
class FrozenBufferedDeletes { // TODO (DVU_RENAME) FrozenBufferedUpdates?
/* Query we often undercount (say 24 bytes), plus int. */
final static int BYTES_PER_DEL_QUERY = RamUsageEstimator.NUM_BYTES_OBJECT_REF + RamUsageEstimator.NUM_BYTES_INT + 24;
// Terms, in sorted order:
final PrefixCodedTerms terms;
int termCount; // just for debugging
// Parallel array of deleted query, and the docIDUpto for
// each
// Parallel array of deleted query, and the docIDUpto for each
final Query[] queries;
final int[] queryLimits;
// numeric DV update term and their updates
final NumericUpdate[] updates;
final int bytesUsed;
final int numTermDeletes;
private long gen = -1; // assigned by BufferedDeletesStream once pushed
@ -72,7 +78,17 @@ class FrozenBufferedDeletes {
upto++;
}
bytesUsed = (int) terms.getSizeInBytes() + queries.length * BYTES_PER_DEL_QUERY;
List<NumericUpdate> allUpdates = new ArrayList<NumericUpdate>();
int numericUpdatesSize = 0;
for (Map<String,NumericUpdate> fieldUpdates : deletes.numericUpdates.values()) {
for (NumericUpdate update : fieldUpdates.values()) {
allUpdates.add(update);
numericUpdatesSize += update.sizeInBytes();
}
}
updates = allUpdates.toArray(new NumericUpdate[allUpdates.size()]);
bytesUsed = (int) terms.getSizeInBytes() + queries.length * BYTES_PER_DEL_QUERY + numericUpdatesSize + updates.length * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
numTermDeletes = deletes.numTermDeletes.get();
}
@ -140,6 +156,6 @@ class FrozenBufferedDeletes {
}
boolean any() {
return termCount > 0 || queries.length > 0;
return termCount > 0 || queries.length > 0 || updates.length > 0;
}
}

View File

@ -36,6 +36,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.FieldInfos.FieldNumbers;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.index.MergePolicy.MergeTrigger;
@ -459,10 +460,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
// This is the last ref to this RLD, and we're not
// pooling, so remove it:
if (rld.writeLiveDocs(directory)) {
// Make sure we only write del docs for a live segment:
// Make sure we only write del docs and field updates for a live segment:
assert infoIsLive(rld.info);
// Must checkpoint w/ deleter, because we just
// created created new _X_N.del file.
// created new _X_N.del and field updates files.
deleter.checkpoint(segmentInfos, false);
}
@ -481,10 +482,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
try {
if (doSave && rld.writeLiveDocs(directory)) {
// Make sure we only write del docs for a live segment:
// Make sure we only write del docs and field updates for a live segment:
assert infoIsLive(rld.info);
// Must checkpoint w/ deleter, because we just
// created created new _X_N.del file.
// created created new _X_N.del and field updates files.
deleter.checkpoint(segmentInfos, false);
}
} catch (Throwable t) {
@ -529,11 +530,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
if (rld != null) {
assert rld.info == info;
if (rld.writeLiveDocs(directory)) {
// Make sure we only write del docs for a live segment:
// Make sure we only write del docs and updates for a live segment:
assert infoIsLive(info);
// Must checkpoint w/ deleter, because we just
// created created new _X_N.del file.
// created new _X_N.del and field updates files.
deleter.checkpoint(segmentInfos, false);
// we wrote liveDocs and field updates, reopen the reader
rld.reopenReader(IOContext.READ);
}
}
}
@ -1532,6 +1537,48 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
}
}
/**
* Updates a document's NumericDocValue for <code>field</code> to the given
* <code>value</code>. This method can be used to 'unset' a document's value
* by passing {@code null} as the new value. Also, you can only update fields
* that already exist in the index, not add new fields through this method.
*
* <p>
* <b>NOTE:</b> it is currently not allowed to update the value of documents
* in a segment where the field does not exist (even though it may exist in
* other segments). If you try that, you will hit an
* {@link UnsupportedOperationException} when the segment is later flushed
* (following an NRT reader reopen, commit, forceMerge etc.).
*
* <p>
* <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
* close the writer. See <a href="#OOME">above</a> for details.
* </p>
*
* @param term
* the term to identify the document(s) to be updated
* @param field
* field name of the NumericDocValues field
* @param value
* new value for the field
* @throws CorruptIndexException
* if the index is corrupt
* @throws IOException
* if there is a low-level IO error
*/
// TODO (DVU_FIELDINFOS_GEN) remove the paragraph on updating segments without the field not allowed
public void updateNumericDocValue(Term term, String field, Long value) throws IOException {
ensureOpen();
if (!globalFieldNumberMap.contains(field, DocValuesType.NUMERIC)) {
throw new IllegalArgumentException("can only update existing numeric-docvalues fields!");
}
try {
docWriter.updateNumericDocValue(term, field, value);
} catch (OutOfMemoryError oom) {
handleOOM(oom, "updateNumericDocValue");
}
}
// for test purpose
final synchronized int getSegmentCount(){
return segmentInfos.size();
@ -1915,7 +1962,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
merge.maxNumSegments = maxNumSegments;
}
}
} else {
spec = mergePolicy.findMerges(trigger, segmentInfos);
}
@ -2512,7 +2558,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
}
}
SegmentInfoPerCommit infoPerCommit = new SegmentInfoPerCommit(info, 0, -1L);
SegmentInfoPerCommit infoPerCommit = new SegmentInfoPerCommit(info, 0, -1L, -1L);
info.setFiles(new HashSet<String>(trackingDir.getCreatedFiles()));
trackingDir.getCreatedFiles().clear();
@ -2599,7 +2645,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
SegmentInfo newInfo = new SegmentInfo(directory, info.info.getVersion(), segName, info.info.getDocCount(),
info.info.getUseCompoundFile(),
info.info.getCodec(), info.info.getDiagnostics(), attributes);
SegmentInfoPerCommit newInfoPerCommit = new SegmentInfoPerCommit(newInfo, info.getDelCount(), info.getDelGen());
SegmentInfoPerCommit newInfoPerCommit = new SegmentInfoPerCommit(newInfo, info.getDelCount(), info.getDelGen(), info.getDocValuesGen());
Set<String> segFiles = new HashSet<String>();
@ -3011,7 +3057,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
flushDeletesCount.incrementAndGet();
final BufferedDeletesStream.ApplyDeletesResult result;
result = bufferedDeletesStream.applyDeletes(readerPool, segmentInfos.asList());
if (result.anyDeletes) {
if (result.anyDeletes || result.anyNumericDVUpdates) {
checkpoint();
}
if (!keepFullyDeletedSegments && result.allDeleted != null) {
@ -3063,15 +3109,23 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
}
}
/** Carefully merges deletes for the segments we just
* merged. This is tricky because, although merging will
* clear all deletes (compacts the documents), new
* deletes may have been flushed to the segments since
* the merge was started. This method "carries over"
* such new deletes onto the newly merged segment, and
* saves the resulting deletes file (incrementing the
* delete generation for merge.info). If no deletes were
* flushed, no new deletes file is saved. */
private MergePolicy.DocMap getDocMap(MergePolicy.OneMerge merge, MergeState mergeState) {
MergePolicy.DocMap docMap = merge.getDocMap(mergeState);
assert docMap.isConsistent(merge.info.info.getDocCount());
return docMap;
}
/**
* Carefully merges deletes and updates for the segments we just merged. This
* is tricky because, although merging will clear all deletes (compacts the
* documents) and compact all the updates, new deletes and updates may have
* been flushed to the segments since the merge was started. This method
* "carries over" such new deletes and updates onto the newly merged segment,
* and saves the resulting deletes and updates files (incrementing the delete
* and DV generations for merge.info). If no deletes were flushed, no new
* deletes file is saved.
*/
// TODO (DVU_RENAME) to commitMergedDeletesAndUpdates
synchronized private ReadersAndLiveDocs commitMergedDeletes(MergePolicy.OneMerge merge, MergeState mergeState) throws IOException {
assert testPoint("startCommitMergeDeletes");
@ -3088,20 +3142,22 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
long minGen = Long.MAX_VALUE;
// Lazy init (only when we find a delete to carry over):
ReadersAndLiveDocs mergedDeletes = null;
ReadersAndLiveDocs mergedDeletes = null; // TODO (DVU_RENAME) to mergedDeletesAndUpdates
boolean initWritableLiveDocs = false;
MergePolicy.DocMap docMap = null;
for(int i=0; i < sourceSegments.size(); i++) {
final Map<Integer,Map<String,Long>> mergedUpdates = new HashMap<Integer,Map<String,Long>>();
for (int i = 0; i < sourceSegments.size(); i++) {
SegmentInfoPerCommit info = sourceSegments.get(i);
minGen = Math.min(info.getBufferedDeletesGen(), minGen);
final int docCount = info.info.getDocCount();
final Bits prevLiveDocs = merge.readers.get(i).getLiveDocs();
final Bits currentLiveDocs;
final ReadersAndLiveDocs rld = readerPool.get(info, false);
// We hold a ref so it should still be in the pool:
assert rld != null: "seg=" + info.info.name;
currentLiveDocs = rld.getLiveDocs();
final Bits currentLiveDocs = rld.getLiveDocs();
final Map<Integer,Map<String,Long>> mergingUpdates = rld.getMergingUpdates();
if (prevLiveDocs != null) {
// If we had deletions on starting the merge we must
@ -3123,11 +3179,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
// If so, we must carefully merge the liveDocs one
// doc at a time:
if (currentLiveDocs != prevLiveDocs) {
// This means this segment received new deletes
// since we started the merge, so we
// must merge them:
for(int j=0;j<docCount;j++) {
for (int j = 0; j < docCount; j++) {
if (!prevLiveDocs.get(j)) {
assert !currentLiveDocs.get(j);
} else {
@ -3135,14 +3190,44 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
if (mergedDeletes == null) {
mergedDeletes = readerPool.get(merge.info, true);
mergedDeletes.initWritableLiveDocs();
docMap = merge.getDocMap(mergeState);
assert docMap.isConsistent(merge.info.info.getDocCount());
initWritableLiveDocs = true;
docMap = getDocMap(merge, mergeState);
} else if (!initWritableLiveDocs) { // mergedDeletes was initialized by field-updates changes
mergedDeletes.initWritableLiveDocs();
initWritableLiveDocs = true;
}
mergedDeletes.delete(docMap.map(docUpto));
} else if (mergingUpdates != null) {
// document isn't deleted, check if it has updates
Map<String,Long> docUpdates = mergingUpdates.get(Integer.valueOf(j));
if (docUpdates != null) {
if (mergedDeletes == null) {
mergedDeletes = readerPool.get(merge.info, true);
docMap = getDocMap(merge, mergeState);
}
mergedUpdates.put(Integer.valueOf(docMap.map(docUpto)), docUpdates);
}
}
docUpto++;
}
}
} else if (mergingUpdates != null) {
// need to check each non-deleted document if it has any updates
for (int j = 0; j < docCount; j++) {
if (prevLiveDocs.get(j)) {
// document isn't deleted, check if it has updates
Map<String,Long> docUpdates = mergingUpdates.get(Integer.valueOf(j));
if (docUpdates != null) {
if (mergedDeletes == null) {
mergedDeletes = readerPool.get(merge.info, true);
docMap = getDocMap(merge, mergeState);
}
mergedUpdates.put(Integer.valueOf(docMap.map(docUpto)), docUpdates);
}
// advance docUpto for every non-deleted document
docUpto++;
}
}
} else {
docUpto += info.info.getDocCount() - info.getDelCount() - rld.getPendingDeleteCount();
}
@ -3150,18 +3235,45 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
assert currentLiveDocs.length() == docCount;
// This segment had no deletes before but now it
// does:
for(int j=0; j<docCount; j++) {
for (int j = 0; j < docCount; j++) {
if (!currentLiveDocs.get(j)) {
if (mergedDeletes == null) {
mergedDeletes = readerPool.get(merge.info, true);
mergedDeletes.initWritableLiveDocs();
docMap = merge.getDocMap(mergeState);
assert docMap.isConsistent(merge.info.info.getDocCount());
initWritableLiveDocs = true;
docMap = getDocMap(merge, mergeState);
} else if (!initWritableLiveDocs) { // mergedDeletes was initialized by field-updates changes
mergedDeletes.initWritableLiveDocs();
initWritableLiveDocs = true;
}
mergedDeletes.delete(docMap.map(docUpto));
} else if (mergingUpdates != null) {
// document isn't deleted, check if it has updates
Map<String,Long> docUpdates = mergingUpdates.get(Integer.valueOf(j));
if (docUpdates != null) {
if (mergedDeletes == null) {
mergedDeletes = readerPool.get(merge.info, true);
docMap = getDocMap(merge, mergeState);
}
mergedUpdates.put(Integer.valueOf(docMap.map(docUpto)), docUpdates);
}
}
docUpto++;
}
} else if (mergingUpdates != null) {
// no deletions before or after, but there were updates
for (int j = 0; j < docCount; j++) {
Map<String,Long> docUpdates = mergingUpdates.get(Integer.valueOf(j));
if (docUpdates != null) {
if (mergedDeletes == null) {
mergedDeletes = readerPool.get(merge.info, true);
docMap = getDocMap(merge, mergeState);
}
mergedUpdates.put(Integer.valueOf(docMap.map(docUpto)), docUpdates);
}
// advance docUpto for every non-deleted document
docUpto++;
}
} else {
// No deletes before or after
docUpto += info.info.getDocCount();
@ -3170,11 +3282,18 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
assert docUpto == merge.info.info.getDocCount();
// set any updates that came while the segment was merging
if (!mergedUpdates.isEmpty()) {
assert mergedDeletes != null;
mergedDeletes.setMergingUpdates(mergedUpdates);
}
if (infoStream.isEnabled("IW")) {
if (mergedDeletes == null) {
infoStream.message("IW", "no new deletes since merge started");
infoStream.message("IW", "no new deletes or field updates since merge started");
} else {
infoStream.message("IW", mergedDeletes.getPendingDeleteCount() + " new deletes since merge started");
infoStream.message("IW", mergedDeletes.getPendingDeleteCount() + " new deletes since merge started and "
+ mergedDeletes.getPendingUpdatesCount() + " new field updates since merge started");
}
}
@ -3213,7 +3332,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
final ReadersAndLiveDocs mergedDeletes = merge.info.info.getDocCount() == 0 ? null : commitMergedDeletes(merge, mergeState);
assert mergedDeletes == null || mergedDeletes.getPendingDeleteCount() != 0;
assert mergedDeletes == null || mergedDeletes.getPendingDeleteCount() != 0 || mergedDeletes.hasFieldUpdates();
// If the doc store we are using has been closed and
// is in now compound format (but wasn't when we
@ -3527,8 +3646,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
// Lock order: IW -> BD
final BufferedDeletesStream.ApplyDeletesResult result = bufferedDeletesStream.applyDeletes(readerPool, merge.segments);
if (result.anyDeletes) {
if (result.anyDeletes || result.anyNumericDVUpdates) {
checkpoint();
}
@ -3556,7 +3675,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
details.put("mergeMaxNumSegments", "" + merge.maxNumSegments);
details.put("mergeFactor", Integer.toString(merge.segments.size()));
setDiagnostics(si, SOURCE_MERGE, details);
merge.setInfo(new SegmentInfoPerCommit(si, 0, -1L));
merge.setInfo(new SegmentInfoPerCommit(si, 0, -1L, -1L));
// System.out.println("[" + Thread.currentThread().getName() + "] _mergeInit: " + segString(merge.segments) + " into " + si);
// Lock order: IW -> BD
bufferedDeletesStream.prune(segmentInfos);
@ -3598,7 +3719,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
// exception inside mergeInit
if (merge.registerDone) {
final List<SegmentInfoPerCommit> sourceSegments = merge.segments;
for(SegmentInfoPerCommit info : sourceSegments) {
for (SegmentInfoPerCommit info : sourceSegments) {
mergingSegments.remove(info);
}
merge.registerDone = false;
@ -3623,6 +3744,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
if (drop) {
rld.dropChanges();
}
rld.setMerging(false);
rld.release(sr);
readerPool.release(rld);
if (drop) {
@ -3680,9 +3802,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
// Hold onto the "live" reader; we will use this to
// commit merged deletes
final ReadersAndLiveDocs rld = readerPool.get(info, true);
SegmentReader reader = rld.getReader(context);
SegmentReader reader = rld.getReader(true, context);
assert reader != null;
// Notify that we are merging, so that we can later copy the updates
// that were received while merging to the merged segment.
rld.setMerging(true);
// Carefully pull the most recent live docs:
final Bits liveDocs;
final int delCount;
@ -3715,7 +3841,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
// fix the reader's live docs and del count
assert delCount > reader.numDeletedDocs(); // beware of zombies
SegmentReader newReader = new SegmentReader(info, reader.core, liveDocs, info.info.getDocCount() - delCount);
SegmentReader newReader = new SegmentReader(info, reader, liveDocs, info.info.getDocCount() - delCount);
boolean released = false;
try {
rld.release(reader);
@ -3877,7 +4003,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
final IndexReaderWarmer mergedSegmentWarmer = config.getMergedSegmentWarmer();
if (poolReaders && mergedSegmentWarmer != null && merge.info.info.getDocCount() != 0) {
final ReadersAndLiveDocs rld = readerPool.get(merge.info, true);
final SegmentReader sr = rld.getReader(IOContext.READ);
final SegmentReader sr = rld.getReader(true, IOContext.READ);
try {
mergedSegmentWarmer.warm(sr);
} finally {

View File

@ -105,11 +105,11 @@ public class MergeState {
}
private static class NoDelDocMap extends DocMap {
private static final class NoDelDocMap extends DocMap {
private final int maxDoc;
private NoDelDocMap(int maxDoc) {
NoDelDocMap(int maxDoc) {
this.maxDoc = maxDoc;
}

View File

@ -0,0 +1,68 @@
package org.apache.lucene.index;
import static org.apache.lucene.util.RamUsageEstimator.*;
import org.apache.lucene.document.NumericDocValuesField;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/** An in-place update to a numeric docvalues field */
final class NumericUpdate {
/* Rough logic: OBJ_HEADER + 3*PTR + INT
* Term: OBJ_HEADER + 2*PTR
* Term.field: 2*OBJ_HEADER + 4*INT + PTR + string.length*CHAR
* Term.bytes: 2*OBJ_HEADER + 2*INT + PTR + bytes.length
* String: 2*OBJ_HEADER + 4*INT + PTR + string.length*CHAR
* Long: OBJ_HEADER + LONG
*/
private static final int RAW_SIZE_IN_BYTES = 9*NUM_BYTES_OBJECT_HEADER + 8*NUM_BYTES_OBJECT_REF + 8*NUM_BYTES_INT + NUM_BYTES_LONG;
static final Long MISSING = new Long(0);
Term term;
String field;
Long value;
int docIDUpto = -1; // unassigned until applied, and confusing that it's here, when it's just used in BufferedDeletes...
/**
* Constructor.
*
* @param term the {@link Term} which determines the documents that will be updated
* @param field the {@link NumericDocValuesField} to update
* @param value the updated value
*/
NumericUpdate(Term term, String field, Long value) {
this.term = term;
this.field = field;
this.value = value == null ? MISSING : value;
}
int sizeInBytes() {
int sizeInBytes = RAW_SIZE_IN_BYTES;
sizeInBytes += term.field.length() * NUM_BYTES_CHAR;
sizeInBytes += term.bytes.bytes.length;
sizeInBytes += field.length() * NUM_BYTES_CHAR;
return sizeInBytes;
}
@Override
public String toString() {
return "term=" + term + ",field=" + field + ",value=" + value;
}
}

View File

@ -18,19 +18,29 @@ 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.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.index.FieldInfo.DocValuesType;
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,25 +53,38 @@ 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).
private Bits liveDocs;
// Holds the numeric DocValues updates.
private final Map<String,Map<Integer,Long>> numericUpdates = new HashMap<String,Map<Integer,Long>>();
// How many further deletions we've done against
// liveDocs vs when we loaded it or last wrote it:
private int pendingDeleteCount;
// 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 mergingUpdates
// map. Also, calls to writeLiveDocs merge the updates with mergingUpdates.
// That way, when the segment is done merging, IndexWriter can apply the
// updates on the merged segment too.
private boolean isMerging = false;
// Holds any updates that come through while this segment was being merged.
private final Map<String,Map<Integer,Long>> mergingUpdates = new HashMap<String,Map<Integer,Long>>();
public ReadersAndLiveDocs(IndexWriter writer, SegmentInfoPerCommit info) {
this.info = info;
this.writer = writer;
shared = true;
liveDocsShared = true;
}
public void incRef() {
@ -83,7 +106,19 @@ class ReadersAndLiveDocs {
public synchronized int getPendingDeleteCount() {
return pendingDeleteCount;
}
public synchronized boolean hasFieldUpdates() {
return numericUpdates.size() > 0;
}
public synchronized int getPendingUpdatesCount() {
int pendingUpdatesCount = 0;
for (Entry<String,Map<Integer,Long>> e : numericUpdates.entrySet()) {
pendingUpdatesCount += e.getValue().size();
}
return pendingUpdatesCount;
}
// Call only from assert!
public synchronized boolean verifyDocCounts() {
int count;
@ -102,35 +137,109 @@ class ReadersAndLiveDocs {
return true;
}
// Get reader for searching/deleting
public synchronized SegmentReader getReader(IOContext context) throws IOException {
//System.out.println(" livedocs=" + rld.liveDocs);
public synchronized void reopenReader(IOContext context) throws IOException {
if (reader != null) {
SegmentReader newReader = new SegmentReader(info, reader, liveDocs, info.info.getDocCount() - info.getDelCount() - pendingDeleteCount);
boolean reopened = false;
try {
reader.decRef();
reader = newReader;
if (liveDocs == null) {
liveDocs = reader.getLiveDocs();
}
reopened = true;
} finally {
if (!reopened) {
newReader.decRef();
}
}
}
}
private synchronized SegmentReader doGetReader(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;
}
private SegmentReader doGetReaderWithUpdates(IOContext context) throws IOException {
boolean checkpoint = false;
try {
// don't synchronize the entire method because we cannot call
// writer.checkpoint() while holding the RLD lock, otherwise we might hit
// a deadlock w/ e.g. a concurrent merging thread.
synchronized (this) {
checkpoint = writeLiveDocs(info.info.dir);
if (reader == null) {
// We steal returned ref:
reader = new SegmentReader(info, context);
if (liveDocs == null) {
liveDocs = reader.getLiveDocs();
}
} else if (checkpoint) {
// enroll a new reader with the applied updates
reopenReader(context);
}
// Ref for caller
reader.incRef();
return reader;
}
} finally {
if (checkpoint) {
writer.checkpoint();
}
}
}
/** Returns a {@link SegmentReader} while applying field updates if requested. */
public SegmentReader getReader(boolean applyFieldUpdates, IOContext context) throws IOException {
// if we need to apply field updates, we call writeLiveDocs and change
// SegmentInfos. Therefore must hold the lock on IndexWriter. This code
// ensures that readers that don't need to apply updates don't pay the
// cost of obtaining it.
if (applyFieldUpdates && hasFieldUpdates()) {
synchronized (writer) {
return doGetReaderWithUpdates(context);
}
} else {
return doGetReader(context);
}
}
public synchronized void release(SegmentReader sr) throws IOException {
assert info == sr.getSegmentInfo();
sr.decRef();
}
/**
* Updates the numeric doc value of {@code docID} under {@code field} to the
* given {@code value}.
*/
public synchronized void updateNumericDocValue(String field, int docID, Long value) {
assert Thread.holdsLock(writer);
assert docID >= 0 && docID < reader.maxDoc() : "out of bounds: docid=" + docID + " maxDoc=" + reader.maxDoc() + " seg=" + info.info.name + " docCount=" + info.info.getDocCount();
Map<Integer,Long> updates = numericUpdates.get(field);
if (updates == null) {
updates = new HashMap<Integer,Long>();
numericUpdates.put(field, updates);
}
updates.put(docID, value);
}
public synchronized boolean delete(int docID) {
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);
@ -162,13 +271,11 @@ class ReadersAndLiveDocs {
* it when you're done (ie, do not call release()).
*/
public synchronized SegmentReader getReadOnlyClone(IOContext context) throws IOException {
if (reader == null) {
getReader(context).decRef();
assert reader != null;
}
shared = true;
getReader(true, context).decRef(); // make sure we enroll a new reader if there are field updates
assert reader != null;
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 +287,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 +299,7 @@ class ReadersAndLiveDocs {
} else {
liveDocs = liveDocsFormat.newLiveDocs(liveDocs);
}
shared = false;
} else {
assert liveDocs != null;
liveDocsShared = false;
}
}
@ -206,7 +311,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 +327,270 @@ class ReadersAndLiveDocs {
// deletes onto the newly merged segment, so we can
// discard them on the sub-readers:
pendingDeleteCount = 0;
numericUpdates.clear();
mergingUpdates.clear();
}
// 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 Thread.holdsLock(writer);
//System.out.println("rld.writeLiveDocs seg=" + info + " pendingDelCount=" + pendingDeleteCount + " numericUpdates=" + numericUpdates);
final boolean hasFieldUpdates = hasFieldUpdates();
if (pendingDeleteCount == 0 && !hasFieldUpdates) {
return false;
}
// We have new deletes or updates
if (pendingDeleteCount > 0) {
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 {
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
}
// 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();
if (pendingDeleteCount > 0) {
codec.liveDocsFormat().writeLiveDocs((MutableBits)liveDocs, trackingDir, info, pendingDeleteCount, IOContext.DEFAULT);
}
// apply numeric updates if there are any
if (hasFieldUpdates) {
// 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 {
// clone FieldInfos so that we can update their numericUpdatesGen
// 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
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());
}
}
}
// create new fields or update existing ones to have NumericDV type
// for (String f : numericUpdates.keySet()) {
// builder.addOrUpdate(f, NumericDocValuesField.TYPE);
// }
final FieldInfos fieldInfos = builder.finish();
final long nextDocValuesGen = info.getNextDocValuesGen();
final String segmentSuffix = Long.toString(nextDocValuesGen, Character.MAX_RADIX);
final SegmentWriteState state = new SegmentWriteState(null, trackingDir, info.info, fieldInfos, null, IOContext.DEFAULT, segmentSuffix, true);
final DocValuesFormat docValuesFormat = codec.docValuesFormat();
final DocValuesConsumer fieldsConsumer = docValuesFormat.fieldsConsumer(state);
boolean fieldsConsumerSuccess = false;
try {
for (Entry<String,Map<Integer,Long>> e : numericUpdates.entrySet()) {
final String field = e.getKey();
final Map<Integer,Long> updates = e.getValue();
final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
if (fieldInfo == null || fieldInfo.getDocValuesType() != DocValuesType.NUMERIC) {
throw new UnsupportedOperationException(
"cannot update docvalues in a segment with no docvalues field: segment=" + info + ", field=" + field);
}
// assert fieldInfo != null;
info.setDocValuesGen(fieldInfo.number, nextDocValuesGen);
// write the numeric updates to a new gen'd docvalues file
fieldsConsumer.addNumericField(fieldInfo, new Iterable<Number>() {
@SuppressWarnings("synthetic-access")
final NumericDocValues currentValues = reader.getNumericDocValues(field);
@Override
public Iterator<Number> iterator() {
return new Iterator<Number>() {
@SuppressWarnings("synthetic-access")
final int maxDoc = reader.maxDoc();
int curDoc = -1;
@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");
}
Long updatedValue = updates.get(curDoc);
if (updatedValue == null) {
updatedValue = Long.valueOf(currentValues.get(curDoc));
} else if (updatedValue == NumericUpdate.MISSING) {
updatedValue = null;
}
return updatedValue;
}
@Override
public void remove() {
throw new UnsupportedOperationException("this iterator does not support removing elements");
}
};
}
});
}
fieldsConsumerSuccess = true;
} finally {
if (fieldsConsumerSuccess) {
fieldsConsumer.close();
} else {
IOUtils.closeWhileHandlingException(fieldsConsumer);
}
}
} finally {
if (reader != this.reader) {
reader.close();
}
}
}
// 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:
success = true;
} finally {
if (!success) {
// Advance only the nextWriteDelGen so that a 2nd
// attempt to write will write to a new file
if (pendingDeleteCount > 0) {
info.advanceNextWriteDelGen();
}
// Advance only the nextWriteDocValuesGen so that a 2nd
// attempt to write will write to a new file
if (hasFieldUpdates) {
info.advanceNextWriteDocValuesGen();
}
// 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:
if (pendingDeleteCount > 0) {
info.advanceDelGen();
info.setDelCount(info.getDelCount() + pendingDeleteCount);
pendingDeleteCount = 0;
return true;
} else {
return false;
}
if (hasFieldUpdates) {
info.advanceDocValuesGen();
// copy all the updates to mergingUpdates, so they can later be applied to the merged segment
if (isMerging) {
copyUpdatesToMerging();
}
numericUpdates.clear();
}
info.addUpdatesFiles(trackingDir.getCreatedFiles());
return true;
}
private void copyUpdatesToMerging() {
for (Entry<String,Map<Integer,Long>> e : numericUpdates.entrySet()) {
String field = e.getKey();
Map<Integer,Long> merging = mergingUpdates.get(field);
if (merging == null) {
mergingUpdates.put(field, new HashMap<Integer,Long>(e.getValue()));
} else {
merging.putAll(e.getValue());
}
}
}
/**
* Indicates whether this segment is currently being merged. Call this just
* before the segment is being merged with {@code true} and when the merge has
* finished and all updates have been applied to the merged segment, call this
* with {@code false}.
*/
public synchronized void setMerging(boolean isMerging) {
this.isMerging = isMerging;
if (!isMerging) {
mergingUpdates.clear();
}
}
/**
* Called from IndexWriter after applying deletes to the merged segment, while
* it was being merged.
*/
public synchronized void setMergingUpdates(Map<Integer,Map<String,Long>> updates) {
for (Entry<Integer,Map<String,Long>> e : updates.entrySet()) {
int doc = e.getKey().intValue();
for (Entry<String,Long> docUpdates : e.getValue().entrySet()) {
String field = docUpdates.getKey();
Long value = docUpdates.getValue();
Map<Integer,Long> fieldUpdates = numericUpdates.get(field);
if (fieldUpdates == null) {
fieldUpdates = new HashMap<Integer,Long>();
numericUpdates.put(field, fieldUpdates);
}
fieldUpdates.put(doc, value);
}
}
}
/** Returns updates that came in while this segment was merging. */
public synchronized Map<Integer,Map<String,Long>> getMergingUpdates() {
copyUpdatesToMerging();
if (mergingUpdates.isEmpty()) {
return null;
}
Map<Integer,Map<String,Long>> updates = new HashMap<Integer,Map<String,Long>>();
for (Entry<String,Map<Integer,Long>> e : mergingUpdates.entrySet()) {
String field = e.getKey();
for (Entry<Integer,Long> fieldUpdates : e.getValue().entrySet()) {
Integer doc = fieldUpdates.getKey();
Long value = fieldUpdates.getValue();
Map<String,Long> docUpdates = updates.get(doc);
if (docUpdates == null) {
docUpdates = new HashMap<String,Long>();
updates.put(doc, docUpdates);
}
docUpdates.put(field, value);
}
}
mergingUpdates.clear();
return updates;
}
@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);
sb.append(" pendingUpdatesCount=").append(getPendingUpdatesCount());
return sb.toString();
}
}

View File

@ -26,17 +26,15 @@ 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;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.IOUtils;
@ -44,18 +42,16 @@ 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;
@ -66,7 +62,7 @@ final class SegmentCoreReaders {
// TODO: make a single thread local w/ a
// Thingy class holding fieldsReader, termVectorsReader,
// normsProducer, dvProducer
// normsProducer
final CloseableThreadLocal<StoredFieldsReader> fieldsReaderLocal = new CloseableThreadLocal<StoredFieldsReader>() {
@Override
@ -82,20 +78,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,Bits>> docsWithFieldLocal = new CloseableThreadLocal<Map<String,Bits>>() {
@Override
protected Map<String,Bits> initialValue() {
return new HashMap<String,Bits>();
}
};
final CloseableThreadLocal<Map<String,Object>> normsLocal = new CloseableThreadLocal<Map<String,Object>>() {
@Override
protected Map<String,Object> initialValue() {
@ -120,8 +102,9 @@ final class SegmentCoreReaders {
cfsReader = null;
cfsDir = dir;
}
fieldInfos = codec.fieldInfosFormat().getFieldInfosReader().read(cfsDir, si.info.name, IOContext.READONCE);
fieldInfos = codec.fieldInfosFormat().getFieldInfosReader().read(cfsDir, si.info.name, IOContext.READONCE);
final PostingsFormat format = codec.postingsFormat();
final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si.info, fieldInfos, context);
// Ask codec for its Fields
@ -131,13 +114,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;
@ -167,146 +143,14 @@ final class SegmentCoreReaders {
this.owner = owner;
}
int getRefCount() {
return ref.get();
}
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;
}
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;
}
Bits getDocsWithField(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;
}
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;
}
NumericDocValues getNormValues(String field) throws IOException {
FieldInfo fi = fieldInfos.fieldInfo(field);
if (fi == null) {
@ -332,8 +176,9 @@ final class SegmentCoreReaders {
void decRef() throws IOException {
if (ref.decrementAndGet() == 0) {
IOUtils.close(termVectorsLocal, fieldsReaderLocal, docValuesLocal, normsLocal, docsWithFieldLocal, fields,
dvProducer, termVectorsReaderOrig, fieldsReaderOrig, cfsReader, normsProducer);
// System.err.println("--- closing core readers");
IOUtils.close(termVectorsLocal, fieldsReaderLocal, normsLocal, fields, termVectorsReaderOrig, fieldsReaderOrig,
cfsReader, normsProducer);
notifyCoreClosedListeners();
}
}
@ -356,12 +201,12 @@ final class SegmentCoreReaders {
/** Returns approximate RAM bytes used */
public long ramBytesUsed() {
return ((dvProducer!=null) ? dvProducer.ramBytesUsed() : 0) +
((normsProducer!=null) ? normsProducer.ramBytesUsed() : 0) +
return ((normsProducer!=null) ? normsProducer.ramBytesUsed() : 0) +
((fields!=null) ? fields.ramBytesUsed() : 0) +
((fieldsReaderOrig!=null)? fieldsReaderOrig.ramBytesUsed() : 0) +
((termVectorsReaderOrig!=null) ? termVectorsReaderOrig.ramBytesUsed() : 0);
}
@Override
public String toString() {
return "SegmentCoreReader(owner=" + owner + ")";

View File

@ -119,7 +119,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 +170,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';

View File

@ -19,7 +19,10 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.store.Directory;
@ -27,9 +30,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 +46,35 @@ public class SegmentInfoPerCommit {
// attempt to write:
private long nextWriteDelGen;
// holds field.number to docValuesGen mapping
// TODO (DVU_FIELDINFOS_GEN) once we gen FieldInfos, get rid of this; every FieldInfo will record its dvGen
private final Map<Integer,Long> fieldDocValuesGens = new HashMap<Integer,Long>();
// Generation number of the docValues (-1 if there are no field updates)
private long docValuesGen;
// Normally 1 + docValuesGen, unless an exception was hit on last attempt to
// write
private long nextWriteDocValuesGen;
// Tracks the files with field updates
private Set<String> updatesFiles = new HashSet<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 docValuesGen
* doc-values generation number (used to name docvalues files)
**/
public SegmentInfoPerCommit(SegmentInfo info, int delCount, long delGen) {
public SegmentInfoPerCommit(SegmentInfo info, int delCount, long delGen, long docValuesGen) {
this.info = info;
this.delCount = delCount;
this.delGen = delGen;
@ -61,8 +83,25 @@ public class SegmentInfoPerCommit {
} else {
nextWriteDelGen = delGen+1;
}
this.docValuesGen = docValuesGen;
if (docValuesGen == -1) {
nextWriteDocValuesGen = 1;
} else {
nextWriteDocValuesGen = docValuesGen + 1;
}
}
/** Returns the files which contains field updates. */
public Set<String> getUpdatesFiles() {
return new HashSet<String>(updatesFiles);
}
/** Called when we succeed in writing field updates. */
public void addUpdatesFiles(Set<String> files) {
updatesFiles.addAll(files);
}
/** Called when we succeed in writing deletes */
void advanceDelGen() {
delGen = nextWriteDelGen;
@ -76,6 +115,21 @@ public class SegmentInfoPerCommit {
void advanceNextWriteDelGen() {
nextWriteDelGen++;
}
/** Called when we succeed in writing docvalues updates */
void advanceDocValuesGen() {
docValuesGen = nextWriteDocValuesGen;
nextWriteDocValuesGen = docValuesGen + 1;
sizeInBytes = -1;
}
/**
* Called if there was an exception while writing docvalues updates, so that
* we don't try to write to the same file more than once.
*/
void advanceNextWriteDocValuesGen() {
nextWriteDocValuesGen++;
}
/** Returns total size in bytes of all files for this
* segment. */
@ -96,9 +150,15 @@ 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
files.addAll(updatesFiles);
return files;
}
@ -115,26 +175,53 @@ 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 docValuesGen != -1;
}
/** Returns the next available generation number of the docvalues files. */
public long getNextDocValuesGen() {
return nextWriteDocValuesGen;
}
/**
* Returns the docvalues generation of this field, or -1 if there are
* no updates to it.
*/
public long getDocValuesGen(int fieldNumber) {
Long gen = fieldDocValuesGens.get(fieldNumber);
return gen == null ? -1 : gen.longValue();
}
/** Sets the docvalues generation for this field. */
public void setDocValuesGen(int fieldNumber, long gen) {
fieldDocValuesGens.put(fieldNumber, gen);
}
/**
* Returns a mapping from a field number to its DV generation.
*
* @see #getDocValuesGen(int)
*/
public Map<Integer,Long> getFieldDocValuesGens() {
return fieldDocValuesGens;
}
/**
* Returns the generation number of the field infos file or -1 if there are no
* field updates yet.
*/
public long getDocValuesGen() {
return docValuesGen;
}
/**
* Returns the next available generation number
* of the live docs file.
@ -174,17 +261,25 @@ public class SegmentInfoPerCommit {
if (delGen != -1) {
s += ":delGen=" + delGen;
}
if (docValuesGen != -1) {
s += ":docValuesGen=" + docValuesGen;
}
return s;
}
@Override
public SegmentInfoPerCommit clone() {
SegmentInfoPerCommit other = new SegmentInfoPerCommit(info, delCount, delGen);
SegmentInfoPerCommit other = new SegmentInfoPerCommit(info, delCount, delGen, docValuesGen);
// 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.nextWriteDocValuesGen = nextWriteDocValuesGen;
other.updatesFiles.addAll(updatesFiles);
other.fieldDocValuesGens.putAll(fieldDocValuesGens);
return other;
}
}

View File

@ -28,6 +28,7 @@ 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;
@ -35,14 +36,13 @@ import org.apache.lucene.codecs.CodecUtil;
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
@ -111,11 +111,12 @@ import org.apache.lucene.util.ThreadInterruptedException;
*/
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.4. */
public static final int VERSION_40 = 0;
/** The file format version for the segments_N codec header, since 4.5+. */
public static final int VERSION_45 = 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 +320,14 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCom
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_45);
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 +338,19 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCom
if (delCount < 0 || delCount > info.getDocCount()) {
throw new CorruptIndexException("invalid deletion count: " + delCount + " (resource: " + input + ")");
}
add(new SegmentInfoPerCommit(info, delCount, delGen));
long docValuesGen = -1;
if (format >= VERSION_45) {
docValuesGen = input.readLong();
}
SegmentInfoPerCommit siPerCommit = new SegmentInfoPerCommit(info, delCount, delGen, docValuesGen);
if (format >= VERSION_45) {
int numUpdates = input.readInt();
for (int i = 0; i < numUpdates; i++) {
siPerCommit.setDocValuesGen(input.readInt(), input.readLong());
}
siPerCommit.addUpdatesFiles(input.readStringSet());
}
add(siPerCommit);
}
userData = input.readStringStringMap();
@ -395,7 +408,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCom
try {
segnOutput = new ChecksumIndexOutput(directory.createOutput(segmentFileName, IOContext.DEFAULT));
CodecUtil.writeHeader(segnOutput, "segments", VERSION_40);
CodecUtil.writeHeader(segnOutput, "segments", VERSION_45);
segnOutput.writeLong(version);
segnOutput.writeInt(counter); // write counter
segnOutput.writeInt(size()); // write infos
@ -405,6 +418,14 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCom
segnOutput.writeString(si.getCodec().getName());
segnOutput.writeLong(siPerCommit.getDelGen());
segnOutput.writeInt(siPerCommit.getDelCount());
segnOutput.writeLong(siPerCommit.getDocValuesGen());
Map<Integer,Long> docValuesUpdatesGen = siPerCommit.getFieldDocValuesGens();
segnOutput.writeInt(docValuesUpdatesGen.size());
for (Entry<Integer,Long> e : docValuesUpdatesGen.entrySet()) {
segnOutput.writeInt(e.getKey());
segnOutput.writeLong(e.getValue());
}
segnOutput.writeStringSet(siPerCommit.getUpdatesFiles());
assert si.dir == directory;
assert siPerCommit.getDelCount() <= si.getDocCount();
@ -815,6 +836,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCom
files.addAll(info.files());
}
}
return files;
}

View File

@ -18,13 +18,24 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import org.apache.lucene.store.Directory;
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.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.RefCount;
/**
* IndexReader implementation over a single segment.
@ -45,6 +56,23 @@ public final class SegmentReader extends AtomicReader {
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>>();
/**
* Constructs a new SegmentReader with a new core.
* @throws CorruptIndexException if the index is corrupt
@ -54,16 +82,41 @@ public final class SegmentReader extends AtomicReader {
public SegmentReader(SegmentInfoPerCommit si, IOContext context) throws IOException {
this.si = 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, IOContext.READONCE);
liveDocs = codec.liveDocsFormat().readLiveDocs(directory(), si, IOContext.READONCE);
} else {
assert si.getDelCount() == 0;
liveDocs = null;
}
numDocs = si.info.getDocCount() - si.getDelCount();
if (core.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(si);
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 +125,7 @@ public final class SegmentReader extends AtomicReader {
// 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,8 +133,8 @@ public final class SegmentReader extends AtomicReader {
/** 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) throws IOException {
this(si, core,
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 +143,88 @@ public final class SegmentReader extends AtomicReader {
* 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;
core.incRef();
assert liveDocs != null;
this.liveDocs = liveDocs;
this.numDocs = numDocs;
this.core = sr.core;
core.incRef();
// increment refCount of DocValuesProducers that are used by this reader
boolean success = false;
try {
if (core.fieldInfos.hasDocValues()) {
final Codec codec = si.info.getCodec();
final Directory dir = core.cfsReader != null ? core.cfsReader : si.info.dir;
final DocValuesFormat dvFormat = codec.docValuesFormat();
final Map<Long,List<FieldInfo>> genInfos = getGenInfos(si);
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();
} else {
// this gen is not used by given reader, initialize a new one
dvp = newDocValuesProducer(si, IOContext.READ, dir, dvFormat, gen, infos);
}
assert dvp != null;
genDVProducers.put(gen, dvp);
}
for (FieldInfo fi : infos) {
dvProducers.put(fi.name, dvp.get());
}
}
}
success = true;
} finally {
if (!success) {
doClose();
}
}
}
// returns a gen->List<FieldInfo> mapping. Fields without DV updates have gen=-1
private Map<Long,List<FieldInfo>> getGenInfos(SegmentInfoPerCommit si) {
final Map<Long,List<FieldInfo>> genInfos = new HashMap<Long,List<FieldInfo>>();
for (FieldInfo fi : core.fieldInfos) {
if (fi.getDocValuesType() == null) {
continue;
}
long gen = si.getDocValuesGen(fi.number);
List<FieldInfo> infos = genInfos.get(gen);
if (infos == null) {
infos = new ArrayList<FieldInfo>();
genInfos.put(gen, infos);
}
infos.add(fi);
}
return genInfos;
}
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 RefCount<DocValuesProducer>(dvFormat.fieldsProducer(srs)) {
@Override
protected void release() throws IOException {
object.close();
}
};
}
@Override
@ -110,7 +236,26 @@ public final class SegmentReader extends AtomicReader {
@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;
}
}
}
if (t != null) {
if (t instanceof IOException) throw (IOException) t;
if (t instanceof RuntimeException) throw (RuntimeException) t;
if (t instanceof Error) throw (Error) t;
throw new RuntimeException(t);
}
}
}
@Override
@ -217,34 +362,140 @@ public final class SegmentReader extends AtomicReader {
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 = core.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();
return core.getDocsWithField(field);
FieldInfo fi = core.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
@ -284,9 +535,21 @@ public final class SegmentReader extends AtomicReader {
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();
return (core!=null) ? core.ramBytesUsed() : 0;
long ramBytesUsed = dvRamBytesUsed();
if (core != null) {
ramBytesUsed += core.ramBytesUsed();
}
return ramBytesUsed;
}
}

View File

@ -71,16 +71,32 @@ public class SegmentWriteState {
* to {@link Directory#createOutput(String,IOContext)}. */
public final IOContext context;
/** True is this instance represents a field update. */
public final boolean isFieldUpdate; // TODO (DVU_FIELDINFOS_GEN) once we gen FieldInfos, get rid of this
/** Sole constructor. */
public SegmentWriteState(InfoStream infoStream, Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos,
BufferedDeletes segDeletes, IOContext context) {
this(infoStream, directory, segmentInfo, fieldInfos, segDeletes, context, "", false);
}
/**
* Constructor which takes segment suffix and isFieldUpdate in addition to the
* other parameters.
*
* @see #SegmentWriteState(InfoStream, Directory, SegmentInfo, FieldInfos,
* BufferedDeletes, IOContext)
*/
public SegmentWriteState(InfoStream infoStream, Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos,
BufferedDeletes segDeletes, IOContext context, String segmentSuffix, boolean isFieldUpdate) {
this.infoStream = infoStream;
this.segDeletes = segDeletes;
this.directory = directory;
this.segmentInfo = segmentInfo;
this.fieldInfos = fieldInfos;
segmentSuffix = "";
this.segmentSuffix = segmentSuffix;
this.context = context;
this.isFieldUpdate = isFieldUpdate;
}
/** Create a shallow copy of {@link SegmentWriteState} with a new segment suffix. */
@ -93,5 +109,6 @@ public class SegmentWriteState {
this.segmentSuffix = segmentSuffix;
segDeletes = state.segDeletes;
delCountOnFlush = state.delCountOnFlush;
isFieldUpdate = state.isFieldUpdate;
}
}

View File

@ -162,18 +162,26 @@ final class StandardDirectoryReader extends DirectoryReader {
readerShared[i] = false;
newReaders[i] = newReader;
} else {
if (newReaders[i].getSegmentInfo().getDelGen() == infos.info(i).getDelGen()) {
if (newReaders[i].getSegmentInfo().getDelGen() == infos.info(i).getDelGen()
&& newReaders[i].getSegmentInfo().getDocValuesGen() == infos.info(i).getDocValuesGen()) {
// No change; this reader will be shared between
// the old and the new one, so we must incRef
// it:
readerShared[i] = true;
newReaders[i].incRef();
} else {
// there are changes to the reader, either liveDocs or DV updates
readerShared[i] = false;
// Steal the ref returned by SegmentReader ctor:
assert infos.info(i).info.dir == newReaders[i].getSegmentInfo().info.dir;
assert infos.info(i).hasDeletions();
newReaders[i] = new SegmentReader(infos.info(i), newReaders[i].core);
assert infos.info(i).hasDeletions() || infos.info(i).hasFieldUpdates();
if (newReaders[i].getSegmentInfo().getDelGen() == infos.info(i).getDelGen()) {
// only DV updates
newReaders[i] = new SegmentReader(infos.info(i), newReaders[i], newReaders[i].getLiveDocs(), newReaders[i].numDocs());
} else {
// both DV and liveDocs have changed
newReaders[i] = new SegmentReader(infos.info(i), newReaders[i]);
}
}
}
success = true;

View File

@ -0,0 +1,84 @@
package org.apache.lucene.util;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicInteger;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* Manages reference counting for a given object. Extensions can override
* {@link #release()} to do custom logic when reference counting hits 0.
*/
public class RefCount<T> {
private final AtomicInteger refCount = new AtomicInteger(1);
protected final T object;
public RefCount(T object) {
this.object = object;
}
/**
* Called when reference counting hits 0. By default this method does nothing,
* but extensions can override to e.g. release resources attached to object
* that is managed by this class.
*/
protected void release() throws IOException {}
/**
* Decrements the reference counting of this object. When reference counting
* hits 0, calls {@link #release()}.
*/
public final void decRef() throws IOException {
final int rc = refCount.decrementAndGet();
if (rc == 0) {
boolean success = false;
try {
release();
success = true;
} finally {
if (!success) {
// Put reference back on failure
refCount.incrementAndGet();
}
}
} else if (rc < 0) {
throw new IllegalStateException("too many decRef calls: refCount is " + rc + " after decrement");
}
}
public final T get() {
return object;
}
/** Returns the current reference count. */
public final int getRefCount() {
return refCount.get();
}
/**
* Increments the reference count. Calls to this method must be matched with
* calls to {@link #decRef()}.
*/
public final void incRef() {
refCount.incrementAndGet();
}
}

View File

@ -239,7 +239,7 @@ public class TestDoc extends LuceneTestCase {
}
}
return new SegmentInfoPerCommit(info, 0, -1L);
return new SegmentInfoPerCommit(info, 0, -1L, -1L);
}

File diff suppressed because it is too large Load Diff

View File

@ -90,7 +90,7 @@ public class TestSegmentMerger extends LuceneTestCase {
SegmentReader mergedReader = new SegmentReader(new SegmentInfoPerCommit(
new SegmentInfo(mergedDir, Constants.LUCENE_MAIN_VERSION, mergedSegment, docsMerged,
false, codec, null, null),
0, -1L),
0, -1L, -1L),
newIOContext(random()));
assertTrue(mergedReader != null);
assertTrue(mergedReader.numDocs() == 2);

View File

@ -141,7 +141,7 @@ public class IndexSplitter {
SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.name, info.getDocCount(),
info.getUseCompoundFile(),
info.getCodec(), info.getDiagnostics(), info.attributes());
destInfos.add(new SegmentInfoPerCommit(newInfo, infoPerCommit.getDelCount(), infoPerCommit.getDelGen()));
destInfos.add(new SegmentInfoPerCommit(newInfo, infoPerCommit.getDelCount(), infoPerCommit.getDelGen(), infoPerCommit.getDocValuesGen()));
// now copy files over
Collection<String> files = infoPerCommit.files();
for (final String srcName : files) {

View File

@ -37,7 +37,6 @@ import org.apache.lucene.index.LogMergePolicy;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.index.SlowCompositeReaderWrapper;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TieredMergePolicy;
@ -121,9 +120,23 @@ public class TestSortingMergePolicy extends LuceneTestCase {
iw1.commit();
iw2.commit();
final Document doc = randomDocument();
iw1.addDocument(doc);
iw2.addDocument(doc);
// NOTE: don't use RIW.addDocument directly, since it sometimes commits
// which may trigger a merge, at which case forceMerge may not do anything.
// With field updates this is a problem, since the updates can go into the
// single segment in the index, and threefore the index won't be sorted.
// This hurts the assumption of the test later on, that the index is sorted
// by SortingMP.
iw1.w.addDocument(doc);
iw2.w.addDocument(doc);
if (defaultCodecSupportsFieldUpdates()) {
// update NDV of docs belonging to one term (covers many documents)
final long value = random().nextLong();
final String term = RandomPicks.randomFrom(random(), terms);
iw1.w.updateNumericDocValue(new Term("s", term), "ndv", value);
iw2.w.updateNumericDocValue(new Term("s", term), "ndv", value);
}
iw1.forceMerge(1);
iw2.forceMerge(1);
iw1.close();
@ -144,7 +157,7 @@ public class TestSortingMergePolicy extends LuceneTestCase {
private static void assertSorted(AtomicReader reader) throws IOException {
final NumericDocValues ndv = reader.getNumericDocValues("ndv");
for (int i = 1; i < reader.maxDoc(); ++i) {
assertTrue(ndv.get(i-1) <= ndv.get(i));
assertTrue("ndv(" + (i-1) + ")=" + ndv.get(i-1) + ",ndv(" + i + ")=" + ndv.get(i), ndv.get(i-1) <= ndv.get(i));
}
}
@ -154,6 +167,7 @@ public class TestSortingMergePolicy extends LuceneTestCase {
assertSorted(sortedReader1);
assertSorted(sortedReader2);
assertReaderEquals("", sortedReader1, sortedReader2);
}

View File

@ -1382,6 +1382,15 @@ public abstract class LuceneTestCase extends Assert {
}
return true;
}
/** Returns true if the codec "supports" field updates. */
public static boolean defaultCodecSupportsFieldUpdates() {
String name = Codec.getDefault().getName();
if (name.equals("Lucene40") || name.equals("Lucene41") || name.equals("Lucene42")) {
return false;
}
return true;
}
public void assertReaderEquals(String info, IndexReader leftReader, IndexReader rightReader) throws IOException {
assertReaderStatisticsEquals(info, leftReader, rightReader);