LUCENE-4055: fix more nocommits

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4055@1340710 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2012-05-20 13:24:29 +00:00
parent 5687147186
commit fb6c3cfeac
16 changed files with 111 additions and 209 deletions

View File

@ -42,5 +42,5 @@ public abstract class SegmentInfosReader {
* @param infos empty instance to be populated with data
* @throws IOException
*/
public abstract SegmentInfo read(Directory directory, String segmentName) throws IOException;
public abstract SegmentInfo read(Directory directory, String segmentName, IOContext context) throws IOException;
}

View File

@ -35,9 +35,6 @@ import org.apache.lucene.index.SegmentInfo;
public class Lucene3xSegmentInfosFormat extends SegmentInfosFormat {
private final SegmentInfosReader reader = new Lucene3xSegmentInfosReader();
// nocommit explain or remove this!:
public static final String SI_EXTENSION = "si";
/** This format adds optional per-segment String
* diagnostics storage, and switches userData to Map */
public static final int FORMAT_DIAGNOSTICS = -9;
@ -48,20 +45,14 @@ public class Lucene3xSegmentInfosFormat extends SegmentInfosFormat {
/** Each segment records the Lucene version that created it. */
public static final int FORMAT_3_1 = -11;
// nocommit we should nuke FORMAT_4_0!?
/** Each segment records whether its postings are written
* in the new flex format */
public static final int FORMAT_4_0 = -12;
public static final int FORMAT_4X_UPGRADE = -12;
/** This must always point to the most recent file format.
* whenever you add a new format, make it 1 smaller (negative version logic)! */
// TODO: move this, as its currently part of required preamble
public static final int FORMAT_CURRENT = FORMAT_4_0;
/** Extension used for saving each SegmentInfo, once a 3.x
* index is first committed to with 4.0. */
public static final String SI_EXTENSION = "si";
/** This must always point to the first supported file format. */
public static final int FORMAT_MINIMUM = FORMAT_DIAGNOSTICS;
@Override
public SegmentInfosReader getSegmentInfosReader() {
return reader;

View File

@ -89,19 +89,18 @@ public class Lucene3xSegmentInfosReader extends SegmentInfosReader {
}
@Override
public SegmentInfo read(Directory directory, String segmentName) throws IOException {
return read(directory, segmentName, Lucene3xSegmentInfosFormat.FORMAT_4_0);
public SegmentInfo read(Directory directory, String segmentName, IOContext context) throws IOException {
return read(directory, segmentName, Lucene3xSegmentInfosFormat.FORMAT_4X_UPGRADE, context);
}
public SegmentInfo read(Directory directory, String segmentName, int format) throws IOException {
public SegmentInfo read(Directory directory, String segmentName, int format, IOContext context) throws IOException {
// NOTE: this is NOT how 3.x is really written...
String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene3xSegmentInfosFormat.SI_EXTENSION);
// nocommit what IOCtx
boolean success = false;
IndexInput input = directory.openInput(fileName, IOContext.READONCE);
IndexInput input = directory.openInput(fileName, context);
try {
SegmentInfo si = readSegmentInfo(segmentName, directory, format, input);
@ -118,13 +117,13 @@ public class Lucene3xSegmentInfosReader extends SegmentInfosReader {
private SegmentInfo readSegmentInfo(String segmentName, Directory dir, int format, IndexInput input) throws IOException {
// check that it is a format we can understand
if (format > Lucene3xSegmentInfosFormat.FORMAT_MINIMUM) {
if (format > Lucene3xSegmentInfosFormat.FORMAT_DIAGNOSTICS) {
throw new IndexFormatTooOldException(input, format,
Lucene3xSegmentInfosFormat.FORMAT_MINIMUM, Lucene3xSegmentInfosFormat.FORMAT_CURRENT);
Lucene3xSegmentInfosFormat.FORMAT_DIAGNOSTICS, Lucene3xSegmentInfosFormat.FORMAT_4X_UPGRADE);
}
if (format < Lucene3xSegmentInfosFormat.FORMAT_CURRENT) {
if (format < Lucene3xSegmentInfosFormat.FORMAT_4X_UPGRADE) {
throw new IndexFormatTooNewException(input, format,
Lucene3xSegmentInfosFormat.FORMAT_MINIMUM, Lucene3xSegmentInfosFormat.FORMAT_CURRENT);
Lucene3xSegmentInfosFormat.FORMAT_DIAGNOSTICS, Lucene3xSegmentInfosFormat.FORMAT_4X_UPGRADE);
}
final String version;
if (format <= Lucene3xSegmentInfosFormat.FORMAT_3_1) {

View File

@ -19,7 +19,6 @@ package org.apache.lucene.codecs.lucene3x;
import java.io.IOException;
import java.util.Set;
import java.util.Arrays; //nocommit
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.TermVectorsReader;

View File

@ -58,10 +58,11 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
// private String segment;
// nocommit don't pass FIS here...
public Lucene40PostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException {
freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION),
ioContext);
// nocommit don't consult FieldInfos here... then nuke
// fieldInfos arg...
// nocommit we can assert FIS.hasProx == our hasProx here...
// this.segment = segmentInfo.name;
if (fieldInfos.hasProx()) {

View File

@ -41,9 +41,9 @@ import org.apache.lucene.util.IOUtils;
public class Lucene40SegmentInfosReader extends SegmentInfosReader {
@Override
public SegmentInfo read(Directory dir, String segment) throws IOException {
public SegmentInfo read(Directory dir, String segment, IOContext context) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segment, "", Lucene40SegmentInfosFormat.SI_EXTENSION);
final IndexInput input = dir.openInput(fileName, IOContext.READONCE);
final IndexInput input = dir.openInput(fileName, context);
boolean success = false;
try {
final String version = input.readString();

View File

@ -49,13 +49,13 @@ public class SimpleTextNormsFormat extends NormsFormat {
@Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
return new SimpleTextNormsPerDocConsumer(state, NORMS_SEG_SUFFIX);
return new SimpleTextNormsPerDocConsumer(state);
}
@Override
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
return new SimpleTextNormsPerDocProducer(state,
BytesRef.getUTF8SortedAsUnicodeComparator(), NORMS_SEG_SUFFIX);
BytesRef.getUTF8SortedAsUnicodeComparator());
}
@Override
@ -74,8 +74,8 @@ public class SimpleTextNormsFormat extends NormsFormat {
SimpleTextPerDocProducer {
public SimpleTextNormsPerDocProducer(SegmentReadState state,
Comparator<BytesRef> comp, String segmentSuffix) throws IOException {
super(state, comp, segmentSuffix);
Comparator<BytesRef> comp) throws IOException {
super(state, comp, NORMS_SEG_SUFFIX);
}
@Override
@ -105,9 +105,9 @@ public class SimpleTextNormsFormat extends NormsFormat {
public static class SimpleTextNormsPerDocConsumer extends
SimpleTextPerDocConsumer {
public SimpleTextNormsPerDocConsumer(PerDocWriteState state,
String segmentSuffix) throws IOException {
super(state, segmentSuffix);
public SimpleTextNormsPerDocConsumer(PerDocWriteState state)
throws IOException {
super(state, NORMS_SEG_SUFFIX);
}
@Override
@ -129,25 +129,21 @@ public class SimpleTextNormsFormat extends NormsFormat {
@Override
public void abort() {
Set<String> files = new HashSet<String>();
filesInternal(state.segmentName, files, segmentSuffix);
filesInternal(state.segmentName, files);
IOUtils.deleteFilesIgnoringExceptions(state.directory,
SegmentInfo.findMatchingFiles(state.segmentName, state.directory, files).toArray(new String[0]));
}
public static void files(SegmentInfo segmentInfo, Set<String> files)
throws IOException {
filesInternal(segmentInfo.name, files,
NORMS_SEG_SUFFIX);
filesInternal(segmentInfo.name, files);
}
public static void filesInternal(String segmentName,
Set<String> files, String segmentSuffix) {
// nocommit simplify this: weird that we get suffix as
// an arg... it's always a constant
assert segmentSuffix.equals(NORMS_SEG_SUFFIX);
Set<String> files) {
String id = docValuesIdRegexp(segmentName);
files.add(IndexFileNames.segmentFileName(id, "",
segmentSuffix));
NORMS_SEG_SUFFIX));
}
}
}

View File

@ -47,10 +47,10 @@ import static org.apache.lucene.codecs.simpletext.SimpleTextSegmentInfosWriter.*
public class SimpleTextSegmentInfosReader extends SegmentInfosReader {
@Override
public SegmentInfo read(Directory directory, String segmentName) throws IOException {
public SegmentInfo read(Directory directory, String segmentName, IOContext context) throws IOException {
BytesRef scratch = new BytesRef();
String fileName = IndexFileNames.segmentFileName(segmentName, "", SimpleTextSegmentInfosFormat.SI_EXTENSION);
IndexInput input = directory.openInput(fileName, IOContext.READONCE);
IndexInput input = directory.openInput(fileName, context);
boolean success = false;
try {
SimpleTextUtil.readLine(input, scratch);

View File

@ -663,7 +663,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
rollbackSegments = segmentInfos.createBackupSegmentInfos(true);
// start with previous field numbers, but new FieldInfos
globalFieldNumberMap = getOrLoadGlobalFieldNumberMap();
globalFieldNumberMap = getFieldNumberMap();
docWriter = new DocumentsWriter(codec, config, directory, this, globalFieldNumberMap, bufferedDeletesStream);
// Default deleter (for backwards compatibility) is
@ -730,34 +730,27 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
* Loads or returns the already loaded the global field number map for this {@link SegmentInfos}.
* If this {@link SegmentInfos} has no global field number map the returned instance is empty
*/
private FieldNumberBiMap getOrLoadGlobalFieldNumberMap() throws IOException {
private FieldNumberBiMap getFieldNumberMap() throws IOException {
final FieldNumberBiMap map = new FieldNumberBiMap();
if (segmentInfos.size() > 0) {
// nocommit fixme for 3.x indices...
/*
if (segmentInfos.getFormat() > SegmentInfos.FORMAT_DIAGNOSTICS) {
// Pre-3.1 index. In this case we sweep all
// segments, merging their FieldInfos:
for(SegmentInfo info : segmentInfos) {
for(FieldInfo fi : getFieldInfos(info)) {
map.addOrGet(fi.name, fi.number);
}
}
} else {
*/
// Already >= 3.1 index; just seed the FieldInfos
// from the last segment
for(FieldInfo fi : getFieldInfos(segmentInfos.info(segmentInfos.size()-1))) {
map.addOrGet(fi.name, fi.number);
}
// nocommit we can also pull the DV types of the
// fields... and catch DV type change on addDoc
// instead of much later in merge
//}
SegmentInfo biggest = null;
for(SegmentInfo info : segmentInfos) {
if (biggest == null || (info.docCount-info.getDelCount()) > (biggest.docCount-biggest.getDelCount())) {
biggest = info;
}
}
if (biggest != null) {
for(FieldInfo fi : getFieldInfos(biggest)) {
map.addOrGet(fi.name, fi.number);
}
}
// nocommit we can also pull the DV types of the
// fields... and catch DV type change on addDoc
// instead of much later in merge
//}
return map;
}
@ -2025,29 +2018,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
try {
if (useCompoundFile(newSegment)) {
String compoundFileName = IndexFileNames.segmentFileName(newSegment.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "creating compound file " + compoundFileName);
}
// Now build compound file
// nocommit factor to use craeteCompoundFile method!?
final Directory cfsDir = new CompoundFileDirectory(directory, compoundFileName, context, true);
IOException prior = null;
try {
for(String fileName : newSegment.files()) {
directory.copy(cfsDir, fileName, fileName, context);
}
} catch(IOException ex) {
prior = ex;
} finally {
IOUtils.closeWhileHandlingException(prior, cfsDir);
}
// Perform the merge
Collection<String> files = createCompoundFile(infoStream, directory, compoundFileName, MergeState.CheckAbort.NONE, newSegment, context);
newSegment.setUseCompoundFile(true);
synchronized(this) {
deleter.deleteNewFiles(newSegment.files());
deleter.deleteNewFiles(files);
}
newSegment.setUseCompoundFile(true);
}
// Have codec write SegmentInfo. Must do this after
@ -2326,7 +2304,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// Now create the compound file if needed
if (useCompoundFile) {
createCompoundFile(directory, IndexFileNames.segmentFileName(mergedName, "", IndexFileNames.COMPOUND_FILE_EXTENSION), MergeState.CheckAbort.NONE, info, context);
createCompoundFile(infoStream, directory, IndexFileNames.segmentFileName(mergedName, "", IndexFileNames.COMPOUND_FILE_EXTENSION), MergeState.CheckAbort.NONE, info, context);
// delete new non cfs files directly: they were never
// registered with IFD
@ -3422,15 +3400,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
final String mergedName = merge.info.name;
int mergedDocCount = 0;
List<SegmentInfo> sourceSegments = merge.segments;
IOContext context = new IOContext(merge.getMergeInfo());
final MergeState.CheckAbort checkAbort = new MergeState.CheckAbort(merge, directory);
SegmentMerger merger = new SegmentMerger(infoStream, directory, config.getTermIndexInterval(), mergedName, checkAbort,
// nocommit
payloadProcessorProvider, new FieldInfos.Builder(globalFieldNumberMap), codec, context);
if (infoStream.isEnabled("IW")) {
@ -3491,7 +3466,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// This is where all the work happens:
MergeState mergeState = merger.merge();
mergedDocCount = merge.info.docCount = mergeState.mergedDocCount;
merge.info.docCount = mergeState.mergedDocCount;
// Record which codec was used to write the segment
@ -3499,11 +3474,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// ctor when we make the merge.info:
merge.info.setCodec(codec);
// nocommit should segment merger do this!? else
// other places must do so...??? addIndexes...
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "merge codec=" + codec + " docCount=" + mergedDocCount + "; merged segment has " +
infoStream.message("IW", "merge codec=" + codec + " docCount=" + merge.info.docCount + "; merged segment has " +
(mergeState.fieldInfos.hasVectors() ? "vectors" : "no vectors") + "; " +
(mergeState.fieldInfos.hasNorms() ? "norms" : "no norms") + "; " +
(mergeState.fieldInfos.hasDocValues() ? "docValues" : "no docValues") + "; " +
@ -3525,10 +3497,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
final String compoundFileName = IndexFileNames.segmentFileName(mergedName, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
try {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "create compound file " + compoundFileName);
}
createCompoundFile(directory, compoundFileName, checkAbort, merge.info, context);
createCompoundFile(infoStream, directory, compoundFileName, checkAbort, merge.info, context);
success = true;
} catch (IOException ioe) {
synchronized(this) {
@ -3556,6 +3525,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
}
// nocommit why do we set success back to false here!?
success = false;
synchronized(this) {
@ -3576,14 +3546,21 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
merge.info.setUseCompoundFile(true);
}
// nocommit need try/success thingy...? ie must
// remove all seg files if we fail to write the .si?
// Have codec write SegmentInfo. Must do this after
// creating CFS so that 1) .si isn't slurped into CFS,
// and 2) .si reflects useCompoundFile=true change
// above:
codec.segmentInfosFormat().getSegmentInfosWriter().write(directory, merge.info, mergeState.fieldInfos, context);
boolean success2 = false;
try {
codec.segmentInfosFormat().getSegmentInfosWriter().write(directory, merge.info, mergeState.fieldInfos, context);
success2 = true;
} finally {
if (!success2) {
synchronized(this) {
deleter.deleteNewFiles(merge.info.files());
}
}
}
merge.info.clearFilesCache();
// nocommit ideally we would freeze merge.info here!!
@ -3626,7 +3603,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
}
}
return mergedDocCount;
return merge.info.docCount;
}
synchronized void addMergeException(MergePolicy.OneMerge merge) {
@ -3801,9 +3778,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
pendingCommit = toSync;
}
// nocommit move this back above...? problem is
// prepareCommit writes on the _X.si files... which
// of course need to be sync'd too...
// This call can take a long time -- 10s of seconds
// or more. We do it without sync:
boolean success = false;
@ -3999,19 +3973,26 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
* deletion files, this SegmentInfo must not reference such files when this
* method is called, because they are not allowed within a compound file.
*/
static final Collection<String> createCompoundFile(Directory directory, String fileName, CheckAbort checkAbort, final SegmentInfo info, IOContext context)
static final Collection<String> createCompoundFile(InfoStream infoStream, Directory directory, String fileName, CheckAbort checkAbort, final SegmentInfo info, IOContext context)
throws IOException {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "create compound file " + fileName);
}
assert info.getDocStoreOffset() == -1;
// Now merge all added files
Collection<String> files = info.files();
CompoundFileDirectory cfsDir = new CompoundFileDirectory(directory, fileName, context, true);
IOException prior = null;
try {
for (String file : files) {
directory.copy(cfsDir, file, file, context);
checkAbort.work(directory.fileLength(file));
}
} catch(IOException ex) {
prior = ex;
} finally {
cfsDir.close();
IOUtils.closeWhileHandlingException(prior, cfsDir);
}
return files;

View File

@ -60,6 +60,7 @@ public class SegmentInfo implements Cloneable {
* - NO if there are no deletes
* - YES or higher if there are deletes at generation N
*/
// nocommit explain that codec need not save this....:
private long delGen;
/*
@ -86,6 +87,7 @@ public class SegmentInfo implements Cloneable {
//TODO: LUCENE-2555: remove once we don't need to support shared doc stores (pre 4.0)
private boolean docStoreIsCompoundFile; // whether doc store files are stored in compound file (*.cfx)
// nocommit explain that codec need not save this....:
private int delCount; // How many deleted docs in this segment
private Codec codec;

View File

@ -280,7 +280,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
String segName = input.readString();
Codec codec = Codec.forName(input.readString());
//System.out.println("SIS.read seg=" + seg + " codec=" + codec);
SegmentInfo info = codec.segmentInfosFormat().getSegmentInfosReader().read(directory, segName);
SegmentInfo info = codec.segmentInfosFormat().getSegmentInfosReader().read(directory, segName, IOContext.READ);
info.setCodec(codec);
info.setDelGen(input.readLong());
info.setDelCount(input.readInt());
@ -352,9 +352,10 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
// list-of-segs plus delGen plus other stuff
// "generically" and then codec gets to write SI
final Set<String> upgradedSIFiles = new HashSet<String>();
try {
// nocommit what IOCtx to use...
segnOutput = new ChecksumIndexOutput(directory.createOutput(segmentFileName, new IOContext(new FlushInfo(totalDocCount(), 0))));
segnOutput = new ChecksumIndexOutput(directory.createOutput(segmentFileName, IOContext.DEFAULT));
CodecUtil.writeHeader(segnOutput, "segments", VERSION_40);
segnOutput.writeLong(version);
segnOutput.writeInt(counter); // write counter
@ -367,19 +368,19 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
segnOutput.writeInt(si.getDelCount());
assert si.dir == directory;
// nocommit hacky!
// If this segment is pre-4.x, perform a one-time
// "ugprade" to write the .si file for it:
String version = si.getVersion();
if (version == null || version.startsWith("3.")) {
String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene3xSegmentInfosFormat.SI_EXTENSION);
if (!directory.fileExists(fileName)) {
//System.out.println("write 3x info seg=" + si.name + " version=" + si.getVersion() + " codec=" + si.getCodec().getName());
write3xInfo(si);
upgradedSIFiles.add(write3xInfo(directory, si, IOContext.DEFAULT));
si.clearFilesCache();
}
}
}
segnOutput.writeStringStringMap(userData);
segnOutput.prepareCommit();
pendingSegnOutput = segnOutput;
success = true;
} finally {
@ -388,7 +389,13 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
// but suppress any exception:
IOUtils.closeWhileHandlingException(segnOutput);
// nocommit must also remove any written .si files...
for(String fileName : upgradedSIFiles) {
try {
directory.deleteFile(fileName);
} catch (Throwable t) {
// Suppress so we keep throwing the original exception
}
}
try {
// Try not to leave a truncated segments_N file in
@ -401,18 +408,14 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
}
}
// nocommit copy of PreflexRWSegmentInfosWriter.write!!
@Deprecated
public void write3xInfo(SegmentInfo si) throws IOException {
public static String write3xInfo(Directory dir, SegmentInfo si, IOContext context) throws IOException {
// NOTE: this is NOT how 3.x is really written...
String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene3xSegmentInfosFormat.SI_EXTENSION);
//System.out.println("UPGRADE write " + fileName);
// nocommit what IOCtx
boolean success = false;
IndexOutput output = si.dir.createOutput(fileName, new IOContext(new FlushInfo(0, 0)));
IndexOutput output = dir.createOutput(fileName, context);
try {
// we are about to write this SI in 3.x format, dropping all codec information, etc.
// so it had better be a 3.x segment or you will get very confusing errors later.
@ -454,11 +457,17 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(output);
si.dir.deleteFile(fileName);
try {
si.dir.deleteFile(fileName);
} catch (Throwable t) {
// Suppress so we keep throwing the original exception
}
} else {
output.close();
}
}
return fileName;
}
/**

View File

@ -26,7 +26,6 @@ import java.util.zip.Checksum;
*
* @lucene.internal
*/
// nocommit fixme to not seek backwards...
public class ChecksumIndexOutput extends IndexOutput {
IndexOutput main;
Checksum digest;
@ -72,29 +71,6 @@ public class ChecksumIndexOutput extends IndexOutput {
throw new UnsupportedOperationException();
}
/**
* Starts but does not complete the commit of this file (=
* writing of the final checksum at the end). After this
* is called must call {@link #finishCommit} and the
* {@link #close} to complete the commit.
*/
public void prepareCommit() throws IOException {
final long checksum = getChecksum();
// Intentionally write a mismatched checksum. This is
// because we want to 1) test, as best we can, that we
// are able to write a long to the file, but 2) not
// actually "commit" the file yet. This (prepare
// commit) is phase 1 of a two-phase commit.
// nocommit fixme... or just nuke? appending codec
// fails w/ this:
/*
final long pos = main.getFilePointer();
main.writeLong(checksum-1);
main.flush();
main.seek(pos);
*/
}
/** See {@link #prepareCommit} */
public void finishCommit() throws IOException {
main.writeLong(getChecksum());

View File

@ -109,7 +109,7 @@ public final class TrackingDirectoryWrapper extends Directory implements Closeab
@Override
public void copy(Directory to, String src, String dest, IOContext context) throws IOException {
// nocommit add dest to created files!?
createdFileNames.add(dest);
other.copy(to, src, dest, context);
}
@ -118,11 +118,6 @@ public final class TrackingDirectoryWrapper extends Directory implements Closeab
return other.createSlicer(name, context);
}
// nocommit no?
//@Override
//protected final void ensureOpen() throws
//AlreadyClosedException {
public Set<String> getCreatedFiles() {
return createdFileNames;
}

View File

@ -129,7 +129,10 @@ public final class CodecUtil {
return checkHeaderNoMagic(in, codec, minVersion, maxVersion);
}
// nocommit jdocs
/** Like {@link
* checkHeader(DataInput,String,int,int)} except this
* version assumes the first int has already been read
* and validated from the input. */
public static int checkHeaderNoMagic(DataInput in, String codec, int minVersion, int maxVersion) throws IOException {
final String actualCodec = in.readString();
if (!actualCodec.equals(codec)) {

View File

@ -205,7 +205,7 @@ public class TestDoc extends LuceneTestCase {
false, null, false, 0, codec, null);
if (useCompoundFile) {
Collection<String> filesToDelete = IndexWriter.createCompoundFile(dir, merged + ".cfs", MergeState.CheckAbort.NONE, info, newIOContext(random()));
Collection<String> filesToDelete = IndexWriter.createCompoundFile(InfoStream.getDefault(), dir, merged + ".cfs", MergeState.CheckAbort.NONE, info, newIOContext(random()));
info.setUseCompoundFile(true);
for (final String fileToDelete : filesToDelete) {
si1.dir.deleteFile(fileToDelete);

View File

@ -46,57 +46,7 @@ class PreFlexRWSegmentInfosWriter extends SegmentInfosWriter {
/** Save a single segment's info. */
@Override
public void write(Directory dir ,SegmentInfo si, FieldInfos fis, IOContext ioContext) throws IOException {
String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene3xSegmentInfosFormat.SI_EXTENSION);
boolean success = false;
IndexOutput output = dir.createOutput(fileName, ioContext);
try {
// we are about to write this SI in 3.x format, dropping all codec information, etc.
// so it had better be a 3.x segment or you will get very confusing errors later.
assert si.getCodec() instanceof Lucene3xCodec : "broken test, trying to mix preflex with other codecs";
assert si.getDelCount() <= si.docCount: "delCount=" + si.getDelCount() + " docCount=" + si.docCount + " segment=" + si.name;
// Write the Lucene version that created this segment, since 3.1
output.writeString(si.getVersion());
output.writeString(si.name);
output.writeInt(si.docCount);
output.writeLong(si.getDelGen());
output.writeInt(si.getDocStoreOffset());
if (si.getDocStoreOffset() != -1) {
output.writeString(si.getDocStoreSegment());
output.writeByte((byte) (si.getDocStoreIsCompoundFile() ? 1:0));
}
// pre-4.0 indexes write a byte if there is a single norms file
output.writeByte((byte) 1);
Map<Integer,Long> normGen = si.getNormGen();
if (normGen == null) {
output.writeInt(SegmentInfo.NO);
} else {
output.writeInt(normGen.size());
for (Entry<Integer,Long> entry : normGen.entrySet()) {
output.writeLong(entry.getValue());
}
}
output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
output.writeInt(si.getDelCount());
// hasProx:
output.writeByte((byte) 1);
output.writeStringStringMap(si.getDiagnostics());
// hasVectors:
output.writeByte((byte) 1);
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(output);
} else {
output.close();
}
}
public void write(Directory dir, SegmentInfo si, FieldInfos fis, IOContext ioContext) throws IOException {
SegmentInfos.write3xInfo(dir, si, ioContext);
}
}