Index: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java (revision 1406075) +++ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java (working copy) @@ -20,6 +20,7 @@ import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.FieldInfosFormat; +import org.apache.lucene.codecs.GenerationReplacementsFormat; import org.apache.lucene.codecs.LiveDocsFormat; import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.PostingsFormat; @@ -44,6 +45,7 @@ // TODO: need a plain-text impl (using the above) private final NormsFormat normsFormat = new SimpleTextNormsFormat(); private final LiveDocsFormat liveDocs = new SimpleTextLiveDocsFormat(); + private final GenerationReplacementsFormat generationReplacements = new SimpleTextGenerationReplacementsFormat(); public SimpleTextCodec() { super("SimpleText"); @@ -88,4 +90,9 @@ public LiveDocsFormat liveDocsFormat() { return liveDocs; } + + @Override + public GenerationReplacementsFormat generationReplacementsFormat() { + return generationReplacements; + } } Index: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextGenerationReplacementsFormat.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextGenerationReplacementsFormat.java (revision 0) +++ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextGenerationReplacementsFormat.java (working copy) @@ -0,0 +1,86 @@ +package org.apache.lucene.codecs.simpletext; + +import java.io.IOException; +import java.util.Collection; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.GenerationReplacementsFormat; +import org.apache.lucene.index.FieldGenerationReplacements; +import org.apache.lucene.index.SegmentInfoPerCommit; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.StringHelper; + +/* + * 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. + */ + +public class SimpleTextGenerationReplacementsFormat extends + GenerationReplacementsFormat { + final static BytesRef FGR_DOCCOUNT = new BytesRef(" number of documents "); + final static BytesRef FGR_DOC = new BytesRef(" doc "); + final static BytesRef FGR_GENERATION = new BytesRef(" generation "); + + @Override + protected FieldGenerationReplacements readPersistedGeneration(IndexInput input) + throws IOException { + FieldGenerationReplacements reps = new FieldGenerationReplacements(); + + BytesRef scratch = new BytesRef(); + SimpleTextUtil.readLine(input, scratch); + assert StringHelper.startsWith(scratch, FGR_DOCCOUNT); + final int size = Integer.parseInt(readString(FGR_DOCCOUNT.length, scratch)); + + for (int i = 0; i < size; i++) { + SimpleTextUtil.readLine(input, scratch); + assert StringHelper.startsWith(scratch, FGR_DOC); + final int doc = Integer.parseInt(readString(FGR_DOC.length, scratch)); + + SimpleTextUtil.readLine(input, scratch); + assert StringHelper.startsWith(scratch, FGR_GENERATION); + final long generation = Integer.parseInt(readString(FGR_GENERATION.length, scratch)); + + reps.set(doc, generation); + } + + return reps; + } + + private String readString(int offset, BytesRef scratch) { + return new String(scratch.bytes, scratch.offset+offset, scratch.length-offset, IOUtils.CHARSET_UTF_8); + } + + @Override + protected void persistGeneration(FieldGenerationReplacements reps, + IndexOutput output) throws IOException { + BytesRef scratch = new BytesRef(); + SimpleTextUtil.write(output, FGR_DOCCOUNT); + SimpleTextUtil.write(output, Integer.toString(reps.size()), scratch); + SimpleTextUtil.writeNewline(output); + + for (Entry entry : reps){ + SimpleTextUtil.write(output, FGR_DOC); + SimpleTextUtil.write(output, Integer.toString(entry.getKey()), scratch); + SimpleTextUtil.writeNewline(output); + SimpleTextUtil.write(output, FGR_GENERATION); + SimpleTextUtil.write(output, Long.toString(entry.getValue()), scratch); + SimpleTextUtil.writeNewline(output); + } + } + +} Index: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java (revision 1406075) +++ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java (working copy) @@ -32,6 +32,7 @@ private final SegmentInfoWriter writer = new SimpleTextSegmentInfoWriter(); public static final String SI_EXTENSION = "si"; + public static final String SI_FILES_LIST_EXTENSION = "sif"; @Override public SegmentInfoReader getSegmentInfoReader() { Index: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoReader.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoReader.java (revision 1406075) +++ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoReader.java (working copy) @@ -25,6 +25,7 @@ import java.util.Set; import org.apache.lucene.codecs.SegmentInfoReader; +import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.store.Directory; @@ -95,21 +96,23 @@ attributes.put(key, value); } - SimpleTextUtil.readLine(input, scratch); - assert StringHelper.startsWith(scratch, SI_NUM_FILES); - int numFiles = Integer.parseInt(readString(SI_NUM_FILES.length, scratch)); - Set files = new HashSet(); + Set files = actualReadFiles(input, scratch); - for (int i = 0; i < numFiles; i++) { - SimpleTextUtil.readLine(input, scratch); - assert StringHelper.startsWith(scratch, SI_FILE); - String fileName = readString(SI_FILE.length, scratch); - files.add(fileName); - } - SegmentInfo info = new SegmentInfo(directory, version, segmentName, docCount, isCompoundFile, null, diagnostics, Collections.unmodifiableMap(attributes)); info.setFiles(files); + + int updatesIndex = 1; + while (updatesIndex > 0) { + files = readFilesList(directory, segmentName, updatesIndex, context); + if (files == null) { + updatesIndex = -1; + } else { + info.addFiles(files); + updatesIndex++; + } + } + success = true; return info; } finally { @@ -121,6 +124,46 @@ } } + @Override + public Set readFilesList(Directory dir, String segment, long generation, IOContext context) throws IOException { + final String segFileName = IndexFileNames.fileNameFromGeneration(segment, Lucene40SegmentInfoFormat.SI_FILES_LIST_EXTENSION, generation); + if (!dir.fileExists(segFileName)) { + return null; + } + + IndexInput input = dir.openInput(segFileName, context); + boolean success = false; + try { + BytesRef scratch = new BytesRef(); + Set files = actualReadFiles(input, scratch); + + success = true; + return files; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(input); + } else { + input.close(); + } + } + } + + public Set actualReadFiles(IndexInput input, BytesRef scratch) + throws IOException { + SimpleTextUtil.readLine(input, scratch); + assert StringHelper.startsWith(scratch, SI_NUM_FILES); + int numFiles = Integer.parseInt(readString(SI_NUM_FILES.length, scratch)); + Set files = new HashSet(); + + for (int i = 0; i < numFiles; i++) { + SimpleTextUtil.readLine(input, scratch); + assert StringHelper.startsWith(scratch, SI_FILE); + String fileName = readString(SI_FILE.length, scratch); + files.add(fileName); + } + return files; + } + private String readString(int offset, BytesRef scratch) { return new String(scratch.bytes, scratch.offset+offset, scratch.length-offset, IOUtils.CHARSET_UTF_8); } Index: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoWriter.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoWriter.java (revision 1406075) +++ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoWriter.java (working copy) @@ -22,6 +22,7 @@ import java.util.Set; import org.apache.lucene.codecs.SegmentInfoWriter; +import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat; import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.SegmentInfo; @@ -111,19 +112,32 @@ } } - Set files = si.files(); - int numFiles = files == null ? 0 : files.size(); - SimpleTextUtil.write(output, SI_NUM_FILES); - SimpleTextUtil.write(output, Integer.toString(numFiles), scratch); - SimpleTextUtil.writeNewline(output); + actualWriteFiles(si, output, scratch); + + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(output); + } else { + output.close(); + } + } + } + + @Override + public void writeFilesList(Directory dir, SegmentInfo si, long generation, IOContext ioContext) throws IOException { + final String segFileName = IndexFileNames.fileNameFromGeneration(si.name, + Lucene40SegmentInfoFormat.SI_FILES_LIST_EXTENSION, generation); + si.addFile(segFileName); - if (numFiles > 0) { - for(String fileName : files) { - SimpleTextUtil.write(output, SI_FILE); - SimpleTextUtil.write(output, fileName, scratch); - SimpleTextUtil.writeNewline(output); - } - } + boolean success = false; + IndexOutput output = dir.createOutput(segFileName, ioContext); + + try { + BytesRef scratch = new BytesRef(); + + actualWriteFiles(si, output, scratch); + success = true; } finally { if (!success) { @@ -137,4 +151,22 @@ } } } + + public void actualWriteFiles(SegmentInfo si, IndexOutput output, + BytesRef scratch) throws IOException { + Set files = si.files(); + int numFiles = files == null ? 0 : files.size(); + SimpleTextUtil.write(output, SI_NUM_FILES); + SimpleTextUtil.write(output, Integer.toString(numFiles), scratch); + SimpleTextUtil.writeNewline(output); + + if (numFiles > 0) { + for(String fileName : files) { + SimpleTextUtil.write(output, SI_FILE); + SimpleTextUtil.write(output, fileName, scratch); + SimpleTextUtil.writeNewline(output); + } + } + } + } Index: lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java =================================================================== --- lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java (revision 1406075) +++ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java (working copy) @@ -18,6 +18,7 @@ */ import java.io.IOException; +import java.util.Set; import org.apache.lucene.codecs.StoredFieldsReader; import org.apache.lucene.index.FieldInfo; @@ -91,7 +92,7 @@ } @Override - public void visitDocument(int n, StoredFieldVisitor visitor) throws IOException { + public void visitDocument(int n, StoredFieldVisitor visitor, Set ignoreFields) throws IOException { in.seek(offsets[n]); readLine(); assert StringHelper.startsWith(scratch, NUM); @@ -124,15 +125,20 @@ throw new RuntimeException("unknown field type"); } - switch (visitor.needsField(fieldInfo)) { - case YES: - readField(type, fieldInfo, visitor); - break; - case NO: - readLine(); - assert StringHelper.startsWith(scratch, VALUE); - break; - case STOP: return; + if (ignoreFields != null && ignoreFields.contains(fieldInfo.name)) { + readLine(); + } else { + switch (visitor.needsField(fieldInfo)) { + case YES: + readField(type, fieldInfo, visitor); + break; + case NO: + readLine(); + assert StringHelper.startsWith(scratch, VALUE); + break; + case STOP: + return; + } } } } Index: lucene/core/src/java/org/apache/lucene/codecs/Codec.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/Codec.java (revision 1408903) +++ lucene/core/src/java/org/apache/lucene/codecs/Codec.java (working copy) @@ -86,6 +86,9 @@ /** Encodes/decodes live docs */ public abstract LiveDocsFormat liveDocsFormat(); + /** Encodes/decodes live docs */ + public abstract GenerationReplacementsFormat generationReplacementsFormat(); + /** looks up a codec by name */ public static Codec forName(String name) { if (loader == null) { Index: lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java (revision 1408903) +++ lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java (working copy) @@ -75,6 +75,11 @@ } @Override + public GenerationReplacementsFormat generationReplacementsFormat() { + return delegate.generationReplacementsFormat(); + } + + @Override public NormsFormat normsFormat() { return delegate.normsFormat(); } Index: lucene/core/src/java/org/apache/lucene/codecs/GenerationReplacementsFormat.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/GenerationReplacementsFormat.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/codecs/GenerationReplacementsFormat.java (working copy) @@ -0,0 +1,165 @@ +package org.apache.lucene.codecs; + +/* + * 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 org.apache.lucene.index.FieldGenerationReplacements; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.SegmentInfoPerCommit; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.IOUtils; + +/** + * Format for field replacements of certain generation + * + * @lucene.experimental + */ +public abstract class GenerationReplacementsFormat { + + /** Extension of generation replacements vectors */ + static final String FIELD_GENERATION_REPLACEMENT_EXTENSION = "fgr"; + + /** + * Sole constructor. (For invocation by subclass constructors, typically + * implicit.) + */ + protected GenerationReplacementsFormat() {} + + /** + * Read field generation replacements. If no replacements exist return + * {@code null}. + */ + public FieldGenerationReplacements readGenerationReplacements(String field, + Directory dir, SegmentInfoPerCommit info, IOContext context) + throws IOException { + String fileName = getLastGenerationFileName(field, dir, info); + if (fileName == null) { + return null; + } + + return internalReadGeneration(dir, fileName, context); + } + + private FieldGenerationReplacements internalReadGeneration(Directory dir, + String fileName, IOContext context) throws IOException { + IndexInput input = dir.openInput(fileName, context); + + boolean success = false; + try { + return readPersistedGeneration(input); + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(input); + } else { + input.close(); + } + } + } + + private String getLastGenerationFileName(String field, Directory dir, + SegmentInfoPerCommit info) throws IOException { + for (long i = info.getUpdateGen(); i > 0; i--) { + final String nameWithField = IndexFileNames.segmentFileName( + info.info.name, field, ""); + final String fileName = IndexFileNames.fileNameFromGeneration( + nameWithField, FIELD_GENERATION_REPLACEMENT_EXTENSION, i); + if (dir.fileExists(fileName)) { + return fileName; + } + } + return null; + } + + /** + * Read persisted field generation replacements from a given input. + */ + protected abstract FieldGenerationReplacements readPersistedGeneration( + IndexInput input) throws IOException; + + /** + * Persist field generation replacements. Use + * {@link SegmentInfoPerCommit#getNextUpdateGen()} to determine the generation + * of the deletes file you should write to. + */ + public void writeGenerationReplacement(String field, + FieldGenerationReplacements reps, Directory dir, + SegmentInfoPerCommit info, IOContext context) throws IOException { + if (reps == null) { + // nothing new to write + return; + } + + String inputFileName = getLastGenerationFileName(field, dir, info); + final FieldGenerationReplacements existing; + if (inputFileName != null) { + existing = internalReadGeneration(dir, inputFileName, context); + existing.merge(reps); + } else { + existing = reps; + } + + final String nameWithGeneration = IndexFileNames.fileNameFromGeneration( + info.info.name, "", info.getNextDelGen()); + final String fileName = IndexFileNames.segmentFileName(nameWithGeneration, + field, FIELD_GENERATION_REPLACEMENT_EXTENSION); + + final IndexOutput output = dir.createOutput(fileName, context); + + persistGeneration(reps, output); + + boolean success = false; + try {} finally { + if (!success) { + IOUtils.closeWhileHandlingException(output); + info.info.dir.deleteFile(fileName); + } else { + output.close(); + if (inputFileName != null) { + info.info.removeFile(inputFileName); + } + info.info.addFile(fileName); + } + } + } + + /** + * Persist field generation replacements to a given output. + */ + protected abstract void persistGeneration(FieldGenerationReplacements reps, + IndexOutput output) throws IOException; + + /** + * Records all files in use by this {@link SegmentInfoPerCommit} into the + * files argument. + */ + public void files(SegmentInfoPerCommit info, Directory dir, + Collection files) throws IOException { + final String[] dirFiles = dir.listAll(); + for (int i = 0; i < dirFiles.length; i++) { + if (dirFiles[i].startsWith(info.info.name) + && dirFiles[i].endsWith(FIELD_GENERATION_REPLACEMENT_EXTENSION)) { + files.add(dirFiles[i]); + } + } + } +} Index: lucene/core/src/java/org/apache/lucene/codecs/SegmentInfoReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/SegmentInfoReader.java (revision 1408903) +++ lucene/core/src/java/org/apache/lucene/codecs/SegmentInfoReader.java (working copy) @@ -18,6 +18,7 @@ */ import java.io.IOException; +import java.util.Set; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.store.Directory; @@ -39,8 +40,22 @@ * Read {@link SegmentInfo} data from a directory. * @param directory directory to read from * @param segmentName name of the segment to read + * @param context IO context to use * @return infos instance to be populated with data * @throws IOException If an I/O error occurs */ public abstract SegmentInfo read(Directory directory, String segmentName, IOContext context) throws IOException; + + /** + * Read names of files that belong to a an updates segment of the segment with + * {@link SegmentInfo}, with the given updates generation. + * + * @param directory directory to read from + * @param segmentName name of the segment to read + * @param generation updates generation, or 0 for segment base + * @param context IO context to use + * @return list of file names + * @throws IOException If an I/O error occurs + */ + public abstract Set readFilesList(Directory directory, String segmentName, long generation, IOContext context) throws IOException; } Index: lucene/core/src/java/org/apache/lucene/codecs/SegmentInfoWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/SegmentInfoWriter.java (revision 1408903) +++ lucene/core/src/java/org/apache/lucene/codecs/SegmentInfoWriter.java (working copy) @@ -37,7 +37,25 @@ /** * Write {@link SegmentInfo} data. + + * @param dir directory to write to + * @param info the segment info to write + * @param fis field infos to use + * @param ioContext IO context to use * @throws IOException If an I/O error occurs */ public abstract void write(Directory dir, SegmentInfo info, FieldInfos fis, IOContext ioContext) throws IOException; + + /** + * Write the list of files belonging to an updates segment of the segment with + * {@link SegmentInfo}, with the given updates generation. + * + * @param dir directory to write to + * @param info info of the segment to write + * @param generation updates generation, or 0 for segment base + * @param ioContext IO context to use + * @throws IOException + * If an I/O error occurs + */ + public abstract void writeFilesList(Directory dir, SegmentInfo info, long generation, IOContext ioContext) throws IOException; } Index: lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsReader.java (revision 1408903) +++ lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsReader.java (working copy) @@ -18,13 +18,14 @@ import java.io.Closeable; import java.io.IOException; +import java.util.Set; import org.apache.lucene.index.StoredFieldVisitor; /** * Codec API for reading stored fields. *

