Index: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java (revision 1537835) +++ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java (working copy) @@ -23,7 +23,7 @@ import org.apache.lucene.codecs.LiveDocsFormat; import org.apache.lucene.index.IndexFileNames; -import org.apache.lucene.index.SegmentInfoPerCommit; +import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; @@ -63,7 +63,7 @@ } @Override - public Bits readLiveDocs(Directory dir, SegmentInfoPerCommit info, IOContext context) throws IOException { + public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException { assert info.hasDeletions(); BytesRef scratch = new BytesRef(); CharsRef scratchUTF16 = new CharsRef(); @@ -105,7 +105,7 @@ } @Override - public void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfoPerCommit info, int newDelCount, IOContext context) throws IOException { + public void writeLiveDocs(MutableBits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException { BitSet set = ((SimpleTextBits) bits).bits; int size = bits.length(); BytesRef scratch = new BytesRef(); @@ -138,7 +138,7 @@ } @Override - public void files(SegmentInfoPerCommit info, Collection files) throws IOException { + public void files(SegmentCommitInfo info, Collection files) throws IOException { if (info.hasDeletions()) { files.add(IndexFileNames.fileNameFromGeneration(info.info.name, LIVEDOCS_EXTENSION, info.getDelGen())); } Index: lucene/core/src/java/org/apache/lucene/codecs/LiveDocsFormat.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/LiveDocsFormat.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/codecs/LiveDocsFormat.java (working copy) @@ -20,7 +20,7 @@ import java.io.IOException; import java.util.Collection; -import org.apache.lucene.index.SegmentInfoPerCommit; +import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.util.Bits; @@ -42,13 +42,13 @@ public abstract MutableBits newLiveDocs(Bits existing) throws IOException; /** Read live docs bits. */ - public abstract Bits readLiveDocs(Directory dir, SegmentInfoPerCommit info, IOContext context) throws IOException; + public abstract Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException; /** Persist live docs bits. Use {@link - * SegmentInfoPerCommit#getNextDelGen} to determine the + * SegmentCommitInfo#getNextDelGen} to determine the * generation of the deletes file you should write to. */ - public abstract void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfoPerCommit info, int newDelCount, IOContext context) throws IOException; + public abstract void writeLiveDocs(MutableBits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException; - /** Records all files in use by this {@link SegmentInfoPerCommit} into the files argument. */ - public abstract void files(SegmentInfoPerCommit info, Collection files) throws IOException; + /** Records all files in use by this {@link SegmentCommitInfo} into the files argument. */ + public abstract void files(SegmentCommitInfo info, Collection files) throws IOException; } Index: lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40LiveDocsFormat.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40LiveDocsFormat.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40LiveDocsFormat.java (working copy) @@ -23,7 +23,7 @@ import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.LiveDocsFormat; import org.apache.lucene.index.IndexFileNames; -import org.apache.lucene.index.SegmentInfoPerCommit; +import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.store.DataOutput; // javadocs import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; @@ -85,7 +85,7 @@ } @Override - public Bits readLiveDocs(Directory dir, SegmentInfoPerCommit info, IOContext context) throws IOException { + public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException { String filename = IndexFileNames.fileNameFromGeneration(info.info.name, DELETES_EXTENSION, info.getDelGen()); final BitVector liveDocs = new BitVector(dir, filename, context); assert liveDocs.count() == info.info.getDocCount() - info.getDelCount(): @@ -95,7 +95,7 @@ } @Override - public void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfoPerCommit info, int newDelCount, IOContext context) throws IOException { + public void writeLiveDocs(MutableBits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException { String filename = IndexFileNames.fileNameFromGeneration(info.info.name, DELETES_EXTENSION, info.getNextDelGen()); final BitVector liveDocs = (BitVector) bits; assert liveDocs.count() == info.info.getDocCount() - info.getDelCount() - newDelCount; @@ -104,7 +104,7 @@ } @Override - public void files(SegmentInfoPerCommit info, Collection files) throws IOException { + public void files(SegmentCommitInfo info, Collection files) throws IOException { if (info.hasDeletions()) { files.add(IndexFileNames.fileNameFromGeneration(info.info.name, DELETES_EXTENSION, info.getDelGen())); } Index: lucene/core/src/java/org/apache/lucene/index/BufferedDeletes.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/BufferedDeletes.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/BufferedDeletes.java (working copy) @@ -1,229 +0,0 @@ -package org.apache.lucene.index; - -/* - * 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. - */ - -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.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.lucene.search.Query; -import org.apache.lucene.util.RamUsageEstimator; - -/* Holds buffered deletes and updates, by docID, term or query for a - * single segment. This is used to hold buffered pending - * 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 { // TODO (DVU_RENAME) BufferedUpdates? - - /* Rough logic: HashMap has an array[Entry] w/ varying - load factor (say 2 * POINTER). Entry is object w/ Term - key, Integer val, int hash, Entry next - (OBJ_HEADER + 3*POINTER + INT). Term is object w/ - String field and String text (OBJ_HEADER + 2*POINTER). - Term's field is String (OBJ_HEADER + 4*INT + POINTER + - OBJ_HEADER + string.length*CHAR). - Term's text is String (OBJ_HEADER + 4*INT + POINTER + - OBJ_HEADER + string.length*CHAR). Integer is - OBJ_HEADER + INT. */ - final static int BYTES_PER_DEL_TERM = 9*RamUsageEstimator.NUM_BYTES_OBJECT_REF + 7*RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 10*RamUsageEstimator.NUM_BYTES_INT; - - /* Rough logic: del docIDs are List. Say list - allocates ~2X size (2*POINTER). Integer is OBJ_HEADER - + int */ - final static int BYTES_PER_DEL_DOCID = 2*RamUsageEstimator.NUM_BYTES_OBJECT_REF + RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + RamUsageEstimator.NUM_BYTES_INT; - - /* Rough logic: HashMap has an array[Entry] w/ varying - load factor (say 2 * POINTER). Entry is object w/ - Query key, Integer val, int hash, Entry next - (OBJ_HEADER + 3*POINTER + INT). Query we often - 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-field map holds a reference to the updated field, and - * therefore we only account for the object reference and - * map space itself. This is incremented when we first see - * an updated field. - * - * HashMap has an array[Entry] w/ varying load - * factor (say 2*POINTER). Entry is an object w/ String key, - * LinkedHashMap val, int hash, Entry next (OBJ_HEADER + 3*POINTER + INT). - * - * LinkedHashMap (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_FIELD_ENTRY = - 7*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 Term for an already updated - * field. - * LinkedHashMap has an array[Entry] w/ varying load factor - * (say 2*POINTER). Entry is an object w/ Term key, NumericUpdate val, - * int hash, Entry next, Entry before, Entry after (OBJ_HEADER + 5*POINTER + INT). - * - * Term (key) is counted only as POINTER. - * NumericUpdate (val) counts its own size and isn't accounted for here. - */ - final static int BYTES_PER_NUMERIC_UPDATE_ENTRY = 7*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 terms = new HashMap(); - final Map queries = new HashMap(); - final List docIDs = new ArrayList(); - - // Map> - // For each field we keep an ordered list of NumericUpdates, key'd by the - // update Term. LinkedHashMap guarantees we will later traverse the map in - // insertion order (so that if two terms affect the same document, the last - // one that came in wins), and helps us detect faster if the same Term is - // used to update the same field multiple times (so we later traverse it - // only once). - final Map> numericUpdates = new HashMap>(); - - public static final Integer MAX_INT = Integer.valueOf(Integer.MAX_VALUE); - - final AtomicLong bytesUsed; - - private final static boolean VERBOSE_DELETES = false; - - long gen; - - public BufferedDeletes() { - this.bytesUsed = new AtomicLong(); - } - - @Override - public String toString() { - if (VERBOSE_DELETES) { - return "gen=" + gen + " numTerms=" + numTermDeletes + ", terms=" + terms - + ", queries=" + queries + ", docIDs=" + docIDs + ", numericUpdates=" + numericUpdates - + ", bytesUsed=" + bytesUsed; - } else { - String s = "gen=" + gen; - if (numTermDeletes.get() != 0) { - s += " " + numTermDeletes.get() + " deleted terms (unique count=" + terms.size() + ")"; - } - if (queries.size() != 0) { - s += " " + queries.size() + " deleted queries"; - } - 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(); - } - - return s; - } - } - - public void addQuery(Query query, int docIDUpto) { - Integer current = queries.put(query, docIDUpto); - // increment bytes used only if the query wasn't added so far. - if (current == null) { - bytesUsed.addAndGet(BYTES_PER_DEL_QUERY); - } - } - - public void addDocID(int docID) { - docIDs.add(Integer.valueOf(docID)); - bytesUsed.addAndGet(BYTES_PER_DEL_DOCID); - } - - public void addTerm(Term term, int docIDUpto) { - Integer current = terms.get(term); - if (current != null && docIDUpto < current) { - // Only record the new number if it's greater than 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. If we blindly replace than we can - // incorrectly get both docs indexed. - return; - } - - terms.put(term, Integer.valueOf(docIDUpto)); - // note that if current != null then it means there's already a buffered - // delete on that term, therefore we seem to over-count. this over-counting - // is done to respect IndexWriterConfig.setMaxBufferedDeleteTerms. - numTermDeletes.incrementAndGet(); - if (current == null) { - bytesUsed.addAndGet(BYTES_PER_DEL_TERM + term.bytes.length + (RamUsageEstimator.NUM_BYTES_CHAR * term.field().length())); - } - } - - public void addNumericUpdate(NumericUpdate update, int docIDUpto) { - LinkedHashMap fieldUpdates = numericUpdates.get(update.field); - if (fieldUpdates == null) { - fieldUpdates = new LinkedHashMap(); - numericUpdates.put(update.field, fieldUpdates); - bytesUsed.addAndGet(BYTES_PER_NUMERIC_FIELD_ENTRY); - } - final NumericUpdate current = fieldUpdates.get(update.term); - 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; - // since it's a LinkedHashMap, we must first remove the Term entry so that - // it's added last (we're interested in insertion-order). - if (current != null) { - fieldUpdates.remove(update.term); - } - fieldUpdates.put(update.term, 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); - } - - boolean any() { - return terms.size() > 0 || docIDs.size() > 0 || queries.size() > 0 || numericUpdates.size() > 0; - } -} Index: lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java (working copy) @@ -1,585 +0,0 @@ -package org.apache.lucene.index; - -/* - * 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. - */ - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.lucene.search.DocIdSet; -import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.QueryWrapperFilter; -import org.apache.lucene.store.IOContext; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.InfoStream; - -/* Tracks the stream of {@link BufferedDeletes}. - * When DocumentsWriterPerThread flushes, its buffered - * 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 or updates are - * buffered and must be flushed (by RAM usage or by count). - * - * Each packet is assigned a generation, and each flushed or - * merged segment is also assigned a generation, so we can - * track which BufferedDeletes packets to apply to any given - * segment. */ - -class BufferedDeletesStream { // TODO (DVU_RENAME) BufferedUpdatesStream - - // TODO: maybe linked list? - private final List deletes = new ArrayList(); - - // Starts at 1 so that SegmentInfos that have never had - // deletes applied (whose bufferedDelGen defaults to 0) - // will be correct: - private long nextGen = 1; - - // used only by assert - private Term lastDeleteTerm; - - private final InfoStream infoStream; - private final AtomicLong bytesUsed = new AtomicLong(); - private final AtomicInteger numTerms = new AtomicInteger(); - - public BufferedDeletesStream(InfoStream infoStream) { - this.infoStream = infoStream; - } - - // Appends a new packet of buffered deletes to the stream, - // setting its generation: - public synchronized long push(FrozenBufferedDeletes packet) { - /* - * The insert operation must be atomic. If we let threads increment the gen - * and push the packet afterwards we risk that packets are out of order. - * With DWPT this is possible if two or more flushes are racing for pushing - * updates. If the pushed packets get our of order would loose documents - * since deletes are applied to the wrong segments. - */ - packet.setDelGen(nextGen++); - assert packet.any(); - assert checkDeleteStats(); - assert packet.delGen() < nextGen; - assert deletes.isEmpty() || deletes.get(deletes.size()-1).delGen() < packet.delGen() : "Delete packets must be in order"; - deletes.add(packet); - numTerms.addAndGet(packet.numTermDeletes); - bytesUsed.addAndGet(packet.bytesUsed); - if (infoStream.isEnabled("BD")) { - infoStream.message("BD", "push deletes " + packet + " delGen=" + packet.delGen() + " packetCount=" + deletes.size() + " totBytesUsed=" + bytesUsed.get()); - } - assert checkDeleteStats(); - return packet.delGen(); - } - - public synchronized void clear() { - deletes.clear(); - nextGen = 1; - numTerms.set(0); - bytesUsed.set(0); - } - - public boolean any() { - return bytesUsed.get() != 0; - } - - public int numTerms() { - return numTerms.get(); - } - - public long bytesUsed() { - return bytesUsed.get(); - } - - public static class ApplyDeletesResult { - - // True if any actual deletes took place: - public final boolean anyDeletes; - - // Current gen, for the merged segment: - public final long gen; - - // If non-null, contains segments that are 100% deleted - public final List allDeleted; - - ApplyDeletesResult(boolean anyDeletes, long gen, List allDeleted) { - this.anyDeletes = anyDeletes; - this.gen = gen; - this.allDeleted = allDeleted; - } - } - - // Sorts SegmentInfos from smallest to biggest bufferedDelGen: - private static final Comparator sortSegInfoByDelGen = new Comparator() { - @Override - public int compare(SegmentInfoPerCommit si1, SegmentInfoPerCommit si2) { - return Long.compare(si1.getBufferedDeletesGen(), si2.getBufferedDeletesGen()); - } - }; - - /** Resolves the buffered deleted Term/Query/docIDs, into - * actual deleted docIDs in the liveDocs MutableBits for - * each SegmentReader. */ - public synchronized ApplyDeletesResult applyDeletes(IndexWriter.ReaderPool readerPool, List infos) throws IOException { - final long t0 = System.currentTimeMillis(); - - if (infos.size() == 0) { - return new ApplyDeletesResult(false, nextGen++, null); - } - - assert checkDeleteStats(); - - if (!any()) { - if (infoStream.isEnabled("BD")) { - infoStream.message("BD", "applyDeletes: no deletes; skipping"); - } - return new ApplyDeletesResult(false, nextGen++, null); - } - - if (infoStream.isEnabled("BD")) { - infoStream.message("BD", "applyDeletes: infos=" + infos + " packetCount=" + deletes.size()); - } - - final long gen = nextGen++; - - List infos2 = new ArrayList(); - infos2.addAll(infos); - Collections.sort(infos2, sortSegInfoByDelGen); - - CoalescedDeletes coalescedDeletes = null; - boolean anyNewDeletes = false; - - int infosIDX = infos2.size()-1; - int delIDX = deletes.size()-1; - - List allDeleted = null; - - while (infosIDX >= 0) { - //System.out.println("BD: cycle delIDX=" + delIDX + " infoIDX=" + infosIDX); - - final FrozenBufferedDeletes packet = delIDX >= 0 ? deletes.get(delIDX) : null; - final SegmentInfoPerCommit info = infos2.get(infosIDX); - final long segGen = info.getBufferedDeletesGen(); - - if (packet != null && segGen < packet.delGen()) { -// System.out.println(" coalesce"); - if (coalescedDeletes == null) { - coalescedDeletes = new CoalescedDeletes(); - } - if (!packet.isSegmentPrivate) { - /* - * Only coalesce if we are NOT on a segment private del packet: the segment private del packet - * must only applied to segments with the same delGen. Yet, if a segment is already deleted - * from the SI since it had no more documents remaining after some del packets younger than - * its segPrivate packet (higher delGen) have been applied, the segPrivate packet has not been - * removed. - */ - coalescedDeletes.update(packet); - } - - delIDX--; - } else if (packet != null && segGen == packet.delGen()) { - assert packet.isSegmentPrivate : "Packet and Segments deletegen can only match on a segment private del packet gen=" + segGen; - //System.out.println(" eq"); - - // Lock order: IW -> BD -> RP - assert readerPool.infoIsLive(info); - final ReadersAndLiveDocs rld = readerPool.get(info, true); - final SegmentReader reader = rld.getReader(IOContext.READ); - int delCount = 0; - final boolean segAllDeletes; - try { - Map fieldUpdates = null; - if (coalescedDeletes != null) { - //System.out.println(" del coalesced"); - delCount += applyTermDeletes(coalescedDeletes.termsIterable(), rld, reader); - delCount += applyQueryDeletes(coalescedDeletes.queriesIterable(), rld, reader); - fieldUpdates = applyNumericDocValuesUpdates(coalescedDeletes.numericDVUpdates, rld, reader, fieldUpdates); - } - //System.out.println(" del exact"); - // Don't delete by Term here; DocumentsWriterPerThread - // already did that on flush: - delCount += applyQueryDeletes(packet.queriesIterable(), rld, reader); - fieldUpdates = applyNumericDocValuesUpdates(Arrays.asList(packet.updates), rld, reader, fieldUpdates); - if (!fieldUpdates.isEmpty()) { - rld.writeFieldUpdates(info.info.dir, fieldUpdates); - } - final int fullDelCount = rld.info.getDelCount() + rld.getPendingDeleteCount(); - assert fullDelCount <= rld.info.info.getDocCount(); - segAllDeletes = fullDelCount == rld.info.info.getDocCount(); - } finally { - rld.release(reader); - readerPool.release(rld); - } - anyNewDeletes |= delCount > 0; - - if (segAllDeletes) { - if (allDeleted == null) { - allDeleted = new ArrayList(); - } - allDeleted.add(info); - } - - if (infoStream.isEnabled("BD")) { - infoStream.message("BD", "seg=" + info + " segGen=" + segGen + " segDeletes=[" + packet + "]; coalesced deletes=[" + (coalescedDeletes == null ? "null" : coalescedDeletes) + "] newDelCount=" + delCount + (segAllDeletes ? " 100% deleted" : "")); - } - - if (coalescedDeletes == null) { - coalescedDeletes = new CoalescedDeletes(); - } - - /* - * Since we are on a segment private del packet we must not - * update the coalescedDeletes here! We can simply advance to the - * next packet and seginfo. - */ - delIDX--; - infosIDX--; - info.setBufferedDeletesGen(gen); - - } else { - //System.out.println(" gt"); - - if (coalescedDeletes != null) { - // Lock order: IW -> BD -> RP - assert readerPool.infoIsLive(info); - final ReadersAndLiveDocs rld = readerPool.get(info, true); - final SegmentReader reader = rld.getReader(IOContext.READ); - int delCount = 0; - final boolean segAllDeletes; - try { - delCount += applyTermDeletes(coalescedDeletes.termsIterable(), rld, reader); - delCount += applyQueryDeletes(coalescedDeletes.queriesIterable(), rld, reader); - Map fieldUpdates = applyNumericDocValuesUpdates(coalescedDeletes.numericDVUpdates, rld, reader, null); - if (!fieldUpdates.isEmpty()) { - rld.writeFieldUpdates(info.info.dir, fieldUpdates); - } - final int fullDelCount = rld.info.getDelCount() + rld.getPendingDeleteCount(); - assert fullDelCount <= rld.info.info.getDocCount(); - segAllDeletes = fullDelCount == rld.info.info.getDocCount(); - } finally { - rld.release(reader); - readerPool.release(rld); - } - anyNewDeletes |= delCount > 0; - - if (segAllDeletes) { - if (allDeleted == null) { - allDeleted = new ArrayList(); - } - allDeleted.add(info); - } - - if (infoStream.isEnabled("BD")) { - infoStream.message("BD", "seg=" + info + " segGen=" + segGen + " coalesced deletes=[" + coalescedDeletes + "] newDelCount=" + delCount + (segAllDeletes ? " 100% deleted" : "")); - } - } - info.setBufferedDeletesGen(gen); - - infosIDX--; - } - } - - assert checkDeleteStats(); - if (infoStream.isEnabled("BD")) { - infoStream.message("BD", "applyDeletes took " + (System.currentTimeMillis()-t0) + " msec"); - } - // assert infos != segmentInfos || !any() : "infos=" + infos + " segmentInfos=" + segmentInfos + " any=" + any; - - return new ApplyDeletesResult(anyNewDeletes, gen, allDeleted); - } - - synchronized long getNextGen() { - return nextGen++; - } - - // Lock order IW -> BD - /* Removes any BufferedDeletes that we no longer need to - * store because all segments in the index have had the - * deletes applied. */ - public synchronized void prune(SegmentInfos segmentInfos) { - assert checkDeleteStats(); - long minGen = Long.MAX_VALUE; - for(SegmentInfoPerCommit info : segmentInfos) { - minGen = Math.min(info.getBufferedDeletesGen(), minGen); - } - - if (infoStream.isEnabled("BD")) { - infoStream.message("BD", "prune sis=" + segmentInfos + " minGen=" + minGen + " packetCount=" + deletes.size()); - } - final int limit = deletes.size(); - for(int delIDX=0;delIDX= minGen) { - prune(delIDX); - assert checkDeleteStats(); - return; - } - } - - // All deletes pruned - prune(limit); - assert !any(); - assert checkDeleteStats(); - } - - private synchronized void prune(int count) { - if (count > 0) { - if (infoStream.isEnabled("BD")) { - infoStream.message("BD", "pruneDeletes: prune " + count + " packets; " + (deletes.size() - count) + " packets remain"); - } - for(int delIDX=0;delIDX= 0; - bytesUsed.addAndGet(-packet.bytesUsed); - assert bytesUsed.get() >= 0; - } - deletes.subList(0, count).clear(); - } - } - - // Delete by Term - private synchronized long applyTermDeletes(Iterable termsIter, ReadersAndLiveDocs rld, SegmentReader reader) throws IOException { - long delCount = 0; - Fields fields = reader.fields(); - if (fields == null) { - // This reader has no postings - return 0; - } - - TermsEnum termsEnum = null; - - String currentField = null; - DocsEnum docs = null; - - assert checkDeleteTerm(null); - - boolean any = false; - - //System.out.println(Thread.currentThread().getName() + " del terms reader=" + reader); - for (Term term : termsIter) { - // Since we visit terms sorted, we gain performance - // by re-using the same TermsEnum and seeking only - // forwards - if (!term.field().equals(currentField)) { - assert currentField == null || currentField.compareTo(term.field()) < 0; - currentField = term.field(); - Terms terms = fields.terms(currentField); - if (terms != null) { - termsEnum = terms.iterator(termsEnum); - } else { - termsEnum = null; - } - } - - if (termsEnum == null) { - continue; - } - assert checkDeleteTerm(term); - - // 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); - - if (docsEnum != null) { - while (true) { - final int docID = docsEnum.nextDoc(); - //System.out.println(Thread.currentThread().getName() + " del term=" + term + " doc=" + docID); - 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 (rld.delete(docID)) { - delCount++; - } - } - } - } - } - - return delCount; - } - - // NumericDocValues Updates - // If otherFieldUpdates != null, we need to merge the updates into them - private synchronized Map applyNumericDocValuesUpdates(Iterable updates, - ReadersAndLiveDocs rld, SegmentReader reader, Map otherFieldUpdates) throws IOException { - Fields fields = reader.fields(); - if (fields == null) { - // This reader has no postings - return Collections.emptyMap(); - } - - // TODO: we can process the updates per DV field, from last to first so that - // if multiple terms affect same document for the same field, we add an update - // only once (that of the last term). To do that, we can keep a bitset which - // marks which documents have already been updated. So e.g. if term T1 - // updates doc 7, and then we process term T2 and it updates doc 7 as well, - // we don't apply the update since we know T1 came last and therefore wins - // the update. - // We can also use that bitset as 'liveDocs' to pass to TermEnum.docs(), so - // that these documents aren't even returned. - - String currentField = null; - TermsEnum termsEnum = null; - DocsEnum docs = null; - final Map result = otherFieldUpdates == null ? new HashMap() : otherFieldUpdates; - //System.out.println(Thread.currentThread().getName() + " numericDVUpdate reader=" + reader); - for (NumericUpdate update : updates) { - Term term = update.term; - int limit = update.docIDUpto; - - // TODO: we traverse the terms in update order (not term order) so that we - // apply the updates in the correct order, i.e. if two terms udpate the - // same document, the last one that came in wins, irrespective of the - // terms lexical order. - // we can apply the updates in terms order if we keep an updatesGen (and - // increment it with every update) and attach it to each NumericUpdate. Note - // that we cannot rely only on docIDUpto because an app may send two updates - // which will get same docIDUpto, yet will still need to respect the order - // those updates arrived. - - if (!term.field().equals(currentField)) { - // if we change the code to process updates in terms order, enable this assert -// assert currentField == null || currentField.compareTo(term.field()) < 0; - currentField = term.field(); - Terms terms = fields.terms(currentField); - if (terms != null) { - termsEnum = terms.iterator(termsEnum); - } else { - termsEnum = null; - continue; // no terms in that field - } - } - - if (termsEnum == null) { - continue; - } - // 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); - - NumericFieldUpdates fieldUpdates = result.get(update.field); - if (fieldUpdates == null) { - fieldUpdates = new NumericFieldUpdates.PackedNumericFieldUpdates(reader.maxDoc()); - result.put(update.field, fieldUpdates); - } - 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 - } - fieldUpdates.add(doc, update.value); - } - } - } - return result; - } - - public static class QueryAndLimit { - public final Query query; - public final int limit; - public QueryAndLimit(Query query, int limit) { - this.query = query; - this.limit = limit; - } - } - - // Delete by query - private static long applyQueryDeletes(Iterable queriesIter, ReadersAndLiveDocs rld, final SegmentReader reader) throws IOException { - long delCount = 0; - final AtomicReaderContext readerContext = reader.getContext(); - boolean any = false; - for (QueryAndLimit ent : queriesIter) { - Query query = ent.query; - int limit = ent.limit; - final DocIdSet docs = new QueryWrapperFilter(query).getDocIdSet(readerContext, reader.getLiveDocs()); - if (docs != null) { - final DocIdSetIterator it = docs.iterator(); - if (it != null) { - while(true) { - int doc = it.nextDoc(); - if (doc >= limit) { - break; - } - - if (!any) { - rld.initWritableLiveDocs(); - any = true; - } - - if (rld.delete(doc)) { - delCount++; - } - } - } - } - } - - return delCount; - } - - // used only by assert - private boolean checkDeleteTerm(Term term) { - if (term != null) { - assert lastDeleteTerm == null || term.compareTo(lastDeleteTerm) > 0: "lastTerm=" + lastDeleteTerm + " vs term=" + term; - } - // TODO: we re-use term now in our merged iterable, but we shouldn't clone, instead copy for this assert - lastDeleteTerm = term == null ? null : new Term(term.field(), BytesRef.deepCopyOf(term.bytes)); - return true; - } - - // only for assert - private boolean checkDeleteStats() { - int numTerms2 = 0; - long bytesUsed2 = 0; - for(FrozenBufferedDeletes packet : deletes) { - numTerms2 += packet.numTermDeletes; - bytesUsed2 += packet.bytesUsed; - } - assert numTerms2 == numTerms.get(): "numTerms2=" + numTerms2 + " vs " + numTerms.get(); - assert bytesUsed2 == bytesUsed.get(): "bytesUsed2=" + bytesUsed2 + " vs " + bytesUsed; - return true; - } -} Index: lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java (working copy) @@ -38,7 +38,7 @@ // instance on DocumentWriterPerThread, or via sync'd code by // DocumentsWriterDeleteQueue -class BufferedDeletes { // TODO (DVU_RENAME) BufferedUpdates? +class BufferedUpdates { /* Rough logic: HashMap has an array[Entry] w/ varying load factor (say 2 * POINTER). Entry is object w/ Term @@ -116,7 +116,7 @@ long gen; - public BufferedDeletes() { + public BufferedUpdates() { this.bytesUsed = new AtomicLong(); } Index: lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (revision 1537832) +++ lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (working copy) @@ -51,10 +51,10 @@ * track which BufferedDeletes packets to apply to any given * segment. */ -class BufferedDeletesStream { // TODO (DVU_RENAME) BufferedUpdatesStream +class BufferedUpdatesStream { // TODO: maybe linked list? - private final List deletes = new ArrayList(); + private final List updates = new ArrayList(); // Starts at 1 so that SegmentInfos that have never had // deletes applied (whose bufferedDelGen defaults to 0) @@ -68,13 +68,13 @@ private final AtomicLong bytesUsed = new AtomicLong(); private final AtomicInteger numTerms = new AtomicInteger(); - public BufferedDeletesStream(InfoStream infoStream) { + public BufferedUpdatesStream(InfoStream infoStream) { this.infoStream = infoStream; } // Appends a new packet of buffered deletes to the stream, // setting its generation: - public synchronized long push(FrozenBufferedDeletes packet) { + public synchronized long push(FrozenBufferedUpdates packet) { /* * The insert operation must be atomic. If we let threads increment the gen * and push the packet afterwards we risk that packets are out of order. @@ -86,12 +86,12 @@ assert packet.any(); assert checkDeleteStats(); assert packet.delGen() < nextGen; - assert deletes.isEmpty() || deletes.get(deletes.size()-1).delGen() < packet.delGen() : "Delete packets must be in order"; - deletes.add(packet); + assert updates.isEmpty() || updates.get(updates.size()-1).delGen() < packet.delGen() : "Delete packets must be in order"; + updates.add(packet); numTerms.addAndGet(packet.numTermDeletes); bytesUsed.addAndGet(packet.bytesUsed); if (infoStream.isEnabled("BD")) { - infoStream.message("BD", "push deletes " + packet + " delGen=" + packet.delGen() + " packetCount=" + deletes.size() + " totBytesUsed=" + bytesUsed.get()); + infoStream.message("BD", "push deletes " + packet + " delGen=" + packet.delGen() + " packetCount=" + updates.size() + " totBytesUsed=" + bytesUsed.get()); } assert checkDeleteStats(); return packet.delGen(); @@ -98,7 +98,7 @@ } public synchronized void clear() { - deletes.clear(); + updates.clear(); nextGen = 1; numTerms.set(0); bytesUsed.set(0); @@ -125,9 +125,9 @@ public final long gen; // If non-null, contains segments that are 100% deleted - public final List allDeleted; + public final List allDeleted; - ApplyDeletesResult(boolean anyDeletes, long gen, List allDeleted) { + ApplyDeletesResult(boolean anyDeletes, long gen, List allDeleted) { this.anyDeletes = anyDeletes; this.gen = gen; this.allDeleted = allDeleted; @@ -135,9 +135,9 @@ } // Sorts SegmentInfos from smallest to biggest bufferedDelGen: - private static final Comparator sortSegInfoByDelGen = new Comparator() { + private static final Comparator sortSegInfoByDelGen = new Comparator() { @Override - public int compare(SegmentInfoPerCommit si1, SegmentInfoPerCommit si2) { + public int compare(SegmentCommitInfo si1, SegmentCommitInfo si2) { return Long.compare(si1.getBufferedDeletesGen(), si2.getBufferedDeletesGen()); } }; @@ -145,7 +145,7 @@ /** Resolves the buffered deleted Term/Query/docIDs, into * actual deleted docIDs in the liveDocs MutableBits for * each SegmentReader. */ - public synchronized ApplyDeletesResult applyDeletes(IndexWriter.ReaderPool readerPool, List infos) throws IOException { + public synchronized ApplyDeletesResult applyDeletesAndUpdates(IndexWriter.ReaderPool readerPool, List infos) throws IOException { final long t0 = System.currentTimeMillis(); if (infos.size() == 0) { @@ -162,34 +162,34 @@ } if (infoStream.isEnabled("BD")) { - infoStream.message("BD", "applyDeletes: infos=" + infos + " packetCount=" + deletes.size()); + infoStream.message("BD", "applyDeletes: infos=" + infos + " packetCount=" + updates.size()); } final long gen = nextGen++; - List infos2 = new ArrayList(); + List infos2 = new ArrayList(); infos2.addAll(infos); Collections.sort(infos2, sortSegInfoByDelGen); - CoalescedDeletes coalescedDeletes = null; + CoalescedUpdates coalescedDeletes = null; boolean anyNewDeletes = false; int infosIDX = infos2.size()-1; - int delIDX = deletes.size()-1; + int delIDX = updates.size()-1; - List allDeleted = null; + List allDeleted = null; while (infosIDX >= 0) { //System.out.println("BD: cycle delIDX=" + delIDX + " infoIDX=" + infosIDX); - final FrozenBufferedDeletes packet = delIDX >= 0 ? deletes.get(delIDX) : null; - final SegmentInfoPerCommit info = infos2.get(infosIDX); + final FrozenBufferedUpdates packet = delIDX >= 0 ? updates.get(delIDX) : null; + final SegmentCommitInfo info = infos2.get(infosIDX); final long segGen = info.getBufferedDeletesGen(); if (packet != null && segGen < packet.delGen()) { // System.out.println(" coalesce"); if (coalescedDeletes == null) { - coalescedDeletes = new CoalescedDeletes(); + coalescedDeletes = new CoalescedUpdates(); } if (!packet.isSegmentPrivate) { /* @@ -209,7 +209,7 @@ // Lock order: IW -> BD -> RP assert readerPool.infoIsLive(info); - final ReadersAndLiveDocs rld = readerPool.get(info, true); + final ReadersAndUpdates rld = readerPool.get(info, true); final SegmentReader reader = rld.getReader(IOContext.READ); int delCount = 0; final boolean segAllDeletes; @@ -240,7 +240,7 @@ if (segAllDeletes) { if (allDeleted == null) { - allDeleted = new ArrayList(); + allDeleted = new ArrayList(); } allDeleted.add(info); } @@ -250,7 +250,7 @@ } if (coalescedDeletes == null) { - coalescedDeletes = new CoalescedDeletes(); + coalescedDeletes = new CoalescedUpdates(); } /* @@ -268,7 +268,7 @@ if (coalescedDeletes != null) { // Lock order: IW -> BD -> RP assert readerPool.infoIsLive(info); - final ReadersAndLiveDocs rld = readerPool.get(info, true); + final ReadersAndUpdates rld = readerPool.get(info, true); final SegmentReader reader = rld.getReader(IOContext.READ); int delCount = 0; final boolean segAllDeletes; @@ -290,7 +290,7 @@ if (segAllDeletes) { if (allDeleted == null) { - allDeleted = new ArrayList(); + allDeleted = new ArrayList(); } allDeleted.add(info); } @@ -325,16 +325,16 @@ public synchronized void prune(SegmentInfos segmentInfos) { assert checkDeleteStats(); long minGen = Long.MAX_VALUE; - for(SegmentInfoPerCommit info : segmentInfos) { + for(SegmentCommitInfo info : segmentInfos) { minGen = Math.min(info.getBufferedDeletesGen(), minGen); } if (infoStream.isEnabled("BD")) { - infoStream.message("BD", "prune sis=" + segmentInfos + " minGen=" + minGen + " packetCount=" + deletes.size()); + infoStream.message("BD", "prune sis=" + segmentInfos + " minGen=" + minGen + " packetCount=" + updates.size()); } - final int limit = deletes.size(); + final int limit = updates.size(); for(int delIDX=0;delIDX= minGen) { + if (updates.get(delIDX).delGen() >= minGen) { prune(delIDX); assert checkDeleteStats(); return; @@ -350,21 +350,21 @@ private synchronized void prune(int count) { if (count > 0) { if (infoStream.isEnabled("BD")) { - infoStream.message("BD", "pruneDeletes: prune " + count + " packets; " + (deletes.size() - count) + " packets remain"); + infoStream.message("BD", "pruneDeletes: prune " + count + " packets; " + (updates.size() - count) + " packets remain"); } for(int delIDX=0;delIDX= 0; bytesUsed.addAndGet(-packet.bytesUsed); assert bytesUsed.get() >= 0; } - deletes.subList(0, count).clear(); + updates.subList(0, count).clear(); } } // Delete by Term - private synchronized long applyTermDeletes(Iterable termsIter, ReadersAndLiveDocs rld, SegmentReader reader) throws IOException { + private synchronized long applyTermDeletes(Iterable termsIter, ReadersAndUpdates rld, SegmentReader reader) throws IOException { long delCount = 0; Fields fields = reader.fields(); if (fields == null) { @@ -439,7 +439,7 @@ // NumericDocValues Updates // If otherFieldUpdates != null, we need to merge the updates into them private synchronized Map applyNumericDocValuesUpdates(Iterable updates, - ReadersAndLiveDocs rld, SegmentReader reader, Map otherFieldUpdates) throws IOException { + ReadersAndUpdates rld, SegmentReader reader, Map otherFieldUpdates) throws IOException { Fields fields = reader.fields(); if (fields == null) { // This reader has no postings @@ -527,7 +527,7 @@ } // Delete by query - private static long applyQueryDeletes(Iterable queriesIter, ReadersAndLiveDocs rld, final SegmentReader reader) throws IOException { + private static long applyQueryDeletes(Iterable queriesIter, ReadersAndUpdates rld, final SegmentReader reader) throws IOException { long delCount = 0; final AtomicReaderContext readerContext = reader.getContext(); boolean any = false; @@ -574,7 +574,7 @@ private boolean checkDeleteStats() { int numTerms2 = 0; long bytesUsed2 = 0; - for(FrozenBufferedDeletes packet : deletes) { + for(FrozenBufferedUpdates packet : updates) { numTerms2 += packet.numTermDeletes; bytesUsed2 += packet.bytesUsed; } Index: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (working copy) @@ -389,7 +389,7 @@ String oldSegs = null; boolean foundNonNullVersion = false; Comparator versionComparator = StringHelper.getVersionComparator(); - for (SegmentInfoPerCommit si : sis) { + for (SegmentCommitInfo si : sis) { String version = si.info.getVersion(); if (version == null) { // pre-3.1 segment @@ -483,7 +483,7 @@ result.maxSegmentName = -1; for(int i=0;i result.maxSegmentName) { result.maxSegmentName = segmentName; Index: lucene/core/src/java/org/apache/lucene/index/CoalescedDeletes.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/CoalescedDeletes.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/CoalescedDeletes.java (working copy) @@ -1,97 +0,0 @@ -package org.apache.lucene.index; - -/* - * 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. - */ - -import java.util.ArrayList; -import java.util.Iterator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.lucene.search.Query; -import org.apache.lucene.index.BufferedDeletesStream.QueryAndLimit; -import org.apache.lucene.util.MergedIterator; - -class CoalescedDeletes { - final Map queries = new HashMap(); - final List> iterables = new ArrayList>(); - final List numericDVUpdates = new ArrayList(); - - @Override - public String toString() { - // note: we could add/collect more debugging information - 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++) { - 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 termsIterable() { - return new Iterable() { - @SuppressWarnings({"unchecked","rawtypes"}) - @Override - public Iterator iterator() { - Iterator subs[] = new Iterator[iterables.size()]; - for (int i = 0; i < iterables.size(); i++) { - subs[i] = iterables.get(i).iterator(); - } - return new MergedIterator(subs); - } - }; - } - - public Iterable queriesIterable() { - return new Iterable() { - - @Override - public Iterator iterator() { - return new Iterator() { - private final Iterator> iter = queries.entrySet().iterator(); - - @Override - public boolean hasNext() { - return iter.hasNext(); - } - - @Override - public QueryAndLimit next() { - final Map.Entry ent = iter.next(); - return new QueryAndLimit(ent.getKey(), ent.getValue()); - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - }; - } - }; - } -} Index: lucene/core/src/java/org/apache/lucene/index/CoalescedUpdates.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/CoalescedUpdates.java (revision 1537832) +++ lucene/core/src/java/org/apache/lucene/index/CoalescedUpdates.java (working copy) @@ -24,10 +24,10 @@ import java.util.Map; import org.apache.lucene.search.Query; -import org.apache.lucene.index.BufferedDeletesStream.QueryAndLimit; +import org.apache.lucene.index.BufferedUpdatesStream.QueryAndLimit; import org.apache.lucene.util.MergedIterator; -class CoalescedDeletes { +class CoalescedUpdates { final Map queries = new HashMap(); final List> iterables = new ArrayList>(); final List numericDVUpdates = new ArrayList(); @@ -35,15 +35,15 @@ @Override public String toString() { // note: we could add/collect more debugging information - return "CoalescedDeletes(termSets=" + iterables.size() + ",queries=" + queries.size() + ",numericUpdates=" + numericDVUpdates.size() + ")"; + return "CoalescedUpdates(termSets=" + iterables.size() + ",queries=" + queries.size() + ",numericUpdates=" + numericDVUpdates.size() + ")"; } - void update(FrozenBufferedDeletes in) { + void update(FrozenBufferedUpdates in) { iterables.add(in.termsIterable()); for (int queryIdx = 0; queryIdx < in.queries.length; queryIdx++) { final Query query = in.queries[queryIdx]; - queries.put(query, BufferedDeletes.MAX_INT); + queries.put(query, BufferedUpdates.MAX_INT); } for (NumericUpdate nu : in.updates) { Index: lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (working copy) @@ -136,7 +136,7 @@ flushPolicy = config.getFlushPolicy(); this.writer = writer; this.events = new ConcurrentLinkedQueue(); - flushControl = new DocumentsWriterFlushControl(this, config, writer.bufferedDeletesStream); + flushControl = new DocumentsWriterFlushControl(this, config, writer.bufferedUpdatesStream); } synchronized boolean deleteQueries(final Query... queries) throws IOException { @@ -323,7 +323,7 @@ } public int getBufferedDeleteTermsSize() { - return deleteQueue.getBufferedDeleteTermsSize(); + return deleteQueue.getBufferedUpdatesTermsSize(); } //for testing Index: lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java (working copy) @@ -39,7 +39,7 @@ * single linked the garbage collector takes care of pruning the list for us. * All nodes in the list that are still relevant should be either directly or * indirectly referenced by one of the DWPT's private {@link DeleteSlice} or by - * the global {@link BufferedDeletes} slice. + * the global {@link BufferedUpdates} slice. *

* Each DWPT as well as the global delete pool maintain their private * DeleteSlice instance. In the DWPT case updating a slice is equivalent to @@ -52,7 +52,7 @@ *

  • updates its private {@link DeleteSlice} either by calling * {@link #updateSlice(DeleteSlice)} or {@link #add(Term, DeleteSlice)} (if the * document has a delTerm)
  • - *
  • applies all deletes in the slice to its private {@link BufferedDeletes} + *
  • applies all deletes in the slice to its private {@link BufferedUpdates} * and resets it
  • *
  • increments its internal document id
  • * @@ -72,7 +72,7 @@ .newUpdater(DocumentsWriterDeleteQueue.class, Node.class, "tail"); private final DeleteSlice globalSlice; - private final BufferedDeletes globalBufferedDeletes; + private final BufferedUpdates globalBufferedUpdates; /* only acquired to update the global deletes */ private final ReentrantLock globalBufferLock = new ReentrantLock(); @@ -83,11 +83,11 @@ } DocumentsWriterDeleteQueue(long generation) { - this(new BufferedDeletes(), generation); + this(new BufferedUpdates(), generation); } - DocumentsWriterDeleteQueue(BufferedDeletes globalBufferedDeletes, long generation) { - this.globalBufferedDeletes = globalBufferedDeletes; + DocumentsWriterDeleteQueue(BufferedUpdates globalBufferedUpdates, long generation) { + this.globalBufferedUpdates = globalBufferedUpdates; this.generation = generation; /* * we use a sentinel instance as our initial tail. No slice will ever try to @@ -177,9 +177,9 @@ /* * check if all items in the global slice were applied * and if the global slice is up-to-date - * and if globalBufferedDeletes has changes + * and if globalBufferedUpdates has changes */ - return globalBufferedDeletes.any() || !globalSlice.isEmpty() || globalSlice.sliceTail != tail + return globalBufferedUpdates.any() || !globalSlice.isEmpty() || globalSlice.sliceTail != tail || tail.next != null; } finally { globalBufferLock.unlock(); @@ -197,7 +197,7 @@ try { if (updateSlice(globalSlice)) { // System.out.println(Thread.currentThread() + ": apply globalSlice"); - globalSlice.apply(globalBufferedDeletes, BufferedDeletes.MAX_INT); + globalSlice.apply(globalBufferedUpdates, BufferedUpdates.MAX_INT); } } finally { globalBufferLock.unlock(); @@ -205,7 +205,7 @@ } } - FrozenBufferedDeletes freezeGlobalBuffer(DeleteSlice callerSlice) { + FrozenBufferedUpdates freezeGlobalBuffer(DeleteSlice callerSlice) { globalBufferLock.lock(); /* * Here we freeze the global buffer so we need to lock it, apply all @@ -222,13 +222,13 @@ try { if (globalSlice.sliceTail != currentTail) { globalSlice.sliceTail = currentTail; - globalSlice.apply(globalBufferedDeletes, BufferedDeletes.MAX_INT); + globalSlice.apply(globalBufferedUpdates, BufferedUpdates.MAX_INT); } // System.out.println(Thread.currentThread().getName() + ": now freeze global buffer " + globalBufferedDeletes); - final FrozenBufferedDeletes packet = new FrozenBufferedDeletes( - globalBufferedDeletes, false); - globalBufferedDeletes.clear(); + final FrozenBufferedUpdates packet = new FrozenBufferedUpdates( + globalBufferedUpdates, false); + globalBufferedUpdates.clear(); return packet; } finally { globalBufferLock.unlock(); @@ -262,7 +262,7 @@ sliceHead = sliceTail = currentTail; } - void apply(BufferedDeletes del, int docIDUpto) { + void apply(BufferedUpdates del, int docIDUpto) { if (sliceHead == sliceTail) { // 0 length slice return; @@ -302,7 +302,7 @@ } public int numGlobalTermDeletes() { - return globalBufferedDeletes.numTermDeletes.get(); + return globalBufferedUpdates.numTermDeletes.get(); } void clear() { @@ -310,7 +310,7 @@ try { final Node currentTail = tail; globalSlice.sliceHead = globalSlice.sliceTail = currentTail; - globalBufferedDeletes.clear(); + globalBufferedUpdates.clear(); } finally { globalBufferLock.unlock(); } @@ -328,7 +328,7 @@ static final AtomicReferenceFieldUpdater nextUpdater = AtomicReferenceFieldUpdater .newUpdater(Node.class, Node.class, "next"); - void apply(BufferedDeletes bufferedDeletes, int docIDUpto) { + void apply(BufferedUpdates bufferedDeletes, int docIDUpto) { throw new IllegalStateException("sentinel item must never be applied"); } @@ -344,7 +344,7 @@ } @Override - void apply(BufferedDeletes bufferedDeletes, int docIDUpto) { + void apply(BufferedUpdates bufferedDeletes, int docIDUpto) { bufferedDeletes.addTerm(item, docIDUpto); } @@ -360,9 +360,9 @@ } @Override - void apply(BufferedDeletes bufferedDeletes, int docIDUpto) { + void apply(BufferedUpdates bufferedUpdates, int docIDUpto) { for (Query query : item) { - bufferedDeletes.addQuery(query, docIDUpto); + bufferedUpdates.addQuery(query, docIDUpto); } } } @@ -373,9 +373,9 @@ } @Override - void apply(BufferedDeletes bufferedDeletes, int docIDUpto) { + void apply(BufferedUpdates bufferedUpdates, int docIDUpto) { for (Term term : item) { - bufferedDeletes.addTerm(term, docIDUpto); + bufferedUpdates.addTerm(term, docIDUpto); } } @@ -392,8 +392,8 @@ } @Override - void apply(BufferedDeletes bufferedDeletes, int docIDUpto) { - bufferedDeletes.addNumericUpdate(item, docIDUpto); + void apply(BufferedUpdates bufferedUpdates, int docIDUpto) { + bufferedUpdates.addNumericUpdate(item, docIDUpto); } @Override @@ -408,19 +408,19 @@ try { if (globalSlice.sliceTail != currentTail) { globalSlice.sliceTail = currentTail; - globalSlice.apply(globalBufferedDeletes, BufferedDeletes.MAX_INT); + globalSlice.apply(globalBufferedUpdates, BufferedUpdates.MAX_INT); } - return globalBufferedDeletes.any(); + return globalBufferedUpdates.any(); } finally { globalBufferLock.unlock(); } } - public int getBufferedDeleteTermsSize() { + public int getBufferedUpdatesTermsSize() { globalBufferLock.lock(); try { forceApplyGlobalSlice(); - return globalBufferedDeletes.terms.size(); + return globalBufferedUpdates.terms.size(); } finally { globalBufferLock.unlock(); } @@ -427,7 +427,7 @@ } public long bytesUsed() { - return globalBufferedDeletes.bytesUsed.get(); + return globalBufferedUpdates.bytesUsed.get(); } @Override Index: lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (working copy) @@ -68,10 +68,10 @@ private boolean closed = false; private final DocumentsWriter documentsWriter; private final LiveIndexWriterConfig config; - private final BufferedDeletesStream bufferedDeletesStream; + private final BufferedUpdatesStream bufferedUpdatesStream; private final InfoStream infoStream; - DocumentsWriterFlushControl(DocumentsWriter documentsWriter, LiveIndexWriterConfig config, BufferedDeletesStream bufferedDeletesStream) { + DocumentsWriterFlushControl(DocumentsWriter documentsWriter, LiveIndexWriterConfig config, BufferedUpdatesStream bufferedUpdatesStream) { this.infoStream = config.getInfoStream(); this.stallControl = new DocumentsWriterStallControl(); this.perThreadPool = documentsWriter.perThreadPool; @@ -79,7 +79,7 @@ this.config = config; this.hardMaxBytesPerDWPT = config.getRAMPerThreadHardLimitMB() * 1024 * 1024; this.documentsWriter = documentsWriter; - this.bufferedDeletesStream = bufferedDeletesStream; + this.bufferedUpdatesStream = bufferedUpdatesStream; } public synchronized long activeBytes() { @@ -418,11 +418,11 @@ * Returns the number of delete terms in the global pool */ public int getNumGlobalTermDeletes() { - return documentsWriter.deleteQueue.numGlobalTermDeletes() + bufferedDeletesStream.numTerms(); + return documentsWriter.deleteQueue.numGlobalTermDeletes() + bufferedUpdatesStream.numTerms(); } public long getDeleteBytesUsed() { - return documentsWriter.deleteQueue.bytesUsed() + bufferedDeletesStream.bytesUsed(); + return documentsWriter.deleteQueue.bytesUsed() + bufferedUpdatesStream.bytesUsed(); } synchronized int numFlushingDWPT() { Index: lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java (working copy) @@ -164,12 +164,12 @@ } static abstract class FlushTicket { - protected FrozenBufferedDeletes frozenDeletes; + protected FrozenBufferedUpdates frozenUpdates; protected boolean published = false; - protected FlushTicket(FrozenBufferedDeletes frozenDeletes) { - assert frozenDeletes != null; - this.frozenDeletes = frozenDeletes; + protected FlushTicket(FrozenBufferedUpdates frozenUpdates) { + assert frozenUpdates != null; + this.frozenUpdates = frozenUpdates; } protected abstract void publish(IndexWriter writer) throws IOException; @@ -181,36 +181,36 @@ * publishing operation is synced on IW -> BDS so that the {@link SegmentInfo}'s * delete generation is always GlobalPacket_deleteGeneration + 1 */ - protected final void publishFlushedSegment(IndexWriter indexWriter, FlushedSegment newSegment, FrozenBufferedDeletes globalPacket) + protected final void publishFlushedSegment(IndexWriter indexWriter, FlushedSegment newSegment, FrozenBufferedUpdates globalPacket) throws IOException { assert newSegment != null; assert newSegment.segmentInfo != null; - final FrozenBufferedDeletes segmentDeletes = newSegment.segmentDeletes; + final FrozenBufferedUpdates segmentUpdates = newSegment.segmentUpdates; //System.out.println("FLUSH: " + newSegment.segmentInfo.info.name); if (indexWriter.infoStream.isEnabled("DW")) { - indexWriter.infoStream.message("DW", "publishFlushedSegment seg-private deletes=" + segmentDeletes); + indexWriter.infoStream.message("DW", "publishFlushedSegment seg-private updates=" + segmentUpdates); } - if (segmentDeletes != null && indexWriter.infoStream.isEnabled("DW")) { - indexWriter.infoStream.message("DW", "flush: push buffered seg private deletes: " + segmentDeletes); + if (segmentUpdates != null && indexWriter.infoStream.isEnabled("DW")) { + indexWriter.infoStream.message("DW", "flush: push buffered seg private updates: " + segmentUpdates); } // now publish! - indexWriter.publishFlushedSegment(newSegment.segmentInfo, segmentDeletes, globalPacket); + indexWriter.publishFlushedSegment(newSegment.segmentInfo, segmentUpdates, globalPacket); } - protected final void finishFlush(IndexWriter indexWriter, FlushedSegment newSegment, FrozenBufferedDeletes bufferedDeletes) + protected final void finishFlush(IndexWriter indexWriter, FlushedSegment newSegment, FrozenBufferedUpdates bufferedUpdates) throws IOException { // Finish the flushed segment and publish it to IndexWriter if (newSegment == null) { - assert bufferedDeletes != null; - if (bufferedDeletes != null && bufferedDeletes.any()) { - indexWriter.publishFrozenDeletes(bufferedDeletes); + assert bufferedUpdates != null; + if (bufferedUpdates != null && bufferedUpdates.any()) { + indexWriter.publishFrozenUpdates(bufferedUpdates); if (indexWriter.infoStream.isEnabled("DW")) { - indexWriter.infoStream.message("DW", "flush: push buffered deletes: " + bufferedDeletes); + indexWriter.infoStream.message("DW", "flush: push buffered updates: " + bufferedUpdates); } } } else { - publishFlushedSegment(indexWriter, newSegment, bufferedDeletes); + publishFlushedSegment(indexWriter, newSegment, bufferedUpdates); } } } @@ -217,8 +217,8 @@ static final class GlobalDeletesTicket extends FlushTicket { - protected GlobalDeletesTicket(FrozenBufferedDeletes frozenDeletes) { - super(frozenDeletes); + protected GlobalDeletesTicket(FrozenBufferedUpdates frozenUpdates) { + super(frozenUpdates); } @Override protected void publish(IndexWriter writer) throws IOException { @@ -225,7 +225,7 @@ assert !published : "ticket was already publised - can not publish twice"; published = true; // its a global ticket - no segment to publish - finishFlush(writer, null, frozenDeletes); + finishFlush(writer, null, frozenUpdates); } @Override @@ -238,7 +238,7 @@ private FlushedSegment segment; private boolean failed = false; - protected SegmentFlushTicket(FrozenBufferedDeletes frozenDeletes) { + protected SegmentFlushTicket(FrozenBufferedUpdates frozenDeletes) { super(frozenDeletes); } @@ -246,7 +246,7 @@ protected void publish(IndexWriter writer) throws IOException { assert !published : "ticket was already publised - can not publish twice"; published = true; - finishFlush(writer, segment, frozenDeletes); + finishFlush(writer, segment, frozenUpdates); } protected void setSegment(FlushedSegment segment) { Index: lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (working copy) @@ -124,17 +124,17 @@ } static class FlushedSegment { - final SegmentInfoPerCommit segmentInfo; + final SegmentCommitInfo segmentInfo; final FieldInfos fieldInfos; - final FrozenBufferedDeletes segmentDeletes; + final FrozenBufferedUpdates segmentUpdates; final MutableBits liveDocs; final int delCount; - private FlushedSegment(SegmentInfoPerCommit segmentInfo, FieldInfos fieldInfos, - BufferedDeletes segmentDeletes, MutableBits liveDocs, int delCount) { + private FlushedSegment(SegmentCommitInfo segmentInfo, FieldInfos fieldInfos, + BufferedUpdates segmentUpdates, MutableBits liveDocs, int delCount) { this.segmentInfo = segmentInfo; this.fieldInfos = fieldInfos; - this.segmentDeletes = segmentDeletes != null && segmentDeletes.any() ? new FrozenBufferedDeletes(segmentDeletes, true) : null; + this.segmentUpdates = segmentUpdates != null && segmentUpdates.any() ? new FrozenBufferedUpdates(segmentUpdates, true) : null; this.liveDocs = liveDocs; this.delCount = delCount; } @@ -156,7 +156,7 @@ } catch (Throwable t) { } - pendingDeletes.clear(); + pendingUpdates.clear(); createdFiles.addAll(directory.getCreatedFiles()); } finally { aborting = false; @@ -174,8 +174,8 @@ final Counter bytesUsed; SegmentWriteState flushState; - // Deletes for our still-in-RAM (to be flushed next) segment - final BufferedDeletes pendingDeletes; + // Updates for our still-in-RAM (to be flushed next) segment + final BufferedUpdates pendingUpdates; 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 @@ -203,11 +203,11 @@ this.docState.similarity = indexWriterConfig.getSimilarity(); bytesUsed = Counter.newCounter(); byteBlockAllocator = new DirectTrackingAllocator(bytesUsed); - pendingDeletes = new BufferedDeletes(); + pendingUpdates = new BufferedUpdates(); intBlockAllocator = new IntBlockAllocator(bytesUsed); this.deleteQueue = deleteQueue; assert numDocsInRAM == 0 : "num docs " + numDocsInRAM; - pendingDeletes.clear(); + pendingUpdates.clear(); deleteSlice = deleteQueue.newSlice(); segmentInfo = new SegmentInfo(directoryOrig, Constants.LUCENE_MAIN_VERSION, segmentName, -1, false, codec, null); @@ -330,7 +330,7 @@ if (delTerm != null) { deleteQueue.add(delTerm, deleteSlice); assert deleteSlice.isTailItem(delTerm) : "expected the delete term as the tail item"; - deleteSlice.apply(pendingDeletes, numDocsInRAM-docCount); + deleteSlice.apply(pendingUpdates, numDocsInRAM-docCount); } } finally { @@ -368,7 +368,7 @@ } if (applySlice) { - deleteSlice.apply(pendingDeletes, numDocsInRAM); + deleteSlice.apply(pendingUpdates, numDocsInRAM); } else { // if we don't need to apply we must reset! deleteSlice.reset(); } @@ -378,7 +378,7 @@ // Buffer a specific docID for deletion. Currently only // used when we hit an exception when adding a document void deleteDocID(int docIDUpto) { - pendingDeletes.addDocID(docIDUpto); + pendingUpdates.addDocID(docIDUpto); // NOTE: we do not trigger flush here. This is // potentially a RAM leak, if you have an app that tries // to add docs but every single doc always hits a @@ -395,7 +395,7 @@ */ public int numDeleteTerms() { // public for FlushPolicy - return pendingDeletes.numTermDeletes.get(); + return pendingUpdates.numTermDeletes.get(); } /** @@ -412,18 +412,18 @@ * {@link DocumentsWriterDeleteQueue}s global buffer and apply all pending * deletes to this DWPT. */ - FrozenBufferedDeletes prepareFlush() { + FrozenBufferedUpdates prepareFlush() { assert numDocsInRAM > 0; - final FrozenBufferedDeletes globalDeletes = deleteQueue.freezeGlobalBuffer(deleteSlice); + final FrozenBufferedUpdates globalUpdates = deleteQueue.freezeGlobalBuffer(deleteSlice); /* deleteSlice can possibly be null if we have hit non-aborting exceptions during indexing and never succeeded adding a document. */ if (deleteSlice != null) { // apply all deletes before we flush and release the delete slice - deleteSlice.apply(pendingDeletes, numDocsInRAM); + deleteSlice.apply(pendingUpdates, numDocsInRAM); assert deleteSlice.isEmpty(); deleteSlice.reset(); } - return globalDeletes; + return globalUpdates; } /** Flush all pending docs to a new segment */ @@ -432,20 +432,20 @@ assert deleteSlice.isEmpty() : "all deletes must be applied in prepareFlush"; segmentInfo.setDocCount(numDocsInRAM); final SegmentWriteState flushState = new SegmentWriteState(infoStream, directory, segmentInfo, fieldInfos.finish(), - pendingDeletes, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed()))); + pendingUpdates, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed()))); final double startMBUsed = bytesUsed() / 1024. / 1024.; // Apply delete-by-docID now (delete-byDocID only // happens when an exception is hit processing that // doc, eg if analyzer has some problem w/ the text): - if (pendingDeletes.docIDs.size() > 0) { + if (pendingUpdates.docIDs.size() > 0) { flushState.liveDocs = codec.liveDocsFormat().newLiveDocs(numDocsInRAM); - for(int delDocID : pendingDeletes.docIDs) { + for(int delDocID : pendingUpdates.docIDs) { flushState.liveDocs.clear(delDocID); } - flushState.delCountOnFlush = pendingDeletes.docIDs.size(); - pendingDeletes.bytesUsed.addAndGet(-pendingDeletes.docIDs.size() * BufferedDeletes.BYTES_PER_DEL_DOCID); - pendingDeletes.docIDs.clear(); + flushState.delCountOnFlush = pendingUpdates.docIDs.size(); + pendingUpdates.bytesUsed.addAndGet(-pendingUpdates.docIDs.size() * BufferedUpdates.BYTES_PER_DEL_DOCID); + pendingUpdates.docIDs.clear(); } if (aborting) { @@ -463,10 +463,10 @@ try { consumer.flush(flushState); - pendingDeletes.terms.clear(); + pendingUpdates.terms.clear(); segmentInfo.setFiles(new HashSet(directory.getCreatedFiles())); - final SegmentInfoPerCommit segmentInfoPerCommit = new SegmentInfoPerCommit(segmentInfo, 0, -1L, -1L); + final SegmentCommitInfo segmentInfoPerCommit = new SegmentCommitInfo(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 " + @@ -479,12 +479,12 @@ infoStream.message("DWPT", "flushed codec=" + codec); } - final BufferedDeletes segmentDeletes; - if (pendingDeletes.queries.isEmpty() && pendingDeletes.numericUpdates.isEmpty()) { - pendingDeletes.clear(); + final BufferedUpdates segmentDeletes; + if (pendingUpdates.queries.isEmpty() && pendingUpdates.numericUpdates.isEmpty()) { + pendingUpdates.clear(); segmentDeletes = null; } else { - segmentDeletes = pendingDeletes; + segmentDeletes = pendingUpdates; } if (infoStream.isEnabled("DWPT")) { @@ -522,7 +522,7 @@ void sealFlushedSegment(FlushedSegment flushedSegment) throws IOException { assert flushedSegment != null; - SegmentInfoPerCommit newSegment = flushedSegment.segmentInfo; + SegmentCommitInfo newSegment = flushedSegment.segmentInfo; IndexWriter.setDiagnostics(newSegment.info, IndexWriter.SOURCE_FLUSH); @@ -564,7 +564,7 @@ // carry the changes; there's no reason to use // filesystem as intermediary here. - SegmentInfoPerCommit info = flushedSegment.segmentInfo; + SegmentCommitInfo info = flushedSegment.segmentInfo; Codec codec = info.info.getCodec(); codec.liveDocsFormat().writeLiveDocs(flushedSegment.liveDocs, directory, info, delCount, context); newSegment.setDelCount(delCount); @@ -588,7 +588,7 @@ } long bytesUsed() { - return bytesUsed.get() + pendingDeletes.bytesUsed.get(); + return bytesUsed.get() + pendingUpdates.bytesUsed.get(); } /* Initial chunks size of the shared byte[] blocks used to @@ -626,7 +626,7 @@ @Override public String toString() { - return "DocumentsWriterPerThread [pendingDeletes=" + pendingDeletes + return "DocumentsWriterPerThread [pendingDeletes=" + pendingUpdates + ", segment=" + (segmentInfo != null ? segmentInfo.name : "null") + ", aborting=" + aborting + ", numDocsInRAM=" + numDocsInRAM + ", deleteQueue=" + deleteQueue + "]"; } Index: lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (working copy) @@ -34,8 +34,8 @@ private void applyDeletes(SegmentWriteState state, Fields fields) throws IOException { // Process any pending Term deletes for this newly // flushed segment: - if (state.segDeletes != null && state.segDeletes.terms.size() > 0) { - Map segDeletes = state.segDeletes.terms; + if (state.segUpdates != null && state.segUpdates.terms.size() > 0) { + Map segDeletes = state.segUpdates.terms; List deleteTerms = new ArrayList(segDeletes.keySet()); Collections.sort(deleteTerms); String lastField = null; Index: lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java (working copy) @@ -1,166 +0,0 @@ -package org.apache.lucene.index; - -/* - * 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. - */ - -import java.util.ArrayList; -import java.util.Iterator; -import java.util.LinkedHashMap; -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; - -/** - * 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 - 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 - final boolean isSegmentPrivate; // set to true iff this frozen packet represents - // a segment private deletes. in that case is should - // only have Queries - - - public FrozenBufferedDeletes(BufferedDeletes deletes, boolean isSegmentPrivate) { - this.isSegmentPrivate = isSegmentPrivate; - assert !isSegmentPrivate || deletes.terms.size() == 0 : "segment private package should only have del queries"; - Term termsArray[] = deletes.terms.keySet().toArray(new Term[deletes.terms.size()]); - termCount = termsArray.length; - ArrayUtil.timSort(termsArray); - PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder(); - for (Term term : termsArray) { - builder.add(term); - } - terms = builder.finish(); - - queries = new Query[deletes.queries.size()]; - queryLimits = new int[deletes.queries.size()]; - int upto = 0; - for(Map.Entry ent : deletes.queries.entrySet()) { - queries[upto] = ent.getKey(); - queryLimits[upto] = ent.getValue(); - upto++; - } - - // TODO if a Term affects multiple fields, we could keep the updates key'd by Term - // so that it maps to all fields it affects, sorted by their docUpto, and traverse - // that Term only once, applying the update to all fields that still need to be - // updated. - List allUpdates = new ArrayList(); - int numericUpdatesSize = 0; - for (LinkedHashMap 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(); - } - - public void setDelGen(long gen) { - assert this.gen == -1; - this.gen = gen; - } - - public long delGen() { - assert gen != -1; - return gen; - } - - public Iterable termsIterable() { - return new Iterable() { - @Override - public Iterator iterator() { - return terms.iterator(); - } - }; - } - - public Iterable queriesIterable() { - return new Iterable() { - @Override - public Iterator iterator() { - return new Iterator() { - private int upto; - - @Override - public boolean hasNext() { - return upto < queries.length; - } - - @Override - public QueryAndLimit next() { - QueryAndLimit ret = new QueryAndLimit(queries[upto], queryLimits[upto]); - upto++; - return ret; - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - }; - } - }; - } - - @Override - public String toString() { - String s = ""; - if (numTermDeletes != 0) { - s += " " + numTermDeletes + " deleted terms (unique count=" + termCount + ")"; - } - if (queries.length != 0) { - s += " " + queries.length + " deleted queries"; - } - if (bytesUsed != 0) { - s += " bytesUsed=" + bytesUsed; - } - - return s; - } - - boolean any() { - return termCount > 0 || queries.length > 0 || updates.length > 0; - } -} Index: lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (revision 1537832) +++ lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (working copy) @@ -23,7 +23,7 @@ import java.util.List; import java.util.Map; -import org.apache.lucene.index.BufferedDeletesStream.QueryAndLimit; +import org.apache.lucene.index.BufferedUpdatesStream.QueryAndLimit; import org.apache.lucene.search.Query; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.RamUsageEstimator; @@ -34,7 +34,7 @@ * structure to hold them. We don't hold docIDs because these are applied on * flush. */ -class FrozenBufferedDeletes { // TODO (DVU_RENAME) FrozenBufferedUpdates? +class 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; @@ -58,7 +58,7 @@ // only have Queries - public FrozenBufferedDeletes(BufferedDeletes deletes, boolean isSegmentPrivate) { + public FrozenBufferedUpdates(BufferedUpdates deletes, boolean isSegmentPrivate) { this.isSegmentPrivate = isSegmentPrivate; assert !isSegmentPrivate || deletes.terms.size() == 0 : "segment private package should only have del queries"; Term termsArray[] = deletes.terms.keySet().toArray(new Term[deletes.terms.size()]); Index: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (working copy) @@ -220,7 +220,7 @@ private volatile long changeCount; // increments every time a change is completed private volatile long lastCommitChangeCount; // last changeCount that was committed - private List rollbackSegments; // list of segmentInfo we will fallback to if the commit fails + private List rollbackSegments; // list of segmentInfo we will fallback to if the commit fails volatile SegmentInfos pendingCommit; // set when a commit is pending (after prepareCommit() & before commit()) volatile long pendingCommitChangeCount; @@ -235,7 +235,7 @@ final IndexFileDeleter deleter; // used by forceMerge to note those needing merging - private Map segmentsToMerge = new HashMap(); + private Map segmentsToMerge = new HashMap(); private int mergeMaxNumSegments; private Lock writeLock; @@ -245,7 +245,7 @@ // Holds all SegmentInfo instances currently involved in // merges - private HashSet mergingSegments = new HashSet(); + private HashSet mergingSegments = new HashSet(); private MergePolicy mergePolicy; private final MergeScheduler mergeScheduler; @@ -259,7 +259,7 @@ final AtomicInteger flushDeletesCount = new AtomicInteger(); final ReaderPool readerPool = new ReaderPool(); - final BufferedDeletesStream bufferedDeletesStream; + final BufferedUpdatesStream bufferedUpdatesStream; // This is a "write once" variable (like the organic dye // on a DVD-R that may or may not be heated by a laser and @@ -421,10 +421,10 @@ class ReaderPool { - private final Map readerMap = new HashMap(); + private final Map readerMap = new HashMap(); // used only by asserts - public synchronized boolean infoIsLive(SegmentInfoPerCommit info) { + public synchronized boolean infoIsLive(SegmentCommitInfo info) { int idx = segmentInfos.indexOf(info); assert idx != -1: "info=" + info + " isn't live"; assert segmentInfos.info(idx) == info: "info=" + info + " doesn't match live info in segmentInfos"; @@ -431,8 +431,8 @@ return true; } - public synchronized void drop(SegmentInfoPerCommit info) throws IOException { - final ReadersAndLiveDocs rld = readerMap.get(info); + public synchronized void drop(SegmentCommitInfo info) throws IOException { + final ReadersAndUpdates rld = readerMap.get(info); if (rld != null) { assert info == rld.info; // System.out.println("[" + Thread.currentThread().getName() + "] ReaderPool.drop: " + info); @@ -442,7 +442,7 @@ } public synchronized boolean anyPendingDeletes() { - for(ReadersAndLiveDocs rld : readerMap.values()) { + for(ReadersAndUpdates rld : readerMap.values()) { if (rld.getPendingDeleteCount() != 0) { return true; } @@ -451,11 +451,11 @@ return false; } - public synchronized void release(ReadersAndLiveDocs rld) throws IOException { + public synchronized void release(ReadersAndUpdates rld) throws IOException { release(rld, true); } - public synchronized void release(ReadersAndLiveDocs rld, boolean assertInfoLive) throws IOException { + public synchronized void release(ReadersAndUpdates rld, boolean assertInfoLive) throws IOException { // Matches incRef in get: rld.decRef(); @@ -491,9 +491,9 @@ * any pending changes. */ synchronized void dropAll(boolean doSave) throws IOException { Throwable priorE = null; - final Iterator> it = readerMap.entrySet().iterator(); + final Iterator> it = readerMap.entrySet().iterator(); while(it.hasNext()) { - final ReadersAndLiveDocs rld = it.next().getValue(); + final ReadersAndUpdates rld = it.next().getValue(); try { if (doSave && rld.writeLiveDocs(directory)) { @@ -547,8 +547,8 @@ * @throws IOException If there is a low-level I/O error */ public synchronized void commit(SegmentInfos infos) throws IOException { - for (SegmentInfoPerCommit info : infos) { - final ReadersAndLiveDocs rld = readerMap.get(info); + for (SegmentCommitInfo info : infos) { + final ReadersAndUpdates rld = readerMap.get(info); if (rld != null) { assert rld.info == info; if (rld.writeLiveDocs(directory)) { @@ -571,18 +571,18 @@ /** * Obtain a ReadersAndLiveDocs instance from the * readerPool. If create is true, you must later call - * {@link #release(ReadersAndLiveDocs)}. + * {@link #release(ReadersAndUpdates)}. */ - public synchronized ReadersAndLiveDocs get(SegmentInfoPerCommit info, boolean create) { + public synchronized ReadersAndUpdates get(SegmentCommitInfo info, boolean create) { assert info.info.dir == directory: "info.dir=" + info.info.dir + " vs " + directory; - ReadersAndLiveDocs rld = readerMap.get(info); + ReadersAndUpdates rld = readerMap.get(info); if (rld == null) { if (!create) { return null; } - rld = new ReadersAndLiveDocs(IndexWriter.this, info); + rld = new ReadersAndUpdates(IndexWriter.this, info); // Steal initial reference: readerMap.put(info, rld); } else { @@ -603,7 +603,7 @@ // pool: private boolean noDups() { Set seen = new HashSet(); - for(SegmentInfoPerCommit info : readerMap.keySet()) { + for(SegmentCommitInfo info : readerMap.keySet()) { assert !seen.contains(info.info.name); seen.add(info.info.name); } @@ -616,11 +616,11 @@ * If the reader isn't being pooled, the segmentInfo's * delCount is returned. */ - public int numDeletedDocs(SegmentInfoPerCommit info) { + public int numDeletedDocs(SegmentCommitInfo info) { ensureOpen(false); int delCount = info.getDelCount(); - final ReadersAndLiveDocs rld = readerPool.get(info, false); + final ReadersAndUpdates rld = readerPool.get(info, false); if (rld != null) { delCount += rld.getPendingDeleteCount(); } @@ -692,7 +692,7 @@ mergeScheduler = config.getMergeScheduler(); codec = config.getCodec(); - bufferedDeletesStream = new BufferedDeletesStream(infoStream); + bufferedUpdatesStream = new BufferedUpdatesStream(infoStream); poolReaders = config.getReaderPooling(); writeLock = directory.makeLock(WRITE_LOCK_NAME); @@ -811,7 +811,7 @@ private FieldNumbers getFieldNumberMap() throws IOException { final FieldNumbers map = new FieldNumbers(); - for(SegmentInfoPerCommit info : segmentInfos) { + for(SegmentCommitInfo info : segmentInfos) { for(FieldInfo fi : SegmentReader.readFieldInfos(info)) { map.addOrGet(fi.name, fi.number, fi.getDocValuesType()); } @@ -1093,7 +1093,7 @@ public synchronized int numDocs() { ensureOpen(); int count = docWriter.getNumDocs(); - for (final SegmentInfoPerCommit info : segmentInfos) { + for (final SegmentCommitInfo info : segmentInfos) { count += info.info.getDocCount() - numDeletedDocs(info); } return count; @@ -1104,7 +1104,7 @@ */ public synchronized boolean hasDeletions() { ensureOpen(); - if (bufferedDeletesStream.any()) { + if (bufferedUpdatesStream.any()) { return true; } if (docWriter.anyDeletions()) { @@ -1113,7 +1113,7 @@ if (readerPool.anyPendingDeletes()) { return true; } - for (final SegmentInfoPerCommit info : segmentInfos) { + for (final SegmentCommitInfo info : segmentInfos) { if (info.hasDeletions()) { return true; } @@ -1354,7 +1354,7 @@ throw new IllegalArgumentException("the reader must be a SegmentReader or composite reader containing only SegmentReaders"); } - final SegmentInfoPerCommit info = ((SegmentReader) reader).getSegmentInfo(); + final SegmentCommitInfo info = ((SegmentReader) reader).getSegmentInfo(); // TODO: this is a slow linear search, but, number of // segments should be contained unless something is @@ -1362,9 +1362,9 @@ // cost: if (segmentInfos.indexOf(info) != -1) { - ReadersAndLiveDocs rld = readerPool.get(info, false); + ReadersAndUpdates rld = readerPool.get(info, false); if (rld != null) { - synchronized(bufferedDeletesStream) { + synchronized(bufferedUpdatesStream) { rld.initWritableLiveDocs(); if (rld.delete(docID)) { final int fullDelCount = rld.info.getDelCount() + rld.getPendingDeleteCount(); @@ -1713,7 +1713,7 @@ synchronized(this) { resetMergeExceptions(); segmentsToMerge.clear(); - for(SegmentInfoPerCommit info : segmentInfos) { + for(SegmentCommitInfo info : segmentInfos) { segmentsToMerge.put(info, Boolean.TRUE); } mergeMaxNumSegments = maxNumSegments; @@ -1968,7 +1968,7 @@ * MergePolicy). * *

    Do not alter the returned collection! */ - public synchronized Collection getMergingSegments() { + public synchronized Collection getMergingSegments() { return mergingSegments; } @@ -2046,7 +2046,7 @@ mergePolicy.close(); mergeScheduler.close(); - bufferedDeletesStream.clear(); + bufferedUpdatesStream.clear(); processEvents(false, true); docWriter.close(); // mark it as closed first to prevent subsequent indexing actions/flushes docWriter.abort(this); // don't sync on IW here @@ -2281,10 +2281,10 @@ segmentInfos.changed(); } - synchronized void publishFrozenDeletes(FrozenBufferedDeletes packet) { + synchronized void publishFrozenUpdates(FrozenBufferedUpdates packet) { assert packet != null && packet.any(); - synchronized (bufferedDeletesStream) { - bufferedDeletesStream.push(packet); + synchronized (bufferedUpdatesStream) { + bufferedUpdatesStream.push(packet); } } @@ -2292,28 +2292,28 @@ * Atomically adds the segment private delete packet and publishes the flushed * segments SegmentInfo to the index writer. */ - void publishFlushedSegment(SegmentInfoPerCommit newSegment, - FrozenBufferedDeletes packet, FrozenBufferedDeletes globalPacket) throws IOException { + void publishFlushedSegment(SegmentCommitInfo newSegment, + FrozenBufferedUpdates packet, FrozenBufferedUpdates globalPacket) throws IOException { try { synchronized (this) { // Lock order IW -> BDS - synchronized (bufferedDeletesStream) { + synchronized (bufferedUpdatesStream) { if (infoStream.isEnabled("IW")) { infoStream.message("IW", "publishFlushedSegment"); } if (globalPacket != null && globalPacket.any()) { - bufferedDeletesStream.push(globalPacket); + bufferedUpdatesStream.push(globalPacket); } // Publishing the segment must be synched on IW -> BDS to make the sure // that no merge prunes away the seg. private delete packet final long nextGen; if (packet != null && packet.any()) { - nextGen = bufferedDeletesStream.push(packet); + nextGen = bufferedUpdatesStream.push(packet); } else { // Since we don't have a delete packet to apply we can get a new // generation right away - nextGen = bufferedDeletesStream.getNextGen(); + nextGen = bufferedUpdatesStream.getNextGen(); } if (infoStream.isEnabled("IW")) { infoStream.message("IW", "publish sets newSegment delGen=" + nextGen + " seg=" + segString(newSegment)); @@ -2401,7 +2401,7 @@ flush(false, true); - List infos = new ArrayList(); + List infos = new ArrayList(); boolean success = false; try { @@ -2412,7 +2412,7 @@ SegmentInfos sis = new SegmentInfos(); // read infos from dir sis.read(dir); - for (SegmentInfoPerCommit info : sis) { + for (SegmentCommitInfo info : sis) { assert !infos.contains(info): "dup info dir=" + info.info.dir + " name=" + info.info.name; String newSegName = newSegmentName(); @@ -2432,7 +2432,7 @@ success = true; } finally { if (!success) { - for(SegmentInfoPerCommit sipc : infos) { + for(SegmentCommitInfo sipc : infos) { for(String file : sipc.files()) { try { directory.deleteFile(file); @@ -2450,7 +2450,7 @@ success = true; } finally { if (!success) { - for(SegmentInfoPerCommit sipc : infos) { + for(SegmentCommitInfo sipc : infos) { for(String file : sipc.files()) { try { directory.deleteFile(file); @@ -2553,7 +2553,7 @@ } } - SegmentInfoPerCommit infoPerCommit = new SegmentInfoPerCommit(info, 0, -1L, -1L); + SegmentCommitInfo infoPerCommit = new SegmentCommitInfo(info, 0, -1L, -1L); info.setFiles(new HashSet(trackingDir.getCreatedFiles())); trackingDir.getCreatedFiles().clear(); @@ -2619,7 +2619,7 @@ } /** Copies the segment files as-is into the IndexWriter's directory. */ - private SegmentInfoPerCommit copySegmentAsIs(SegmentInfoPerCommit info, String segName, IOContext context) + private SegmentCommitInfo copySegmentAsIs(SegmentCommitInfo info, String segName, IOContext context) throws IOException { // note: we don't really need this fis (its copied), but we load it up @@ -2631,7 +2631,7 @@ SegmentInfo newInfo = new SegmentInfo(directory, info.info.getVersion(), segName, info.info.getDocCount(), info.info.getUseCompoundFile(), info.info.getCodec(), info.info.getDiagnostics()); - SegmentInfoPerCommit newInfoPerCommit = new SegmentInfoPerCommit(newInfo, info.getDelCount(), info.getDelGen(), info.getFieldInfosGen()); + SegmentCommitInfo newInfoPerCommit = new SegmentCommitInfo(newInfo, info.getDelCount(), info.getDelGen(), info.getFieldInfosGen()); Set segFiles = new HashSet(); @@ -2887,7 +2887,7 @@ * this method may return true right after you * had just called {@link #commit}. */ public final boolean hasUncommittedChanges() { - return changeCount != lastCommitChangeCount || docWriter.anyChanges() || bufferedDeletesStream.any(); + return changeCount != lastCommitChangeCount || docWriter.anyChanges() || bufferedUpdatesStream.any(); } private final void commitInternal() throws IOException { @@ -3040,16 +3040,16 @@ if (infoStream.isEnabled("IW")) { infoStream.message("IW", "apply all deletes during flush"); } - applyAllDeletes(); + applyAllDeletesAndUpdates(); } else if (infoStream.isEnabled("IW")) { - infoStream.message("IW", "don't apply deletes now delTermCount=" + bufferedDeletesStream.numTerms() + " bytesUsed=" + bufferedDeletesStream.bytesUsed()); + infoStream.message("IW", "don't apply deletes now delTermCount=" + bufferedUpdatesStream.numTerms() + " bytesUsed=" + bufferedUpdatesStream.bytesUsed()); } } - final synchronized void applyAllDeletes() throws IOException { + final synchronized void applyAllDeletesAndUpdates() throws IOException { flushDeletesCount.incrementAndGet(); - final BufferedDeletesStream.ApplyDeletesResult result; - result = bufferedDeletesStream.applyDeletes(readerPool, segmentInfos.asList()); + final BufferedUpdatesStream.ApplyDeletesResult result; + result = bufferedUpdatesStream.applyDeletesAndUpdates(readerPool, segmentInfos.asList()); if (result.anyDeletes) { checkpoint(); } @@ -3057,7 +3057,7 @@ if (infoStream.isEnabled("IW")) { infoStream.message("IW", "drop 100% deleted segments: " + segString(result.allDeleted)); } - for (SegmentInfoPerCommit info : result.allDeleted) { + for (SegmentCommitInfo info : result.allDeleted) { // If a merge has already registered for this // segment, we leave it in the readerPool; the // merge will skip merging it and will then drop @@ -3069,7 +3069,7 @@ } checkpoint(); } - bufferedDeletesStream.prune(segmentInfos); + bufferedUpdatesStream.prune(segmentInfos); } /** Expert: Return the total size of all index files currently cached in memory. @@ -3077,7 +3077,7 @@ */ public final long ramSizeInBytes() { ensureOpen(); - return docWriter.flushControl.netBytes() + bufferedDeletesStream.bytesUsed(); + return docWriter.flushControl.netBytes() + bufferedUpdatesStream.bytesUsed(); } // for testing only @@ -3095,7 +3095,7 @@ } private synchronized void ensureValidMerge(MergePolicy.OneMerge merge) { - for(SegmentInfoPerCommit info : merge.segments) { + for(SegmentCommitInfo info : merge.segments) { if (!segmentInfos.contains(info)) { throw new MergePolicy.MergeException("MergePolicy selected a segment (" + info.info.name + ") that is not in the current index " + segString(), directory); } @@ -3130,12 +3130,11 @@ * 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 { + synchronized private ReadersAndUpdates commitMergedDeletesAndUpdates(MergePolicy.OneMerge merge, MergeState mergeState) throws IOException { assert testPoint("startCommitMergeDeletes"); - final List sourceSegments = merge.segments; + final List sourceSegments = merge.segments; if (infoStream.isEnabled("IW")) { infoStream.message("IW", "commitMergeDeletes " + segString(merge.segments)); @@ -3147,17 +3146,17 @@ long minGen = Long.MAX_VALUE; // Lazy init (only when we find a delete to carry over): - ReadersAndLiveDocs mergedDeletes = null; // TODO (DVU_RENAME) to mergedDeletesAndUpdates + ReadersAndUpdates mergedDeletesAndUpdates = null; boolean initWritableLiveDocs = false; MergePolicy.DocMap docMap = null; final Map mergedFieldUpdates = new HashMap(); for (int i = 0; i < sourceSegments.size(); i++) { - SegmentInfoPerCommit info = sourceSegments.get(i); + SegmentCommitInfo 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 ReadersAndLiveDocs rld = readerPool.get(info, false); + final ReadersAndUpdates rld = readerPool.get(info, false); // We hold a ref so it should still be in the pool: assert rld != null: "seg=" + info.info.name; final Bits currentLiveDocs = rld.getLiveDocs(); @@ -3209,16 +3208,16 @@ assert !currentLiveDocs.get(j); } else { if (!currentLiveDocs.get(j)) { - if (mergedDeletes == null) { - mergedDeletes = readerPool.get(merge.info, true); - mergedDeletes.initWritableLiveDocs(); + if (mergedDeletesAndUpdates == null) { + mergedDeletesAndUpdates = readerPool.get(merge.info, true); + mergedDeletesAndUpdates.initWritableLiveDocs(); initWritableLiveDocs = true; docMap = getDocMap(merge, mergeState); } else if (!initWritableLiveDocs) { // mergedDeletes was initialized by field-updates changes - mergedDeletes.initWritableLiveDocs(); + mergedDeletesAndUpdates.initWritableLiveDocs(); initWritableLiveDocs = true; } - mergedDeletes.delete(docMap.map(docUpto)); + mergedDeletesAndUpdates.delete(docMap.map(docUpto)); if (mergingFields != null) { // advance all iters beyond the deleted document skipDeletedDoc(updatesIters, j); } @@ -3228,8 +3227,8 @@ for (int idx = 0; idx < mergingFields.length; idx++) { UpdatesIterator updatesIter = updatesIters[idx]; if (updatesIter.doc() == j) { // document has an update - if (mergedDeletes == null) { - mergedDeletes = readerPool.get(merge.info, true); + if (mergedDeletesAndUpdates == null) { + mergedDeletesAndUpdates = readerPool.get(merge.info, true); docMap = getDocMap(merge, mergeState); } if (newDoc == -1) { // map once per all field updates, but only if there are any updates @@ -3261,8 +3260,8 @@ for (int idx = 0; idx < mergingFields.length; idx++) { UpdatesIterator updatesIter = updatesIters[idx]; if (updatesIter.doc() == j) { // document has an update - if (mergedDeletes == null) { - mergedDeletes = readerPool.get(merge.info, true); + if (mergedDeletesAndUpdates == null) { + mergedDeletesAndUpdates = readerPool.get(merge.info, true); docMap = getDocMap(merge, mergeState); } if (newDoc == -1) { // map once per all field updates, but only if there are any updates @@ -3297,16 +3296,16 @@ // does: for (int j = 0; j < docCount; j++) { if (!currentLiveDocs.get(j)) { - if (mergedDeletes == null) { - mergedDeletes = readerPool.get(merge.info, true); - mergedDeletes.initWritableLiveDocs(); + if (mergedDeletesAndUpdates == null) { + mergedDeletesAndUpdates = readerPool.get(merge.info, true); + mergedDeletesAndUpdates.initWritableLiveDocs(); initWritableLiveDocs = true; docMap = getDocMap(merge, mergeState); } else if (!initWritableLiveDocs) { // mergedDeletes was initialized by field-updates changes - mergedDeletes.initWritableLiveDocs(); + mergedDeletesAndUpdates.initWritableLiveDocs(); initWritableLiveDocs = true; } - mergedDeletes.delete(docMap.map(docUpto)); + mergedDeletesAndUpdates.delete(docMap.map(docUpto)); if (mergingFields != null) { // advance all iters beyond the deleted document skipDeletedDoc(updatesIters, j); } @@ -3316,8 +3315,8 @@ for (int idx = 0; idx < mergingFields.length; idx++) { UpdatesIterator updatesIter = updatesIters[idx]; if (updatesIter.doc() == j) { // document has an update - if (mergedDeletes == null) { - mergedDeletes = readerPool.get(merge.info, true); + if (mergedDeletesAndUpdates == null) { + mergedDeletesAndUpdates = readerPool.get(merge.info, true); docMap = getDocMap(merge, mergeState); } if (newDoc == -1) { // map once per all field updates, but only if there are any updates @@ -3346,8 +3345,8 @@ for (int idx = 0; idx < mergingFields.length; idx++) { UpdatesIterator updatesIter = updatesIters[idx]; if (updatesIter.doc() == j) { // document has an update - if (mergedDeletes == null) { - mergedDeletes = readerPool.get(merge.info, true); + if (mergedDeletesAndUpdates == null) { + mergedDeletesAndUpdates = readerPool.get(merge.info, true); docMap = getDocMap(merge, mergeState); } if (newDoc == -1) { // map once per all field updates, but only if there are any updates @@ -3387,11 +3386,11 @@ // NOTE: currently this is the only place which throws a true // IOException. If this ever changes, we need to extend that try/finally // block to the rest of the method too. - mergedDeletes.writeFieldUpdates(directory, mergedFieldUpdates); + mergedDeletesAndUpdates.writeFieldUpdates(directory, mergedFieldUpdates); success = true; } finally { if (!success) { - mergedDeletes.dropChanges(); + mergedDeletesAndUpdates.dropChanges(); readerPool.drop(merge.info); } } @@ -3398,10 +3397,10 @@ } if (infoStream.isEnabled("IW")) { - if (mergedDeletes == null) { + if (mergedDeletesAndUpdates == null) { infoStream.message("IW", "no new deletes or field updates since merge started"); } else { - String msg = mergedDeletes.getPendingDeleteCount() + " new deletes"; + String msg = mergedDeletesAndUpdates.getPendingDeleteCount() + " new deletes"; if (!mergedFieldUpdates.isEmpty()) { msg += " and " + mergedFieldUpdates.size() + " new field updates"; } @@ -3412,7 +3411,7 @@ merge.info.setBufferedDeletesGen(minGen); - return mergedDeletes; + return mergedDeletesAndUpdates; } synchronized private boolean commitMerge(MergePolicy.OneMerge merge, MergeState mergeState) throws IOException { @@ -3443,7 +3442,7 @@ return false; } - final ReadersAndLiveDocs mergedDeletes = merge.info.info.getDocCount() == 0 ? null : commitMergedDeletes(merge, mergeState); + final ReadersAndUpdates mergedUpdates = merge.info.info.getDocCount() == 0 ? null : commitMergedDeletesAndUpdates(merge, mergeState); // System.out.println("[" + Thread.currentThread().getName() + "] IW.commitMerge: mergedDeletes=" + mergedDeletes); // If the doc store we are using has been closed and @@ -3455,8 +3454,8 @@ final boolean allDeleted = merge.segments.size() == 0 || merge.info.info.getDocCount() == 0 || - (mergedDeletes != null && - mergedDeletes.getPendingDeleteCount() == merge.info.info.getDocCount()); + (mergedUpdates != null && + mergedUpdates.getPendingDeleteCount() == merge.info.info.getDocCount()); if (infoStream.isEnabled("IW")) { if (allDeleted) { @@ -3472,20 +3471,20 @@ assert merge.info.info.getDocCount() != 0 || keepFullyDeletedSegments || dropSegment; - if (mergedDeletes != null) { + if (mergedUpdates != null) { boolean success = false; try { if (dropSegment) { - mergedDeletes.dropChanges(); + mergedUpdates.dropChanges(); } // Pass false for assertInfoLive because the merged // segment is not yet live (only below do we commit it // to the segmentInfos): - readerPool.release(mergedDeletes, false); + readerPool.release(mergedUpdates, false); success = true; } finally { if (!success) { - mergedDeletes.dropChanges(); + mergedUpdates.dropChanges(); readerPool.drop(merge.info); } } @@ -3652,7 +3651,7 @@ } boolean isExternal = false; - for(SegmentInfoPerCommit info : merge.segments) { + for(SegmentCommitInfo info : merge.segments) { if (mergingSegments.contains(info)) { if (infoStream.isEnabled("IW")) { infoStream.message("IW", "reject merge " + segString(merge.segments) + ": segment " + segString(info) + " is already marked for merge"); @@ -3690,7 +3689,7 @@ // threads, start if (infoStream.isEnabled("IW")) { StringBuilder builder = new StringBuilder("registerMerge merging= ["); - for (SegmentInfoPerCommit info : mergingSegments) { + for (SegmentCommitInfo info : mergingSegments) { builder.append(info.info.name).append(", "); } builder.append("]"); @@ -3700,7 +3699,7 @@ infoStream.message("IW", builder.toString()); } } - for(SegmentInfoPerCommit info : merge.segments) { + for(SegmentCommitInfo info : merge.segments) { if (infoStream.isEnabled("IW")) { infoStream.message("IW", "registerMerge info=" + segString(info)); } @@ -3709,7 +3708,7 @@ assert merge.estimatedMergeBytes == 0; assert merge.totalMergeBytes == 0; - for(SegmentInfoPerCommit info : merge.segments) { + for(SegmentCommitInfo info : merge.segments) { if (info.info.getDocCount() > 0) { final int delCount = numDeletedDocs(info); assert delCount <= info.info.getDocCount(); @@ -3768,7 +3767,7 @@ // could pre-pool them somehow in that case... // Lock order: IW -> BD - final BufferedDeletesStream.ApplyDeletesResult result = bufferedDeletesStream.applyDeletes(readerPool, merge.segments); + final BufferedUpdatesStream.ApplyDeletesResult result = bufferedUpdatesStream.applyDeletesAndUpdates(readerPool, merge.segments); if (result.anyDeletes) { checkpoint(); @@ -3778,7 +3777,7 @@ if (infoStream.isEnabled("IW")) { infoStream.message("IW", "drop 100% deleted segments: " + result.allDeleted); } - for(SegmentInfoPerCommit info : result.allDeleted) { + for(SegmentCommitInfo info : result.allDeleted) { segmentInfos.remove(info); if (merge.segments.contains(info)) { mergingSegments.remove(info); @@ -3798,12 +3797,12 @@ 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, -1L)); + merge.setInfo(new SegmentCommitInfo(si, 0, -1L, -1L)); // System.out.println("[" + Thread.currentThread().getName() + "] IW._mergeInit: " + segString(merge.segments) + " into " + si); // Lock order: IW -> BD - bufferedDeletesStream.prune(segmentInfos); + bufferedUpdatesStream.prune(segmentInfos); if (infoStream.isEnabled("IW")) { infoStream.message("IW", "merge seg=" + merge.info.info.name + " " + segString(merge.segments)); @@ -3841,8 +3840,8 @@ // It's possible we are called twice, eg if there was an // exception inside mergeInit if (merge.registerDone) { - final List sourceSegments = merge.segments; - for (SegmentInfoPerCommit info : sourceSegments) { + final List sourceSegments = merge.segments; + for (SegmentCommitInfo info : sourceSegments) { mergingSegments.remove(info); } merge.registerDone = false; @@ -3861,7 +3860,7 @@ final SegmentReader sr = merge.readers.get(i); if (sr != null) { try { - final ReadersAndLiveDocs rld = readerPool.get(sr.getSegmentInfo(), false); + final ReadersAndUpdates rld = readerPool.get(sr.getSegmentInfo(), false); // We still hold a ref so it should not have been removed: assert rld != null; if (drop) { @@ -3898,7 +3897,7 @@ final String mergedName = merge.info.info.name; - List sourceSegments = merge.segments; + List sourceSegments = merge.segments; IOContext context = new IOContext(merge.getMergeInfo()); @@ -3918,11 +3917,11 @@ int segUpto = 0; while(segUpto < sourceSegments.size()) { - final SegmentInfoPerCommit info = sourceSegments.get(segUpto); + final SegmentCommitInfo info = sourceSegments.get(segUpto); // Hold onto the "live" reader; we will use this to // commit merged deletes - final ReadersAndLiveDocs rld = readerPool.get(info, true); + final ReadersAndUpdates rld = readerPool.get(info, true); // Carefully pull the most recent live docs and reader SegmentReader reader; @@ -4121,7 +4120,7 @@ final IndexReaderWarmer mergedSegmentWarmer = config.getMergedSegmentWarmer(); if (poolReaders && mergedSegmentWarmer != null && merge.info.info.getDocCount() != 0) { - final ReadersAndLiveDocs rld = readerPool.get(merge.info, true); + final ReadersAndUpdates rld = readerPool.get(merge.info, true); final SegmentReader sr = rld.getReader(IOContext.READ); try { mergedSegmentWarmer.warm(sr); @@ -4171,7 +4170,7 @@ } // utility routines for tests - synchronized SegmentInfoPerCommit newestSegment() { + synchronized SegmentCommitInfo newestSegment() { return segmentInfos.size() > 0 ? segmentInfos.info(segmentInfos.size()-1) : null; } @@ -4187,9 +4186,9 @@ * segments, for debugging. * * @lucene.internal */ - public synchronized String segString(Iterable infos) { + public synchronized String segString(Iterable infos) { final StringBuilder buffer = new StringBuilder(); - for(final SegmentInfoPerCommit info : infos) { + for(final SegmentCommitInfo info : infos) { if (buffer.length() > 0) { buffer.append(' '); } @@ -4202,7 +4201,7 @@ * segment, for debugging. * * @lucene.internal */ - public synchronized String segString(SegmentInfoPerCommit info) { + public synchronized String segString(SegmentCommitInfo info) { return info.toString(info.info.dir, numDeletedDocs(info) - info.getDelCount()); } @@ -4252,12 +4251,12 @@ // For infoStream output synchronized SegmentInfos toLiveInfos(SegmentInfos sis) { final SegmentInfos newSIS = new SegmentInfos(); - final Map liveSIS = new HashMap(); - for(SegmentInfoPerCommit info : segmentInfos) { + final Map liveSIS = new HashMap(); + for(SegmentCommitInfo info : segmentInfos) { liveSIS.put(info, info); } - for(SegmentInfoPerCommit info : sis) { - SegmentInfoPerCommit liveInfo = liveSIS.get(info); + for(SegmentCommitInfo info : sis) { + SegmentCommitInfo liveInfo = liveSIS.get(info); if (liveInfo != null) { info = liveInfo; } @@ -4455,9 +4454,9 @@ //System.out.println("IW.nrtIsCurrent " + (infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedDeletesStream.any())); ensureOpen(); if (infoStream.isEnabled("IW")) { - infoStream.message("IW", "nrtIsCurrent: infoVersion matches: " + (infos.version == segmentInfos.version) + "; DW changes: " + docWriter.anyChanges() + "; BD changes: "+ bufferedDeletesStream.any()); + infoStream.message("IW", "nrtIsCurrent: infoVersion matches: " + (infos.version == segmentInfos.version) + "; DW changes: " + docWriter.anyChanges() + "; BD changes: "+ bufferedUpdatesStream.any()); } - return infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedDeletesStream.any(); + return infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedUpdatesStream.any(); } synchronized boolean isClosed() { @@ -4578,7 +4577,7 @@ try { purge(forcePurge); } finally { - applyAllDeletes(); + applyAllDeletesAndUpdates(); flushCount.incrementAndGet(); } } Index: lucene/core/src/java/org/apache/lucene/index/LogByteSizeMergePolicy.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/LogByteSizeMergePolicy.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/LogByteSizeMergePolicy.java (working copy) @@ -43,7 +43,7 @@ } @Override - protected long size(SegmentInfoPerCommit info) throws IOException { + protected long size(SegmentCommitInfo info) throws IOException { return sizeBytes(info); } Index: lucene/core/src/java/org/apache/lucene/index/LogDocMergePolicy.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/LogDocMergePolicy.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/LogDocMergePolicy.java (working copy) @@ -40,7 +40,7 @@ } @Override - protected long size(SegmentInfoPerCommit info) throws IOException { + protected long size(SegmentCommitInfo info) throws IOException { return sizeDocs(info); } Index: lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/LogMergePolicy.java (working copy) @@ -153,10 +153,10 @@ /** Return the number of documents in the provided {@link - * SegmentInfoPerCommit}, pro-rated by percentage of + * SegmentCommitInfo}, pro-rated by percentage of * non-deleted documents if {@link * #setCalibrateSizeByDeletes} is set. */ - protected long sizeDocs(SegmentInfoPerCommit info) throws IOException { + protected long sizeDocs(SegmentCommitInfo info) throws IOException { if (calibrateSizeByDeletes) { int delCount = writer.get().numDeletedDocs(info); assert delCount <= info.info.getDocCount(); @@ -167,10 +167,10 @@ } /** Return the byte size of the provided {@link - * SegmentInfoPerCommit}, pro-rated by percentage of + * SegmentCommitInfo}, pro-rated by percentage of * non-deleted documents if {@link * #setCalibrateSizeByDeletes} is set. */ - protected long sizeBytes(SegmentInfoPerCommit info) throws IOException { + protected long sizeBytes(SegmentCommitInfo info) throws IOException { if (calibrateSizeByDeletes) { return super.size(info); } @@ -180,13 +180,13 @@ /** Returns true if the number of segments eligible for * merging is less than or equal to the specified {@code * maxNumSegments}. */ - protected boolean isMerged(SegmentInfos infos, int maxNumSegments, Map segmentsToMerge) throws IOException { + protected boolean isMerged(SegmentInfos infos, int maxNumSegments, Map segmentsToMerge) throws IOException { final int numSegments = infos.size(); int numToMerge = 0; - SegmentInfoPerCommit mergeInfo = null; + SegmentCommitInfo mergeInfo = null; boolean segmentIsOriginal = false; for(int i=0;i segments = infos.asList(); + final List segments = infos.asList(); int start = last - 1; while (start >= 0) { - SegmentInfoPerCommit info = infos.info(start); + SegmentCommitInfo info = infos.info(start); if (size(info) > maxMergeSizeForForcedMerge || sizeDocs(info) > maxMergeDocs) { if (verbose()) { message("findForcedMergesSizeLimit: skip segment=" + info + ": size is > maxMergeSize (" + maxMergeSizeForForcedMerge + ") or sizeDocs is > maxMergeDocs (" + maxMergeDocs + ")"); @@ -251,7 +251,7 @@ */ private MergeSpecification findForcedMergesMaxNumSegments(SegmentInfos infos, int maxNumSegments, int last) throws IOException { MergeSpecification spec = new MergeSpecification(); - final List segments = infos.asList(); + final List segments = infos.asList(); // First, enroll all "full" merges (size // mergeFactor) to potentially be run concurrently: @@ -316,7 +316,7 @@ * in use may make use of concurrency. */ @Override public MergeSpecification findForcedMerges(SegmentInfos infos, - int maxNumSegments, Map segmentsToMerge) throws IOException { + int maxNumSegments, Map segmentsToMerge) throws IOException { assert maxNumSegments > 0; if (verbose()) { @@ -337,7 +337,7 @@ // since merging started): int last = infos.size(); while (last > 0) { - final SegmentInfoPerCommit info = infos.info(--last); + final SegmentCommitInfo info = infos.info(--last); if (segmentsToMerge.get(info) != null) { last++; break; @@ -362,7 +362,7 @@ // Check if there are any segments above the threshold boolean anyTooLarge = false; for (int i = 0; i < last; i++) { - SegmentInfoPerCommit info = infos.info(i); + SegmentCommitInfo info = infos.info(i); if (size(info) > maxMergeSizeForForcedMerge || sizeDocs(info) > maxMergeDocs) { anyTooLarge = true; break; @@ -384,7 +384,7 @@ @Override public MergeSpecification findForcedDeletesMerges(SegmentInfos segmentInfos) throws IOException { - final List segments = segmentInfos.asList(); + final List segments = segmentInfos.asList(); final int numSegments = segments.size(); if (verbose()) { @@ -396,7 +396,7 @@ IndexWriter w = writer.get(); assert w != null; for(int i=0;i 0) { if (verbose()) { @@ -436,11 +436,11 @@ } private static class SegmentInfoAndLevel implements Comparable { - SegmentInfoPerCommit info; + SegmentCommitInfo info; float level; int index; - public SegmentInfoAndLevel(SegmentInfoPerCommit info, float level, int index) { + public SegmentInfoAndLevel(SegmentCommitInfo info, float level, int index) { this.info = info; this.level = level; this.index = index; @@ -473,10 +473,10 @@ final List levels = new ArrayList(); final float norm = (float) Math.log(mergeFactor); - final Collection mergingSegments = writer.get().getMergingSegments(); + final Collection mergingSegments = writer.get().getMergingSegments(); for(int i=0;i= maxMergeSize || sizeDocs(info) >= maxMergeDocs); if (mergingSegments.contains(info)) { anyMerging = true; @@ -572,7 +572,7 @@ } else if (!anyTooLarge) { if (spec == null) spec = new MergeSpecification(); - final List mergeInfos = new ArrayList(); + final List mergeInfos = new ArrayList(); for(int i=start;i readers; // used by IndexWriter /** Segments to be merged. */ - public final List segments; + public final List segments; /** Number of documents in the merged segment. */ public final int totalDocCount; @@ -116,15 +116,15 @@ boolean paused; /** Sole constructor. - * @param segments List of {@link SegmentInfoPerCommit}s + * @param segments List of {@link SegmentCommitInfo}s * to be merged. */ - public OneMerge(List segments) { + public OneMerge(List segments) { if (0 == segments.size()) throw new RuntimeException("segments must include at least one segment"); // clone the list, as the in list may be based off original SegmentInfos and may be modified - this.segments = new ArrayList(segments); + this.segments = new ArrayList(segments); int count = 0; - for(SegmentInfoPerCommit info : segments) { + for(SegmentCommitInfo info : segments) { count += info.info.getDocCount(); } totalDocCount = count; @@ -150,10 +150,10 @@ } /** - * Expert: Sets the {@link SegmentInfoPerCommit} of this {@link OneMerge}. + * Expert: Sets the {@link SegmentCommitInfo} of this {@link OneMerge}. * Allows sub-classes to e.g. set diagnostics properties. */ - public void setInfo(SegmentInfoPerCommit info) { + public void setInfo(SegmentCommitInfo info) { this.info = info; } @@ -271,7 +271,7 @@ * */ public int totalNumDocs() throws IOException { int total = 0; - for (SegmentInfoPerCommit info : segments) { + for (SegmentCommitInfo info : segments) { total += info.info.getDocCount(); } return total; @@ -462,7 +462,7 @@ * produced by a cascaded merge. */ public abstract MergeSpecification findForcedMerges( - SegmentInfos segmentInfos, int maxSegmentCount, Map segmentsToMerge) + SegmentInfos segmentInfos, int maxSegmentCount, Map segmentsToMerge) throws IOException; /** @@ -488,7 +488,7 @@ * {@link #getMaxCFSSegmentSizeMB()} and the size is less or equal to the * TotalIndexSize * {@link #getNoCFSRatio()} otherwise false. */ - public boolean useCompoundFile(SegmentInfos infos, SegmentInfoPerCommit mergedInfo) throws IOException { + public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo) throws IOException { if (getNoCFSRatio() == 0.0) { return false; } @@ -500,7 +500,7 @@ return true; } long totalSize = 0; - for (SegmentInfoPerCommit info : infos) { + for (SegmentCommitInfo info : infos) { totalSize += size(info); } return mergedInfoSize <= getNoCFSRatio() * totalSize; @@ -507,9 +507,9 @@ } /** Return the byte size of the provided {@link - * SegmentInfoPerCommit}, pro-rated by percentage of + * SegmentCommitInfo}, pro-rated by percentage of * non-deleted documents is set. */ - protected long size(SegmentInfoPerCommit info) throws IOException { + protected long size(SegmentCommitInfo info) throws IOException { long byteSize = info.sizeInBytes(); int delCount = writer.get().numDeletedDocs(info); double delRatio = (info.info.getDocCount() <= 0 ? 0.0f : ((float)delCount / (float)info.info.getDocCount())); @@ -520,7 +520,7 @@ /** Returns true if this single info is already fully merged (has no * pending deletes, is in the same dir as the * writer, and matches the current compound file setting */ - protected final boolean isMerged(SegmentInfoPerCommit info) { + protected final boolean isMerged(SegmentCommitInfo info) { IndexWriter w = writer.get(); assert w != null; boolean hasDeletions = w.numDeletedDocs(info) > 0; Index: lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java (working copy) @@ -60,19 +60,19 @@ @Override public MergeSpecification findForcedMerges(SegmentInfos segmentInfos, - int maxSegmentCount, Map segmentsToMerge) { return null; } + int maxSegmentCount, Map segmentsToMerge) { return null; } @Override public MergeSpecification findForcedDeletesMerges(SegmentInfos segmentInfos) { return null; } @Override - public boolean useCompoundFile(SegmentInfos segments, SegmentInfoPerCommit newSegment) { return useCompoundFile; } + public boolean useCompoundFile(SegmentInfos segments, SegmentCommitInfo newSegment) { return useCompoundFile; } @Override public void setIndexWriter(IndexWriter writer) {} @Override - protected long size(SegmentInfoPerCommit info) throws IOException { + protected long size(SegmentCommitInfo info) throws IOException { return Long.MAX_VALUE; } Index: lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java (working copy) @@ -1,524 +0,0 @@ -package org.apache.lucene.index; - -/* - * 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. - */ - -import java.io.IOException; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; -import java.util.Map.Entry; -import java.util.NoSuchElementException; -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.lucene.codecs.Codec; -import org.apache.lucene.codecs.DocValuesConsumer; -import org.apache.lucene.codecs.DocValuesFormat; -import org.apache.lucene.codecs.LiveDocsFormat; -import org.apache.lucene.document.NumericDocValuesField; -import org.apache.lucene.index.NumericFieldUpdates.UpdatesIterator; -import org.apache.lucene.store.Directory; -import org.apache.lucene.store.IOContext; -import org.apache.lucene.store.TrackingDirectoryWrapper; -import org.apache.lucene.util.Bits; -import org.apache.lucene.util.IOUtils; -import org.apache.lucene.util.MutableBits; - -// Used by IndexWriter to hold open SegmentReaders (for -// searching or merging), plus pending deletes and updates, -// for a given segment -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; - - // Tracks how many consumers are using this instance: - private final AtomicInteger refCount = new AtomicInteger(1); - - private final IndexWriter writer; - - // 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 - // 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; - - // 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 liveDocsShared; - - // Indicates whether this segment is currently being merged. While a segment - // is merging, all field updates are also registered in the - // mergingNumericUpdates map. Also, calls to writeFieldUpdates merge the - // updates with mergingNumericUpdates. - // That way, when the segment is done merging, IndexWriter can apply the - // updates on the merged segment too. - private boolean isMerging = false; - - private final Map mergingNumericUpdates = new HashMap(); - - public ReadersAndLiveDocs(IndexWriter writer, SegmentInfoPerCommit info) { - this.info = info; - this.writer = writer; - liveDocsShared = true; - } - - public void incRef() { - final int rc = refCount.incrementAndGet(); - assert rc > 1; - } - - public void decRef() { - final int rc = refCount.decrementAndGet(); - assert rc >= 0; - } - - public int refCount() { - final int rc = refCount.get(); - assert rc >= 0; - return rc; - } - - public synchronized int getPendingDeleteCount() { - return pendingDeleteCount; - } - - // Call only from assert! - public synchronized boolean verifyDocCounts() { - int count; - if (liveDocs != null) { - count = 0; - for(int docID=0;docID e : numericFieldUpdates.entrySet()) { - NumericFieldUpdates fieldUpdates = mergingNumericUpdates.get(e.getKey()); - if (fieldUpdates == null) { - mergingNumericUpdates.put(e.getKey(), e.getValue()); - } else { - fieldUpdates.merge(e.getValue()); - } - } - } - - // create a new map, keeping only the gens that are in use - Map> genUpdatesFiles = info.getUpdatesFiles(); - Map> newGenUpdatesFiles = new HashMap>(); - final long fieldInfosGen = info.getFieldInfosGen(); - for (FieldInfo fi : fieldInfos) { - long dvGen = fi.getDocValuesGen(); - if (dvGen != -1 && !newGenUpdatesFiles.containsKey(dvGen)) { - if (dvGen == fieldInfosGen) { - newGenUpdatesFiles.put(fieldInfosGen, trackingDir.getCreatedFiles()); - } else { - newGenUpdatesFiles.put(dvGen, genUpdatesFiles.get(dvGen)); - } - } - } - - info.setGenUpdatesFiles(newGenUpdatesFiles); - - // wrote new files, should checkpoint() - writer.checkpoint(); - - // if there is a reader open, reopen it to reflect the updates - if (reader != null) { - SegmentReader newReader = new SegmentReader(info, reader, liveDocs, info.info.getDocCount() - info.getDelCount() - pendingDeleteCount); - boolean reopened = false; - try { - reader.decRef(); - reader = newReader; - reopened = true; - } finally { - if (!reopened) { - newReader.decRef(); - } - } - } - } - - /** - * Returns a reader for merge. This method applies field updates if there are - * any and marks that this segment is currently merging. - */ - synchronized SegmentReader getReaderForMerge(IOContext context) throws IOException { - assert Thread.holdsLock(writer); - // must execute these two statements as atomic operation, otherwise we - // could lose updates if e.g. another thread calls writeFieldUpdates in - // between, or the updates are applied to the obtained reader, but then - // re-applied in IW.commitMergedDeletes (unnecessary work and potential - // bugs). - isMerging = true; - return getReader(context); - } - - /** - * Drops all merging updates. Called from IndexWriter after this segment - * finished merging (whether successfully or not). - */ - public synchronized void dropMergingUpdates() { - mergingNumericUpdates.clear(); - isMerging = false; - } - - /** Returns updates that came in while this segment was merging. */ - public synchronized Map getMergingFieldUpdates() { - return mergingNumericUpdates; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("ReadersAndLiveDocs(seg=").append(info); - sb.append(" pendingDeleteCount=").append(pendingDeleteCount); - sb.append(" liveDocsShared=").append(liveDocsShared); - return sb.toString(); - } - -} Index: lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (revision 1537832) +++ lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (working copy) @@ -42,10 +42,10 @@ // Used by IndexWriter to hold open SegmentReaders (for // searching or merging), plus pending deletes and updates, // for a given segment -class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates +class ReadersAndUpdates { // Not final because we replace (clone) when we need to // change it and it's been shared: - public final SegmentInfoPerCommit info; + public final SegmentCommitInfo info; // Tracks how many consumers are using this instance: private final AtomicInteger refCount = new AtomicInteger(1); @@ -80,7 +80,7 @@ private final Map mergingNumericUpdates = new HashMap(); - public ReadersAndLiveDocs(IndexWriter writer, SegmentInfoPerCommit info) { + public ReadersAndUpdates(IndexWriter writer, SegmentCommitInfo info) { this.info = info; this.writer = writer; liveDocsShared = true; @@ -243,7 +243,6 @@ // 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 { assert Thread.holdsLock(writer); //System.out.println("rld.writeLiveDocs seg=" + info + " pendingDelCount=" + pendingDeleteCount + " numericUpdates=" + numericUpdates); Index: lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java (revision 1537832) +++ lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java (working copy) @@ -32,7 +32,7 @@ * fields. * * @lucene.experimental */ -public class SegmentInfoPerCommit { // TODO (DVU_RENAME) to SegmentCommitInfo +public class SegmentCommitInfo { /** The {@link SegmentInfo} that we wrap. */ public final SegmentInfo info; @@ -72,7 +72,7 @@ * @param fieldInfosGen * FieldInfos generation number (used to name field-infos files) **/ - public SegmentInfoPerCommit(SegmentInfo info, int delCount, long delGen, long fieldInfosGen) { + public SegmentCommitInfo(SegmentInfo info, int delCount, long delGen, long fieldInfosGen) { this.info = info; this.delCount = delCount; this.delGen = delGen; @@ -246,8 +246,8 @@ } @Override - public SegmentInfoPerCommit clone() { - SegmentInfoPerCommit other = new SegmentInfoPerCommit(info, delCount, delGen, fieldInfosGen); + public SegmentCommitInfo clone() { + SegmentCommitInfo other = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen); // Not clear that we need to carry over nextWriteDelGen // (i.e. do we ever clone after a failed write and // before the next successful write?), but just do it to Index: lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (working copy) @@ -85,7 +85,7 @@ private final Set coreClosedListeners = Collections.synchronizedSet(new LinkedHashSet()); - SegmentCoreReaders(SegmentReader owner, Directory dir, SegmentInfoPerCommit si, IOContext context) throws IOException { + SegmentCoreReaders(SegmentReader owner, Directory dir, SegmentCommitInfo si, IOContext context) throws IOException { final Codec codec = si.info.getCodec(); final Directory cfsDir; // confusing name: if (cfs) its the cfsdir, otherwise its the segment's directory. Index: lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java (working copy) @@ -37,7 +37,7 @@ private final Map> genDVProducers = new HashMap>(); - private RefCount newDocValuesProducer(SegmentInfoPerCommit si, IOContext context, Directory dir, + private RefCount newDocValuesProducer(SegmentCommitInfo si, IOContext context, Directory dir, DocValuesFormat dvFormat, final Long gen, List infos) throws IOException { Directory dvDir = dir; String segmentSuffix = ""; @@ -61,7 +61,7 @@ } /** Returns the {@link DocValuesProducer} for the given generation. */ - synchronized DocValuesProducer getDocValuesProducer(long gen, SegmentInfoPerCommit si, IOContext context, Directory dir, + synchronized DocValuesProducer getDocValuesProducer(long gen, SegmentCommitInfo si, IOContext context, Directory dir, DocValuesFormat dvFormat, List infos) throws IOException { RefCount dvp = genDVProducers.get(gen); if (dvp == null) { Index: lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java (working copy) @@ -1,265 +0,0 @@ -package org.apache.lucene.index; - -/* - * 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. - */ - -import java.io.IOException; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; - -import org.apache.lucene.store.Directory; - -/** Embeds a [read-only] SegmentInfo and adds per-commit - * fields. - * - * @lucene.experimental */ -public class SegmentInfoPerCommit { // TODO (DVU_RENAME) to SegmentCommitInfo - - /** The {@link SegmentInfo} that we wrap. */ - public final SegmentInfo info; - - // How many deleted docs in the segment: - private int delCount; - - // Generation number of the live docs file (-1 if there - // are no deletes yet): - private long delGen; - - // Normally 1+delGen, unless an exception was hit on last - // attempt to write: - private long nextWriteDelGen; - - // Generation number of the FieldInfos (-1 if there are no updates) - private long fieldInfosGen; - - // Normally 1 + fieldInfosGen, unless an exception was hit on last attempt to - // write - private long nextWriteFieldInfosGen; - - // Track the per-generation updates files - private final Map> genUpdatesFiles = new HashMap>(); - - 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) - * @param fieldInfosGen - * FieldInfos generation number (used to name field-infos files) - **/ - public SegmentInfoPerCommit(SegmentInfo info, int delCount, long delGen, long fieldInfosGen) { - this.info = info; - this.delCount = delCount; - this.delGen = delGen; - if (delGen == -1) { - nextWriteDelGen = 1; - } else { - nextWriteDelGen = delGen+1; - } - - this.fieldInfosGen = fieldInfosGen; - if (fieldInfosGen == -1) { - nextWriteFieldInfosGen = 1; - } else { - nextWriteFieldInfosGen = fieldInfosGen + 1; - } - } - - /** Returns the per generation updates files. */ - public Map> getUpdatesFiles() { - return Collections.unmodifiableMap(genUpdatesFiles); - } - - /** Sets the updates file names per generation. Does not deep clone the map. */ - public void setGenUpdatesFiles(Map> genUpdatesFiles) { - this.genUpdatesFiles.clear(); - this.genUpdatesFiles.putAll(genUpdatesFiles); - } - - /** Called when we succeed in writing deletes */ - void advanceDelGen() { - delGen = nextWriteDelGen; - nextWriteDelGen = delGen+1; - sizeInBytes = -1; - } - - /** Called if there was an exception while writing - * deletes, so that we don't try to write to the same - * file more than once. */ - void advanceNextWriteDelGen() { - nextWriteDelGen++; - } - - /** Called when we succeed in writing a new FieldInfos generation. */ - void advanceFieldInfosGen() { - fieldInfosGen = nextWriteFieldInfosGen; - nextWriteFieldInfosGen = fieldInfosGen + 1; - sizeInBytes = -1; - } - - /** - * Called if there was an exception while writing a new generation of - * FieldInfos, so that we don't try to write to the same file more than once. - */ - void advanceNextWriteFieldInfosGen() { - nextWriteFieldInfosGen++; - } - - /** Returns total size in bytes of all files for this - * segment. */ - public long sizeInBytes() throws IOException { - if (sizeInBytes == -1) { - long sum = 0; - for (final String fileName : files()) { - sum += info.dir.fileLength(fileName); - } - sizeInBytes = sum; - } - - return sizeInBytes; - } - - /** Returns all files in use by this segment. */ - public Collection files() throws IOException { - // Start from the wrapped info's files: - Collection files = new HashSet(info.files()); - - // TODO we could rely on TrackingDir.getCreatedFiles() (like we do for - // updates) and then maybe even be able to remove LiveDocsFormat.files(). - - // Must separately add any live docs files: - info.getCodec().liveDocsFormat().files(this, files); - - // Must separately add any field updates files - for (Set updateFiles : genUpdatesFiles.values()) { - files.addAll(updateFiles); - } - - return files; - } - - // NOTE: only used in-RAM by IW to track buffered deletes; - // this is never written to/read from the Directory - private long bufferedDeletesGen; - - long getBufferedDeletesGen() { - return bufferedDeletesGen; - } - - void setBufferedDeletesGen(long v) { - bufferedDeletesGen = v; - sizeInBytes = -1; - } - - /** Returns true if there are any deletions for the - * segment at this commit. */ - public boolean hasDeletions() { - return delGen != -1; - } - - /** Returns true if there are any field updates for the segment in this commit. */ - public boolean hasFieldUpdates() { - return fieldInfosGen != -1; - } - - /** Returns the next available generation number of the FieldInfos files. */ - public long getNextFieldInfosGen() { - return nextWriteFieldInfosGen; - } - - /** - * Returns the generation number of the field infos file or -1 if there are no - * field updates yet. - */ - public long getFieldInfosGen() { - return fieldInfosGen; - } - - /** - * Returns the next available generation number - * of the live docs file. - */ - public long getNextDelGen() { - return nextWriteDelGen; - } - - /** - * Returns generation number of the live docs file - * or -1 if there are no deletes yet. - */ - public long getDelGen() { - return delGen; - } - - /** - * Returns the number of deleted docs in the segment. - */ - public int getDelCount() { - return delCount; - } - - void setDelCount(int delCount) { - this.delCount = delCount; - assert delCount <= info.getDocCount(); - } - - /** Returns a description of this segment. */ - public String toString(Directory dir, int pendingDelCount) { - String s = info.toString(dir, delCount + pendingDelCount); - if (delGen != -1) { - s += ":delGen=" + delGen; - } - if (fieldInfosGen != -1) { - s += ":fieldInfosGen=" + fieldInfosGen; - } - return s; - } - - @Override - public String toString() { - return toString(info.dir, 0); - } - - @Override - public SegmentInfoPerCommit clone() { - SegmentInfoPerCommit other = new SegmentInfoPerCommit(info, delCount, delGen, fieldInfosGen); - // Not clear that we need to carry over nextWriteDelGen - // (i.e. do we ever clone after a failed write and - // before the next successful write?), but just do it to - // be safe: - other.nextWriteDelGen = nextWriteDelGen; - other.nextWriteFieldInfosGen = nextWriteFieldInfosGen; - - // deep clone - for (Entry> e : genUpdatesFiles.entrySet()) { - other.genUpdatesFiles.put(e.getKey(), new HashSet(e.getValue())); - } - - return other; - } -} Index: lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (working copy) @@ -115,7 +115,7 @@ * * @lucene.experimental */ -public final class SegmentInfos implements Cloneable, Iterable { +public final class SegmentInfos implements Cloneable, Iterable { /** The file format version for the segments_N codec header, up to 4.5. */ public static final int VERSION_40 = 0; @@ -141,7 +141,7 @@ /** Opaque Map<String, String> that user can specify during IndexWriter.commit */ public Map userData = Collections.emptyMap(); - private List segments = new ArrayList(); + private List segments = new ArrayList(); /** * If non-null, information about loading segments_N files @@ -152,14 +152,14 @@ /** Sole constructor. Typically you call this and then * use {@link #read(Directory) or * #read(Directory,String)} to populate each {@link - * SegmentInfoPerCommit}. Alternatively, you can add/remove your - * own {@link SegmentInfoPerCommit}s. */ + * SegmentCommitInfo}. Alternatively, you can add/remove your + * own {@link SegmentCommitInfo}s. */ public SegmentInfos() { } - /** Returns {@link SegmentInfoPerCommit} at the provided + /** Returns {@link SegmentCommitInfo} at the provided * index. */ - public SegmentInfoPerCommit info(int i) { + public SegmentCommitInfo info(int i) { return segments.get(i); } @@ -348,7 +348,7 @@ if (format >= VERSION_46) { fieldInfosGen = input.readLong(); } - SegmentInfoPerCommit siPerCommit = new SegmentInfoPerCommit(info, delCount, delGen, fieldInfosGen); + SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen); if (format >= VERSION_46) { int numGensUpdatesFiles = input.readInt(); final Map> genUpdatesFiles; @@ -386,7 +386,7 @@ } /** Find the latest commit ({@code segments_N file}) and - * load all {@link SegmentInfoPerCommit}s. */ + * load all {@link SegmentCommitInfo}s. */ public final void read(Directory directory) throws IOException { generation = lastGeneration = -1; @@ -424,7 +424,7 @@ segnOutput.writeLong(version); segnOutput.writeInt(counter); // write counter segnOutput.writeInt(size()); // write infos - for (SegmentInfoPerCommit siPerCommit : this) { + for (SegmentCommitInfo siPerCommit : this) { SegmentInfo si = siPerCommit.info; segnOutput.writeString(si.name); segnOutput.writeString(si.getCodec().getName()); @@ -471,8 +471,8 @@ try { final SegmentInfos sis = (SegmentInfos) super.clone(); // deep clone, first recreate all collections: - sis.segments = new ArrayList(size()); - for(final SegmentInfoPerCommit info : this) { + sis.segments = new ArrayList(size()); + for(final SegmentCommitInfo info : this) { assert info.info.getCodec() != null; // dont directly access segments, use add method!!! sis.add(info.clone()); @@ -841,7 +841,7 @@ } final int size = size(); for(int i=0;i 0) { buffer.append(' '); } - final SegmentInfoPerCommit info = info(i); + final SegmentCommitInfo info = info(i); buffer.append(info.toString(directory, 0)); } return buffer.toString(); @@ -964,7 +964,7 @@ * this does not include deletions */ public int totalDocCount() { int count = 0; - for(SegmentInfoPerCommit info : this) { + for(SegmentCommitInfo info : this) { count += info.info.getDocCount(); } return count; @@ -978,12 +978,12 @@ /** applies all changes caused by committing a merge to this SegmentInfos */ void applyMergeChanges(MergePolicy.OneMerge merge, boolean dropSegment) { - final Set mergedAway = new HashSet(merge.segments); + final Set mergedAway = new HashSet(merge.segments); boolean inserted = false; int newSegIdx = 0; for (int segIdx = 0, cnt = segments.size(); segIdx < cnt; segIdx++) { assert segIdx >= newSegIdx; - final SegmentInfoPerCommit info = segments.get(segIdx); + final SegmentCommitInfo info = segments.get(segIdx); if (mergedAway.contains(info)) { if (!inserted && !dropSegment) { segments.set(segIdx, merge.info); @@ -1009,9 +1009,9 @@ } } - List createBackupSegmentInfos() { - final List list = new ArrayList(size()); - for(final SegmentInfoPerCommit info : this) { + List createBackupSegmentInfos() { + final List list = new ArrayList(size()); + for(final SegmentCommitInfo info : this) { assert info.info.getCodec() != null; list.add(info.clone()); } @@ -1018,7 +1018,7 @@ return list; } - void rollbackSegmentInfos(List infos) { + void rollbackSegmentInfos(List infos) { this.clear(); this.addAll(infos); } @@ -1026,45 +1026,45 @@ /** Returns an unmodifiable {@link Iterator} of contained segments in order. */ // @Override (comment out until Java 6) @Override - public Iterator iterator() { + public Iterator iterator() { return asList().iterator(); } /** Returns all contained segments as an unmodifiable {@link List} view. */ - public List asList() { + public List asList() { return Collections.unmodifiableList(segments); } - /** Returns number of {@link SegmentInfoPerCommit}s. */ + /** Returns number of {@link SegmentCommitInfo}s. */ public int size() { return segments.size(); } - /** Appends the provided {@link SegmentInfoPerCommit}. */ - public void add(SegmentInfoPerCommit si) { + /** Appends the provided {@link SegmentCommitInfo}. */ + public void add(SegmentCommitInfo si) { segments.add(si); } - /** Appends the provided {@link SegmentInfoPerCommit}s. */ - public void addAll(Iterable sis) { - for (final SegmentInfoPerCommit si : sis) { + /** Appends the provided {@link SegmentCommitInfo}s. */ + public void addAll(Iterable sis) { + for (final SegmentCommitInfo si : sis) { this.add(si); } } - /** Clear all {@link SegmentInfoPerCommit}s. */ + /** Clear all {@link SegmentCommitInfo}s. */ public void clear() { segments.clear(); } - /** Remove the provided {@link SegmentInfoPerCommit}. + /** Remove the provided {@link SegmentCommitInfo}. * *

    WARNING: O(N) cost */ - public void remove(SegmentInfoPerCommit si) { + public void remove(SegmentCommitInfo si) { segments.remove(si); } - /** Remove the {@link SegmentInfoPerCommit} at the + /** Remove the {@link SegmentCommitInfo} at the * provided index. * *

    WARNING: O(N) cost */ @@ -1073,18 +1073,18 @@ } /** Return true if the provided {@link - * SegmentInfoPerCommit} is contained. + * SegmentCommitInfo} is contained. * *

    WARNING: O(N) cost */ - boolean contains(SegmentInfoPerCommit si) { + boolean contains(SegmentCommitInfo si) { return segments.contains(si); } /** Returns index of the provided {@link - * SegmentInfoPerCommit}. + * SegmentCommitInfo}. * *

    WARNING: O(N) cost */ - int indexOf(SegmentInfoPerCommit si) { + int indexOf(SegmentCommitInfo si) { return segments.indexOf(si); } } Index: lucene/core/src/java/org/apache/lucene/index/SegmentReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (working copy) @@ -46,7 +46,7 @@ */ public final class SegmentReader extends AtomicReader { - private final SegmentInfoPerCommit si; + private final SegmentCommitInfo si; private final Bits liveDocs; // Normally set to si.docCount - si.delDocCount, unless we @@ -83,7 +83,7 @@ * @throws IOException if there is a low-level IO error */ // TODO: why is this public? - public SegmentReader(SegmentInfoPerCommit si, IOContext context) throws IOException { + public SegmentReader(SegmentCommitInfo si, IOContext context) throws IOException { this.si = si; // TODO if the segment uses CFS, we may open the CFS file twice: once for // reading the FieldInfos (if they are not gen'd) and second time by @@ -127,7 +127,7 @@ /** 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, SegmentReader sr) throws IOException { + SegmentReader(SegmentCommitInfo si, SegmentReader sr) throws IOException { this(si, sr, si.info.getCodec().liveDocsFormat().readLiveDocs(si.info.dir, si, IOContext.READONCE), si.info.getDocCount() - si.getDelCount()); @@ -137,7 +137,7 @@ * SegmentReader and using the provided in-memory * liveDocs. Used by IndexWriter to provide a new NRT * reader */ - SegmentReader(SegmentInfoPerCommit si, SegmentReader sr, Bits liveDocs, int numDocs) throws IOException { + SegmentReader(SegmentCommitInfo si, SegmentReader sr, Bits liveDocs, int numDocs) throws IOException { this.si = si; this.liveDocs = liveDocs; this.numDocs = numDocs; @@ -193,7 +193,7 @@ * * @lucene.internal */ - static FieldInfos readFieldInfos(SegmentInfoPerCommit info) throws IOException { + static FieldInfos readFieldInfos(SegmentCommitInfo info) throws IOException { final Directory dir; final boolean closeDir; if (info.getFieldInfosGen() == -1 && info.info.getUseCompoundFile()) { @@ -335,7 +335,7 @@ /** * Return the SegmentInfoPerCommit of the segment this reader is reading. */ - public SegmentInfoPerCommit getSegmentInfo() { + public SegmentCommitInfo getSegmentInfo() { return si; } Index: lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java (working copy) @@ -48,12 +48,13 @@ * segment. */ public int delCountOnFlush; - /** Deletes to apply while we are flushing the segment. A - * Term is enrolled in here if it was deleted at one - * point, and it's mapped to the docIDUpto, meaning any - * docID < docIDUpto containing this term should be - * deleted. */ - public final BufferedDeletes segDeletes; + /** + * Deletes and updates to apply while we are flushing the segment. A Term is + * enrolled in here if it was deleted/updated at one point, and it's mapped to + * the docIDUpto, meaning any docID < docIDUpto containing this term should + * be deleted/updated. + */ + public final BufferedUpdates segUpdates; /** {@link MutableBits} recording live documents; this is * only set if there is one or more deleted documents. */ @@ -73,8 +74,8 @@ /** Sole constructor. */ public SegmentWriteState(InfoStream infoStream, Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos, - BufferedDeletes segDeletes, IOContext context) { - this(infoStream, directory, segmentInfo, fieldInfos, segDeletes, context, ""); + BufferedUpdates segUpdates, IOContext context) { + this(infoStream, directory, segmentInfo, fieldInfos, segUpdates, context, ""); } /** @@ -81,12 +82,12 @@ * Constructor which takes segment suffix. * * @see #SegmentWriteState(InfoStream, Directory, SegmentInfo, FieldInfos, - * BufferedDeletes, IOContext) + * BufferedUpdates, IOContext) */ public SegmentWriteState(InfoStream infoStream, Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos, - BufferedDeletes segDeletes, IOContext context, String segmentSuffix) { + BufferedUpdates segUpdates, IOContext context, String segmentSuffix) { this.infoStream = infoStream; - this.segDeletes = segDeletes; + this.segUpdates = segUpdates; this.directory = directory; this.segmentInfo = segmentInfo; this.fieldInfos = fieldInfos; @@ -102,7 +103,7 @@ fieldInfos = state.fieldInfos; context = state.context; this.segmentSuffix = segmentSuffix; - segDeletes = state.segDeletes; + segUpdates = state.segUpdates; delCountOnFlush = state.delCountOnFlush; } } Index: lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java (working copy) @@ -90,9 +90,9 @@ // segmentInfos here, so that we are passing the // actual instance of SegmentInfoPerCommit in // IndexWriter's segmentInfos: - final SegmentInfoPerCommit info = infos.info(i); + final SegmentCommitInfo info = infos.info(i); assert info.info.dir == dir; - final ReadersAndLiveDocs rld = writer.readerPool.get(info, true); + final ReadersAndUpdates rld = writer.readerPool.get(info, true); try { final SegmentReader reader = rld.getReadOnlyClone(IOContext.READ); if (reader.numDocs() > 0 || writer.getKeepFullyDeletedSegments()) { Index: lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java (working copy) @@ -232,9 +232,9 @@ return segsPerTier; } - private class SegmentByteSizeDescending implements Comparator { + private class SegmentByteSizeDescending implements Comparator { @Override - public int compare(SegmentInfoPerCommit o1, SegmentInfoPerCommit o2) { + public int compare(SegmentCommitInfo o1, SegmentCommitInfo o2) { try { final long sz1 = size(o1); final long sz2 = size(o2); @@ -272,16 +272,16 @@ if (infos.size() == 0) { return null; } - final Collection merging = writer.get().getMergingSegments(); - final Collection toBeMerged = new HashSet(); + final Collection merging = writer.get().getMergingSegments(); + final Collection toBeMerged = new HashSet(); - final List infosSorted = new ArrayList(infos.asList()); + final List infosSorted = new ArrayList(infos.asList()); Collections.sort(infosSorted, new SegmentByteSizeDescending()); // Compute total index bytes & print details about the index long totIndexBytes = 0; long minSegmentBytes = Long.MAX_VALUE; - for(SegmentInfoPerCommit info : infosSorted) { + for(SegmentCommitInfo info : infosSorted) { final long segBytes = size(info); if (verbose()) { String extra = merging.contains(info) ? " [merging]" : ""; @@ -334,9 +334,9 @@ // Gather eligible segments for merging, ie segments // not already being merged and not already picked (by // prior iteration of this loop) for merging: - final List eligible = new ArrayList(); + final List eligible = new ArrayList(); for(int idx = tooBigCount; idx best = null; + List best = null; boolean bestTooLarge = false; long bestMergeBytes = 0; @@ -367,10 +367,10 @@ long totAfterMergeBytes = 0; - final List candidate = new ArrayList(); + final List candidate = new ArrayList(); boolean hitTooLarge = false; for(int idx = startIdx;idx maxMergedSegmentBytes) { @@ -409,7 +409,7 @@ } final OneMerge merge = new OneMerge(best); spec.add(merge); - for(SegmentInfoPerCommit info : merge.segments) { + for(SegmentCommitInfo info : merge.segments) { toBeMerged.add(info); } @@ -426,11 +426,11 @@ } /** Expert: scores one merge; subclasses can override. */ - protected MergeScore score(List candidate, boolean hitTooLarge, long mergingBytes) throws IOException { + protected MergeScore score(List candidate, boolean hitTooLarge, long mergingBytes) throws IOException { long totBeforeMergeBytes = 0; long totAfterMergeBytes = 0; long totAfterMergeBytesFloored = 0; - for(SegmentInfoPerCommit info : candidate) { + for(SegmentCommitInfo info : candidate) { final long segBytes = size(info); totAfterMergeBytes += segBytes; totAfterMergeBytesFloored += floorSize(segBytes); @@ -482,16 +482,16 @@ } @Override - public MergeSpecification findForcedMerges(SegmentInfos infos, int maxSegmentCount, Map segmentsToMerge) throws IOException { + public MergeSpecification findForcedMerges(SegmentInfos infos, int maxSegmentCount, Map segmentsToMerge) throws IOException { if (verbose()) { message("findForcedMerges maxSegmentCount=" + maxSegmentCount + " infos=" + writer.get().segString(infos) + " segmentsToMerge=" + segmentsToMerge); } - List eligible = new ArrayList(); + List eligible = new ArrayList(); boolean forceMergeRunning = false; - final Collection merging = writer.get().getMergingSegments(); + final Collection merging = writer.get().getMergingSegments(); boolean segmentIsOriginal = false; - for(SegmentInfoPerCommit info : infos) { + for(SegmentCommitInfo info : infos) { final Boolean isOriginal = segmentsToMerge.get(info); if (isOriginal != null) { segmentIsOriginal = isOriginal; @@ -558,9 +558,9 @@ if (verbose()) { message("findForcedDeletesMerges infos=" + writer.get().segString(infos) + " forceMergeDeletesPctAllowed=" + forceMergeDeletesPctAllowed); } - final List eligible = new ArrayList(); - final Collection merging = writer.get().getMergingSegments(); - for(SegmentInfoPerCommit info : infos) { + final List eligible = new ArrayList(); + final Collection merging = writer.get().getMergingSegments(); + for(SegmentCommitInfo info : infos) { double pctDeletes = 100.*((double) writer.get().numDeletedDocs(info))/info.info.getDocCount(); if (pctDeletes > forceMergeDeletesPctAllowed && !merging.contains(info)) { eligible.add(info); Index: lucene/core/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java (revision 1537835) +++ lucene/core/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java (working copy) @@ -65,7 +65,7 @@ * so all segments created with a different version number than this Lucene version will * get upgraded. */ - protected boolean shouldUpgradeSegment(SegmentInfoPerCommit si) { + protected boolean shouldUpgradeSegment(SegmentCommitInfo si) { return !Constants.LUCENE_MAIN_VERSION.equals(si.info.getVersion()); } @@ -81,10 +81,10 @@ } @Override - public MergeSpecification findForcedMerges(SegmentInfos segmentInfos, int maxSegmentCount, Map segmentsToMerge) throws IOException { + public MergeSpecification findForcedMerges(SegmentInfos segmentInfos, int maxSegmentCount, Map segmentsToMerge) throws IOException { // first find all old segments - final Map oldSegments = new HashMap(); - for (final SegmentInfoPerCommit si : segmentInfos) { + final Map oldSegments = new HashMap(); + for (final SegmentCommitInfo si : segmentInfos) { final Boolean v = segmentsToMerge.get(si); if (v != null && shouldUpgradeSegment(si)) { oldSegments.put(si, v); @@ -114,8 +114,8 @@ message("findForcedMerges: " + base.getClass().getSimpleName() + " does not want to merge all old segments, merge remaining ones into new segment: " + oldSegments); } - final List newInfos = new ArrayList(); - for (final SegmentInfoPerCommit si : segmentInfos) { + final List newInfos = new ArrayList(); + for (final SegmentCommitInfo si : segmentInfos) { if (oldSegments.containsKey(si)) { newInfos.add(si); } @@ -136,7 +136,7 @@ } @Override - public boolean useCompoundFile(SegmentInfos segments, SegmentInfoPerCommit newSegment) throws IOException { + public boolean useCompoundFile(SegmentInfos segments, SegmentCommitInfo newSegment) throws IOException { return base.useCompoundFile(segments, newSegment); } Index: lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (revision 1537835) +++ lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (working copy) @@ -913,7 +913,7 @@ if (VERBOSE) { System.out.println("checkAllSegmentsUpgraded: " + infos); } - for (SegmentInfoPerCommit si : infos) { + for (SegmentCommitInfo si : infos) { assertEquals(Constants.LUCENE_MAIN_VERSION, si.info.getVersion()); } return infos.size(); Index: lucene/core/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java (revision 1537835) +++ lucene/core/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java (working copy) @@ -273,7 +273,7 @@ SegmentInfos sis = new SegmentInfos(); sis.read(dir); - for (SegmentInfoPerCommit si : sis) { + for (SegmentCommitInfo si : sis) { FieldInfos fis = SegmentReader.readFieldInfos(si); for (FieldInfo fi : fis) { Index: lucene/core/src/test/org/apache/lucene/index/TestDoc.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestDoc.java (revision 1537835) +++ lucene/core/src/test/org/apache/lucene/index/TestDoc.java (working copy) @@ -129,20 +129,20 @@ setMergePolicy(newLogMergePolicy(10)) ); - SegmentInfoPerCommit si1 = indexDoc(writer, "test.txt"); + SegmentCommitInfo si1 = indexDoc(writer, "test.txt"); printSegment(out, si1); - SegmentInfoPerCommit si2 = indexDoc(writer, "test2.txt"); + SegmentCommitInfo si2 = indexDoc(writer, "test2.txt"); printSegment(out, si2); writer.close(); - SegmentInfoPerCommit siMerge = merge(directory, si1, si2, "_merge", false); + SegmentCommitInfo siMerge = merge(directory, si1, si2, "_merge", false); printSegment(out, siMerge); - SegmentInfoPerCommit siMerge2 = merge(directory, si1, si2, "_merge2", false); + SegmentCommitInfo siMerge2 = merge(directory, si1, si2, "_merge2", false); printSegment(out, siMerge2); - SegmentInfoPerCommit siMerge3 = merge(directory, siMerge, siMerge2, "_merge3", false); + SegmentCommitInfo siMerge3 = merge(directory, siMerge, siMerge2, "_merge3", false); printSegment(out, siMerge3); directory.close(); @@ -195,7 +195,7 @@ assertEquals(multiFileOutput, singleFileOutput); } - private SegmentInfoPerCommit indexDoc(IndexWriter writer, String fileName) + private SegmentCommitInfo indexDoc(IndexWriter writer, String fileName) throws Exception { File file = new File(workDir, fileName); @@ -209,7 +209,7 @@ } - private SegmentInfoPerCommit merge(Directory dir, SegmentInfoPerCommit si1, SegmentInfoPerCommit si2, String merged, boolean useCompoundFile) + private SegmentCommitInfo merge(Directory dir, SegmentCommitInfo si1, SegmentCommitInfo si2, String merged, boolean useCompoundFile) throws Exception { IOContext context = newIOContext(random()); SegmentReader r1 = new SegmentReader(si1, context); @@ -239,11 +239,11 @@ } } - return new SegmentInfoPerCommit(info, 0, -1L, -1L); + return new SegmentCommitInfo(info, 0, -1L, -1L); } - private void printSegment(PrintWriter out, SegmentInfoPerCommit si) + private void printSegment(PrintWriter out, SegmentCommitInfo si) throws Exception { SegmentReader reader = new SegmentReader(si, newIOContext(random())); Index: lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java (revision 1537835) +++ lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java (working copy) @@ -62,7 +62,7 @@ IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()))); writer.addDocument(testDoc); writer.commit(); - SegmentInfoPerCommit info = writer.newestSegment(); + SegmentCommitInfo info = writer.newestSegment(); writer.close(); //After adding the document, we should be able to read it back in SegmentReader reader = new SegmentReader(info, newIOContext(random())); @@ -124,7 +124,7 @@ writer.addDocument(doc); writer.commit(); - SegmentInfoPerCommit info = writer.newestSegment(); + SegmentCommitInfo info = writer.newestSegment(); writer.close(); SegmentReader reader = new SegmentReader(info, newIOContext(random())); @@ -196,7 +196,7 @@ writer.addDocument(doc); writer.commit(); - SegmentInfoPerCommit info = writer.newestSegment(); + SegmentCommitInfo info = writer.newestSegment(); writer.close(); SegmentReader reader = new SegmentReader(info, newIOContext(random())); @@ -239,7 +239,7 @@ writer.addDocument(doc); writer.commit(); - SegmentInfoPerCommit info = writer.newestSegment(); + SegmentCommitInfo info = writer.newestSegment(); writer.close(); SegmentReader reader = new SegmentReader(info, newIOContext(random())); Index: lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java (revision 1537835) +++ lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java (working copy) @@ -43,8 +43,8 @@ } DeleteSlice slice1 = queue.newSlice(); DeleteSlice slice2 = queue.newSlice(); - BufferedDeletes bd1 = new BufferedDeletes(); - BufferedDeletes bd2 = new BufferedDeletes(); + BufferedUpdates bd1 = new BufferedUpdates(); + BufferedUpdates bd2 = new BufferedUpdates(); int last1 = 0; int last2 = 0; Set uniqueValues = new HashSet(); @@ -83,7 +83,7 @@ .numGlobalTermDeletes()); } - private void assertAllBetween(int start, int end, BufferedDeletes deletes, + private void assertAllBetween(int start, int end, BufferedUpdates deletes, Integer[] ids) { for (int i = start; i <= end; i++) { assertEquals(Integer.valueOf(end), deletes.terms.get(new Term("id", @@ -134,8 +134,7 @@ } assertTrue(queue.anyChanges()); if (random().nextInt(5) == 0) { - FrozenBufferedDeletes freezeGlobalBuffer = queue - .freezeGlobalBuffer(null); + FrozenBufferedUpdates freezeGlobalBuffer = queue.freezeGlobalBuffer(null); assertEquals(termsSinceFreeze, freezeGlobalBuffer.termCount); assertEquals(queriesSinceFreeze, freezeGlobalBuffer.queries.length); queriesSinceFreeze = 0; @@ -166,7 +165,7 @@ assertTrue("changes in del queue but not in slice yet", queue.anyChanges()); queue.tryApplyGlobalSlice(); assertTrue("changes in global buffer", queue.anyChanges()); - FrozenBufferedDeletes freezeGlobalBuffer = queue.freezeGlobalBuffer(null); + FrozenBufferedUpdates freezeGlobalBuffer = queue.freezeGlobalBuffer(null); assertTrue(freezeGlobalBuffer.any()); assertEquals(1, freezeGlobalBuffer.termCount); assertFalse("all changes applied", queue.anyChanges()); @@ -197,8 +196,8 @@ for (UpdateThread updateThread : threads) { DeleteSlice slice = updateThread.slice; queue.updateSlice(slice); - BufferedDeletes deletes = updateThread.deletes; - slice.apply(deletes, BufferedDeletes.MAX_INT); + BufferedUpdates deletes = updateThread.deletes; + slice.apply(deletes, BufferedUpdates.MAX_INT); assertEquals(uniqueValues, deletes.terms.keySet()); } queue.tryApplyGlobalSlice(); @@ -220,7 +219,7 @@ final AtomicInteger index; final Integer[] ids; final DeleteSlice slice; - final BufferedDeletes deletes; + final BufferedUpdates deletes; final CountDownLatch latch; protected UpdateThread(DocumentsWriterDeleteQueue queue, @@ -229,7 +228,7 @@ this.index = index; this.ids = ids; this.slice = queue.newSlice(); - deletes = new BufferedDeletes(); + deletes = new BufferedUpdates(); this.latch = latch; } @@ -245,7 +244,7 @@ Term term = new Term("id", ids[i].toString()); queue.add(term, slice); assertTrue(slice.isTailItem(term)); - slice.apply(deletes, BufferedDeletes.MAX_INT); + slice.apply(deletes, BufferedUpdates.MAX_INT); } } } Index: lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java (revision 1537835) +++ lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java (working copy) @@ -346,7 +346,7 @@ if (state.flushPending) { toFlush = state; } else if (flushOnDeleteTerms() - && state.dwpt.pendingDeletes.numTermDeletes.get() >= indexWriterConfig + && state.dwpt.pendingUpdates.numTermDeletes.get() >= indexWriterConfig .getMaxBufferedDeleteTerms()) { toFlush = state; } else { Index: lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java (revision 1537835) +++ lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java (working copy) @@ -74,12 +74,12 @@ // flushing without applying deletes means // there will still be deletes in the segment infos writer.flush(false, false); - assertTrue(writer.bufferedDeletesStream.any()); + assertTrue(writer.bufferedUpdatesStream.any()); // get reader flushes pending deletes // so there should not be anymore IndexReader r1 = writer.getReader(); - assertFalse(writer.bufferedDeletesStream.any()); + assertFalse(writer.bufferedUpdatesStream.any()); r1.close(); // delete id:2 from the first segment @@ -208,8 +208,8 @@ //System.out.println("segdels4:" + writer.docWriter.deletesToString()); } - boolean segThere(SegmentInfoPerCommit info, SegmentInfos infos) { - for (SegmentInfoPerCommit si : infos) { + boolean segThere(SegmentCommitInfo info, SegmentInfos infos) { + for (SegmentCommitInfo si : infos) { if (si.info.name.equals(info.info.name)) return true; } return false; @@ -272,7 +272,7 @@ @Override public MergeSpecification findForcedMerges(SegmentInfos segmentInfos, - int maxSegmentCount, Map segmentsToMerge) + int maxSegmentCount, Map segmentsToMerge) throws IOException { return null; } @@ -284,7 +284,7 @@ } @Override - public boolean useCompoundFile(SegmentInfos segments, SegmentInfoPerCommit newSegment) { + public boolean useCompoundFile(SegmentInfos segments, SegmentCommitInfo newSegment) { return useCompoundFile; } } Index: lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java (revision 1537835) +++ lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java (working copy) @@ -138,7 +138,7 @@ SegmentInfos infos = new SegmentInfos(); infos.read(dir); long totalBytes = 0; - for(SegmentInfoPerCommit sipc : infos) { + for(SegmentCommitInfo sipc : infos) { totalBytes += sipc.sizeInBytes(); } long totalBytes2 = 0; Index: lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java (revision 1537835) +++ lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java (working copy) @@ -51,9 +51,9 @@ merge1Dir = newDirectory(); merge2Dir = newDirectory(); DocHelper.setupDoc(doc1); - SegmentInfoPerCommit info1 = DocHelper.writeDoc(random(), merge1Dir, doc1); + SegmentCommitInfo info1 = DocHelper.writeDoc(random(), merge1Dir, doc1); DocHelper.setupDoc(doc2); - SegmentInfoPerCommit info2 = DocHelper.writeDoc(random(), merge2Dir, doc2); + SegmentCommitInfo info2 = DocHelper.writeDoc(random(), merge2Dir, doc2); reader1 = new SegmentReader(info1, newIOContext(random())); reader2 = new SegmentReader(info2, newIOContext(random())); } @@ -87,7 +87,7 @@ int docsMerged = mergeState.segmentInfo.getDocCount(); assertTrue(docsMerged == 2); //Should be able to open a new SegmentReader against the new directory - SegmentReader mergedReader = new SegmentReader(new SegmentInfoPerCommit( + SegmentReader mergedReader = new SegmentReader(new SegmentCommitInfo( new SegmentInfo(mergedDir, Constants.LUCENE_MAIN_VERSION, mergedSegment, docsMerged, false, codec, null), 0, -1L, -1L), Index: lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java (revision 1537835) +++ lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java (working copy) @@ -42,7 +42,7 @@ super.setUp(); dir = newDirectory(); DocHelper.setupDoc(testDoc); - SegmentInfoPerCommit info = DocHelper.writeDoc(random(), dir, testDoc); + SegmentCommitInfo info = DocHelper.writeDoc(random(), dir, testDoc); reader = new SegmentReader(info, IOContext.READ); } Index: lucene/core/src/test/org/apache/lucene/index/TestSegmentTermDocs.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestSegmentTermDocs.java (revision 1537835) +++ lucene/core/src/test/org/apache/lucene/index/TestSegmentTermDocs.java (working copy) @@ -31,7 +31,7 @@ public class TestSegmentTermDocs extends LuceneTestCase { private Document testDoc = new Document(); private Directory dir; - private SegmentInfoPerCommit info; + private SegmentCommitInfo info; @Override public void setUp() throws Exception { Index: lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java (revision 1537835) +++ lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java (working copy) @@ -45,7 +45,7 @@ private String[] testTerms = {"this", "is", "a", "test"}; private int[][] positions = new int[testTerms.length][]; private Directory dir; - private SegmentInfoPerCommit seg; + private SegmentCommitInfo seg; private FieldInfos fieldInfos = new FieldInfos(new FieldInfo[0]); private static int TERM_FREQ = 3; Index: lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java =================================================================== --- lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java (revision 1537835) +++ lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java (working copy) @@ -98,7 +98,7 @@ public void listSegments() throws IOException { DecimalFormat formatter = new DecimalFormat("###,###.###", DecimalFormatSymbols.getInstance(Locale.ROOT)); for (int x = 0; x < infos.size(); x++) { - SegmentInfoPerCommit info = infos.info(x); + SegmentCommitInfo info = infos.info(x); String sizeStr = formatter.format(info.sizeInBytes()); System.out.println(info.info.name + " " + sizeStr); } @@ -112,7 +112,7 @@ return -1; } - private SegmentInfoPerCommit getInfo(String name) { + private SegmentCommitInfo getInfo(String name) { for (int x = 0; x < infos.size(); x++) { if (name.equals(infos.info(x).info.name)) return infos.info(x); @@ -135,12 +135,12 @@ SegmentInfos destInfos = new SegmentInfos(); destInfos.counter = infos.counter; for (String n : segs) { - SegmentInfoPerCommit infoPerCommit = getInfo(n); + SegmentCommitInfo infoPerCommit = getInfo(n); SegmentInfo info = infoPerCommit.info; // Same info just changing the dir: SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.name, info.getDocCount(), info.getUseCompoundFile(), info.getCodec(), info.getDiagnostics()); - destInfos.add(new SegmentInfoPerCommit(newInfo, infoPerCommit.getDelCount(), infoPerCommit.getDelGen(), infoPerCommit.getFieldInfosGen())); + destInfos.add(new SegmentCommitInfo(newInfo, infoPerCommit.getDelCount(), infoPerCommit.getDelGen(), infoPerCommit.getFieldInfosGen())); // now copy files over Collection files = infoPerCommit.files(); for (final String srcName : files) { Index: lucene/misc/src/java/org/apache/lucene/index/sorter/SortingMergePolicy.java =================================================================== --- lucene/misc/src/java/org/apache/lucene/index/sorter/SortingMergePolicy.java (revision 1537835) +++ lucene/misc/src/java/org/apache/lucene/index/sorter/SortingMergePolicy.java (working copy) @@ -29,7 +29,7 @@ import org.apache.lucene.index.MergeState; import org.apache.lucene.index.MultiReader; import org.apache.lucene.index.SegmentInfo; -import org.apache.lucene.index.SegmentInfoPerCommit; +import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.SegmentReader; import org.apache.lucene.index.SlowCompositeReaderWrapper; @@ -64,7 +64,7 @@ Sorter.DocMap docMap; AtomicReader sortedView; - SortingOneMerge(List segments) { + SortingOneMerge(List segments) { super(segments); } @@ -87,7 +87,7 @@ } @Override - public void setInfo(SegmentInfoPerCommit info) { + public void setInfo(SegmentCommitInfo info) { Map diagnostics = info.info.getDiagnostics(); diagnostics.put(SORTER_ID_PROP, sorter.getID()); super.setInfo(info); @@ -187,7 +187,7 @@ @Override public MergeSpecification findForcedMerges(SegmentInfos segmentInfos, - int maxSegmentCount, Map segmentsToMerge) + int maxSegmentCount, Map segmentsToMerge) throws IOException { return sortedMergeSpecification(in.findForcedMerges(segmentInfos, maxSegmentCount, segmentsToMerge)); } @@ -210,7 +210,7 @@ @Override public boolean useCompoundFile(SegmentInfos segments, - SegmentInfoPerCommit newSegment) throws IOException { + SegmentCommitInfo newSegment) throws IOException { return in.useCompoundFile(segments, newSegment); } Index: lucene/test-framework/src/java/org/apache/lucene/index/AlcoholicMergePolicy.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/index/AlcoholicMergePolicy.java (revision 1537835) +++ lucene/test-framework/src/java/org/apache/lucene/index/AlcoholicMergePolicy.java (working copy) @@ -54,7 +54,7 @@ @Override //@BlackMagic(level=Voodoo); - protected long size(SegmentInfoPerCommit info) throws IOException { + protected long size(SegmentCommitInfo info) throws IOException { int hourOfDay = calendar.get(Calendar.HOUR_OF_DAY); if (hourOfDay < 6 || hourOfDay > 20 || Index: lucene/test-framework/src/java/org/apache/lucene/index/DocHelper.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/index/DocHelper.java (revision 1537835) +++ lucene/test-framework/src/java/org/apache/lucene/index/DocHelper.java (working copy) @@ -259,7 +259,7 @@ * named "test"; returns the SegmentInfo describing the new * segment */ - public static SegmentInfoPerCommit writeDoc(Random random, Directory dir, Document doc) throws IOException + public static SegmentCommitInfo writeDoc(Random random, Directory dir, Document doc) throws IOException { return writeDoc(random, dir, new MockAnalyzer(random, MockTokenizer.WHITESPACE, false), null, doc); } @@ -269,13 +269,13 @@ * and the similarity score; returns the SegmentInfo * describing the new segment */ - public static SegmentInfoPerCommit writeDoc(Random random, Directory dir, Analyzer analyzer, Similarity similarity, Document doc) throws IOException { + public static SegmentCommitInfo writeDoc(Random random, Directory dir, Analyzer analyzer, Similarity similarity, Document doc) throws IOException { IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig( /* LuceneTestCase.newIndexWriterConfig(random, */ TEST_VERSION_CURRENT, analyzer).setSimilarity(similarity == null ? IndexSearcher.getDefaultSimilarity() : similarity)); //writer.setNoCFSRatio(0.0); writer.addDocument(doc); writer.commit(); - SegmentInfoPerCommit info = writer.newestSegment(); + SegmentCommitInfo info = writer.newestSegment(); writer.close(); return info; } Index: lucene/test-framework/src/java/org/apache/lucene/index/MockRandomMergePolicy.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/index/MockRandomMergePolicy.java (revision 1537835) +++ lucene/test-framework/src/java/org/apache/lucene/index/MockRandomMergePolicy.java (working copy) @@ -47,10 +47,10 @@ int numSegments = segmentInfos.size(); - List segments = new ArrayList(); - final Collection merging = writer.get().getMergingSegments(); + List segments = new ArrayList(); + final Collection merging = writer.get().getMergingSegments(); - for(SegmentInfoPerCommit sipc : segmentInfos) { + for(SegmentCommitInfo sipc : segmentInfos) { if (!merging.contains(sipc)) { segments.add(sipc); } @@ -73,11 +73,11 @@ @Override public MergeSpecification findForcedMerges( - SegmentInfos segmentInfos, int maxSegmentCount, Map segmentsToMerge) + SegmentInfos segmentInfos, int maxSegmentCount, Map segmentsToMerge) throws IOException { - final List eligibleSegments = new ArrayList(); - for(SegmentInfoPerCommit info : segmentInfos) { + final List eligibleSegments = new ArrayList(); + for(SegmentCommitInfo info : segmentInfos) { if (segmentsToMerge.containsKey(info)) { eligibleSegments.add(info); } @@ -101,7 +101,7 @@ if (mergeSpec != null) { for(OneMerge merge : mergeSpec.merges) { - for(SegmentInfoPerCommit info : merge.segments) { + for(SegmentCommitInfo info : merge.segments) { assert segmentsToMerge.containsKey(info); } } @@ -119,7 +119,7 @@ } @Override - public boolean useCompoundFile(SegmentInfos infos, SegmentInfoPerCommit mergedInfo) throws IOException { + public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo) throws IOException { // 80% of the time we create CFS: return random.nextInt(5) != 1; } Index: lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java (revision 1537835) +++ lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java (working copy) @@ -77,7 +77,7 @@ import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.MergeScheduler; import org.apache.lucene.index.MultiFields; -import org.apache.lucene.index.SegmentInfoPerCommit; +import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentReader; import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; Index: solr/test-framework/src/java/org/apache/solr/util/RandomMergePolicy.java =================================================================== --- solr/test-framework/src/java/org/apache/solr/util/RandomMergePolicy.java (revision 1537835) +++ solr/test-framework/src/java/org/apache/solr/util/RandomMergePolicy.java (working copy) @@ -71,7 +71,7 @@ } public MergeSpecification findForcedMerges(SegmentInfos segmentInfos, int maxSegmentCount, - Map segmentsToMerge) + Map segmentsToMerge) throws IOException { return inner.findForcedMerges(segmentInfos, maxSegmentCount, segmentsToMerge); @@ -89,7 +89,7 @@ } public boolean useCompoundFile(SegmentInfos infos, - SegmentInfoPerCommit mergedInfo) + SegmentCommitInfo mergedInfo) throws IOException { return inner.useCompoundFile(infos, mergedInfo);