- * You need to implement {@link #visitDocument(int, StoredFieldVisitor)} to + * You need to implement {@link #visitDocument(int, StoredFieldVisitor, Set)} to * read the stored fields for a document, implement {@link #clone()} (creating * clones of any IndexInputs used, etc), and {@link #close()} * @lucene.experimental @@ -35,8 +36,9 @@ protected StoredFieldsReader() { } - /** Visit the stored fields for document n */ - public abstract void visitDocument(int n, StoredFieldVisitor visitor) throws IOException; + /** Visit the stored fields for document n, ignoring certain + * fields. */ + public abstract void visitDocument(int n, StoredFieldVisitor visitor, Set ignoreFields) throws IOException; public abstract StoredFieldsReader clone(); } Index: lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java (revision 1410796) +++ lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java (working copy) @@ -36,6 +36,7 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Set; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.StoredFieldsReader; @@ -183,7 +184,7 @@ } @Override - public void visitDocument(int docID, StoredFieldVisitor visitor) + public void visitDocument(int docID, StoredFieldVisitor visitor, Set ignoreFields) throws IOException { fieldsStream.seek(indexReader.getStartPointer(docID)); Index: lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java (revision 1408903) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java (working copy) @@ -21,6 +21,7 @@ import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.FieldInfosFormat; import org.apache.lucene.codecs.FilterCodec; +import org.apache.lucene.codecs.GenerationReplacementsFormat; import org.apache.lucene.codecs.LiveDocsFormat; import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.PostingsFormat; @@ -50,6 +51,7 @@ private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat(); private final NormsFormat normsFormat = new Lucene40NormsFormat(); private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat(); + private final GenerationReplacementsFormat generationReplacementsFormat = new Lucene40GenerationReplacementsFormat(); private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() { @Override @@ -103,6 +105,11 @@ return liveDocsFormat; } + @Override + public final GenerationReplacementsFormat generationReplacementsFormat() { + return generationReplacementsFormat; + } + /** Returns the postings format that should be used for writing * new segments of field. * Index: lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40GenerationReplacementsFormat.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40GenerationReplacementsFormat.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40GenerationReplacementsFormat.java (working copy) @@ -0,0 +1,62 @@ +package org.apache.lucene.codecs.lucene40; + +import java.io.IOException; +import java.util.Collection; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.GenerationReplacementsFormat; +import org.apache.lucene.index.FieldGenerationReplacements; +import org.apache.lucene.index.SegmentInfoPerCommit; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; + +/* + * 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. + */ + +public class Lucene40GenerationReplacementsFormat extends + GenerationReplacementsFormat { + + @Override + protected FieldGenerationReplacements readPersistedGeneration(IndexInput input) + throws IOException { + final int size = input.readVInt(); + FieldGenerationReplacements reps = new FieldGenerationReplacements(); + int curr = 0; + for (int i = 0; i < size; i++) { + curr += input.readVInt(); + reps.set(curr, input.readVLong()); + } + return reps; + } + + @Override + protected void persistGeneration(FieldGenerationReplacements reps, + IndexOutput output) throws IOException { + // write number of replacements + output.writeVInt(reps.size()); + + // write replacements + int prev = 0; + for (Entry entry : reps){ + final Integer curr = entry.getKey(); + output.writeVInt(curr - prev); + prev = curr; + output.writeVLong(entry.getValue()); + } + } + +} Index: lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfoFormat.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfoFormat.java (revision 1408903) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfoFormat.java (working copy) @@ -88,6 +88,7 @@ /** File extension used to store {@link SegmentInfo}. */ public final static String SI_EXTENSION = "si"; + public final static String SI_FILES_LIST_EXTENSION = "sif"; static final String CODEC_NAME = "Lucene40SegmentInfo"; static final int VERSION_START = 0; static final int VERSION_CURRENT = VERSION_START; Index: lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfoReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfoReader.java (revision 1408903) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfoReader.java (working copy) @@ -61,7 +61,7 @@ final boolean isCompoundFile = input.readByte() == SegmentInfo.YES; final Map diagnostics = input.readStringStringMap(); final Map attributes = input.readStringStringMap(); - final Set files = input.readStringSet(); + Set files = input.readStringSet(); if (input.getFilePointer() != input.length()) { throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")"); @@ -71,6 +71,17 @@ null, diagnostics, Collections.unmodifiableMap(attributes)); si.setFiles(files); + int updatesIndex = 1; + while (updatesIndex > 0) { + files = readFilesList(dir, segment, updatesIndex, context); + if (files == null) { + updatesIndex = -1; + } else { + si.addFiles(files); + updatesIndex++; + } + } + success = true; return si; @@ -83,4 +94,33 @@ } } } + + @Override + public Set readFilesList(Directory dir, String segment, long generation, IOContext context) throws IOException { + final String fileName = IndexFileNames.fileNameFromGeneration(segment, Lucene40SegmentInfoFormat.SI_FILES_LIST_EXTENSION, generation); + if (!dir.fileExists(fileName)) { + return null; + } + + final IndexInput input = dir.openInput(fileName, context); + boolean success = false; + try { + final Set files = input.readStringSet(); + + if (input.getFilePointer() != input.length()) { + throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")"); + } + + success = true; + + return files; + + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(input); + } else { + input.close(); + } + } + } } Index: lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfoWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfoWriter.java (revision 1408903) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfoWriter.java (working copy) @@ -71,4 +71,28 @@ } } } + + @Override + public void writeFilesList(Directory dir, SegmentInfo si, + long generation, IOContext ioContext) throws IOException { + final String fileName = IndexFileNames.fileNameFromGeneration(si.name, + Lucene40SegmentInfoFormat.SI_FILES_LIST_EXTENSION, generation); + si.addFile(fileName); + + final IndexOutput output = dir.createOutput(fileName, ioContext); + + boolean success = false; + try { + output.writeStringSet(si.files()); + + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(output); + si.dir.deleteFile(fileName); + } else { + output.close(); + } + } + } } Index: lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java (revision 1408903) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java (working copy) @@ -34,6 +34,7 @@ import org.apache.lucene.util.IOUtils; import java.io.Closeable; +import java.util.Set; import static org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsWriter.*; @@ -141,7 +142,7 @@ } @Override - public final void visitDocument(int n, StoredFieldVisitor visitor) throws IOException { + public final void visitDocument(int n, StoredFieldVisitor visitor, Set ignoreFields) throws IOException { seekIndex(n); fieldsStream.seek(indexStream.readLong()); @@ -153,15 +154,19 @@ int bits = fieldsStream.readByte() & 0xFF; assert bits <= (FIELD_IS_NUMERIC_MASK | FIELD_IS_BINARY): "bits=" + Integer.toHexString(bits); - switch(visitor.needsField(fieldInfo)) { - case YES: - readField(visitor, fieldInfo, bits); - break; - case NO: - skipField(bits); - break; - case STOP: - return; + if (ignoreFields != null && ignoreFields.contains(fieldInfo.name)) { + skipField(bits); + } else { + switch (visitor.needsField(fieldInfo)) { + case YES: + readField(visitor, fieldInfo, bits); + break; + case NO: + skipField(bits); + break; + case STOP: + return; + } } } } Index: lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java =================================================================== --- lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java (revision 1408903) +++ lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java (working copy) @@ -21,6 +21,7 @@ import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.FieldInfosFormat; import org.apache.lucene.codecs.FilterCodec; +import org.apache.lucene.codecs.GenerationReplacementsFormat; import org.apache.lucene.codecs.LiveDocsFormat; import org.apache.lucene.codecs.NormsFormat; import org.apache.lucene.codecs.PostingsFormat; @@ -29,6 +30,7 @@ import org.apache.lucene.codecs.TermVectorsFormat; import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat; import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat; +import org.apache.lucene.codecs.lucene40.Lucene40GenerationReplacementsFormat; import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat; import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat; import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat; @@ -55,6 +57,7 @@ private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat(); private final NormsFormat normsFormat = new Lucene40NormsFormat(); private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat(); + private final GenerationReplacementsFormat generationReplacementsFormat = new Lucene40GenerationReplacementsFormat(); private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() { @Override @@ -108,6 +111,11 @@ return liveDocsFormat; } + @Override + public final GenerationReplacementsFormat generationReplacementsFormat() { + return generationReplacementsFormat; + } + /** Returns the postings format that should be used for writing * new segments of field. * Index: lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java (revision 1406075) +++ lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java (working copy) @@ -80,7 +80,7 @@ * since deletes are applied to the wrong segments. */ packet.setDelGen(nextGen++); - assert packet.any(); + assert packet.anyDeletes() || packet.anyUpdates(); assert checkDeleteStats(); assert packet.delGen() < nextGen; assert deletes.isEmpty() || deletes.get(deletes.size()-1).delGen() < packet.delGen() : "Delete packets must be in order"; @@ -187,7 +187,7 @@ final SegmentInfoPerCommit info = infos2.get(infosIDX); final long segGen = info.getBufferedDeletesGen(); - if (packet != null && segGen < packet.delGen()) { + if (packet != null && packet.anyDeletes() && segGen < packet.delGen()) { //System.out.println(" coalesce"); if (coalescedDeletes == null) { coalescedDeletes = new CoalescedDeletes(); @@ -204,7 +204,7 @@ } delIDX--; - } else if (packet != null && segGen == packet.delGen()) { + } else if (packet != null && packet.anyDeletes() && 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"); @@ -294,6 +294,25 @@ infosIDX--; } + + for (SegmentInfoPerCommit updateInfo : infos2) { + //System.out.println("BD: cycle delIDX=" + delIDX + " infoIDX=" + infosIDX); + final long updateSegGen = updateInfo.getBufferedDeletesGen(); + + for (FrozenBufferedDeletes updatePacket : deletes) { + if (updatePacket.anyUpdates() && updatePacket.delGen() <= updateSegGen) { + assert readerPool.infoIsLive(updateInfo); + final ReadersAndLiveDocs rld = readerPool.get(updateInfo, true); + final SegmentReader reader = rld.getReader(IOContext.READ); + try { + anyNewDeletes |= applyTermUpdates(updatePacket.updateTerms, updatePacket.updateArrays, rld, reader); + } finally { + rld.release(reader); + readerPool.release(rld); + } + } + } + } } assert checkDeleteStats(); @@ -427,6 +446,75 @@ return delCount; } + private synchronized boolean applyTermUpdates(PrefixCodedTerms updateTerms, + FieldsUpdate[][] updateArrays, ReadersAndLiveDocs rld, + SegmentReader reader) throws IOException { + Fields fields = reader.fields(); + if (fields == null) { + // This reader has no postings + return false; + } + + TermsEnum termsEnum = null; + + String currentField = null; + DocsEnum docs = null; + + UpdatedSegmentData updatedSegmentData = new UpdatedSegmentData(); + int termIndex = -1; + + // System.out.println(Thread.currentThread().getName() + + // " del terms reader=" + reader); + for (Term term : updateTerms) { + termIndex++; + // 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(null); + } else { + termsEnum = null; + } + } + + if (termsEnum == null) { + continue; + } + assert checkDeleteTerm(term); + + // System.out.println(" term=" + term); + + if (termsEnum.seekExact(term.bytes(), false)) { + // we don't need term frequencies for this + DocsEnum docsEnum = termsEnum.docs(rld.getLiveDocs(), docs, 0); + // 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; + } + updatedSegmentData.addUpdates(docID, updateArrays[termIndex]); + } + } + } + } + + if (updatedSegmentData.hasUpdates()) { + rld.setLiveUpdates(updatedSegmentData); + return true; + } + + return false; + } + public static class QueryAndLimit { public final Query query; public final int limit; Index: lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java (working copy) @@ -0,0 +1,132 @@ +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.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.lucene.util.RamUsageEstimator; + +/* Holds buffered updates by term for a + * single segment. This is used to hold buffered pending + * updates against the to-be-flushed segment. Once the + * updates are pushed (on flush in DocumentsWriter), these + * updates are converted to a FrozenUpdates instance. */ + +// NOTE: we are sync'd by BufferedUpdates, ie, all access to +// instances of this class is via sync'd methods on +// BufferedUpdates + +class 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; + + final AtomicInteger numTermUpdates = new AtomicInteger(); + final TreeMap> terms = new TreeMap>(); + + public static final Integer MAX_INT = Integer.valueOf(Integer.MAX_VALUE); + + final AtomicLong bytesUsed; + + private final static boolean VERBOSE_DELETES = false; + + long gen; + public BufferedUpdates() { + this(new AtomicLong()); + } + + BufferedUpdates(AtomicLong bytesUsed) { + assert bytesUsed != null; + this.bytesUsed = bytesUsed; + } + + @Override + public String toString() { + if (VERBOSE_DELETES) { + return "gen=" + gen + " numTerms=" + numTermUpdates + ", terms=" + terms + + ", bytesUsed=" + bytesUsed; + } else { + String s = "gen=" + gen; + if (numTermUpdates.get() != 0) { + s += " " + numTermUpdates.get() + " updated terms (unique count=" + terms.size() + ")"; + } + if (bytesUsed.get() != 0) { + s += " bytesUsed=" + bytesUsed.get(); + } + + return s; + } + } + + public void addTerm(Term term, FieldsUpdate update) { + SortedSet current = terms.get(term); + //if (current != null && update.docIDUpto < current.peek().docIDUpto) { + // 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; + //} + + if (current == null) { + current = new TreeSet(); + terms.put(term, current); + bytesUsed.addAndGet(BYTES_PER_DEL_TERM + term.bytes.length + (RamUsageEstimator.NUM_BYTES_CHAR * term.field().length())); + } + current.add(update); + numTermUpdates.incrementAndGet(); + } + + void clear() { + terms.clear(); + numTermUpdates.set(0); + bytesUsed.set(0); + } + + boolean any() { + return terms.size() > 0; + } +} Index: lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (revision 1406075) +++ lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (working copy) @@ -369,7 +369,7 @@ } if (perDocConsumer == null) { - PerDocWriteState perDocWriteState = docState.docWriter.newPerDocWriteState(""); + PerDocWriteState perDocWriteState = docState.docWriter.newPerDocWriteState(); perDocConsumer = docState.docWriter.codec.docValuesFormat().docsConsumer(perDocWriteState); if (perDocConsumer == null) { throw new IllegalStateException("codec=" + docState.docWriter.codec + " does not support docValues: from docValuesFormat().docsConsumer(...) returned null; field=" + fieldInfo.name); Index: lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (revision 1406075) +++ lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (working copy) @@ -20,10 +20,13 @@ import java.io.IOException; import java.util.Collection; import java.util.List; +import java.util.Map; +import java.util.Map.Entry; import java.util.concurrent.atomic.AtomicInteger; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.GenerationReplacementsFormat; import org.apache.lucene.index.DocumentsWriterFlushQueue.SegmentFlushTicket; import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment; import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain; @@ -39,83 +42,71 @@ import org.apache.lucene.util.MutableBits; /** - * This class accepts multiple added documents and directly - * writes segment files. - * - * Each added document is passed to the {@link DocConsumer}, - * which in turn processes the document and interacts with - * other consumers in the indexing chain. Certain - * consumers, like {@link StoredFieldsConsumer} and {@link - * TermVectorsConsumer}, digest a document and - * immediately write bytes to the "doc store" files (ie, - * they do not consume RAM per document, except while they - * are processing the document). - * - * Other consumers, eg {@link FreqProxTermsWriter} and - * {@link NormsConsumer}, buffer bytes in RAM and flush only - * when a new segment is produced. - - * Once we have used our allowed RAM buffer, or the number - * of added docs is large enough (in the case we are - * flushing by doc count instead of RAM usage), we create a - * real segment and flush it to the Directory. - * + * This class accepts multiple added documents and directly writes segment + * files. + * + * Each added document is passed to the {@link DocConsumer}, which in turn + * processes the document and interacts with other consumers in the indexing + * chain. Certain consumers, like {@link StoredFieldsConsumer} and + * {@link TermVectorsConsumer}, digest a document and immediately write bytes to + * the "doc store" files (ie, they do not consume RAM per document, except while + * they are processing the document). + * + * Other consumers, eg {@link FreqProxTermsWriter} and {@link NormsConsumer}, + * buffer bytes in RAM and flush only when a new segment is produced. + * + * Once we have used our allowed RAM buffer, or the number of added docs is + * large enough (in the case we are flushing by doc count instead of RAM usage), + * we create a real segment and flush it to the Directory. + * * Threads: - * - * Multiple threads are allowed into addDocument at once. - * There is an initial synchronized call to getThreadState - * which allocates a ThreadState for this thread. The same - * thread will get the same ThreadState over time (thread - * affinity) so that if there are consistent patterns (for - * example each thread is indexing a different content - * source) then we make better use of RAM. Then - * processDocument is called on that ThreadState without - * synchronization (most of the "heavy lifting" is in this - * call). Finally the synchronized "finishDocument" is - * called to flush changes to the directory. - * - * When flush is called by IndexWriter we forcefully idle - * all threads and flush only once they are all idle. This - * means you can call flush with a given thread even while - * other threads are actively adding/deleting documents. - * - * + * + * Multiple threads are allowed into addDocument at once. There is an initial + * synchronized call to getThreadState which allocates a ThreadState for this + * thread. The same thread will get the same ThreadState over time (thread + * affinity) so that if there are consistent patterns (for example each thread + * is indexing a different content source) then we make better use of RAM. Then + * processDocument is called on that ThreadState without synchronization (most + * of the "heavy lifting" is in this call). Finally the synchronized + * "finishDocument" is called to flush changes to the directory. + * + * When flush is called by IndexWriter we forcefully idle all threads and flush + * only once they are all idle. This means you can call flush with a given + * thread even while other threads are actively adding/deleting documents. + * + * * Exceptions: - * - * Because this class directly updates in-memory posting - * lists, and flushes stored fields and term vectors - * directly to files in the directory, there are certain - * limited times when an exception can corrupt this state. - * For example, a disk full while flushing stored fields - * leaves this file in a corrupt state. Or, an OOM - * exception while appending to the in-memory posting lists - * can corrupt that posting list. We call such exceptions - * "aborting exceptions". In these cases we must call - * abort() to discard all docs added since the last flush. - * - * All other exceptions ("non-aborting exceptions") can - * still partially update the index structures. These - * updates are consistent, but, they represent only a part - * of the document seen up until the exception was hit. - * When this happens, we immediately mark the document as - * deleted so that the document is always atomically ("all - * or none") added to the index. + * + * Because this class directly updates in-memory posting lists, and flushes + * stored fields and term vectors directly to files in the directory, there are + * certain limited times when an exception can corrupt this state. For example, + * a disk full while flushing stored fields leaves this file in a corrupt state. + * Or, an OOM exception while appending to the in-memory posting lists can + * corrupt that posting list. We call such exceptions "aborting exceptions". In + * these cases we must call abort() to discard all docs added since the last + * flush. + * + * All other exceptions ("non-aborting exceptions") can still partially update + * the index structures. These updates are consistent, but, they represent only + * a part of the document seen up until the exception was hit. When this + * happens, we immediately mark the document as deleted so that the document is + * always atomically ("all or none") added to the index. */ final class DocumentsWriter { Directory directory; - + private volatile boolean closed; - + final InfoStream infoStream; Similarity similarity; - + List newFiles; - + final IndexWriter indexWriter; - + private AtomicInteger numDocsInRAM = new AtomicInteger(0); - + // TODO: cut over to BytesRefHash in BufferedDeletes volatile DocumentsWriterDeleteQueue deleteQueue = new DocumentsWriterDeleteQueue(); private final DocumentsWriterFlushQueue ticketQueue = new DocumentsWriterFlushQueue(); @@ -126,17 +117,20 @@ * #anyChanges() & #flushAllThreads */ private volatile boolean pendingChangesInCurrentFullFlush; - - private Collection abortedFiles; // List of files that were written before last abort() - + + private Collection abortedFiles; // List of files that were written + // before last abort() + final IndexingChain chain; - + final DocumentsWriterPerThreadPool perThreadPool; final FlushPolicy flushPolicy; final DocumentsWriterFlushControl flushControl; final Codec codec; - DocumentsWriter(Codec codec, LiveIndexWriterConfig config, Directory directory, IndexWriter writer, FieldNumbers globalFieldNumbers, + + DocumentsWriter(Codec codec, LiveIndexWriterConfig config, + Directory directory, IndexWriter writer, FieldNumbers globalFieldNumbers, BufferedDeletesStream bufferedDeletesStream) { this.codec = codec; this.directory = directory; @@ -151,7 +145,7 @@ flushPolicy.init(this); flushControl = new DocumentsWriterFlushControl(this, config); } - + synchronized void deleteQueries(final Query... queries) throws IOException { deleteQueue.addDelete(queries); flushControl.doOnDelete(); @@ -159,7 +153,7 @@ applyAllDeletes(deleteQueue); } } - + // TODO: we could check w/ FreqProxTermsWriter: if the // term doesn't exist, don't bother buffering into the // per-DWPT map (but still must go into the global map) @@ -171,47 +165,49 @@ applyAllDeletes(deleteQueue); } } - + DocumentsWriterDeleteQueue currentDeleteSession() { return deleteQueue; } - private void applyAllDeletes(DocumentsWriterDeleteQueue deleteQueue) throws IOException { + private void applyAllDeletes(DocumentsWriterDeleteQueue deleteQueue) + throws IOException { if (deleteQueue != null && !flushControl.isFullFlush()) { ticketQueue.addDeletesAndPurge(this, deleteQueue); } indexWriter.applyAllDeletes(); indexWriter.flushCount.incrementAndGet(); } - + /** Returns how many docs are currently buffered in RAM. */ int getNumDocs() { return numDocsInRAM.get(); } - + Collection abortedFiles() { return abortedFiles; } - + private void ensureOpen() throws AlreadyClosedException { if (closed) { throw new AlreadyClosedException("this IndexWriter is closed"); } } - - /** Called if we hit an exception at a bad time (when - * updating the index files) and must discard all - * currently buffered docs. This resets our state, - * discarding any docs added since last flush. */ + + /** + * Called if we hit an exception at a bad time (when updating the index files) + * and must discard all currently buffered docs. This resets our state, + * discarding any docs added since last flush. + */ synchronized void abort() { boolean success = false; - + try { deleteQueue.clear(); if (infoStream.isEnabled("DW")) { infoStream.message("DW", "abort"); } - + final int limit = perThreadPool.getActiveThreadState(); for (int i = 0; i < limit; i++) { final ThreadState perThread = perThreadPool.getThreadState(i); @@ -236,53 +232,58 @@ success = true; } finally { if (infoStream.isEnabled("DW")) { - infoStream.message("DW", "done abort; abortedFiles=" + abortedFiles + " success=" + success); + infoStream.message("DW", "done abort; abortedFiles=" + abortedFiles + + " success=" + success); } } } - + boolean anyChanges() { if (infoStream.isEnabled("DW")) { - infoStream.message("DW", "anyChanges? numDocsInRam=" + numDocsInRAM.get() - + " deletes=" + anyDeletions() + " hasTickets:" - + ticketQueue.hasTickets() + " pendingChangesInFullFlush: " - + pendingChangesInCurrentFullFlush); + infoStream.message("DW", + "anyChanges? numDocsInRam=" + numDocsInRAM.get() + " deletes=" + + anyDeletions() + " hasTickets:" + ticketQueue.hasTickets() + + " pendingChangesInFullFlush: " + + pendingChangesInCurrentFullFlush); } /* - * changes are either in a DWPT or in the deleteQueue. - * yet if we currently flush deletes and / or dwpt there - * could be a window where all changes are in the ticket queue - * before they are published to the IW. ie we need to check if the - * ticket queue has any tickets. + * changes are either in a DWPT or in the deleteQueue. yet if we currently + * flush deletes and / or dwpt there could be a window where all changes are + * in the ticket queue before they are published to the IW. ie we need to + * check if the ticket queue has any tickets. */ - return numDocsInRAM.get() != 0 || anyDeletions() || ticketQueue.hasTickets() || pendingChangesInCurrentFullFlush; + return numDocsInRAM.get() != 0 || anyDeletions() + || ticketQueue.hasTickets() || pendingChangesInCurrentFullFlush; } public int getBufferedDeleteTermsSize() { return deleteQueue.getBufferedDeleteTermsSize(); } - - //for testing + + // for testing public int getNumBufferedDeleteTerms() { return deleteQueue.numGlobalTermDeletes(); } - + public boolean anyDeletions() { return deleteQueue.anyChanges(); } - + void close() { closed = true; flushControl.setClosed(); } - + private boolean preUpdate() throws IOException { ensureOpen(); boolean maybeMerge = false; if (flushControl.anyStalledThreads() || flushControl.numQueuedFlushes() > 0) { // Help out flushing any queued DWPTs so we can un-stall: if (infoStream.isEnabled("DW")) { - infoStream.message("DW", "DocumentsWriter has queued dwpt; will hijack this thread to flush pending segment(s)"); + infoStream + .message( + "DW", + "DocumentsWriter has queued dwpt; will hijack this thread to flush pending segment(s)"); } do { // Try pick up pending threads here if possible @@ -291,52 +292,58 @@ // Don't push the delete here since the update could fail! maybeMerge |= doFlush(flushingDWPT); } - + if (infoStream.isEnabled("DW")) { if (flushControl.anyStalledThreads()) { - infoStream.message("DW", "WARNING DocumentsWriter has stalled threads; waiting"); + infoStream.message("DW", + "WARNING DocumentsWriter has stalled threads; waiting"); } } flushControl.waitIfStalled(); // block if stalled - } while (flushControl.numQueuedFlushes() != 0); // still queued DWPTs try help flushing - + } while (flushControl.numQueuedFlushes() != 0); // still queued DWPTs try + // help flushing + if (infoStream.isEnabled("DW")) { - infoStream.message("DW", "continue indexing after helping out flushing DocumentsWriter is healthy"); + infoStream + .message("DW", + "continue indexing after helping out flushing DocumentsWriter is healthy"); } } return maybeMerge; } - - private boolean postUpdate(DocumentsWriterPerThread flushingDWPT, boolean maybeMerge) throws IOException { + + private boolean postUpdate(DocumentsWriterPerThread flushingDWPT, + boolean maybeMerge) throws IOException { if (flushControl.doApplyAllDeletes()) { applyAllDeletes(deleteQueue); } if (flushingDWPT != null) { maybeMerge |= doFlush(flushingDWPT); } else { - final DocumentsWriterPerThread nextPendingFlush = flushControl.nextPendingFlush(); + final DocumentsWriterPerThread nextPendingFlush = flushControl + .nextPendingFlush(); if (nextPendingFlush != null) { maybeMerge |= doFlush(nextPendingFlush); } } - + return maybeMerge; } - - boolean updateDocuments(final Iterable docs, final Analyzer analyzer, - final Term delTerm) throws IOException { + + boolean updateDocuments(final Iterable docs, + final Analyzer analyzer, final Term delTerm) throws IOException { boolean maybeMerge = preUpdate(); - + final ThreadState perThread = flushControl.obtainAndLock(); final DocumentsWriterPerThread flushingDWPT; try { if (!perThread.isActive()) { ensureOpen(); - assert false: "perThread is not active but we are still open"; + assert false : "perThread is not active but we are still open"; } - + final DocumentsWriterPerThread dwpt = perThread.dwpt; try { final int docCount = dwpt.updateDocuments(docs, analyzer, delTerm); @@ -351,29 +358,30 @@ } finally { perThread.unlock(); } - + return postUpdate(flushingDWPT, maybeMerge); } - + boolean updateDocument(final IndexDocument doc, final Analyzer analyzer, final Term delTerm) throws IOException { - + boolean maybeMerge = preUpdate(); - + final ThreadState perThread = flushControl.obtainAndLock(); - + final DocumentsWriterPerThread flushingDWPT; try { - + if (!perThread.isActive()) { ensureOpen(); - throw new IllegalStateException("perThread is not active but we are still open"); + throw new IllegalStateException( + "perThread is not active but we are still open"); } - + final DocumentsWriterPerThread dwpt = perThread.dwpt; try { - dwpt.updateDocument(doc, analyzer, delTerm); + dwpt.updateDocument(doc, analyzer, delTerm); numDocsInRAM.incrementAndGet(); } finally { if (dwpt.checkAndResetHasAborted()) { @@ -385,20 +393,118 @@ } finally { perThread.unlock(); } - + return postUpdate(flushingDWPT, maybeMerge); } - - private boolean doFlush(DocumentsWriterPerThread flushingDWPT) throws IOException { - boolean maybeMerge = false; + + boolean updateFields(Term term, FieldsUpdate fieldsUpdate) throws IOException { + boolean maybeMerge = preUpdate(); + + final ThreadState perThread = flushControl.obtainAndLock(); + + final DocumentsWriterPerThread flushingDWPT; + + try { + + if (!perThread.isActive()) { + ensureOpen(); + throw new IllegalStateException( + "perThread is not active but we are still open"); + } + + final DocumentsWriterPerThread dwpt = perThread.dwpt; + try { + dwpt.updateFields(term, fieldsUpdate); + } finally { + if (dwpt.checkAndResetHasAborted()) { + flushControl.doOnAbort(perThread); + } + } + final boolean isUpdate = term != null; + flushingDWPT = flushControl.doAfterDocument(perThread, isUpdate); + } finally { + perThread.unlock(); + } + return postUpdate(flushingDWPT, maybeMerge); + } + + void writeUpdatedSegment(UpdatedSegmentData liveUpdates, + SegmentInfoPerCommit info) throws IOException { + final ThreadState perThread = flushControl.obtainAndLock(); + + try { + + if (!perThread.isActive()) { + ensureOpen(); + throw new IllegalStateException( + "perThread is not active but we are still open"); + } + + final DocumentsWriterPerThread dwpt = perThread.dwpt; + try { + // start new segment, with update generation in name + dwpt.initSegmentInfo(info.info, + IndexFileNames.generationString(info.getNextUpdateGen())); + + // push documents, including empty ones where needed + liveUpdates.startWriting(info.getNextUpdateGen(), + info.info.getDocCount()); + IndexDocument doc; + while ((doc = liveUpdates.nextDocument()) != null) { + addEmptyDocs(dwpt, liveUpdates); + dwpt.updateDocument(doc, liveUpdates.getAnalyzer(), null); + } + // leftovers + addEmptyDocs(dwpt, liveUpdates); + + // add field generation replacements + final Map generationReplacments = liveUpdates + .getFieldGenerationReplacments(); + if (generationReplacments != null) { + for (Entry field : generationReplacments + .entrySet()) { + final GenerationReplacementsFormat repsFormat = codec + .generationReplacementsFormat(); + repsFormat.writeGenerationReplacement(field.getKey(), + field.getValue(), directory, info, IOContext.DEFAULT); + } + } + + // flush directly + dwpt.clearDeleteSlice(); + dwpt.flush(info.getNextUpdateGen()); + } finally { + if (dwpt.checkAndResetHasAborted()) { + flushControl.doOnAbort(perThread); + } + } + } finally { + perThread.unlock(); + } + } + + private void addEmptyDocs(final DocumentsWriterPerThread dwpt, + UpdatedSegmentData liveUpdates) throws IOException { + final int numMissingDocuments = liveUpdates.numMissingDocuments(); + for (int i = 0; i < numMissingDocuments; i++) { + dwpt.updateDocument(liveUpdates.emptyDocument(), + liveUpdates.getAnalyzer(), null); + } + } + + private boolean doFlush(DocumentsWriterPerThread flushingDWPT) + throws IOException { + int actualFlushes = 0; while (flushingDWPT != null) { - maybeMerge = true; + actualFlushes++; boolean success = false; SegmentFlushTicket ticket = null; try { assert currentFullFlushDelQueue == null || flushingDWPT.deleteQueue == currentFullFlushDelQueue : "expected: " - + currentFullFlushDelQueue + "but was: " + flushingDWPT.deleteQueue + + currentFullFlushDelQueue + + "but was: " + + flushingDWPT.deleteQueue + " " + flushControl.isFullFlush(); /* * Since with DWPT the flush process is concurrent and several DWPT @@ -415,18 +521,25 @@ * might miss to deletes documents in 'A'. */ try { - // Each flush is assigned a ticket in the order they acquire the ticketQueue lock + // Each flush is assigned a ticket in the order they acquire the + // ticketQueue lock ticket = ticketQueue.addFlushTicket(flushingDWPT); - + // flush concurrently without locking - final FlushedSegment newSegment = flushingDWPT.flush(); - ticketQueue.addSegment(ticket, newSegment); - // flush was successful once we reached this point - new seg. has been assigned to the ticket! - success = true; + final FlushedSegment newSegment = flushingDWPT.flush(-1); + if (newSegment == null) { + actualFlushes--; + } else { + ticketQueue.addSegment(ticket, newSegment); + // flush was successful once we reached this point - new seg. has + // been assigned to the ticket! + success = true; + } } finally { if (!success && ticket != null) { // In the case of a failure make sure we are making progress and - // apply all the deletes since the segment flush failed since the flush + // apply all the deletes since the segment flush failed since the + // flush // ticket could hold global deletes see FlushTicket#canPublish() ticketQueue.markTicketFailed(ticket); } @@ -435,59 +548,63 @@ * Now we are done and try to flush the ticket queue if the head of the * queue has already finished the flush. */ - if (ticketQueue.getTicketCount() >= perThreadPool.getActiveThreadState()) { + if (ticketQueue.getTicketCount() >= perThreadPool + .getActiveThreadState()) { // This means there is a backlog: the one // thread in innerPurge can't keep up with all - // other threads flushing segments. In this case + // other threads flushing segments. In this case // we forcefully stall the producers. ticketQueue.forcePurge(this); } else { ticketQueue.tryPurge(this); } - + } finally { flushControl.doAfterFlush(flushingDWPT); flushingDWPT.checkAndResetHasAborted(); indexWriter.flushCount.incrementAndGet(); indexWriter.doAfterFlush(); } - + flushingDWPT = flushControl.nextPendingFlush(); } - + // If deletes alone are consuming > 1/2 our RAM // buffer, force them all to apply now. This is to // prevent too-frequent flushing of a long tail of // tiny segments: final double ramBufferSizeMB = indexWriter.getConfig().getRAMBufferSizeMB(); - if (ramBufferSizeMB != IndexWriterConfig.DISABLE_AUTO_FLUSH && - flushControl.getDeleteBytesUsed() > (1024*1024*ramBufferSizeMB/2)) { + if (ramBufferSizeMB != IndexWriterConfig.DISABLE_AUTO_FLUSH + && flushControl.getDeleteBytesUsed() > (1024 * 1024 * ramBufferSizeMB / 2)) { if (infoStream.isEnabled("DW")) { - infoStream.message("DW", "force apply deletes bytesUsed=" + flushControl.getDeleteBytesUsed() + " vs ramBuffer=" + (1024*1024*ramBufferSizeMB)); + infoStream.message("DW", "force apply deletes bytesUsed=" + + flushControl.getDeleteBytesUsed() + " vs ramBuffer=" + + (1024 * 1024 * ramBufferSizeMB)); } applyAllDeletes(deleteQueue); } - - return maybeMerge; + + return actualFlushes > 0; } - - void finishFlush(FlushedSegment newSegment, FrozenBufferedDeletes bufferedDeletes) - throws IOException { + void finishFlush(FlushedSegment newSegment, + FrozenBufferedDeletes bufferedDeletes) throws IOException { // Finish the flushed segment and publish it to IndexWriter if (newSegment == null) { assert bufferedDeletes != null; - if (bufferedDeletes != null && bufferedDeletes.any()) { + if (bufferedDeletes != null + && (bufferedDeletes.anyDeletes() || bufferedDeletes.anyUpdates())) { indexWriter.publishFrozenDeletes(bufferedDeletes); if (infoStream.isEnabled("DW")) { - infoStream.message("DW", "flush: push buffered deletes: " + bufferedDeletes); + infoStream.message("DW", "flush: push buffered deletes: " + + bufferedDeletes); } } } else { - publishFlushedSegment(newSegment, bufferedDeletes); + publishFlushedSegment(newSegment, bufferedDeletes); } } - + final void subtractFlushedNumDocs(int numFlushed) { int oldValue = numDocsInRAM.get(); while (!numDocsInRAM.compareAndSet(oldValue, oldValue - numFlushed)) { @@ -497,55 +614,62 @@ /** * Publishes the flushed segment, segment private deletes (if any) and its - * associated global delete (if present) to IndexWriter. The actual - * publishing operation is synced on IW -> BDS so that the {@link SegmentInfo}'s - * delete generation is always GlobalPacket_deleteGeneration + 1 + * associated global delete (if present) to IndexWriter. The actual publishing + * operation is synced on IW -> BDS so that the {@link SegmentInfo}'s delete + * generation is always GlobalPacket_deleteGeneration + 1 */ - private void publishFlushedSegment(FlushedSegment newSegment, FrozenBufferedDeletes globalPacket) - throws IOException { + private void publishFlushedSegment(FlushedSegment newSegment, + FrozenBufferedDeletes globalPacket) throws IOException { assert newSegment != null; assert newSegment.segmentInfo != null; final FrozenBufferedDeletes segmentDeletes = newSegment.segmentDeletes; - //System.out.println("FLUSH: " + newSegment.segmentInfo.info.name); + // System.out.println("FLUSH: " + newSegment.segmentInfo.info.name); if (infoStream.isEnabled("DW")) { - infoStream.message("DW", "publishFlushedSegment seg-private deletes=" + segmentDeletes); + infoStream.message("DW", "publishFlushedSegment seg-private deletes=" + + segmentDeletes); } - if (segmentDeletes != null && infoStream.isEnabled("DW")) { - infoStream.message("DW", "flush: push buffered seg private deletes: " + segmentDeletes); + infoStream.message("DW", "flush: push buffered seg private deletes: " + + segmentDeletes); } // now publish! - indexWriter.publishFlushedSegment(newSegment.segmentInfo, segmentDeletes, globalPacket); + indexWriter.publishFlushedSegment(newSegment.segmentInfo, segmentDeletes, + globalPacket); } // for asserts private volatile DocumentsWriterDeleteQueue currentFullFlushDelQueue = null; - + // for asserts - private synchronized boolean setFlushingDeleteQueue(DocumentsWriterDeleteQueue session) { + private synchronized boolean setFlushingDeleteQueue( + DocumentsWriterDeleteQueue session) { currentFullFlushDelQueue = session; return true; } /* * FlushAllThreads is synced by IW fullFlushLock. Flushing all threads is a - * two stage operation; the caller must ensure (in try/finally) that finishFlush - * is called after this method, to release the flush lock in DWFlushControl + * two stage operation; the caller must ensure (in try/finally) that + * finishFlush is called after this method, to release the flush lock in + * DWFlushControl */ - final boolean flushAllThreads() - throws IOException { + final boolean flushAllThreads() throws IOException { final DocumentsWriterDeleteQueue flushingDeleteQueue; if (infoStream.isEnabled("DW")) { - infoStream.message("DW", Thread.currentThread().getName() + " startFullFlush"); + infoStream.message("DW", Thread.currentThread().getName() + + " startFullFlush"); } synchronized (this) { pendingChangesInCurrentFullFlush = anyChanges(); flushingDeleteQueue = deleteQueue; - /* Cutover to a new delete queue. This must be synced on the flush control + /* + * Cutover to a new delete queue. This must be synced on the flush control * otherwise a new DWPT could sneak into the loop with an already flushing - * delete queue */ - flushControl.markForFullFlush(); // swaps the delQueue synced on FlushControl + * delete queue + */ + flushControl.markForFullFlush(); // swaps the delQueue synced on + // FlushControl assert setFlushingDeleteQueue(flushingDeleteQueue); } assert currentFullFlushDelQueue != null; @@ -559,10 +683,15 @@ anythingFlushed |= doFlush(flushingDWPT); } // If a concurrent flush is still in flight wait for it - flushControl.waitForFlush(); - if (!anythingFlushed && flushingDeleteQueue.anyChanges()) { // apply deletes if we did not flush any document + flushControl.waitForFlush(); + if (!anythingFlushed && flushingDeleteQueue.anyChanges()) { // apply + // deletes if + // we did not + // flush any + // document if (infoStream.isEnabled("DW")) { - infoStream.message("DW", Thread.currentThread().getName() + ": flush naked frozen global deletes"); + infoStream.message("DW", Thread.currentThread().getName() + + ": flush naked frozen global deletes"); } ticketQueue.addDeletesAndPurge(this, flushingDeleteQueue); } else { @@ -578,7 +707,8 @@ final void finishFullFlush(boolean success) { try { if (infoStream.isEnabled("DW")) { - infoStream.message("DW", Thread.currentThread().getName() + " finishFullFlush success=" + success); + infoStream.message("DW", Thread.currentThread().getName() + + " finishFullFlush success=" + success); } assert setFlushingDeleteQueue(null); if (success) { Index: lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java (revision 1406075) +++ lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java (working copy) @@ -25,15 +25,15 @@ /** * {@link DocumentsWriterDeleteQueue} is a non-blocking linked pending deletes - * queue. In contrast to other queue implementation we only maintain the - * tail of the queue. A delete queue is always used in a context of a set of - * DWPTs and a global delete pool. Each of the DWPT and the global pool need to - * maintain their 'own' head of the queue (as a DeleteSlice instance per DWPT). - * The difference between the DWPT and the global pool is that the DWPT starts - * maintaining a head once it has added its first document since for its segments - * private deletes only the deletes after that document are relevant. The global - * pool instead starts maintaining the head once this instance is created by - * taking the sentinel instance as its initial head. + * queue. In contrast to other queue implementation we only maintain the tail of + * the queue. A delete queue is always used in a context of a set of DWPTs and a + * global delete pool. Each of the DWPT and the global pool need to maintain + * their 'own' head of the queue (as a DeleteSlice instance per DWPT). The + * difference between the DWPT and the global pool is that the DWPT starts + * maintaining a head once it has added its first document since for its + * segments private deletes only the deletes after that document are relevant. + * The global pool instead starts maintaining the head once this instance is + * created by taking the sentinel instance as its initial head. *

* Since each {@link DeleteSlice} maintains its own head and the list is only * single linked the garbage collector takes care of pruning the list for us. @@ -44,14 +44,15 @@ * Each DWPT as well as the global delete pool maintain their private * DeleteSlice instance. In the DWPT case updating a slice is equivalent to * atomically finishing the document. The slice update guarantees a "happens - * before" relationship to all other updates in the same indexing session. When a - * DWPT updates a document it: + * before" relationship to all other updates in the same indexing session. When + * a DWPT updates a document it: * *

    *
  1. consumes a document and finishes its processing
  2. *
  3. updates its private {@link DeleteSlice} either by calling - * {@link #updateSlice(DeleteSlice)} or {@link #add(Term, DeleteSlice)} (if the - * document has a delTerm)
  4. + * {@link #updateSlice(DeleteSlice)} or + * {@link #add(Term, DeleteSlice, FieldsUpdate)} (if the document has a delTerm) + * *
  5. applies all deletes in the slice to its private {@link BufferedDeletes} * and resets it
  6. *
  7. increments its internal document id
  8. @@ -73,6 +74,7 @@ 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 +85,12 @@ } DocumentsWriterDeleteQueue(long generation) { - this(new BufferedDeletes(), generation); + this(new BufferedDeletes(), new BufferedUpdates(), generation); } - DocumentsWriterDeleteQueue(BufferedDeletes globalBufferedDeletes, long generation) { + DocumentsWriterDeleteQueue(BufferedDeletes globalBufferedDeletes, BufferedUpdates globalBufferedUpdates, long generation) { this.globalBufferedDeletes = globalBufferedDeletes; + this.globalBufferedUpdates = globalBufferedUpdates; this.generation = generation; /* * we use a sentinel instance as our initial tail. No slice will ever try to @@ -110,8 +113,8 @@ /** * invariant for document update */ - void add(Term term, DeleteSlice slice) { - final TermNode termNode = new TermNode(term); + void add(Term term, DeleteSlice slice, FieldsUpdate fieldsUpdate) { + final TermNode termNode = new TermNode(term, fieldsUpdate); // System.out.println(Thread.currentThread().getName() + ": push " + termNode + " this=" + this); add(termNode); /* @@ -174,7 +177,7 @@ * and if the global slice is up-to-date * and if globalBufferedDeletes has changes */ - return globalBufferedDeletes.any() || !globalSlice.isEmpty() || globalSlice.sliceTail != tail + return globalBufferedDeletes.any() || globalBufferedUpdates.any() || !globalSlice.isEmpty() || globalSlice.sliceTail != tail || tail.next != null; } finally { globalBufferLock.unlock(); @@ -192,7 +195,7 @@ try { if (updateSlice(globalSlice)) { // System.out.println(Thread.currentThread() + ": apply globalSlice"); - globalSlice.apply(globalBufferedDeletes, BufferedDeletes.MAX_INT); + globalSlice.apply(globalBufferedDeletes, globalBufferedUpdates, BufferedDeletes.MAX_INT); } } finally { globalBufferLock.unlock(); @@ -217,13 +220,14 @@ try { if (globalSlice.sliceTail != currentTail) { globalSlice.sliceTail = currentTail; - globalSlice.apply(globalBufferedDeletes, BufferedDeletes.MAX_INT); + globalSlice.apply(globalBufferedDeletes, globalBufferedUpdates, BufferedDeletes.MAX_INT); } // System.out.println(Thread.currentThread().getName() + ": now freeze global buffer " + globalBufferedDeletes); final FrozenBufferedDeletes packet = new FrozenBufferedDeletes( - globalBufferedDeletes, false); + globalBufferedDeletes, globalBufferedUpdates, false); globalBufferedDeletes.clear(); + globalBufferedUpdates.clear(); return packet; } finally { globalBufferLock.unlock(); @@ -257,7 +261,7 @@ sliceHead = sliceTail = currentTail; } - void apply(BufferedDeletes del, int docIDUpto) { + void apply(BufferedDeletes del, BufferedUpdates update, int docIDUpto) { if (sliceHead == sliceTail) { // 0 length slice return; @@ -272,7 +276,7 @@ do { current = current.next; assert current != null : "slice property violated between the head on the tail must not be a null node"; - current.apply(del, docIDUpto); + current.apply(del, update, docIDUpto); // System.out.println(Thread.currentThread().getName() + ": pull " + current + " docIDUpto=" + docIDUpto); } while (current != sliceTail); reset(); @@ -323,7 +327,7 @@ static final AtomicReferenceFieldUpdater nextUpdater = AtomicReferenceFieldUpdater .newUpdater(Node.class, Node.class, "next"); - void apply(BufferedDeletes bufferedDeletes, int docIDUpto) { + void apply(BufferedDeletes bufferedDeletes, BufferedUpdates bufferedUpdates, int docIDUpto) { throw new IllegalStateException("sentinel item must never be applied"); } @@ -333,14 +337,20 @@ } private static final class TermNode extends Node { - - TermNode(Term term) { + FieldsUpdate fieldsUpdate; + + TermNode(Term term, FieldsUpdate fieldsUpdate) { super(term); + this.fieldsUpdate = fieldsUpdate; } @Override - void apply(BufferedDeletes bufferedDeletes, int docIDUpto) { - bufferedDeletes.addTerm(item, docIDUpto); + void apply(BufferedDeletes bufferedDeletes, BufferedUpdates bufferedUpdates, int docIDUpto) { + if (fieldsUpdate == null) { + bufferedDeletes.addTerm(item, docIDUpto); + } else { + bufferedUpdates.addTerm(item, new FieldsUpdate(fieldsUpdate, docIDUpto)); + } } @Override @@ -355,7 +365,7 @@ } @Override - void apply(BufferedDeletes bufferedDeletes, int docIDUpto) { + void apply(BufferedDeletes bufferedDeletes, BufferedUpdates bufferedUpdates, int docIDUpto) { for (Query query : item) { bufferedDeletes.addQuery(query, docIDUpto); } @@ -368,7 +378,7 @@ } @Override - void apply(BufferedDeletes bufferedDeletes, int docIDUpto) { + void apply(BufferedDeletes bufferedDeletes, BufferedUpdates bufferedUpdates, int docIDUpto) { for (Term term : item) { bufferedDeletes.addTerm(term, docIDUpto); } @@ -387,7 +397,7 @@ try { if (globalSlice.sliceTail != currentTail) { globalSlice.sliceTail = currentTail; - globalSlice.apply(globalBufferedDeletes, BufferedDeletes.MAX_INT); + globalSlice.apply(globalBufferedDeletes, globalBufferedUpdates, BufferedDeletes.MAX_INT); } return globalBufferedDeletes.any(); } finally { @@ -409,6 +419,10 @@ return globalBufferedDeletes.bytesUsed.get(); } + public long updateBytesUsed() { + return globalBufferedUpdates.bytesUsed.get(); + } + @Override public String toString() { return "DWDQ: [ generation: " + generation + " ]"; Index: lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (revision 1406075) +++ lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (working copy) @@ -25,6 +25,7 @@ import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.SegmentInfoWriter; import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.store.Directory; @@ -121,15 +122,21 @@ final SegmentInfoPerCommit segmentInfo; final FieldInfos fieldInfos; final FrozenBufferedDeletes segmentDeletes; + final BufferedUpdates segmentUpdates; final MutableBits liveDocs; + final UpdatedSegmentData liveUpdates; final int delCount; private FlushedSegment(SegmentInfoPerCommit segmentInfo, FieldInfos fieldInfos, - BufferedDeletes segmentDeletes, MutableBits liveDocs, int delCount) { + BufferedDeletes segmentDeletes, MutableBits liveDocs, + int delCount, BufferedUpdates segmentUpdates, + UpdatedSegmentData liveUpdates) { this.segmentInfo = segmentInfo; this.fieldInfos = fieldInfos; - this.segmentDeletes = segmentDeletes != null && segmentDeletes.any() ? new FrozenBufferedDeletes(segmentDeletes, true) : null; + this.segmentDeletes = segmentDeletes != null && segmentDeletes.any() ? new FrozenBufferedDeletes(segmentDeletes, segmentUpdates, true) : null; + this.segmentUpdates = segmentUpdates; this.liveDocs = liveDocs; + this.liveUpdates = liveUpdates; this.delCount = delCount; } } @@ -175,7 +182,9 @@ SegmentWriteState flushState; //Deletes for our still-in-RAM (to be flushed next) segment BufferedDeletes pendingDeletes; + BufferedUpdates pendingUpdates; SegmentInfo segmentInfo; // Current segment we are working on + SegmentInfo baseSegmentInfo; // name of the base segment for segmentInfo boolean aborting = false; // True if an abort is pending boolean hasAborted = false; // True if the last exception throws by #updateDocument was aborting @@ -204,6 +213,7 @@ bytesUsed = Counter.newCounter(); byteBlockAllocator = new DirectTrackingAllocator(bytesUsed); pendingDeletes = new BufferedDeletes(); + pendingUpdates = new BufferedUpdates(); intBlockAllocator = new IntBlockAllocator(bytesUsed); initialize(); // this should be the last call in the ctor @@ -239,7 +249,7 @@ docState.analyzer = analyzer; docState.docID = numDocsInRAM; if (segmentInfo == null) { - initSegmentInfo(); + initSegmentInfo(null, null); } if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) { infoStream.message("DWPT", Thread.currentThread().getName() + " update delTerm=" + delTerm + " docID=" + docState.docID + " seg=" + segmentInfo.name); @@ -272,16 +282,24 @@ abort(); } } - finishDocument(delTerm); + finishDocument(delTerm, null); } - private void initSegmentInfo() { - String segment = writer.newSegmentName(); - segmentInfo = new SegmentInfo(directoryOrig, Constants.LUCENE_MAIN_VERSION, segment, -1, - false, codec, null, null); + void initSegmentInfo(SegmentInfo info, String segmentSuffix) { + if (info == null) { + String segment = writer.newSegmentName(); + segmentInfo = new SegmentInfo(directoryOrig, + Constants.LUCENE_MAIN_VERSION, segment, -1, false, codec, null, null); + baseSegmentInfo = null; + } else { + baseSegmentInfo = info; + segmentInfo = new SegmentInfo(directoryOrig, + Constants.LUCENE_MAIN_VERSION, IndexFileNames.segmentFileName( + info.name, segmentSuffix, ""), -1, false, codec, null, null); + } assert numDocsInRAM == 0; if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) { - infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segment + " delQueue=" + deleteQueue); + infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segmentInfo.name + " delQueue=" + deleteQueue); } } @@ -290,7 +308,7 @@ assert deleteQueue != null; docState.analyzer = analyzer; if (segmentInfo == null) { - initSegmentInfo(); + initSegmentInfo(null, null); } if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) { infoStream.message("DWPT", Thread.currentThread().getName() + " update delTerm=" + delTerm + " docID=" + docState.docID + " seg=" + segmentInfo.name); @@ -341,16 +359,16 @@ } } - finishDocument(null); + finishDocument(null, null); } // Apply delTerm only after all indexing has // succeeded, but apply it only to docs prior to when // this batch started: if (delTerm != null) { - deleteQueue.add(delTerm, deleteSlice); + deleteQueue.add(delTerm, deleteSlice, null); assert deleteSlice.isTailItem(delTerm) : "expected the delete term as the tail item"; - deleteSlice.apply(pendingDeletes, numDocsInRAM-docCount); + deleteSlice.apply(pendingDeletes, pendingUpdates, numDocsInRAM-docCount); } } finally { @@ -360,7 +378,7 @@ return docCount; } - private void finishDocument(Term delTerm) { + private void finishDocument(Term delTerm, FieldsUpdate fieldsUpdate) { /* * here we actually finish the document in two steps 1. push the delete into * the queue and update our slice. 2. increment the DWPT private document @@ -372,17 +390,16 @@ if (deleteSlice == null) { deleteSlice = deleteQueue.newSlice(); if (delTerm != null) { - deleteQueue.add(delTerm, deleteSlice); + deleteQueue.add(delTerm, deleteSlice, fieldsUpdate); deleteSlice.reset(); } - } else { if (delTerm != null) { - deleteQueue.add(delTerm, deleteSlice); + deleteQueue.add(delTerm, deleteSlice, fieldsUpdate); assert deleteSlice.isTailItem(delTerm) : "expected the delete term as the tail item"; - deleteSlice.apply(pendingDeletes, numDocsInRAM); + deleteSlice.apply(pendingDeletes, pendingUpdates, numDocsInRAM); } else if (deleteQueue.updateSlice(deleteSlice)) { - deleteSlice.apply(pendingDeletes, numDocsInRAM); + deleteSlice.apply(pendingDeletes, pendingUpdates, numDocsInRAM); } } ++numDocsInRAM; @@ -441,7 +458,7 @@ adding a document. */ if (deleteSlice != null) { // apply all deletes before we flush and release the delete slice - deleteSlice.apply(pendingDeletes, numDocsInRAM); + deleteSlice.apply(pendingDeletes, pendingUpdates, numDocsInRAM); assert deleteSlice.isEmpty(); deleteSlice = null; } @@ -449,13 +466,17 @@ } /** Flush all pending docs to a new segment */ - FlushedSegment flush() throws IOException { + FlushedSegment flush(long updateGen) throws IOException { assert numDocsInRAM > 0; assert deleteSlice == null : "all deletes must be applied in prepareFlush"; + if (segmentInfo == null) { + return null; + } segmentInfo.setDocCount(numDocsInRAM); - flushState = new SegmentWriteState(infoStream, directory, segmentInfo, fieldInfos.finish(), - writer.getConfig().getTermIndexInterval(), - pendingDeletes, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed()))); + IOContext context = new IOContext(new FlushInfo(numDocsInRAM, bytesUsed())); + flushState = new SegmentWriteState(infoStream, directory, segmentInfo, 0, fieldInfos.finish(), + writer.getConfig().getTermIndexInterval(), + pendingDeletes, pendingUpdates, context); final double startMBUsed = parent.flushControl.netBytes() / 1024. / 1024.; // Apply delete-by-docID now (delete-byDocID only @@ -487,9 +508,14 @@ try { consumer.flush(flushState); pendingDeletes.terms.clear(); - segmentInfo.setFiles(new HashSet(directory.getCreatedFiles())); - - final SegmentInfoPerCommit segmentInfoPerCommit = new SegmentInfoPerCommit(segmentInfo, 0, -1L); + if (updateGen < 0) { + segmentInfo.setFiles(new HashSet(directory.getCreatedFiles())); + } else { + segmentInfo = baseSegmentInfo; + segmentInfo.addFiles(new HashSet(directory.getCreatedFiles())); + } + + final SegmentInfoPerCommit segmentInfoPerCommit = new SegmentInfoPerCommit(segmentInfo, 0, -1L, updateGen); 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 " + @@ -524,8 +550,9 @@ assert segmentInfo != null; FlushedSegment fs = new FlushedSegment(segmentInfoPerCommit, flushState.fieldInfos, - segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush); - sealFlushedSegment(fs); + segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush, + pendingUpdates, flushState.liveUpdates); + sealFlushedSegment(fs, updateGen); doAfterFlush(); success = true; @@ -544,7 +571,7 @@ * Seals the {@link SegmentInfo} for the new flushed segment and persists * the deleted documents {@link MutableBits}. */ - void sealFlushedSegment(FlushedSegment flushedSegment) throws IOException { + void sealFlushedSegment(FlushedSegment flushedSegment, long updateGen) throws IOException { assert flushedSegment != null; SegmentInfoPerCommit newSegment = flushedSegment.segmentInfo; @@ -558,7 +585,7 @@ if (writer.useCompoundFile(newSegment)) { // Now build compound file - Collection oldFiles = IndexWriter.createCompoundFile(infoStream, directory, MergeState.CheckAbort.NONE, newSegment.info, context); + Collection oldFiles = IndexWriter.createCompoundFile(infoStream, directory, MergeState.CheckAbort.NONE, newSegment.info, context, updateGen); newSegment.info.setUseCompoundFile(true); writer.deleteNewFiles(oldFiles); } @@ -567,7 +594,11 @@ // creating CFS so that 1) .si isn't slurped into CFS, // and 2) .si reflects useCompoundFile=true change // above: - codec.segmentInfoFormat().getSegmentInfoWriter().write(directory, newSegment.info, flushedSegment.fieldInfos, context); + if (updateGen < 0) { + codec.segmentInfoFormat().getSegmentInfoWriter().write(directory, newSegment.info, flushedSegment.fieldInfos, context); + } else { + codec.segmentInfoFormat().getSegmentInfoWriter().writeFilesList(directory, newSegment.info, updateGen, context); + } // TODO: ideally we would freeze newSegment here!! // because any changes after writing the .si will be @@ -649,9 +680,9 @@ } } - PerDocWriteState newPerDocWriteState(String segmentSuffix) { + PerDocWriteState newPerDocWriteState() { assert segmentInfo != null; - return new PerDocWriteState(infoStream, directory, segmentInfo, bytesUsed, segmentSuffix, IOContext.DEFAULT); + return new PerDocWriteState(infoStream, directory, segmentInfo, bytesUsed, "", IOContext.DEFAULT); } @Override @@ -660,4 +691,15 @@ + ", segment=" + (segmentInfo != null ? segmentInfo.name : "null") + ", aborting=" + aborting + ", numDocsInRAM=" + numDocsInRAM + ", deleteQueue=" + deleteQueue + "]"; } + + void updateFields(Term term, FieldsUpdate fieldUpdates) { + finishDocument(term, fieldUpdates); + } + + void clearDeleteSlice() { + if (deleteSlice != null) { + assert deleteSlice.sliceHead == deleteSlice.sliceTail; + deleteSlice = null; + } + } } Index: lucene/core/src/java/org/apache/lucene/index/FieldGenerationReplacements.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/FieldGenerationReplacements.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/index/FieldGenerationReplacements.java (working copy) @@ -0,0 +1,83 @@ +package org.apache.lucene.index; + +import java.util.Iterator; +import java.util.Map.Entry; +import java.util.TreeMap; + +/* + * 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. + */ + +/** + * Holds for a certain field in a stacked segment the documents document the + * generation in which the last replacement (of the relevant field) took place. + */ +public class FieldGenerationReplacements implements Iterable> { + + TreeMap map = null; + + /** + * Set the generation value for a given document. + * + * @param docId + * Document id. + * @param generation + * The requested generation. + */ + public void set(int docId, long generation) { + if (map == null) { + map = new TreeMap(); + } + assert generation > 0 && generation <= Integer.MAX_VALUE; + map.put(docId, generation); + } + + /** + * Get the generation value for a given document. + * + * @param docId + * Document id. + * @return The requested generation, or -1 if the document has no generation. + */ + public long get(int docId) { + if (map == null) { + return -1; + } + final Long val = map.get(docId); + if (val == null) { + return -1; + } + return val; + } + + public void merge(FieldGenerationReplacements other) { + if (map == null) { + map = other.map; + } else if (other != null) { + map.putAll(other.map); + } + } + + @Override + public Iterator> iterator() { + return map.entrySet().iterator(); + } + + public int size() { + return map.size(); + } + +} Index: lucene/core/src/java/org/apache/lucene/index/FieldsUpdate.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/FieldsUpdate.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/index/FieldsUpdate.java (working copy) @@ -0,0 +1,92 @@ +package org.apache.lucene.index; + +import org.apache.lucene.analysis.Analyzer; + +/* + * 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. + */ + +public class FieldsUpdate implements Comparable { + + /** + * Specifies the operation to perform when updating fields. + */ + enum Operation { + /** + * Replace all documents matching the update criterion with a new document + * containing the given fields. + */ + REPLACE_DOCS, + + /** + * Add the given fields to all existing documents matching the update + * criterion. + */ + ADD_FIELDS, + + /** + * Use the given fields to replace fields with same names in all existing + * documents matching the update criterion. + */ + REPLACE_FIELDS + } + + final Operation operation; + final IndexDocument fields; + final Analyzer analyzer; + final int docIDUpto; + + /** + * An update of fields which is not assigned to a specific live segment. + * + * @param operation + * The type of update operation. + * @param fields + * The fields to use in the update. + * @param analyzer + * The analyzer to use in the update. + */ + public FieldsUpdate(Operation operation, IndexDocument fields, + Analyzer analyzer) { + this.operation = operation; + this.fields = fields; + this.analyzer = analyzer; + this.docIDUpto = -1; + } + + /** + * An update of fields for a specific live segment. + * + * @param other + * A non-specific update with the update data. + * @param docIDUpto + * The doc ID in the live segment up to which the update should be + * applied. + */ + public FieldsUpdate(FieldsUpdate other, int docIDUpto) { + this.operation = other.operation; + this.fields = other.fields; + this.analyzer = other.analyzer; + this.docIDUpto = docIDUpto; + } + + /* Order FrieldsUpdate by increasing docIDUpto */ + @Override + public int compareTo(FieldsUpdate other) { + return this.docIDUpto - other.docIDUpto; + } + +} Index: lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (revision 1406075) +++ lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (working copy) @@ -19,7 +19,9 @@ import java.io.IOException; import java.util.Comparator; +import java.util.Iterator; import java.util.Map; +import java.util.SortedSet; import org.apache.lucene.analysis.tokenattributes.OffsetAttribute; import org.apache.lucene.analysis.tokenattributes.PayloadAttribute; @@ -35,8 +37,9 @@ // TODO: break into separate freq and prox writers as // codecs; make separate container (tii/tis/skip/*) that can // be configured as any number of files 1..N -final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implements Comparable { - +final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField + implements Comparable { + final FreqProxTermsWriter parent; final TermsHashPerField termsHashPerField; final FieldInfo fieldInfo; @@ -47,8 +50,9 @@ private boolean hasOffsets; PayloadAttribute payloadAttribute; OffsetAttribute offsetAttribute; - - public FreqProxTermsWriterPerField(TermsHashPerField termsHashPerField, FreqProxTermsWriter parent, FieldInfo fieldInfo) { + + public FreqProxTermsWriterPerField(TermsHashPerField termsHashPerField, + FreqProxTermsWriter parent, FieldInfo fieldInfo) { this.termsHashPerField = termsHashPerField; this.parent = parent; this.fieldInfo = fieldInfo; @@ -56,7 +60,7 @@ fieldState = termsHashPerField.fieldState; setIndexOptions(fieldInfo.getIndexOptions()); } - + @Override int getStreamCount() { if (!hasProx) { @@ -65,23 +69,23 @@ return 2; } } - + @Override void finish() { if (hasPayloads) { fieldInfo.setStorePayloads(); } } - + boolean hasPayloads; - + @Override void skippingLongTerm() {} - + public int compareTo(FreqProxTermsWriterPerField other) { return fieldInfo.name.compareTo(other.fieldInfo.name); } - + // Called after flush void reset() { // Record, up front, whether our in-RAM format will be @@ -89,39 +93,44 @@ setIndexOptions(fieldInfo.getIndexOptions()); payloadAttribute = null; } - + private void setIndexOptions(IndexOptions indexOptions) { if (indexOptions == null) { // field could later be updated with indexed=true, so set everything on hasFreq = hasProx = hasOffsets = true; } else { hasFreq = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; - hasProx = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; - hasOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; + hasProx = indexOptions + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + hasOffsets = indexOptions + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; } } - + @Override boolean start(IndexableField[] fields, int count) { return true; } - + @Override void start(IndexableField f) { if (fieldState.attributeSource.hasAttribute(PayloadAttribute.class)) { - payloadAttribute = fieldState.attributeSource.getAttribute(PayloadAttribute.class); + payloadAttribute = fieldState.attributeSource + .getAttribute(PayloadAttribute.class); } else { payloadAttribute = null; } if (hasOffsets) { - offsetAttribute = fieldState.attributeSource.addAttribute(OffsetAttribute.class); + offsetAttribute = fieldState.attributeSource + .addAttribute(OffsetAttribute.class); } else { offsetAttribute = null; } } - + void writeProx(final int termID, int proxCode) { - //System.out.println("writeProx termID=" + termID + " proxCode=" + proxCode); + // System.out.println("writeProx termID=" + termID + " proxCode=" + + // proxCode); assert hasProx; final BytesRef payload; if (payloadAttribute == null) { @@ -129,39 +138,41 @@ } else { payload = payloadAttribute.getPayload(); } - + if (payload != null && payload.length > 0) { - termsHashPerField.writeVInt(1, (proxCode<<1)|1); + termsHashPerField.writeVInt(1, (proxCode << 1) | 1); termsHashPerField.writeVInt(1, payload.length); - termsHashPerField.writeBytes(1, payload.bytes, payload.offset, payload.length); + termsHashPerField.writeBytes(1, payload.bytes, payload.offset, + payload.length); hasPayloads = true; } else { - termsHashPerField.writeVInt(1, proxCode<<1); + termsHashPerField.writeVInt(1, proxCode << 1); } - + FreqProxPostingsArray postings = (FreqProxPostingsArray) termsHashPerField.postingsArray; postings.lastPositions[termID] = fieldState.position; } - + void writeOffsets(final int termID, int offsetAccum) { assert hasOffsets; final int startOffset = offsetAccum + offsetAttribute.startOffset(); final int endOffset = offsetAccum + offsetAttribute.endOffset(); - //System.out.println("writeOffsets termID=" + termID + " prevOffset=" + prevOffset + " startOff=" + startOffset + " endOff=" + endOffset); + // System.out.println("writeOffsets termID=" + termID + " prevOffset=" + + // prevOffset + " startOff=" + startOffset + " endOff=" + endOffset); FreqProxPostingsArray postings = (FreqProxPostingsArray) termsHashPerField.postingsArray; assert startOffset - postings.lastOffsets[termID] >= 0; termsHashPerField.writeVInt(1, startOffset - postings.lastOffsets[termID]); termsHashPerField.writeVInt(1, endOffset - startOffset); - + postings.lastOffsets[termID] = startOffset; } - + @Override void newTerm(final int termID) { // First time we're seeing this term since the last // flush assert docState.testPoint("FreqProxTermsWriterPerField.newTerm start"); - + FreqProxPostingsArray postings = (FreqProxPostingsArray) termsHashPerField.postingsArray; postings.lastDocIDs[termID] = docState.docID; if (!hasFreq) { @@ -181,34 +192,37 @@ fieldState.maxTermFrequency = Math.max(1, fieldState.maxTermFrequency); fieldState.uniqueTermCount++; } - + @Override void addTerm(final int termID) { - + assert docState.testPoint("FreqProxTermsWriterPerField.addTerm start"); - + FreqProxPostingsArray postings = (FreqProxPostingsArray) termsHashPerField.postingsArray; - + assert !hasFreq || postings.termFreqs[termID] > 0; - + if (!hasFreq) { assert postings.termFreqs == null; if (docState.docID != postings.lastDocIDs[termID]) { assert docState.docID > postings.lastDocIDs[termID]; termsHashPerField.writeVInt(0, postings.lastDocCodes[termID]); - postings.lastDocCodes[termID] = docState.docID - postings.lastDocIDs[termID]; + postings.lastDocCodes[termID] = docState.docID + - postings.lastDocIDs[termID]; postings.lastDocIDs[termID] = docState.docID; fieldState.uniqueTermCount++; } } else if (docState.docID != postings.lastDocIDs[termID]) { - assert docState.docID > postings.lastDocIDs[termID]:"id: "+docState.docID + " postings ID: "+ postings.lastDocIDs[termID] + " termID: "+termID; + assert docState.docID > postings.lastDocIDs[termID] : "id: " + + docState.docID + " postings ID: " + postings.lastDocIDs[termID] + + " termID: " + termID; // Term not yet seen in the current doc but previously // seen in other doc(s) since the last flush - + // Now that we know doc freq for previous doc, // write it & lastDocCode if (1 == postings.termFreqs[termID]) { - termsHashPerField.writeVInt(0, postings.lastDocCodes[termID]|1); + termsHashPerField.writeVInt(0, postings.lastDocCodes[termID] | 1); } else { termsHashPerField.writeVInt(0, postings.lastDocCodes[termID]); termsHashPerField.writeVInt(0, postings.termFreqs[termID]); @@ -228,23 +242,25 @@ } fieldState.uniqueTermCount++; } else { - fieldState.maxTermFrequency = Math.max(fieldState.maxTermFrequency, ++postings.termFreqs[termID]); + fieldState.maxTermFrequency = Math.max(fieldState.maxTermFrequency, + ++postings.termFreqs[termID]); if (hasProx) { - writeProx(termID, fieldState.position-postings.lastPositions[termID]); + writeProx(termID, fieldState.position - postings.lastPositions[termID]); } if (hasOffsets) { writeOffsets(termID, fieldState.offset); } } } - + @Override ParallelPostingsArray createPostingsArray(int size) { return new FreqProxPostingsArray(size, hasFreq, hasProx, hasOffsets); } - + static final class FreqProxPostingsArray extends ParallelPostingsArray { - public FreqProxPostingsArray(int size, boolean writeFreqs, boolean writeProx, boolean writeOffsets) { + public FreqProxPostingsArray(int size, boolean writeFreqs, + boolean writeProx, boolean writeOffsets) { super(size); if (writeFreqs) { termFreqs = new int[size]; @@ -259,27 +275,29 @@ } else { assert !writeOffsets; } - //System.out.println("PA init freqs=" + writeFreqs + " pos=" + writeProx + " offs=" + writeOffsets); + // System.out.println("PA init freqs=" + writeFreqs + " pos=" + writeProx + // + " offs=" + writeOffsets); } - - int termFreqs[]; // # times this term occurs in the current doc - int lastDocIDs[]; // Last docID where this term occurred - int lastDocCodes[]; // Code for prior doc - int lastPositions[]; // Last position where this term occurred - int lastOffsets[]; // Last endOffset where this term occurred - + + int termFreqs[]; // # times this term occurs in the current doc + int lastDocIDs[]; // Last docID where this term occurred + int lastDocCodes[]; // Code for prior doc + int lastPositions[]; // Last position where this term occurred + int lastOffsets[]; // Last endOffset where this term occurred + @Override ParallelPostingsArray newInstance(int size) { - return new FreqProxPostingsArray(size, termFreqs != null, lastPositions != null, lastOffsets != null); + return new FreqProxPostingsArray(size, termFreqs != null, + lastPositions != null, lastOffsets != null); } - + @Override void copyTo(ParallelPostingsArray toArray, int numToCopy) { assert toArray instanceof FreqProxPostingsArray; FreqProxPostingsArray to = (FreqProxPostingsArray) toArray; - + super.copyTo(toArray, numToCopy); - + System.arraycopy(lastDocIDs, 0, to.lastDocIDs, 0, numToCopy); System.arraycopy(lastDocCodes, 0, to.lastDocCodes, 0, numToCopy); if (lastPositions != null) { @@ -295,10 +313,11 @@ System.arraycopy(termFreqs, 0, to.termFreqs, 0, numToCopy); } } - + @Override int bytesPerPosting() { - int bytes = ParallelPostingsArray.BYTES_PER_POSTING + 2 * RamUsageEstimator.NUM_BYTES_INT; + int bytes = ParallelPostingsArray.BYTES_PER_POSTING + 2 + * RamUsageEstimator.NUM_BYTES_INT; if (lastPositions != null) { bytes += RamUsageEstimator.NUM_BYTES_INT; } @@ -308,97 +327,114 @@ if (termFreqs != null) { bytes += RamUsageEstimator.NUM_BYTES_INT; } - + return bytes; } } - + public void abort() {} - + BytesRef payload; - - /* Walk through all unique text tokens (Posting - * instances) found in this field and serialize them - * into a single RAM segment. */ - void flush(String fieldName, FieldsConsumer consumer, final SegmentWriteState state) - throws IOException { - + + /* + * Walk through all unique text tokens (Posting instances) found in this field + * and serialize them into a single RAM segment. + */ + void flush(String fieldName, FieldsConsumer consumer, + final SegmentWriteState state) throws IOException { + if (!fieldInfo.isIndexed()) { - return; // nothing to flush, don't bother the codec with the unindexed field + return; // nothing to flush, don't bother the codec with the unindexed + // field } final TermsConsumer termsConsumer = consumer.addField(fieldInfo); final Comparator termComp = termsConsumer.getComparator(); - + // CONFUSING: this.indexOptions holds the index options - // that were current when we first saw this field. But + // that were current when we first saw this field. But // it's possible this has changed, eg when other // documents are indexed that cause a "downgrade" of the - // IndexOptions. So we must decode the in-RAM buffer + // IndexOptions. So we must decode the in-RAM buffer // according to this.indexOptions, but then write the // new segment to the directory according to // currentFieldIndexOptions: final IndexOptions currentFieldIndexOptions = fieldInfo.getIndexOptions(); assert currentFieldIndexOptions != null; - - final boolean writeTermFreq = currentFieldIndexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; - final boolean writePositions = currentFieldIndexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; - final boolean writeOffsets = currentFieldIndexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; - + + final boolean writeTermFreq = currentFieldIndexOptions + .compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; + final boolean writePositions = currentFieldIndexOptions + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + final boolean writeOffsets = currentFieldIndexOptions + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; + final boolean readTermFreq = this.hasFreq; final boolean readPositions = this.hasProx; final boolean readOffsets = this.hasOffsets; - - //System.out.println("flush readTF=" + readTermFreq + " readPos=" + readPositions + " readOffs=" + readOffsets); - + + // System.out.println("flush readTF=" + readTermFreq + " readPos=" + + // readPositions + " readOffs=" + readOffsets); + // Make sure FieldInfo.update is working correctly!: assert !writeTermFreq || readTermFreq; assert !writePositions || readPositions; assert !writeOffsets || readOffsets; - + assert !writeOffsets || writePositions; - + final Map segDeletes; if (state.segDeletes != null && state.segDeletes.terms.size() > 0) { segDeletes = state.segDeletes.terms; } else { segDeletes = null; } - + + final Map> segUpdates; + if (state.segUpdates != null && state.segUpdates.terms.size() > 0) { + segUpdates = state.segUpdates.terms; + } else { + segUpdates = null; + } + final int[] termIDs = termsHashPerField.sortPostings(termComp); final int numTerms = termsHashPerField.bytesHash.size(); final BytesRef text = new BytesRef(); final FreqProxPostingsArray postings = (FreqProxPostingsArray) termsHashPerField.postingsArray; final ByteSliceReader freq = new ByteSliceReader(); final ByteSliceReader prox = new ByteSliceReader(); - + FixedBitSet visitedDocs = new FixedBitSet(state.segmentInfo.getDocCount()); long sumTotalTermFreq = 0; long sumDocFreq = 0; - + Term protoTerm = new Term(fieldName); for (int i = 0; i < numTerms; i++) { final int termID = termIDs[i]; - //System.out.println("term=" + termID); + // System.out.println("term=" + termID); // Get BytesRef final int textStart = postings.textStarts[termID]; termsHashPerField.bytePool.setBytesRef(text, textStart); - + termsHashPerField.initReader(freq, termID, 0); if (readPositions || readOffsets) { termsHashPerField.initReader(prox, termID, 1); } - + // TODO: really TermsHashPerField should take over most // of this loop, including merge sort of terms from // multiple threads and interacting with the // TermsConsumer, only calling out to us (passing us the // DocsConsumer) to handle delivery of docs/positions - + final PostingsConsumer postingsConsumer = termsConsumer.startTerm(text); - + + Term term = new Term(fieldName, text); + + // calculate doc limit of deletions final int delDocLimit; if (segDeletes != null) { + //final Integer docIDUpto = segDeletes.get(term); protoTerm.bytes = text; final Integer docIDUpto = segDeletes.get(protoTerm); if (docIDUpto != null) { @@ -409,16 +445,29 @@ } else { delDocLimit = 0; } - + + final SortedSet termUpdates; + Iterator updatesIterator = null; + FieldsUpdate nextUpdate = null; + if (segUpdates != null) { + termUpdates = segUpdates.get(term); + if (termUpdates != null && !termUpdates.isEmpty()) { + updatesIterator = termUpdates.iterator(); + nextUpdate = updatesIterator.next(); + } + } else { + termUpdates = null; + } + // Now termStates has numToMerge FieldMergeStates - // which all share the same term. Now we must + // which all share the same term. Now we must // interleave the docID streams. int docFreq = 0; long totTF = 0; int docID = 0; - - while(true) { - //System.out.println(" cycle"); + + while (true) { + // System.out.println(" cycle"); final int termFreq; if (freq.eof()) { if (postings.lastDocCodes[termID] != -1) { @@ -447,85 +496,110 @@ termFreq = freq.readVInt(); } } - + assert docID != postings.lastDocIDs[termID]; } - + docFreq++; - assert docID < state.segmentInfo.getDocCount(): "doc=" + docID + " maxDoc=" + state.segmentInfo.getDocCount(); - + assert docID < state.segmentInfo.getDocCount() : "doc=" + docID + + " maxDoc=" + state.segmentInfo.getDocCount(); + // NOTE: we could check here if the docID was - // deleted, and skip it. However, this is somewhat + // deleted, and skip it. However, this is somewhat // dangerous because it can yield non-deterministic // behavior since we may see the docID before we see - // the term that caused it to be deleted. This + // the term that caused it to be deleted. This // would mean some (but not all) of its postings may // make it into the index, which'd alter the docFreq - // for those terms. We could fix this by doing two + // for those terms. We could fix this by doing two // passes, ie first sweep marks all del docs, and // 2nd sweep does the real flush, but I suspect // that'd add too much time to flush. visitedDocs.set(docID); postingsConsumer.startDoc(docID, writeTermFreq ? termFreq : -1); if (docID < delDocLimit) { - // Mark it deleted. TODO: we could also skip + // Mark it deleted. TODO: we could also skip // writing its postings; this would be // deterministic (just for this Term's docs). // TODO: can we do this reach-around in a cleaner way???? if (state.liveDocs == null) { - state.liveDocs = docState.docWriter.codec.liveDocsFormat().newLiveDocs(state.segmentInfo.getDocCount()); + state.liveDocs = docState.docWriter.codec.liveDocsFormat() + .newLiveDocs(state.segmentInfo.getDocCount()); } if (state.liveDocs.get(docID)) { state.delCountOnFlush++; state.liveDocs.clear(docID); } } - + + // make sure we update the relevant documents according to the doc ID + // in which the updates arrived + while (nextUpdate != null && docID > nextUpdate.docIDUpto) { + if (updatesIterator.hasNext()) { + nextUpdate = updatesIterator.next(); + } else { + nextUpdate = null; + } + } + + if (nextUpdate != null) { + if (state.liveUpdates == null) { + state.liveUpdates = new UpdatedSegmentData(); + } + state.liveUpdates.addUpdates(docID, termUpdates.tailSet(nextUpdate)); + } + totTF += termFreq; // Carefully copy over the prox + payload info, // changing the format to match Lucene's segment // format. - + if (readPositions || readOffsets) { // we did record positions (& maybe payload) and/or offsets int position = 0; int offset = 0; - for(int j=0;j>> 1; - + if ((code & 1) != 0) { - + // This position has a payload final int payloadLength = prox.readVInt(); - + if (payload == null) { payload = new BytesRef(); payload.bytes = new byte[payloadLength]; } else if (payload.bytes.length < payloadLength) { payload.grow(payloadLength); } - + prox.readBytes(payload.bytes, 0, payloadLength); payload.length = payloadLength; thisPayload = payload; - + } else { thisPayload = null; } - + if (readOffsets) { final int startOffset = offset + prox.readVInt(); final int endOffset = startOffset + prox.readVInt(); if (writePositions) { if (writeOffsets) { - assert startOffset >=0 && endOffset >= startOffset : "startOffset=" + startOffset + ",endOffset=" + endOffset + ",offset=" + offset; - postingsConsumer.addPosition(position, thisPayload, startOffset, endOffset); + assert startOffset >= 0 && endOffset >= startOffset : "startOffset=" + + startOffset + + ",endOffset=" + + endOffset + + ",offset=" + + offset; + postingsConsumer.addPosition(position, thisPayload, + startOffset, endOffset); } else { postingsConsumer.addPosition(position, thisPayload, -1, -1); } @@ -539,11 +613,13 @@ } postingsConsumer.finishDoc(); } - termsConsumer.finishTerm(text, new TermStats(docFreq, writeTermFreq ? totTF : -1)); + termsConsumer.finishTerm(text, new TermStats(docFreq, + writeTermFreq ? totTF : -1)); sumTotalTermFreq += totTF; sumDocFreq += docFreq; } - - termsConsumer.finish(writeTermFreq ? sumTotalTermFreq : -1, sumDocFreq, visitedDocs.cardinality()); + + termsConsumer.finish(writeTermFreq ? sumTotalTermFreq : -1, sumDocFreq, + visitedDocs.cardinality()); } } Index: lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java (revision 1382438) +++ lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java (working copy) @@ -1,145 +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.Iterator; -import java.util.Map; - -import org.apache.lucene.search.Query; -import org.apache.lucene.util.ArrayUtil; -import org.apache.lucene.util.RamUsageEstimator; -import org.apache.lucene.index.BufferedDeletesStream.QueryAndLimit; - -/** Holds buffered deletes by term or query, once pushed. - * Pushed deletes are write-once, so we shift to more - * memory efficient data structure to hold them. We don't - * hold docIDs because these are applied on flush. */ - -class FrozenBufferedDeletes { - - /* 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; - 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.mergeSort(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++; - } - - bytesUsed = (int) terms.getSizeInBytes() + queries.length * BYTES_PER_DEL_QUERY; - 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; - } -} Index: lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java (working copy) @@ -0,0 +1,193 @@ +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.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.SortedSet; + +import org.apache.lucene.search.Query; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.index.BufferedDeletesStream.QueryAndLimit; + +/** Holds buffered deletes by term or query, once pushed. + * Pushed deletes are write-once, so we shift to more + * memory efficient data structure to hold them. We don't + * hold docIDs because these are applied on flush. */ + +class FrozenBufferedDeletes { + + /* 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; + + final static List EMPTY_LIST = new ArrayList(0); + // 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; + 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 + + // Updated terms, in sorted order: + final PrefixCodedTerms updateTerms; + // Updated fields per term + final FieldsUpdate[][] updateArrays; + + public FrozenBufferedDeletes(BufferedDeletes deletes, BufferedUpdates updates, boolean isSegmentPrivate) { + this.isSegmentPrivate = isSegmentPrivate; + int localBytesUsed = 0; + if (deletes != null) { + 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.mergeSort(termsArray); + PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder(); + for (Term term : termsArray) { + builder.add(term); + } + terms = builder.finish(); + localBytesUsed += (int) terms.getSizeInBytes(); + + 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++; + } + + localBytesUsed += queries.length * BYTES_PER_DEL_QUERY; + numTermDeletes = deletes.numTermDeletes.get(); + } else { + terms = null; + numTermDeletes = 0; + queries = null; + queryLimits = null; + } + + // freeze updates + if (updates != null && !updates.terms.isEmpty()) { + PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder(); + // TODO : SY : calculate bytes + updateArrays = new FieldsUpdate[updates.terms.size()][]; + int i = 0; + for (Entry> entry : updates.terms.entrySet()) { + builder.add(entry.getKey()); + SortedSet updateList = entry.getValue(); + // TODO : SY : calculate bytes + updateArrays[i++] = updateList.toArray(new FieldsUpdate[updateList.size()]); + } + updateTerms = builder.finish(); + localBytesUsed += (int) updateTerms.getSizeInBytes(); + } else { + updateTerms = null; + updateArrays = null; + } + + bytesUsed = localBytesUsed; + } + + 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() { + if (terms == null) { + return EMPTY_LIST.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 != null && queries.length != 0) { + s += " " + queries.length + " deleted queries"; + } + // TODO : SY : add updates printing + if (bytesUsed != 0) { + s += " bytesUsed=" + bytesUsed; + } + + return s; + } + + boolean anyDeletes() { + return termCount > 0 || (queries != null && queries.length > 0); + } + + boolean anyUpdates() { + return updateTerms != null; + } +} Index: lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java (revision 1382438) +++ lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java (working copy) Property changes on: lucene/core/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java (revision 1406075) +++ lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java (working copy) @@ -95,7 +95,7 @@ // to the gen length as string (hopefully an upper limit so SB won't // expand in the middle. StringBuilder res = new StringBuilder(base.length() + 6 + ext.length()) - .append(base).append('_').append(Long.toString(gen, Character.MAX_RADIX)); + .append(base).append('_').append(generationString(gen)); if (ext.length() > 0) { res.append('.').append(ext); } @@ -103,7 +103,37 @@ } } + public static String generationString(long gen) { + return Long.toString(gen, Character.MAX_RADIX); + } + /** + * Computes the base name of an updated segment from base and generation. If + * the generation < 0, the file name is null. otherwise, the file name is + * <base>_upd_<gen>.
    + * + * @param baseName + * base segment string + * @param gen + * update generation + */ + public static String updatedSegmentFileNameFromGeneration(String baseName, + long gen) { + if (gen <= 0) { + return null; + } else { + assert gen > 0; + // The '10' part in the length is: 3 for '_', 3 for "upd" and 4 as + // estimate to the gen length as string (hopefully an upper limit so SB + // won't expand in the middle. + StringBuilder res = new StringBuilder(baseName.length() + 10) + .append(baseName).append('_') + .append(generationString(gen)); + return res.toString(); + } + } + + /** * Returns a file name that includes the given segment name, your own custom * name and extension. The format of the filename is: * <segmentName>(_<name>)(.<ext>). Index: lucene/core/src/java/org/apache/lucene/index/IndexWriter.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (revision 1408449) +++ lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (working copy) @@ -34,7 +34,9 @@ import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.SegmentInfoWriter; import org.apache.lucene.index.FieldInfos.FieldNumbers; +import org.apache.lucene.index.FieldsUpdate.Operation; import org.apache.lucene.index.IndexWriterConfig.OpenMode; import org.apache.lucene.index.MergePolicy.MergeTrigger; import org.apache.lucene.index.MergeState.CheckAbort; @@ -426,10 +428,12 @@ // Pool still holds a ref: assert rld.refCount() >= 1; + boolean updatesWritten = rld.writeLiveUpdates(directory, codec); + if (!poolReaders && rld.refCount() == 1) { // This is the last ref to this RLD, and we're not // pooling, so remove it: - if (rld.writeLiveDocs(directory)) { + if (rld.writeLiveDocs(directory) || updatesWritten) { // Make sure we only write del docs for a live segment: assert infoIsLive(rld.info); // Must checkpoint w/ deleter, because we just @@ -1197,11 +1201,20 @@ * * @lucene.experimental */ - public void updateDocuments(Term delTerm, Iterable docs) throws IOException { - updateDocuments(delTerm, docs, analyzer); + public void replaceDocuments(Term delTerm, Iterable docs) throws IOException { + replaceDocuments(delTerm, docs, analyzer); } /** + * Replaced by {@link #replaceDocuments(Term, Iterable)}. + * @deprecated use {@link #replaceDocuments(Term, Iterable)}. + */ + @Deprecated + public void updateDocuments(Term delTerm, Iterable docs) throws IOException { + replaceDocuments(delTerm, docs, analyzer); + } + + /** * Atomically deletes documents matching the provided * delTerm and adds a block of documents, analyzed using * the provided analyzer, with sequentially @@ -1215,7 +1228,7 @@ * * @lucene.experimental */ - public void updateDocuments(Term delTerm, Iterable docs, Analyzer analyzer) throws IOException { + public void replaceDocuments(Term delTerm, Iterable docs, Analyzer analyzer) throws IOException { ensureOpen(); try { boolean success = false; @@ -1239,6 +1252,95 @@ } /** + * Replaced by {@link #replaceDocuments(Term, Iterable, Analyzer)}. + * @deprecated use {@link #replaceDocuments(Term, Iterable, Analyzer)}. + */ + @Deprecated + public void updateDocuments(Term delTerm, Iterable docs, Analyzer analyzer) throws IOException { + replaceDocuments(delTerm, docs, analyzer); + } + + /** + * Update fields of documents containing the given term. The operation defines + * whether the new fields are either: + *
      + *
    • + * used as a new document which replaces the existing documents (using + * {@link Operation#REPLACE_DOCS}), + *
    • + * added to the existing documents replacing existing fields with the same name (using + * {@link Operation#REPLACE_FIELDS}), or + *
    • + * added to the existing documents without replacing existing fields (using + * {@link Operation#ADD_FIELDS}). + *
    + * + * @throws CorruptIndexException if the index is corrupt + * @throws IOException if there is a low-level IO error + * + * @lucene.experimental + */ + public void updateFields(Term term, FieldsUpdate.Operation operation, IndexDocument fields) throws IOException { + updateFields(term, operation, fields, analyzer); + } + + /** + * Update fields of documents containing the given term, using the provided + * analyzer instead of the value of {@link #getAnalyzer()}. The operation + * defines whether the new fields are either: + *
      + *
    • + * used as a new document which replaces the existing documents (using + * {@link Operation#REPLACE_DOCS}), + *
    • + * added to the existing documents replacing existing fields with the same + * name (using {@link Operation#REPLACE_FIELDS}), or + *
    • + * added to the existing documents without replacing existing fields (using + * {@link Operation#ADD_FIELDS}). + *
    + * + * @throws CorruptIndexException if the index is corrupt + * @throws IOException if there is a low-level IO error + * + * @lucene.experimental + */ + public void updateFields(Term term, FieldsUpdate.Operation operation, + IndexDocument fields, Analyzer analyzer) + throws IOException { + switch (operation) { + case REPLACE_DOCS: + replaceDocument(term, fields, analyzer); + return; + + case REPLACE_FIELDS: + case ADD_FIELDS: + ensureOpen(); + try { + boolean success = false; + boolean anySegmentFlushed = false; + try { + anySegmentFlushed = docWriter.updateFields(term, + new FieldsUpdate(operation, fields, analyzer)); + success = true; + } finally { + if (!success) { + if (infoStream.isEnabled("IW")) { + infoStream.message("IW", "hit exception updating document"); + } + } + } + + if (anySegmentFlushed) { + maybeMerge(); + } + } catch (OutOfMemoryError oom) { + handleOOM(oom, "updateDocument"); + } + } + } + +/** * Deletes the document(s) containing term. * *

    NOTE: if this method hits an OutOfMemoryError @@ -1416,9 +1518,23 @@ * @throws CorruptIndexException if the index is corrupt * @throws IOException if there is a low-level IO error */ + public void replaceDocument(Term term, IndexDocument doc) throws IOException { + ensureOpen(); + replaceDocument(term, doc, getAnalyzer()); + } + + /** + * Replaced by {@link #replaceDocument(Term, IndexDocument)}. If you wish to + * update fields of existing documents use + * {@link #updateFields(Term, Operation, IndexDocument)}. + * + * @deprecated use {@link #replaceDocument(Term, IndexDocument)} or + * {@link #updateFields(Term, Operation, IndexDocument)}. + */ + @Deprecated public void updateDocument(Term term, IndexDocument doc) throws IOException { ensureOpen(); - updateDocument(term, doc, getAnalyzer()); + replaceDocument(term, doc, getAnalyzer()); } /** @@ -1439,7 +1555,7 @@ * @throws CorruptIndexException if the index is corrupt * @throws IOException if there is a low-level IO error */ - public void updateDocument(Term term, IndexDocument doc, Analyzer analyzer) + public void replaceDocument(Term term, IndexDocument doc, Analyzer analyzer) throws IOException { ensureOpen(); try { @@ -1464,6 +1580,21 @@ } } + /** + * Replaced by {@link #replaceDocument(Term, IndexDocument, Analyzer)}. If you + * wish to update fields of existing documents use + * {@link #updateFields(Term, Operation, IndexDocument, Analyzer)}. + * + * @deprecated use {@link #replaceDocument(Term, IndexDocument, Analyzer)} or + * {@link #updateFields(Term, Operation, IndexDocument, Analyzer)} + * . + */ + @Deprecated + public void updateDocument(Term term, IndexDocument doc, Analyzer analyzer) + throws IOException { + replaceDocument(term, doc, analyzer); + } + // for test purpose final synchronized int getSegmentCount(){ return segmentInfos.size(); @@ -2140,8 +2271,26 @@ deleter.checkpoint(segmentInfos, false); } + void writeSegmentUpdates(SegmentInfoPerCommit segment, + UpdatedSegmentData updates, IOContext context) throws IOException { + docWriter.writeUpdatedSegment(updates, segment); + +// // put updates segment in compound file if required +// if (useCompoundFile(segment)) { +// +// // Now build compound file +// Collection oldFiles = createCompoundFile(infoStream, directory, +// MergeState.CheckAbort.NONE, segment.info, context, segment.getNextUpdateGen()); +// +// synchronized(this) { +// deleter.deleteNewFiles(oldFiles); +// } +// } + segment.advanceUpdateGen(); + } + synchronized void publishFrozenDeletes(FrozenBufferedDeletes packet) { - assert packet != null && packet.any(); + assert packet != null && (packet.anyDeletes() || packet.anyUpdates()); synchronized (bufferedDeletesStream) { bufferedDeletesStream.push(packet); } @@ -2159,13 +2308,13 @@ infoStream.message("IW", "publishFlushedSegment"); } - if (globalPacket != null && globalPacket.any()) { + if (globalPacket != null && (globalPacket.anyDeletes() || globalPacket.anyUpdates())) { bufferedDeletesStream.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()) { + if (packet != null && (packet.anyDeletes())) { nextGen = bufferedDeletesStream.push(packet); } else { // Since we don't have a delete packet to apply we can get a new @@ -2400,7 +2549,7 @@ } } - SegmentInfoPerCommit infoPerCommit = new SegmentInfoPerCommit(info, 0, -1L); + SegmentInfoPerCommit infoPerCommit = new SegmentInfoPerCommit(info, 0, -1L, -1L); info.setFiles(new HashSet(trackingDir.getCreatedFiles())); trackingDir.getCreatedFiles().clear(); @@ -2421,7 +2570,7 @@ if (useCompoundFile) { Collection filesToDelete = infoPerCommit.files(); try { - createCompoundFile(infoStream, directory, MergeState.CheckAbort.NONE, info, context); + createCompoundFile(infoStream, directory, MergeState.CheckAbort.NONE, info, context, -1); } finally { // delete new non cfs files directly: they were never // registered with IFD @@ -2438,15 +2587,16 @@ // above: success = false; try { - codec.segmentInfoFormat().getSegmentInfoWriter().write(trackingDir, info, mergeState.fieldInfos, context); + SegmentInfoWriter segmentInfoWriter = codec.segmentInfoFormat().getSegmentInfoWriter(); + segmentInfoWriter.write(trackingDir, info, mergeState.fieldInfos, context); success = true; } finally { - if (!success) { - synchronized(this) { - deleter.refresh(info.name); + if (!success) { + synchronized(this) { + deleter.refresh(info.name); + } } } - } info.addFiles(trackingDir.getCreatedFiles()); @@ -2487,7 +2637,7 @@ SegmentInfo newInfo = new SegmentInfo(directory, info.info.getVersion(), segName, info.info.getDocCount(), info.info.getUseCompoundFile(), info.info.getCodec(), info.info.getDiagnostics(), attributes); - SegmentInfoPerCommit newInfoPerCommit = new SegmentInfoPerCommit(newInfo, info.getDelCount(), info.getDelGen()); + SegmentInfoPerCommit newInfoPerCommit = new SegmentInfoPerCommit(newInfo, info.getDelCount(), info.getDelGen(), -1L); Set segFiles = new HashSet(); @@ -2507,7 +2657,8 @@ try { - newInfo.getCodec().segmentInfoFormat().getSegmentInfoWriter().write(trackingDir, newInfo, fis, context); + SegmentInfoWriter segmentInfoWriter = newInfo.getCodec().segmentInfoFormat().getSegmentInfoWriter(); + segmentInfoWriter.write(trackingDir, newInfo, fis, context); final Collection siFiles = trackingDir.getCreatedFiles(); @@ -2527,14 +2678,14 @@ } success = true; } finally { - if (!success) { - for(String file : newInfo.files()) { - try { - directory.deleteFile(file); - } catch (Throwable t) { + if (!success) { + for(String file : newInfo.files()) { + try { + directory.deleteFile(file); + } catch (Throwable t) { + } } } - } } return newInfoPerCommit; @@ -3169,6 +3320,7 @@ } deleter.deletePendingFiles(); + deleter.deletePendingFiles(); if (infoStream.isEnabled("IW")) { infoStream.message("IW", "after commitMerge: " + segString()); @@ -3430,7 +3582,7 @@ // names. final String mergeSegmentName = newSegmentName(); SegmentInfo si = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, mergeSegmentName, -1, false, codec, null, null); - merge.info = new SegmentInfoPerCommit(si, 0, -1L); + merge.info = new SegmentInfoPerCommit(si, 0, -1L, -1L); // Lock order: IW -> BD bufferedDeletesStream.prune(segmentInfos); @@ -3606,7 +3758,7 @@ // fix the reader's live docs and del count assert delCount > reader.numDeletedDocs(); // beware of zombies - SegmentReader newReader = new SegmentReader(info, reader.core, liveDocs, info.info.getDocCount() - delCount); + SegmentReader newReader = new SegmentReader(info, context, reader.core, liveDocs, info.info.getDocCount() - delCount); boolean released = false; try { rld.release(reader); @@ -3672,7 +3824,7 @@ Collection filesToRemove = merge.info.files(); try { - filesToRemove = createCompoundFile(infoStream, directory, checkAbort, merge.info.info, context); + filesToRemove = createCompoundFile(infoStream, directory, checkAbort, merge.info.info, context, -1); success = true; } catch (IOException ioe) { synchronized(this) { @@ -3735,7 +3887,8 @@ // above: boolean success2 = false; try { - codec.segmentInfoFormat().getSegmentInfoWriter().write(directory, merge.info.info, mergeState.fieldInfos, context); + SegmentInfoWriter segmentInfoWriter = codec.segmentInfoFormat().getSegmentInfoWriter(); + segmentInfoWriter.write(directory, merge.info.info, mergeState.fieldInfos, context); success2 = true; } finally { if (!success2) { @@ -4141,16 +4294,36 @@ * info.files(). While, generally, this may include separate norms and * deletion files, this SegmentInfo must not reference such files when this * method is called, because they are not allowed within a compound file. + * The value of updateGen for a base segment must be negative. */ - static final Collection createCompoundFile(InfoStream infoStream, Directory directory, CheckAbort checkAbort, final SegmentInfo info, IOContext context) + static final Collection createCompoundFile(InfoStream infoStream, Directory directory, CheckAbort checkAbort, final SegmentInfo info, IOContext context, long updateGen) throws IOException { - final String fileName = IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION); + String fileName = IndexFileNames.fileNameFromGeneration(info.name, IndexFileNames.COMPOUND_FILE_EXTENSION, updateGen); + if (fileName == null) { + fileName = IndexFileNames.segmentFileName(info.name, "", + IndexFileNames.COMPOUND_FILE_EXTENSION); + } if (infoStream.isEnabled("IW")) { infoStream.message("IW", "create compound file " + fileName); } // Now merge all added files - Collection files = info.files(); + String prefix = info.name; + Collection files = null; + if (updateGen < 0) { + files = info.files(); + } else { + // TODO : SY : quick and dirty, better solve by aggregating files in advance + files = new ArrayList(); + prefix = IndexFileNames.fileNameFromGeneration(info.name, "", updateGen); + String[] allFiles = directory.listAll(); + for (int i = 0; i < allFiles.length; i++) { + if (allFiles[i].startsWith(prefix)) { + files.add(allFiles[i]); + } + } + } + final String cfeFileName = IndexFileNames.segmentFileName(prefix, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION); CompoundFileDirectory cfsDir = new CompoundFileDirectory(directory, fileName, context, true); IOException prior = null; try { @@ -4172,7 +4345,7 @@ } catch (Throwable t) { } try { - directory.deleteFile(IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION)); + directory.deleteFile(cfeFileName); } catch (Throwable t) { } } @@ -4181,8 +4354,10 @@ // Replace all previous files with the CFS/CFE files: Set siFiles = new HashSet(); + siFiles.addAll(info.files()); + siFiles.removeAll(files); siFiles.add(fileName); - siFiles.add(IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION)); + siFiles.add(cfeFileName); info.setFiles(siFiles); return files; Index: lucene/core/src/java/org/apache/lucene/index/NormsConsumerPerField.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/NormsConsumerPerField.java (revision 1406075) +++ lucene/core/src/java/org/apache/lucene/index/NormsConsumerPerField.java (working copy) @@ -71,7 +71,7 @@ if (consumer == null) { assert fieldInfo.getNormType() == null || fieldInfo.getNormType() == type; fieldInfo.setNormValueType(type); - consumer = parent.newConsumer(docState.docWriter.newPerDocWriteState(""), fieldInfo, type); + consumer = parent.newConsumer(docState.docWriter.newPerDocWriteState(), fieldInfo, type); this.initType = type; } if (initType != type) { Index: lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java (revision 1406075) +++ lucene/core/src/java/org/apache/lucene/index/ReadersAndLiveDocs.java (working copy) @@ -20,8 +20,11 @@ import java.io.IOException; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.LiveDocsFormat; +import org.apache.lucene.codecs.SegmentInfoWriter; import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FlushInfo; import org.apache.lucene.store.IOContext; import org.apache.lucene.util.Bits; import org.apache.lucene.util.MutableBits; @@ -63,6 +66,8 @@ // liveDocs vs when we loaded it or last wrote it: private int pendingDeleteCount; + private UpdatedSegmentData liveUpdates; + // True if the current liveDocs is referenced by an // external NRT reader: private boolean shared; @@ -207,7 +212,7 @@ } shared = true; if (liveDocs != null) { - return new SegmentReader(reader.getSegmentInfo(), reader.core, liveDocs, info.info.getDocCount() - info.getDelCount() - pendingDeleteCount); + return new SegmentReader(reader.getSegmentInfo(), context, reader.core, liveDocs, info.info.getDocCount() - info.getDelCount() - pendingDeleteCount); } else { assert reader.getLiveDocs() == liveDocs; reader.incRef(); @@ -290,6 +295,23 @@ } } + public synchronized void setLiveUpdates(UpdatedSegmentData updatedSegmentData) { + assert liveUpdates == null; + liveUpdates = updatedSegmentData; + } + + public synchronized boolean writeLiveUpdates(Directory directory, Codec codec) + throws IOException { + if (liveUpdates == null || !liveUpdates.hasUpdates()) { + return false; + } + IOContext context = new IOContext(new FlushInfo(info.info.getDocCount(), + info.info.sizeInBytes())); + writer.writeSegmentUpdates(info, liveUpdates, context); + + return true; + } + @Override public String toString() { return "ReadersAndLiveDocs(seg=" + info + " pendingDeleteCount=" + pendingDeleteCount + " shared=" + shared + ")"; Index: lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (revision 1406075) +++ lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (working copy) @@ -80,29 +80,38 @@ private final Set coreClosedListeners = Collections.synchronizedSet(new LinkedHashSet()); - SegmentCoreReaders(SegmentReader owner, Directory dir, SegmentInfoPerCommit si, IOContext context, int termsIndexDivisor) throws IOException { + SegmentCoreReaders(SegmentReader owner, SegmentInfoPerCommit si, long updageGen, IOContext context, int termsIndexDivisor) throws IOException { if (termsIndexDivisor == 0) { throw new IllegalArgumentException("indexDivisor must be < 0 (don't load terms index) or greater than 0 (got 0)"); } + + final SegmentInfo info; + if (updageGen == -1) { + info = si.info; + } else { + info = new SegmentInfo(si.info, updageGen); + } - final Codec codec = si.info.getCodec(); + Directory dir = info.dir; + + final Codec codec = info.getCodec(); final Directory cfsDir; // confusing name: if (cfs) its the cfsdir, otherwise its the segment's directory. boolean success = false; try { - if (si.info.getUseCompoundFile()) { - cfsDir = cfsReader = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(si.info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false); + if (info.getUseCompoundFile()) { + cfsDir = cfsReader = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false); } else { cfsReader = null; cfsDir = dir; } - fieldInfos = codec.fieldInfosFormat().getFieldInfosReader().read(cfsDir, si.info.name, IOContext.READONCE); + fieldInfos = codec.fieldInfosFormat().getFieldInfosReader().read(cfsDir, info.name, IOContext.READONCE); this.termsIndexDivisor = termsIndexDivisor; final PostingsFormat format = codec.postingsFormat(); - final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si.info, fieldInfos, context, termsIndexDivisor); + final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, info, fieldInfos, context, termsIndexDivisor); // Ask codec for its Fields fields = format.fieldsProducer(segmentReadState); assert fields != null; @@ -112,10 +121,10 @@ norms = codec.normsFormat().docsProducer(segmentReadState); perDocProducer = codec.docValuesFormat().docsProducer(segmentReadState); - fieldsReaderOrig = si.info.getCodec().storedFieldsFormat().fieldsReader(cfsDir, si.info, fieldInfos, context); + fieldsReaderOrig = info.getCodec().storedFieldsFormat().fieldsReader(cfsDir, info, fieldInfos, context); if (fieldInfos.hasVectors()) { // open term vector files only as needed - termVectorsReaderOrig = si.info.getCodec().termVectorsFormat().vectorsReader(cfsDir, si.info, fieldInfos, context); + termVectorsReaderOrig = info.getCodec().termVectorsFormat().vectorsReader(cfsDir, info, fieldInfos, context); } else { termVectorsReaderOrig = null; } Index: lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (revision 1406075) +++ lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (working copy) @@ -100,6 +100,17 @@ this.attributes = attributes; } + SegmentInfo(SegmentInfo info, long gen) { + this.dir = info.dir; + this.version = info.version; + this.name = IndexFileNames.updatedSegmentFileNameFromGeneration(info.name, gen); + this.docCount = info.docCount; + this.isCompoundFile = info.isCompoundFile; + this.codec = info.codec; + this.diagnostics = info.diagnostics; + this.attributes = info.attributes; + } + /** * Returns total size in bytes of all of files used by * this segment. Note that this will not include any live @@ -273,6 +284,14 @@ sizeInBytes = -1; } + /** Remove this file from the set of files written for this + * segment. */ + public void removeFile(String file) { + checkFileNames(Collections.singleton(file)); + setFiles.remove(file); + sizeInBytes = -1; + } + private void checkFileNames(Collection files) { Matcher m = IndexFileNames.CODEC_FILE_PATTERN.matcher(""); for (String file : files) { Index: lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java (revision 1406075) +++ lucene/core/src/java/org/apache/lucene/index/SegmentInfoPerCommit.java (working copy) @@ -40,6 +40,9 @@ // are no deletes yet): private long delGen; + // Generation number of updates (-1 if there are no updates yet): + private long updateGen; + private volatile long sizeInBytes = -1; /** Sole constructor. @@ -48,10 +51,11 @@ * @param delGen deletion generation number (used to name deletion files) **/ - public SegmentInfoPerCommit(SegmentInfo info, int delCount, long delGen) { + public SegmentInfoPerCommit(SegmentInfo info, int delCount, long delGen, long updateGen) { this.info = info; this.delCount = delCount; this.delGen = delGen; + this.updateGen = updateGen; } void advanceDelGen() { @@ -63,6 +67,15 @@ sizeInBytes = -1; } + void advanceUpdateGen() { + if (updateGen == -1) { + updateGen = 1; + } else { + updateGen++; + } + sizeInBytes = -1; + } + /** Returns total size in bytes of all files for this * segment. */ public long sizeInBytes() throws IOException { @@ -106,6 +119,7 @@ sizeInBytes = -1; } + /** * Sets the generation number of the live docs file. * @see #getDelGen() @@ -117,14 +131,14 @@ /** Returns true if there are any deletions for the * segment at this commit. */ - public boolean hasDeletions() { + public boolean hasDeletions() { return delGen != -1; } - /** - * Returns the next available generation number - * of the live docs file. - */ + /** + * Returns the next available generation number + * of the live docs file. + */ public long getNextDelGen() { if (delGen == -1) { return 1; @@ -169,6 +183,31 @@ @Override public SegmentInfoPerCommit clone() { - return new SegmentInfoPerCommit(info, delCount, delGen); + return new SegmentInfoPerCommit(info, delCount, delGen, updateGen); } + + public void setUpdateGen(long updateGen) { + this.updateGen = updateGen; + sizeInBytes = -1; + } + + public boolean hasUpdates() { + return updateGen != -1; + } + + public long getNextUpdateGen() { + if (updateGen == -1) { + return 1; + } + return updateGen + 1; + } + + public long getUpdateGen() { + return updateGen; + } + + void clearUpdateGen() { + updateGen = -1; + sizeInBytes = -1; + } } Index: lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (revision 1406075) +++ lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (working copy) @@ -305,7 +305,8 @@ if (delCount < 0 || delCount > info.getDocCount()) { throw new CorruptIndexException("invalid deletion count: " + delCount + " (resource: " + input + ")"); } - add(new SegmentInfoPerCommit(info, delCount, delGen)); + long updateGen = input.readLong(); + add(new SegmentInfoPerCommit(info, delCount, delGen, updateGen)); } userData = input.readStringStringMap(); @@ -373,6 +374,7 @@ segnOutput.writeString(si.getCodec().getName()); segnOutput.writeLong(siPerCommit.getDelGen()); segnOutput.writeInt(siPerCommit.getDelCount()); + segnOutput.writeLong(siPerCommit.getUpdateGen()); assert si.dir == directory; assert siPerCommit.getDelCount() <= si.getDocCount(); Index: lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (revision 1406075) +++ lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (working copy) @@ -103,7 +103,7 @@ assert numMerged == mergeState.segmentInfo.getDocCount(); final SegmentWriteState segmentWriteState = new SegmentWriteState(mergeState.infoStream, directory, mergeState.segmentInfo, - mergeState.fieldInfos, termIndexInterval, null, context); + 0, mergeState.fieldInfos, termIndexInterval, null, null, context); mergeTerms(segmentWriteState); mergePerDoc(segmentWriteState); Index: lucene/core/src/java/org/apache/lucene/index/SegmentReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (revision 1408451) +++ lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (working copy) @@ -18,48 +18,68 @@ */ import java.io.IOException; +import java.util.HashMap; +import java.util.Map; -import org.apache.lucene.store.Directory; +import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PerDocProducer; import org.apache.lucene.codecs.StoredFieldsReader; import org.apache.lucene.codecs.TermVectorsReader; -import org.apache.lucene.search.FieldCache; // javadocs +import org.apache.lucene.search.FieldCache; +import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.util.Bits; +// javadocs + /** - * IndexReader implementation over a single segment. + * IndexReader implementation over a single segment. *

    - * Instances pointing to the same segment (but with different deletes, etc) - * may share the same core data. + * Instances pointing to the same segment (but with different deletes, etc) may + * share the same core data. + * * @lucene.experimental */ public final class SegmentReader extends AtomicReader { - + private final SegmentInfoPerCommit si; private final Bits liveDocs; - + // Normally set to si.docCount - si.delDocCount, unless we // were created as an NRT reader from IW, in which case IW // tells us the docCount: private final int numDocs; - + final SegmentCoreReaders core; - + + private final SegmentCoreReaders[] updates; + private final IOContext context; + private FieldInfos fieldInfos; + private StoredFieldsReader fieldsReader; + private TermVectorsReader termVectorsReader; + private Map replacementsMap; + /** * Constructs a new SegmentReader with a new core. - * @throws CorruptIndexException if the index is corrupt - * @throws IOException if there is a low-level IO error + * + * @throws CorruptIndexException + * if the index is corrupt + * @throws IOException + * if there is a low-level IO error */ // TODO: why is this public? - public SegmentReader(SegmentInfoPerCommit si, int termInfosIndexDivisor, IOContext context) throws IOException { + public SegmentReader(SegmentInfoPerCommit si, int termInfosIndexDivisor, + IOContext context) throws IOException { this.si = si; - core = new SegmentCoreReaders(this, si.info.dir, si, context, termInfosIndexDivisor); + this.context = context; + core = new SegmentCoreReaders(this, si, -1, context, termInfosIndexDivisor); + updates = initUpdates(si, termInfosIndexDivisor, context); boolean success = false; try { if (si.hasDeletions()) { // NOTE: the bitvector is stored using the regular directory, not cfs - liveDocs = si.info.getCodec().liveDocsFormat().readLiveDocs(directory(), si, new IOContext(IOContext.READ, true)); + liveDocs = si.info.getCodec().liveDocsFormat() + .readLiveDocs(directory(), si, new IOContext(IOContext.READ, true)); } else { assert si.getDelCount() == 0; liveDocs = null; @@ -68,112 +88,221 @@ success = true; } finally { // With lock-less commits, it's entirely possible (and - // fine) to hit a FileNotFound exception above. In + // fine) to hit a FileNotFound exception above. In // this case, we want to explicitly close any subset // of things that were opened so that we don't have to // wait for a GC to do so. if (!success) { core.decRef(); + if (updates != null) { + for (int i = 0; i < updates.length; i++) { + updates[i].decRef(); + } + } } } } - - /** Create new SegmentReader sharing core from a previous - * SegmentReader and loading new live docs from a new - * deletes file. Used by openIfChanged. */ - SegmentReader(SegmentInfoPerCommit si, SegmentCoreReaders core, IOContext context) throws IOException { - this(si, core, - si.info.getCodec().liveDocsFormat().readLiveDocs(si.info.dir, si, context), - si.info.getDocCount() - si.getDelCount()); + + /** + * Create new SegmentReader sharing core from a previous SegmentReader and + * loading new live docs from a new deletes file. Used by openIfChanged. + */ + SegmentReader(SegmentInfoPerCommit si, SegmentCoreReaders core, + IOContext context) throws IOException { + this(si, context, core, si.info.getCodec().liveDocsFormat() + .readLiveDocs(si.info.dir, si, context), si.info.getDocCount() + - si.getDelCount()); } - - /** Create new SegmentReader sharing core from a previous - * SegmentReader and using the provided in-memory - * liveDocs. Used by IndexWriter to provide a new NRT - * reader */ - SegmentReader(SegmentInfoPerCommit si, SegmentCoreReaders core, Bits liveDocs, int numDocs) { + + /** + * Create new SegmentReader sharing core from a previous SegmentReader and + * using the provided in-memory liveDocs. Used by IndexWriter to provide a new + * NRT reader + */ + SegmentReader(SegmentInfoPerCommit si, IOContext context, + SegmentCoreReaders core, Bits liveDocs, int numDocs) { this.si = si; + this.context = context; this.core = core; core.incRef(); - + this.updates = null; // TODO : SY : handle this for NRT updates + assert liveDocs != null; this.liveDocs = liveDocs; - + this.numDocs = numDocs; } - + + private SegmentCoreReaders[] initUpdates(SegmentInfoPerCommit si, + int termInfosIndexDivisor, IOContext context) throws IOException { + SegmentCoreReaders[] updates; + if (si.hasUpdates()) { + updates = new SegmentCoreReaders[(int) si.getUpdateGen()]; + for (int i = 0; i < updates.length; i++) { + updates[i] = new SegmentCoreReaders(this, si, i + 1, context, + termInfosIndexDivisor); + } + } else { + updates = null; + } + return updates; + } + @Override public Bits getLiveDocs() { ensureOpen(); return liveDocs; } - + @Override protected void doClose() throws IOException { - //System.out.println("SR.close seg=" + si); + // System.out.println("SR.close seg=" + si); core.decRef(); + if (updates != null) { + for (int i = 0; i < updates.length; i++) { + updates[i].decRef(); + } + } } - + @Override public boolean hasDeletions() { // Don't call ensureOpen() here (it could affect performance) return liveDocs != null; } - + @Override public FieldInfos getFieldInfos() { ensureOpen(); - return core.fieldInfos; + if (updates == null) { + return core.fieldInfos; + } + + // need to create FieldInfos combining core and updates infos + final FieldInfos.Builder builder = new FieldInfos.Builder(); + builder.add(core.fieldInfos); + for (final SegmentCoreReaders update : updates) { + builder.add(update.fieldInfos); + } + fieldInfos = builder.finish(); + return fieldInfos; } - - /** Expert: retrieve thread-private {@link - * StoredFieldsReader} - * @lucene.internal */ - public StoredFieldsReader getFieldsReader() { + + /** + * Expert: retrieve thread-private {@link StoredFieldsReader} + * + * @lucene.internal + */ + public StoredFieldsReader getFieldsReader() throws IOException { ensureOpen(); - return core.fieldsReaderLocal.get(); + if (fieldsReader == null) { + if (updates == null) { + fieldsReader = core.fieldsReaderLocal.get(); + } else { + // generate readers array + StoredFieldsReader[] allReaders = new StoredFieldsReader[updates.length + 1]; + allReaders[0] = core.fieldsReaderLocal.get(); + for (int i = 0; i < updates.length; i++) { + allReaders[i + 1] = updates[i].fieldsReaderLocal.get(); + } + + // generate replacements map + if (replacementsMap == null) { + generateReplacementsMap(); + } + fieldsReader = new StackedStoredFieldsReader(allReaders, replacementsMap); + } + } + return fieldsReader; } + + private void generateReplacementsMap() throws IOException { + replacementsMap = new HashMap(); + addReplacements(core.fields, replacementsMap); + for (int i = 0; i < updates.length; i++) { + addReplacements(updates[i].fields, replacementsMap); + } + } + private void addReplacements(FieldsProducer fields, + Map replacements) throws IOException { + for (String field : fields) { + if (!replacements.containsKey(field)) { + replacements.put(field, + si.info.getCodec().generationReplacementsFormat() + .readGenerationReplacements(field, directory(), si, context)); + } + } + } + @Override - public void document(int docID, StoredFieldVisitor visitor) throws IOException { + public void document(int docID, StoredFieldVisitor visitor) + throws IOException { checkBounds(docID); - getFieldsReader().visitDocument(docID, visitor); + getFieldsReader().visitDocument(docID, visitor, null); } - + @Override - public Fields fields() { + public Fields fields() throws IOException { ensureOpen(); - return core.fields; + if (updates == null || updates.length == 0) { + return core.fields; + } + + // generate fields array + Fields[] fieldsArray = new Fields[updates.length + 1]; + fieldsArray[0] = core.fields; + for (int i = 0; i < updates.length; i++) { + fieldsArray[i + 1] = updates[i].fields; + } + + // generate replacements map + if (replacementsMap == null) { + generateReplacementsMap(); + } + + return new StackedFields(fieldsArray, replacementsMap, -1); } - + @Override public int numDocs() { // Don't call ensureOpen() here (it could affect performance) return numDocs; } - + @Override public int maxDoc() { // Don't call ensureOpen() here (it could affect performance) return si.info.getDocCount(); } - - /** Expert: retrieve thread-private {@link - * TermVectorsReader} - * @lucene.internal */ - public TermVectorsReader getTermVectorsReader() { + + /** + * Expert: retrieve thread-private {@link TermVectorsReader} + * + * @lucene.internal + */ + public TermVectorsReader getTermVectorsReader() throws IOException { ensureOpen(); - return core.termVectorsLocal.get(); - } + if (termVectorsReader == null) { + if (updates == null) { + termVectorsReader = core.termVectorsLocal.get(); + } else { + // generate readers array + TermVectorsReader[] allReaders = new TermVectorsReader[updates.length + 1]; + allReaders[0] = core.termVectorsLocal.get(); + for (int i = 0; i < updates.length; i++) { + allReaders[i + 1] = updates[i].termVectorsLocal.get(); + } + + // generate replacements map + if (replacementsMap == null) { + generateReplacementsMap(); + } - @Override - public Fields getTermVectors(int docID) throws IOException { - TermVectorsReader termVectorsReader = getTermVectorsReader(); - if (termVectorsReader == null) { - return null; + termVectorsReader = new StackedTermVectorsReader(allReaders, replacementsMap); + } } - checkBounds(docID); - return termVectorsReader.get(docID); + return termVectorsReader; } private void checkBounds(int docID) { @@ -183,10 +312,20 @@ } @Override + public Fields getTermVectors(int docID) throws IOException { + TermVectorsReader termVectorsReader = getTermVectorsReader(); + if (termVectorsReader == null) { + return null; + } + checkBounds(docID); + return termVectorsReader.get(docID); } + + @Override public String toString() { // SegmentInfo.toString takes dir and number of // *pending* deletions; so we reverse compute that here: - return si.toString(si.info.dir, si.info.getDocCount() - numDocs - si.getDelCount()); + return si.toString(si.info.dir, + si.info.getDocCount() - numDocs - si.getDelCount()); } /** @@ -202,7 +341,7 @@ SegmentInfoPerCommit getSegmentInfo() { return si; } - + /** Returns the directory this index resides in. */ public Directory directory() { // Don't ensureOpen here -- in certain cases, when a @@ -210,63 +349,81 @@ // this method on the closed original reader return si.info.dir; } - + // This is necessary so that cloned SegmentReaders (which // share the underlying postings data) will map to the - // same entry in the FieldCache. See LUCENE-1579. + // same entry in the FieldCache. See LUCENE-1579. @Override public Object getCoreCacheKey() { return core; } - + @Override public Object getCombinedCoreAndDeletesKey() { return this; } - - /** Returns term infos index divisor originally passed to - * {@link #SegmentReader(SegmentInfoPerCommit, int, IOContext)}. */ + + /** + * Returns term infos index divisor originally passed to + * {@link #SegmentReader(SegmentInfoPerCommit, int, IOContext)}. + */ public int getTermInfosIndexDivisor() { + // TODO : SY : add updates return core.termsIndexDivisor; } @Override public DocValues docValues(String field) throws IOException { ensureOpen(); - final PerDocProducer perDoc = core.perDocProducer; - if (perDoc == null) { - return null; + if (updates != null) { + for (int i = updates.length - 1; i >= 0; i--) { + DocValues docValues = getDocValues(field, updates[i].perDocProducer); + if (docValues != null) { + return docValues; + } + } } - return perDoc.docValues(field); + return getDocValues(field, core.perDocProducer); } @Override public DocValues normValues(String field) throws IOException { ensureOpen(); - final PerDocProducer perDoc = core.norms; + if (updates != null) { + for (int i = updates.length - 1; i >= 0; i--) { + DocValues docValues = getDocValues(field, updates[i].norms); + if (docValues != null) { + return docValues; + } + } + } + return getDocValues(field, core.norms); + } + + public DocValues getDocValues(String field, final PerDocProducer perDoc) + throws IOException { if (perDoc == null) { return null; } return perDoc.docValues(field); } - /** - * Called when the shared core for this SegmentReader - * is closed. + * Called when the shared core for this SegmentReader is closed. *

    - * This listener is called only once all SegmentReaders - * sharing the same core are closed. At this point it - * is safe for apps to evict this reader from any caches - * keyed on {@link #getCoreCacheKey}. This is the same - * interface that {@link FieldCache} uses, internally, - * to evict entries.

    + * This listener is called only once all SegmentReaders sharing the same core + * are closed. At this point it is safe for apps to evict this reader from any + * caches keyed on {@link #getCoreCacheKey}. This is the same interface that + * {@link FieldCache} uses, internally, to evict entries. + *

    * * @lucene.experimental */ public static interface CoreClosedListener { - /** Invoked when the shared core of the provided {@link - * SegmentReader} has closed. */ + /** + * Invoked when the shared core of the provided {@link SegmentReader} has + * closed. + */ public void onClose(SegmentReader owner); } Index: lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java (revision 1406075) +++ lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java (working copy) @@ -39,6 +39,7 @@ /** {@link SegmentInfo} describing this segment. */ public final SegmentInfo segmentInfo; + public final int updateGen; /** {@link FieldInfos} describing all fields in this * segment. */ @@ -55,10 +56,20 @@ * deleted. */ public final BufferedDeletes segDeletes; + /** Updates to apply while we are flushing the segment. A + * Term is enrolled in here if it was used in update at one + * point, and it's mapped to the docIDUpto, meaning any + * docID < docIDUpto containing this term should be + * deleted. */ + public final BufferedUpdates segUpdates; + /** {@link MutableBits} recording live documents; this is * only set if there is one or more deleted documents. */ public MutableBits liveDocs; + // Lazily created: + public UpdatedSegmentData liveUpdates; + /** Unique suffix for any postings files written for this * segment. {@link PerFieldPostingsFormat} sets this for * each of the postings formats it wraps. If you create @@ -79,12 +90,16 @@ public final IOContext context; /** Sole constructor. */ - public SegmentWriteState(InfoStream infoStream, Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos, - int termIndexInterval, BufferedDeletes segDeletes, IOContext context) { + public SegmentWriteState(InfoStream infoStream, Directory directory, + SegmentInfo segmentInfo, int updateGen, FieldInfos fieldInfos, + int termIndexInterval, BufferedDeletes segDeletes, + BufferedUpdates segUpdates, IOContext context) { this.infoStream = infoStream; this.segDeletes = segDeletes; + this.segUpdates = segUpdates; this.directory = directory; this.segmentInfo = segmentInfo; + this.updateGen = updateGen; this.fieldInfos = fieldInfos; this.termIndexInterval = termIndexInterval; segmentSuffix = ""; @@ -98,11 +113,13 @@ infoStream = state.infoStream; directory = state.directory; segmentInfo = state.segmentInfo; + updateGen = state.updateGen; fieldInfos = state.fieldInfos; termIndexInterval = state.termIndexInterval; context = state.context; this.segmentSuffix = segmentSuffix; segDeletes = state.segDeletes; + segUpdates = state.segUpdates; delCountOnFlush = state.delCountOnFlush; } } Index: lucene/core/src/java/org/apache/lucene/index/StackedDocsEnum.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/StackedDocsEnum.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/index/StackedDocsEnum.java (working copy) @@ -0,0 +1,187 @@ +package org.apache.lucene.index; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.PriorityQueue; + +/* + * 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. + */ + +public class StackedDocsEnum extends DocsAndPositionsEnum { + + final private List active; + final private PriorityQueue queueByDocId; + final private PriorityQueue queueByIndex; + final private FieldGenerationReplacements replacements; + + private static final FieldGenerationReplacements NO_REPLACEMENTS = new FieldGenerationReplacements(); + + public StackedDocsEnum(Map activeMap, + FieldGenerationReplacements replacements) { + active = new ArrayList(); + for (DocsEnum docsEnum : activeMap.keySet()) { + active.add(new DocsEnumWithIndex(docsEnum, activeMap.get(docsEnum))); + } + + queueByDocId = new DocsEnumDocIdPriorityQueue(activeMap.size()); + queueByIndex = new DocsEnumIndexPriorityQueue(activeMap.size()); + if (replacements == null) { + this.replacements = NO_REPLACEMENTS; + } else { + this.replacements = replacements; + } + } + + @Override + public int freq() throws IOException { + int freq = 0; + for (DocsEnumWithIndex docsEnum : active) { + freq += docsEnum.docsEnum.freq(); + } + return freq; + } + + @Override + public int docID() { + return active.get(0).docsEnum.docID(); + } + + @Override + public int nextDoc() throws IOException { + // advance all enums that were active in last docId, and put in queue + for (DocsEnumWithIndex docsEnum : active) { + if (docsEnum.docsEnum.nextDoc() != NO_MORE_DOCS) { + queueByDocId.add(docsEnum); + } + } + active.clear(); + + return queueToActive(); + } + + protected int queueToActive() throws IOException { + while (queueByDocId.size() > 0) { + // put all enums with minimal docId in active list + int docId = queueByDocId.top().docsEnum.docID(); + while (queueByDocId.size() > 0 + && docId == queueByDocId.top().docsEnum.docID()) { + queueByIndex.add(queueByDocId.pop()); + } + + // make sure non-replaced fields exist + while (queueByIndex.size() > 0 + && queueByIndex.top().index > replacements.get(docId)) { + active.add(queueByIndex.pop()); + } + // put replaced fields back in the queue + while (queueByIndex.size() > 0) { + DocsEnumWithIndex docsEnum = queueByIndex.pop(); + if (docsEnum.docsEnum.nextDoc() != NO_MORE_DOCS) { + queueByDocId.add(docsEnum); + } + } + if (!active.isEmpty()) { + return docId; + } + } + + return NO_MORE_DOCS; + } + + @Override + public int advance(int target) throws IOException { + // advance all enums, and put in queue + while (queueByDocId.size() > 0) { + active.add(queueByDocId.pop()); + } + queueByDocId.clear(); + for (DocsEnumWithIndex docsEnum : active) { + if (docsEnum.docsEnum.advance(target) != NO_MORE_DOCS) { + queueByDocId.add(docsEnum); + } + } + active.clear(); + + return queueToActive(); + } + + @Override + public int nextPosition() throws IOException { + return 0; + } + + @Override + public int startOffset() throws IOException { + return 0; + } + + @Override + public int endOffset() throws IOException { + return 0; + } + + @Override + public BytesRef getPayload() throws IOException { + return null; + } + + protected class DocsEnumWithIndex { + + DocsEnum docsEnum; + int index; + + public DocsEnumWithIndex(DocsEnum docsEnum, int index) { + this.docsEnum = docsEnum; + this.index = index; + } + + } + + private class DocsEnumDocIdPriorityQueue extends + PriorityQueue { + + public DocsEnumDocIdPriorityQueue(int maxSize) { + super(maxSize); + } + + @Override + protected boolean lessThan(DocsEnumWithIndex a, DocsEnumWithIndex b) { + return a.docsEnum.docID() < b.docsEnum.docID(); + } + + } + + private class DocsEnumIndexPriorityQueue extends + PriorityQueue { + + public DocsEnumIndexPriorityQueue(int maxSize) { + super(maxSize); + } + + @Override + protected boolean lessThan(DocsEnumWithIndex a, DocsEnumWithIndex b) { + // bigger index should be first + return a.index < b.index; + } + + } + +} Index: lucene/core/src/java/org/apache/lucene/index/StackedFields.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/StackedFields.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/index/StackedFields.java (working copy) @@ -0,0 +1,91 @@ +package org.apache.lucene.index; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.TreeMap; + +/* + * 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. + */ + +/** + * {@link Fields} of a segment with updates. + */ +public class StackedFields extends Fields { + + final Map fields; + + public StackedFields(Fields[] fieldsArray, + Map replacementsMap, int doc) + throws IOException { + fields = new TreeMap(); + final Set ignoreFields = new HashSet(); + + for (int i = fieldsArray.length - 1; i >= 0; i--) { + if (fieldsArray[i] != null) { + final Iterator iterator = fieldsArray[i].iterator(); + while (iterator.hasNext()) { + // handle single field + String field = iterator.next(); + if (!ignoreFields.contains(field)) { + Terms terms = fieldsArray[i].terms(field); + if (terms != null) { + StackedTerms stackedTerms = (StackedTerms) fields.get(field); + if (stackedTerms == null) { + stackedTerms = new StackedTerms(fieldsArray.length, + replacementsMap.get(field)); + fields.put(field, stackedTerms); + } + stackedTerms.addTerms(terms, i); + } + } + } + } + + if (doc >= 0) { + // ignore fields according to replacements for this document + for (Entry entry : replacementsMap + .entrySet()) { + if (!ignoreFields.contains(entry.getKey()) + && entry.getValue() != null && entry.getValue().get(doc) == i) { + ignoreFields.add(entry.getKey()); + } + } + } + } + } + + @Override + public Iterator iterator() { + return Collections.unmodifiableSet(fields.keySet()).iterator(); + } + + @Override + public Terms terms(String field) throws IOException { + return fields.get(field); + } + + @Override + public int size() { + return fields.size(); + } + +} Index: lucene/core/src/java/org/apache/lucene/index/StackedStoredFieldsReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/StackedStoredFieldsReader.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/index/StackedStoredFieldsReader.java (working copy) @@ -0,0 +1,72 @@ +package org.apache.lucene.index; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.lucene.codecs.StoredFieldsReader; + +/* + * 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. + */ + +public class StackedStoredFieldsReader extends StoredFieldsReader { + + private StoredFieldsReader[] allReaders; + private Map replacementsMap; + + public StackedStoredFieldsReader(StoredFieldsReader[] allReaders, + Map replacements) { + this.allReaders = allReaders; + this.replacementsMap = replacements; + } + + @Override + public void close() throws IOException { + for (StoredFieldsReader reader : allReaders) { + reader.close(); + } + } + + @Override + public void visitDocument(int n, StoredFieldVisitor visitor, + Set ignoreFields) throws IOException { + ignoreFields = new HashSet(); + for (int i = allReaders.length - 1; i > 0; i--) { + allReaders[i].visitDocument(n, visitor, ignoreFields); + for (Entry entry : replacementsMap + .entrySet()) { + if (!ignoreFields.contains(entry.getKey()) && entry.getValue() != null + && entry.getValue().get(n) == i) { + ignoreFields.add(entry.getKey()); + } + } + } + allReaders[0].visitDocument(n, visitor, ignoreFields); + } + + @Override + public StoredFieldsReader clone() { + StoredFieldsReader[] newReaders = new StoredFieldsReader[allReaders.length]; + for (int i = 0; i < newReaders.length; i++) { + newReaders[i] = allReaders[i].clone(); + } + return new StackedStoredFieldsReader(newReaders, replacementsMap); + } + +} Index: lucene/core/src/java/org/apache/lucene/index/StackedTermVectorsReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/StackedTermVectorsReader.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/index/StackedTermVectorsReader.java (working copy) @@ -0,0 +1,55 @@ +package org.apache.lucene.index; + +import java.io.IOException; +import java.util.Map; + +import org.apache.lucene.codecs.TermVectorsReader; + +/* + * 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. + */ + +public class StackedTermVectorsReader extends TermVectorsReader { + + private TermVectorsReader[] allReaders; + private Map replacementsMap; + + public StackedTermVectorsReader(TermVectorsReader[] allReaders, + Map replacementsMap) { + this.allReaders = allReaders; + this.replacementsMap = replacementsMap; + } + + @Override + public void close() throws IOException {} + + @Override + public Fields get(int doc) throws IOException { + // generate fields array + Fields[] fieldsArray = new Fields[allReaders.length]; + for (int i = 0; i < fieldsArray.length; i++) { + if (allReaders[i] != null) { + fieldsArray[i] = allReaders[i].get(doc); + } + } + return new StackedFields(fieldsArray, replacementsMap, doc); + } + + @Override + public TermVectorsReader clone() { + return new StackedTermVectorsReader(allReaders, replacementsMap); + } +} Index: lucene/core/src/java/org/apache/lucene/index/StackedTerms.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/StackedTerms.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/index/StackedTerms.java (working copy) @@ -0,0 +1,148 @@ +package org.apache.lucene.index; + +import java.io.IOException; +import java.util.Comparator; + +import org.apache.lucene.util.BytesRef; + +/* + * 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. + */ + +public class StackedTerms extends Terms { + + private final FieldGenerationReplacements replacements; + private final Terms[] subTerms; + private Comparator comparator; + private boolean hasOffsets; + private boolean hasPositions; + private boolean hasPayloads; + + public StackedTerms(int maxTerms, + FieldGenerationReplacements fieldGenerationReplacements) { + this.replacements = fieldGenerationReplacements; + subTerms = new Terms[maxTerms]; + + hasOffsets = false; + hasPositions = false; + hasPayloads = false; + } + + public void addTerms(Terms terms, int generation) throws IOException { + if (terms != null) { + subTerms[generation] = terms; + hasOffsets |= terms.hasOffsets(); + hasPositions |= terms.hasPositions(); + hasPayloads |= terms.hasPayloads(); + if (comparator == null) { + comparator = terms.getComparator(); + } else if (!comparator.equals(terms.getComparator())) { + throw new IllegalStateException( + "sub-readers have different BytesRef.Comparators; cannot merge"); + } + } + } + + @Override + public TermsEnum iterator(TermsEnum reuse) throws IOException { + return new StackedTermsEnum(subTerms, replacements, comparator); + } + + @Override + public Comparator getComparator() throws IOException { + if (comparator == null) { + for (int i = 0; i < subTerms.length; i++) { + if (subTerms[i] != null) { + comparator = subTerms[i].getComparator(); + if (comparator != null) { + return comparator; + } + } + } + } + return comparator; + } + + @Override + public long size() throws IOException { + final TermsEnum iterator = iterator(null); + int size = 0; + while (iterator.next() != null) { + size++; + } + return size; + } + + @Override + public long getSumTotalTermFreq() throws IOException { + long sum = 0; + boolean found = false; + for (int i = 0; i < subTerms.length; i++) { + if (subTerms[i] != null) { + final long subTotalTermFreq = subTerms[i].getSumTotalTermFreq(); + if (subTotalTermFreq != -1) { + sum += subTotalTermFreq; + found = true; + } + } + } + if (!found) { + return -1; + } + return sum; + } + + @Override + public long getSumDocFreq() throws IOException { + long sum = 0; + boolean found = false; + for (int i = 0; i < subTerms.length; i++) { + if (subTerms[i] != null) { + final long sumDocFreq = subTerms[i].getSumDocFreq(); + if (sumDocFreq != -1) { + sum += sumDocFreq; + found = true; + } + } + } + if (!found) { + return -1; + } + return sum; + } + + @Override + public int getDocCount() throws IOException { + // TODO : SY : any intelligent way to do this without full iteration? + return -1; + } + + @Override + public boolean hasOffsets() { + return hasOffsets; + } + + @Override + public boolean hasPositions() { + return hasPositions; + } + + @Override + public boolean hasPayloads() { + return hasPayloads; + } + +} Index: lucene/core/src/java/org/apache/lucene/index/StackedTermsEnum.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/StackedTermsEnum.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/index/StackedTermsEnum.java (working copy) @@ -0,0 +1,256 @@ +package org.apache.lucene.index; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeSet; + +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; + +/* + * 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. + */ + +/** + * TermsEnum for stacked segments (with updates). At the term level the terms + * are merged without taking into consideration fields replacements, so terms + * with no occurrences might return. Given a certain term, all the enumerations + * take into account fields replacements. + */ +public class StackedTermsEnum extends TermsEnum { + + private final Terms[] subTerms; + private final FieldGenerationReplacements replacements; + private Comparator comparator; + private TreeSet activeEnums; + + public StackedTermsEnum(Terms[] subTerms, + FieldGenerationReplacements replacements, Comparator comparator) + throws IOException { + this.subTerms = subTerms; + this.replacements = replacements; + this.comparator = comparator; + } + + @Override + public Comparator getComparator() { + return comparator; + } + + @Override + public BytesRef next() throws IOException { + if (activeEnums == null) { + init(); + return headTerm(); + } + + // get the current term (queue head) + BytesRef headTerm = headTerm(); + final BytesRef currentHead = BytesRef.deepCopyOf(headTerm); + + // advance all enums with same term + while (currentHead.equals(headTerm)) { + if (activeEnums.isEmpty()) { + return null; + } else { + final InnerTermsEnum polled = activeEnums.pollFirst(); + if (polled.advance()) { + activeEnums.add(polled); + headTerm = headTerm(); + } else { + headTerm = null; + } + } + } + + return headTerm; + } + + private void init() throws IOException { + activeEnums = new TreeSet(); + for (int i = 0; i < subTerms.length; i++) { + if (subTerms[i] != null) { + final TermsEnum termsEnum = subTerms[i].iterator(null); + final BytesRef term = termsEnum.next(); + if (term != null) { + activeEnums.add(new InnerTermsEnum(i, termsEnum, term)); + } + } + } + } + + @Override + public BytesRef term() throws IOException { + return headTerm(); + } + + private BytesRef headTerm() { + final InnerTermsEnum head = activeEnums.first(); + if (head == null) { + return null; + } + return head.getTerm(); + } + + @Override + public SeekStatus seekCeil(BytesRef text, boolean useCache) + throws IOException { + // reset active enums + if (activeEnums == null) { + activeEnums = new TreeSet(); + } else { + activeEnums.clear(); + } + + // do seekCeil on all non-null subTerms + SeekStatus status = SeekStatus.END; + for (int i = 0; i < subTerms.length; i++) { + if (subTerms[i] != null) { + final TermsEnum termsEnum = subTerms[i].iterator(null); + final SeekStatus tempStatus = termsEnum.seekCeil(text, useCache); + if (tempStatus != SeekStatus.END) { + // put in new queue + activeEnums.add(new InnerTermsEnum(i, termsEnum, termsEnum.term())); + + // update status if needed + if (tempStatus == SeekStatus.FOUND) { + status = SeekStatus.FOUND; + } else if (status == SeekStatus.END) { + status = SeekStatus.NOT_FOUND; + } + } + } + } + return status; + } + + @Override + public long ord() throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public void seekExact(long ord) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public int docFreq() throws IOException { + int docFreq = 0; + for (InnerTermsEnum inner : activeEnums.headSet(activeEnums.first(), true)) { + docFreq += inner.termsEnum.docFreq(); + } + return docFreq; + } + + @Override + public long totalTermFreq() throws IOException { + int totalTermFreq = 0; + for (InnerTermsEnum inner : activeEnums.headSet(activeEnums.first(), true)) { + totalTermFreq += inner.termsEnum.totalTermFreq(); + } + return totalTermFreq; + } + + @Override + public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) + throws IOException { + // find all enums on current term + List onCurrent = new ArrayList(); + final BytesRef term = term(); + for (InnerTermsEnum inner : activeEnums) { + if (term.equals(inner.getTerm())) { + onCurrent.add(inner); + } + } + + // build map of active enums with indexes + Map activeMap = new HashMap(); + for (InnerTermsEnum innerCurrent : onCurrent) { + final DocsEnum docs = innerCurrent.termsEnum.docs(liveDocs, reuse, flags); + if (docs != null) { + activeMap.put(docs, innerCurrent.getIndex()); + } + } + + return new StackedDocsEnum(activeMap, replacements); + } + + @Override + public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, + DocsAndPositionsEnum reuse, int flags) throws IOException { + List onCurrent = new ArrayList(); + final BytesRef term = term(); + for (InnerTermsEnum inner : activeEnums) { + if (term.equals(inner.getTerm())) { + onCurrent.add(inner); + } + } + + // build map of active enums with indexes + Map activeMap = new HashMap(); + for (InnerTermsEnum innerCurrent : onCurrent) { + final DocsAndPositionsEnum docsAndPositions = innerCurrent.termsEnum + .docsAndPositions(liveDocs, reuse, flags); + if (docsAndPositions != null) { + activeMap.put(docsAndPositions, innerCurrent.getIndex()); + } + } + + if (activeMap.isEmpty()) { + return null; + } + + return new StackedDocsEnum(activeMap, replacements); + } + + private class InnerTermsEnum implements Comparable { + + private int index; + private TermsEnum termsEnum; + private BytesRef term; + + public InnerTermsEnum(int index, TermsEnum termsEnum, BytesRef term) { + this.index = index; + this.termsEnum = termsEnum; + this.term = term; + } + + public int getIndex() { + return index; + } + + public BytesRef getTerm() { + return term; + } + + public boolean advance() throws IOException { + term = termsEnum.next(); + return term != null; + } + + @Override + public int compareTo(InnerTermsEnum o) { + return comparator.compare(this.term, o.term); + } + + } + +} Index: lucene/core/src/java/org/apache/lucene/index/UpdatedFieldsData.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/UpdatedFieldsData.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/index/UpdatedFieldsData.java (working copy) @@ -0,0 +1,51 @@ +package org.apache.lucene.index; + +import java.util.ArrayList; +import java.util.List; +import java.util.TreeMap; + +/* + * 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. + */ + +/** + * Contains the updates for a specific field in an updated segment, ordered by + * doc ID. + */ +public class UpdatedFieldsData { + + // TODO : SY : consider replacing TreeMap with own implementation + TreeMap> docUpdatesMap; + + public UpdatedFieldsData() { + docUpdatesMap = new TreeMap>(); + } + + void addField(int docId, T field, boolean replace) { + List docUpdates = docUpdatesMap.get(docId); + if (docUpdates == null) { + docUpdates = new ArrayList(1); + docUpdatesMap.put(docId, docUpdates); + } else if (replace) { + docUpdates.clear(); + } + docUpdates.add(field); + } + + public List get(int docId) { + return docUpdatesMap.get(docId); + } +} Index: lucene/core/src/java/org/apache/lucene/index/UpdatedSegmentData.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/UpdatedSegmentData.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/index/UpdatedSegmentData.java (working copy) @@ -0,0 +1,245 @@ +package org.apache.lucene.index; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; + +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.index.FieldsUpdate.Operation; + +/* + * 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. + */ + +/** + * Holds updates data for a certain segment. + */ +class UpdatedSegmentData { + + /** Updates mapped by doc ID, for each do sorted list of updates. */ + private TreeMap> updatesMap; + + public long generation; + + private Map fieldGenerationReplacments; + + private Iterator>> updatesIterator; + private int lastDocID; + private int currDocID; + private int baseDocCount; + private Analyzer analyzer; + + UpdatedSegmentData() { + updatesMap = new TreeMap>(); + } + + void addUpdates(int docID, SortedSet sortedUpdates) { + SortedSet prevUpdates = updatesMap.get(docID); + if (prevUpdates == null) { + updatesMap.put(docID, sortedUpdates); + } else { + prevUpdates.addAll(sortedUpdates); + } + } + + void addUpdates(int docID, FieldsUpdate[] updatesArray) { + SortedSet prevUpdates = updatesMap.get(docID); + if (prevUpdates == null) { + prevUpdates = new TreeSet(); + updatesMap.put(docID, prevUpdates); + } + for (int i = 0; i < updatesArray.length; i++) { + prevUpdates.add(updatesArray[i]); + } + } + + boolean hasUpdates() { + return !updatesMap.isEmpty(); + } + + /** + * Start writing updates to updates index. + * + * @param generation + * The updates generation. + * @param baseDocCount + * number of documents in the base segment + */ + void startWriting(long generation, int baseDocCount) { + this.generation = generation; + updatesIterator = updatesMap.entrySet().iterator(); + currDocID = 0; + this.baseDocCount = baseDocCount; + } + + /** + * Get the next document to put in the updates index. Also updates the + * analyzer and the number of missing documents between the current and + * previous documents. + * + * @throws IOException + * If different analyzers were assigned to field updates affecting + * the next document. + */ + IndexDocument nextDocument() throws IOException { + if (!updatesIterator.hasNext()) { + // done with all updates + lastDocID = currDocID; + currDocID = baseDocCount - 1; + return null; + } + Entry> docUpdates = updatesIterator.next(); + lastDocID = currDocID; + currDocID = docUpdates.getKey(); + + return new UpdatesIndexDocument(docUpdates.getValue()); + } + + Analyzer getAnalyzer() { + return analyzer; + } + + /** + * The number of documents missing before the last document returned by + * {@link #nextDocument()}. + */ + int numMissingDocuments() { + return currDocID - lastDocID; + } + + Map getFieldGenerationReplacments() { + return fieldGenerationReplacments; + } + + /** + * An {@link IndexDocument} containing all the updates to a certain document, + * taking into account replacements. + *

    + * Constructing an {@link UpdatesIndexDocument} also updates + * {@link FieldGenerationReplacements} vectors for the relevant fields. + */ + private class UpdatesIndexDocument implements IndexDocument { + + Map> indexablesByField = new HashMap>(); + Map> storablesByField = new HashMap>(); + + public UpdatesIndexDocument(SortedSet fieldsUpdates) + throws IOException { + boolean setAnalyzer = true; + analyzer = null; + for (FieldsUpdate fieldsUpdate : fieldsUpdates) { + // set analyzer and check for analyzer conflict + if (setAnalyzer) { + analyzer = fieldsUpdate.analyzer; + setAnalyzer = false; + } else if (analyzer != fieldsUpdate.analyzer) { + throw new IOException( + "two analyzers assigned to one updated document"); + } + + if (fieldsUpdate.operation == Operation.REPLACE_FIELDS) { + // handle fields replacement + for (IndexableField field : fieldsUpdate.fields.indexableFields()) { + replaceField(field.name()); + } + for (StorableField field : fieldsUpdate.fields.storableFields()) { + replaceField(field.name()); + } + } + + // add new fields + for (IndexableField field : fieldsUpdate.fields.indexableFields()) { + List fieldList = indexablesByField.get(field.name()); + if (fieldList == null) { + fieldList = new ArrayList(); + indexablesByField.put(field.name(), fieldList); + } + fieldList.add(field); + } + for (StorableField field : fieldsUpdate.fields.storableFields()) { + List fieldList = storablesByField.get(field.name()); + if (fieldList == null) { + fieldList = new ArrayList(); + storablesByField.put(field.name(), fieldList); + } + fieldList.add(field); + } + } + } + + private void replaceField(String fieldName) { + // remove previous fields + indexablesByField.remove(fieldName); + storablesByField.remove(fieldName); + + // update field generation replacement vector + if (fieldGenerationReplacments == null) { + fieldGenerationReplacments = new HashMap(); + } + FieldGenerationReplacements fieldReplacement = fieldGenerationReplacments + .get(fieldName); + if (fieldReplacement == null) { + fieldReplacement = new FieldGenerationReplacements(); + fieldGenerationReplacments.put(fieldName, fieldReplacement); + } + fieldReplacement.set(currDocID, generation); + } + + @Override + public Iterable indexableFields() { + List indexableFields = new ArrayList(); + for (List byField : indexablesByField.values()) { + indexableFields.addAll(byField); + } + return indexableFields; + } + + @Override + public Iterable storableFields() { + List storableFields = new ArrayList(); + for (List byField : storablesByField.values()) { + storableFields.addAll(byField); + } + return storableFields; + } + + } + + private static final IndexDocument EMPTY_DOCUMENT = new IndexDocument() { + + @Override + public Iterable storableFields() { + return Collections.emptyList(); + } + + @Override + public Iterable indexableFields() { + return Collections.emptyList(); + } + }; + + public IndexDocument emptyDocument() { + return EMPTY_DOCUMENT; + } +} Index: lucene/core/src/test/org/apache/lucene/index/TestCodecs.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestCodecs.java (revision 1406075) +++ lucene/core/src/test/org/apache/lucene/index/TestCodecs.java (working copy) @@ -619,7 +619,8 @@ final int termIndexInterval = _TestUtil.nextInt(random(), 13, 27); final Codec codec = Codec.getDefault(); final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, false, codec, null, null); - final SegmentWriteState state = new SegmentWriteState(InfoStream.getDefault(), dir, si, fieldInfos, termIndexInterval, null, newIOContext(random())); + final SegmentWriteState state = + new SegmentWriteState(InfoStream.getDefault(), dir, si, 0, fieldInfos, termIndexInterval, null, null, newIOContext(random())); final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(state); Arrays.sort(fields); Index: lucene/core/src/test/org/apache/lucene/index/TestDoc.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestDoc.java (revision 1406075) +++ lucene/core/src/test/org/apache/lucene/index/TestDoc.java (working copy) @@ -232,14 +232,14 @@ info.setFiles(new HashSet(trackingDir.getCreatedFiles())); if (useCompoundFile) { - Collection filesToDelete = IndexWriter.createCompoundFile(InfoStream.getDefault(), dir, MergeState.CheckAbort.NONE, info, newIOContext(random())); + Collection filesToDelete = IndexWriter.createCompoundFile(InfoStream.getDefault(), dir, MergeState.CheckAbort.NONE, info, newIOContext(random()), -1); info.setUseCompoundFile(true); for (final String fileToDelete : filesToDelete) { si1.info.dir.deleteFile(fileToDelete); } } - return new SegmentInfoPerCommit(info, 0, -1L); + return new SegmentInfoPerCommit(info, 0, -1L, -1L); } Index: lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java (revision 1406075) +++ lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java (working copy) @@ -57,14 +57,14 @@ if (random().nextInt(20) == 0 || j == ids.length - 1) { queue.updateSlice(slice1); assertTrue(slice1.isTailItem(term)); - slice1.apply(bd1, j); + slice1.apply(bd1, null, j); assertAllBetween(last1, j, bd1, ids); last1 = j + 1; } if (random().nextInt(10) == 5 || j == ids.length - 1) { queue.updateSlice(slice2); assertTrue(slice2.isTailItem(term)); - slice2.apply(bd2, j); + slice2.apply(bd2, null, j); assertAllBetween(last2, j, bd2, ids); last2 = j + 1; } @@ -166,7 +166,7 @@ queue.tryApplyGlobalSlice(); assertTrue("changes in global buffer", queue.anyChanges()); FrozenBufferedDeletes freezeGlobalBuffer = queue.freezeGlobalBuffer(null); - assertTrue(freezeGlobalBuffer.any()); + assertTrue(freezeGlobalBuffer.anyDeletes()); assertEquals(1, freezeGlobalBuffer.termCount); assertFalse("all changes applied", queue.anyChanges()); } @@ -197,7 +197,7 @@ DeleteSlice slice = updateThread.slice; queue.updateSlice(slice); BufferedDeletes deletes = updateThread.deletes; - slice.apply(deletes, BufferedDeletes.MAX_INT); + slice.apply(deletes, null, BufferedDeletes.MAX_INT); assertEquals(uniqueValues, deletes.terms.keySet()); } queue.tryApplyGlobalSlice(); @@ -242,9 +242,9 @@ int i = 0; while ((i = index.getAndIncrement()) < ids.length) { Term term = new Term("id", ids[i].toString()); - queue.add(term, slice); + queue.add(term, slice, null); assertTrue(slice.isTailItem(term)); - slice.apply(deletes, BufferedDeletes.MAX_INT); + slice.apply(deletes, null, BufferedDeletes.MAX_INT); } } } Index: lucene/core/src/test/org/apache/lucene/index/TestFieldUpdates.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestFieldUpdates.java (revision 0) +++ lucene/core/src/test/org/apache/lucene/index/TestFieldUpdates.java (working copy) @@ -0,0 +1,149 @@ +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.List; + +import org.apache.lucene.analysis.MockAnalyzer; +import org.apache.lucene.document.Document; +import org.apache.lucene.index.FieldsUpdate.Operation; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.LuceneTestCase; + +public class TestFieldUpdates extends LuceneTestCase { + private Directory dir; + + @Override + public void setUp() throws Exception { + super.setUp(); + dir = newDirectory(); + } + + @Override + public void tearDown() throws Exception { + dir.close(); + super.tearDown(); + } + + public void test() { + assertTrue(dir != null); + } + + public void testUpdateDocumentBeforeCommit() throws Exception { + createAndAssertSegment(false); + } + + public void testUpdateDocumentAfterCommit() throws Exception { + createAndAssertSegment(true); + } + + private void createAndAssertSegment(boolean interCommit) throws IOException { + // added doc contains at least the first field, updated field at least the + // last field, other fields split in the middle randomly + int //cutoff = random().nextInt(numFields - 2); + cutoff = 0; + createSegment(cutoff + 1, interCommit); + assertSegment(); + } + + private void createSegment(int cutoff, boolean interCommit) + throws IOException { + // add base document + Document testDoc = new Document(); + DocHelper.setupDoc(testDoc, 0, cutoff); + IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( + TEST_VERSION_CURRENT, new MockAnalyzer(random()))); + writer.addDocument(testDoc); + if (interCommit) { + writer.commit(); + } + + // add updates to base document + Document updateDoc = new Document(); + DocHelper.setupDoc(updateDoc, cutoff, DocHelper.numFields()); + writer.updateFields(new Term(DocHelper.TEXT_FIELD_1_KEY, + DocHelper.FIELD_1_TEXT.split(" ")[0]), Operation.ADD_FIELDS, updateDoc); + writer.commit(); + writer.close(); + } + + private void assertSegment() throws IOException { + // After adding the document, we should be able to read it back in + DirectoryReader directoryReader = DirectoryReader.open(dir); + List leaves = directoryReader.leaves(); + assertEquals("wrong number of atomic readers", 1, leaves.size()); + AtomicReaderContext atomicReaderContext = leaves.get(0); + AtomicReader reader = atomicReaderContext.reader(); + assertTrue(reader != null); + StoredDocument doc = reader.document(0); + assertTrue(doc != null); + + // System.out.println("Document: " + doc); + StorableField[] fields = doc.getFields("textField2"); + assertTrue(fields != null && fields.length == 1); + assertTrue(fields[0].stringValue().equals(DocHelper.FIELD_2_TEXT)); + assertTrue(fields[0].fieldType().storeTermVectors()); + + fields = doc.getFields("textField1"); + assertTrue(fields != null && fields.length == 1); + assertTrue(fields[0].stringValue().equals(DocHelper.FIELD_1_TEXT)); + assertFalse(fields[0].fieldType().storeTermVectors()); + + fields = doc.getFields("keyField"); + assertTrue(fields != null && fields.length == 1); + assertTrue(fields[0].stringValue().equals(DocHelper.KEYWORD_TEXT)); + + fields = doc.getFields(DocHelper.NO_NORMS_KEY); + assertTrue(fields != null && fields.length == 1); + assertTrue(fields[0].stringValue().equals(DocHelper.NO_NORMS_TEXT)); + + fields = doc.getFields(DocHelper.TEXT_FIELD_3_KEY); + assertTrue(fields != null && fields.length == 1); + assertTrue(fields[0].stringValue().equals(DocHelper.FIELD_3_TEXT)); + + // test that the norms are not present in the segment if + // omitNorms is true + for (FieldInfo fi : reader.getFieldInfos()) { + if (fi.isIndexed()) { + assertTrue(fi.omitsNorms() == (reader.normValues(fi.name) == null)); + } + } + reader.close(); + } + + public void testSegmentWithDeletion() throws IOException { + // add base document + IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( + TEST_VERSION_CURRENT, new MockAnalyzer(random()))); + Document testDoc = new Document(); + DocHelper.setupDoc(testDoc, 0, 5); + writer.addDocument(testDoc); + testDoc = new Document(); + DocHelper.setupDoc(testDoc, 5, DocHelper.numFields()); + writer.addDocument(testDoc); + writer.commit(); + + writer.deleteDocuments(new Term(DocHelper.TEXT_FIELD_1_KEY, + DocHelper.FIELD_1_TEXT.split(" ")[0])); + writer.close(); + + assertSegment(); + } + +} Index: lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java (revision 1406075) +++ lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java (working copy) @@ -92,7 +92,7 @@ SegmentReader mergedReader = new SegmentReader(new SegmentInfoPerCommit( new SegmentInfo(mergedDir, Constants.LUCENE_MAIN_VERSION, mergedSegment, docsMerged, false, codec, null, null), - 0, -1L), + 0, -1L, -1L), DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random())); assertTrue(mergedReader != null); assertTrue(mergedReader.numDocs() == 2); Index: lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java =================================================================== --- lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java (revision 1406075) +++ lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java (working copy) @@ -141,7 +141,7 @@ SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.name, info.getDocCount(), info.getUseCompoundFile(), info.getCodec(), info.getDiagnostics(), info.attributes()); - destInfos.add(new SegmentInfoPerCommit(newInfo, infoPerCommit.getDelCount(), infoPerCommit.getDelGen())); + destInfos.add(new SegmentInfoPerCommit(newInfo, infoPerCommit.getDelCount(), infoPerCommit.getDelGen(), -1L)); // now copy files over Collection files = infoPerCommit.files(); for (final String srcName : files) { Index: lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (revision 1406075) +++ lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (working copy) @@ -481,8 +481,8 @@ long bytes = totalPostings * 8 + totalPayloadBytes; SegmentWriteState writeState = new SegmentWriteState(null, dir, - segmentInfo, newFieldInfos, - 32, null, new IOContext(new FlushInfo(maxDoc, bytes))); + segmentInfo, 0, newFieldInfos, + 32, null, null, new IOContext(new FlushInfo(maxDoc, bytes))); FieldsConsumer fieldsConsumer = codec.postingsFormat().fieldsConsumer(writeState); for(Map.Entry> fieldEnt : fields.entrySet()) { Index: lucene/test-framework/src/java/org/apache/lucene/index/DocHelper.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/index/DocHelper.java (revision 1406075) +++ lucene/test-framework/src/java/org/apache/lucene/index/DocHelper.java (working copy) @@ -254,6 +254,23 @@ } /** + * Adds part of the fields above to a document + * @param doc The document to write + * @param from index of the first field to add + * @param to index of the last field to add + 1 + */ + public static void setupDoc(Document doc, int from, int to) { + for (int i=from; i