LUCENE-5969: move CFS to codec

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5969@1628996 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2014-10-02 15:27:13 +00:00
parent 6456948a47
commit af3e1d1c4b
32 changed files with 923 additions and 806 deletions

View File

@ -23,7 +23,6 @@ import java.util.Arrays;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.index.IndexFormatTooOldException;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
@ -166,7 +165,7 @@ final class BitVector implements Cloneable, MutableBits {
<code>d</code>, in a format that can be read by the constructor {@link
#BitVector(Directory, String, IOContext)}. */
final void write(Directory d, String name, IOContext context) throws IOException {
assert !(d instanceof CompoundFileDirectory);
assert !(d instanceof Lucene40CompoundReader);
try (IndexOutput output = d.createOutput(name, context)) {
output.writeInt(-2);
CodecUtil.writeHeader(output, CODEC, VERSION_CURRENT);

View File

@ -18,6 +18,7 @@ package org.apache.lucene.codecs.lucene40;
*/
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.PostingsFormat;
@ -39,6 +40,7 @@ public class Lucene40Codec extends Codec {
private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
private final CompoundFormat compoundFormat = new Lucene40CompoundFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@Override
@ -77,6 +79,11 @@ public class Lucene40Codec extends Codec {
return infosFormat;
}
@Override
public CompoundFormat compoundFormat() {
return compoundFormat;
}
private final DocValuesFormat defaultDVFormat = new Lucene40DocValuesFormat();
@Override

View File

@ -0,0 +1,53 @@
package org.apache.lucene.codecs.lucene40;
/*
* 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.codecs.CompoundFormat;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.MergeState.CheckAbort;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/**
* Lucene 4.0 compound file format
* @deprecated only for reading old 4.x segments
*/
@Deprecated
public final class Lucene40CompoundFormat extends CompoundFormat {
@Override
public Directory getCompoundReader(Directory dir, SegmentInfo si, IOContext context) throws IOException {
String fileName = IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
return new Lucene40CompoundReader(dir, fileName, context, false);
}
@Override
public void write(Directory dir, SegmentInfo si, Collection<String> files, CheckAbort checkAbort, IOContext context) throws IOException {
String fileName = IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
try (Directory cfs = new Lucene40CompoundReader(dir, fileName, context, true)) {
for (String file : files) {
dir.copy(cfs, file, file, context);
checkAbort.work(dir.fileLength(file));
}
}
}
}

View File

@ -0,0 +1,245 @@
package org.apache.lucene.codecs.lucene40;
/*
* 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 org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.store.BaseDirectory;
import org.apache.lucene.store.BufferedIndexInput;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.Lock;
import org.apache.lucene.util.IOUtils;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.io.FileNotFoundException;
import java.io.IOException;
/**
* Lucene 4.x compound file format
* @deprecated only for reading 4.x segments
*/
@Deprecated
final class Lucene40CompoundReader extends BaseDirectory {
// TODO: would be great to move this read-write stuff out of here into test.
/** Offset/Length for a slice inside of a compound file */
public static final class FileEntry {
long offset;
long length;
}
private final Directory directory;
private final String fileName;
protected final int readBufferSize;
private final Map<String,FileEntry> entries;
private final boolean openForWrite;
private static final Map<String,FileEntry> SENTINEL = Collections.emptyMap();
private final Lucene40CompoundWriter writer;
private final IndexInput handle;
private int version;
/**
* Create a new CompoundFileDirectory.
*/
public Lucene40CompoundReader(Directory directory, String fileName, IOContext context, boolean openForWrite) throws IOException {
this.directory = directory;
this.fileName = fileName;
this.readBufferSize = BufferedIndexInput.bufferSize(context);
this.isOpen = false;
this.openForWrite = openForWrite;
if (!openForWrite) {
boolean success = false;
handle = directory.openInput(fileName, context);
try {
this.entries = readEntries(directory, fileName);
if (version >= Lucene40CompoundWriter.VERSION_CHECKSUM) {
CodecUtil.checkHeader(handle, Lucene40CompoundWriter.DATA_CODEC, version, version);
// NOTE: data file is too costly to verify checksum against all the bytes on open,
// but for now we at least verify proper structure of the checksum footer: which looks
// for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
// such as file truncation.
CodecUtil.retrieveChecksum(handle);
}
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(handle);
}
}
this.isOpen = true;
writer = null;
} else {
assert !(directory instanceof Lucene40CompoundReader) : "compound file inside of compound file: " + fileName;
this.entries = SENTINEL;
this.isOpen = true;
writer = new Lucene40CompoundWriter(directory, fileName);
handle = null;
}
}
/** Helper method that reads CFS entries from an input stream */
private final Map<String, FileEntry> readEntries(Directory dir, String name) throws IOException {
ChecksumIndexInput entriesStream = null;
Map<String,FileEntry> mapping = null;
boolean success = false;
try {
final String entriesFileName = IndexFileNames.segmentFileName(
IndexFileNames.stripExtension(name), "",
IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
entriesStream = dir.openChecksumInput(entriesFileName, IOContext.READONCE);
version = CodecUtil.checkHeader(entriesStream, Lucene40CompoundWriter.ENTRY_CODEC, Lucene40CompoundWriter.VERSION_START, Lucene40CompoundWriter.VERSION_CURRENT);
final int numEntries = entriesStream.readVInt();
mapping = new HashMap<>(numEntries);
for (int i = 0; i < numEntries; i++) {
final FileEntry fileEntry = new FileEntry();
final String id = entriesStream.readString();
FileEntry previous = mapping.put(id, fileEntry);
if (previous != null) {
throw new CorruptIndexException("Duplicate cfs entry id=" + id + " in CFS ", entriesStream);
}
fileEntry.offset = entriesStream.readLong();
fileEntry.length = entriesStream.readLong();
}
if (version >= Lucene40CompoundWriter.VERSION_CHECKSUM) {
CodecUtil.checkFooter(entriesStream);
} else {
CodecUtil.checkEOF(entriesStream);
}
success = true;
} finally {
if (success) {
IOUtils.close(entriesStream);
} else {
IOUtils.closeWhileHandlingException(entriesStream);
}
}
return mapping;
}
public Directory getDirectory() {
return directory;
}
public String getName() {
return fileName;
}
@Override
public synchronized void close() throws IOException {
if (!isOpen) {
// allow double close - usually to be consistent with other closeables
return; // already closed
}
isOpen = false;
if (writer != null) {
assert openForWrite;
writer.close();
} else {
IOUtils.close(handle);
}
}
@Override
public synchronized IndexInput openInput(String name, IOContext context) throws IOException {
ensureOpen();
assert !openForWrite;
final String id = IndexFileNames.stripSegmentName(name);
final FileEntry entry = entries.get(id);
if (entry == null) {
throw new FileNotFoundException("No sub-file with id " + id + " found (fileName=" + name + " files: " + entries.keySet() + ")");
}
return handle.slice(name, entry.offset, entry.length);
}
/** Returns an array of strings, one for each file in the directory. */
@Override
public String[] listAll() {
ensureOpen();
String[] res;
if (writer != null) {
res = writer.listAll();
} else {
res = entries.keySet().toArray(new String[entries.size()]);
// Add the segment name
String seg = IndexFileNames.parseSegmentName(fileName);
for (int i = 0; i < res.length; i++) {
res[i] = seg + res[i];
}
}
return res;
}
/** Not implemented
* @throws UnsupportedOperationException always: not supported by CFS */
@Override
public void deleteFile(String name) {
throw new UnsupportedOperationException();
}
/** Not implemented
* @throws UnsupportedOperationException always: not supported by CFS */
public void renameFile(String from, String to) {
throw new UnsupportedOperationException();
}
/** Returns the length of a file in the directory.
* @throws IOException if the file does not exist */
@Override
public long fileLength(String name) throws IOException {
ensureOpen();
if (this.writer != null) {
return writer.fileLength(name);
}
FileEntry e = entries.get(IndexFileNames.stripSegmentName(name));
if (e == null)
throw new FileNotFoundException(name);
return e.length;
}
@Override
public IndexOutput createOutput(String name, IOContext context) throws IOException {
ensureOpen();
return writer.createOutput(name, context);
}
@Override
public void sync(Collection<String> names) {
throw new UnsupportedOperationException();
}
/** Not implemented
* @throws UnsupportedOperationException always: not supported by CFS */
@Override
public Lock makeLock(String name) {
throw new UnsupportedOperationException();
}
@Override
public String toString() {
return "CompoundFileDirectory(file=\"" + fileName + "\" in dir=" + directory + ")";
}
}

View File

@ -0,0 +1,359 @@
package org.apache.lucene.codecs.lucene40;
/*
* 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.Closeable;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FlushInfo;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.IOUtils;
/**
* Combines multiple files into a single compound file.
* @deprecated only for testing
*/
@Deprecated
final class Lucene40CompoundWriter implements Closeable{
private static final class FileEntry {
/** source file */
String file;
long length;
/** temporary holder for the start of this file's data section */
long offset;
/** the directory which contains the file. */
Directory dir;
}
// versioning for the .cfs file
static final String DATA_CODEC = "CompoundFileWriterData";
static final int VERSION_START = 0;
static final int VERSION_CHECKSUM = 1;
static final int VERSION_CURRENT = VERSION_CHECKSUM;
// versioning for the .cfe file
static final String ENTRY_CODEC = "CompoundFileWriterEntries";
private final Directory directory;
private final Map<String, FileEntry> entries = new HashMap<>();
private final Set<String> seenIDs = new HashSet<>();
// all entries that are written to a sep. file but not yet moved into CFS
private final Queue<FileEntry> pendingEntries = new LinkedList<>();
private boolean closed = false;
private IndexOutput dataOut;
private final AtomicBoolean outputTaken = new AtomicBoolean(false);
final String entryTableName;
final String dataFileName;
/**
* Create the compound stream in the specified file. The file name is the
* entire name (no extensions are added).
*
* @throws NullPointerException
* if <code>dir</code> or <code>name</code> is null
*/
Lucene40CompoundWriter(Directory dir, String name) {
if (dir == null)
throw new NullPointerException("directory cannot be null");
if (name == null)
throw new NullPointerException("name cannot be null");
directory = dir;
entryTableName = IndexFileNames.segmentFileName(
IndexFileNames.stripExtension(name), "",
IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
dataFileName = name;
}
private synchronized IndexOutput getOutput(IOContext context) throws IOException {
if (dataOut == null) {
boolean success = false;
try {
dataOut = directory.createOutput(dataFileName, context);
CodecUtil.writeHeader(dataOut, DATA_CODEC, VERSION_CURRENT);
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(dataOut);
}
}
}
return dataOut;
}
/** Returns the directory of the compound file. */
Directory getDirectory() {
return directory;
}
/** Returns the name of the compound file. */
String getName() {
return dataFileName;
}
/**
* Closes all resources and writes the entry table
*
* @throws IllegalStateException
* if close() had been called before or if no file has been added to
* this object
*/
@Override
public void close() throws IOException {
if (closed) {
return;
}
IndexOutput entryTableOut = null;
// TODO this code should clean up after itself
// (remove partial .cfs/.cfe)
boolean success = false;
try {
if (!pendingEntries.isEmpty() || outputTaken.get()) {
throw new IllegalStateException("CFS has pending open files");
}
closed = true;
// open the compound stream; we can safely use IOContext.DEFAULT
// here because this will only open the output if no file was
// added to the CFS
getOutput(IOContext.DEFAULT);
assert dataOut != null;
CodecUtil.writeFooter(dataOut);
success = true;
} finally {
if (success) {
IOUtils.close(dataOut);
} else {
IOUtils.closeWhileHandlingException(dataOut);
}
}
success = false;
try {
entryTableOut = directory.createOutput(entryTableName, IOContext.DEFAULT);
writeEntryTable(entries.values(), entryTableOut);
success = true;
} finally {
if (success) {
IOUtils.close(entryTableOut);
} else {
IOUtils.closeWhileHandlingException(entryTableOut);
}
}
}
private final void ensureOpen() {
if (closed) {
throw new AlreadyClosedException("CFS Directory is already closed");
}
}
/**
* Copy the contents of the file with specified extension into the provided
* output stream.
*/
private final long copyFileEntry(IndexOutput dataOut, FileEntry fileEntry)
throws IOException {
final IndexInput is = fileEntry.dir.openInput(fileEntry.file, IOContext.READONCE);
boolean success = false;
try {
final long startPtr = dataOut.getFilePointer();
final long length = fileEntry.length;
dataOut.copyBytes(is, length);
// Verify that the output length diff is equal to original file
long endPtr = dataOut.getFilePointer();
long diff = endPtr - startPtr;
if (diff != length)
throw new IOException("Difference in the output file offsets " + diff
+ " does not match the original file length " + length);
fileEntry.offset = startPtr;
success = true;
return length;
} finally {
if (success) {
IOUtils.close(is);
// copy successful - delete file
// if we can't we rely on IFD to pick up and retry
IOUtils.deleteFilesIgnoringExceptions(fileEntry.dir, fileEntry.file);
} else {
IOUtils.closeWhileHandlingException(is);
}
}
}
protected void writeEntryTable(Collection<FileEntry> entries,
IndexOutput entryOut) throws IOException {
CodecUtil.writeHeader(entryOut, ENTRY_CODEC, VERSION_CURRENT);
entryOut.writeVInt(entries.size());
for (FileEntry fe : entries) {
entryOut.writeString(IndexFileNames.stripSegmentName(fe.file));
entryOut.writeLong(fe.offset);
entryOut.writeLong(fe.length);
}
CodecUtil.writeFooter(entryOut);
}
IndexOutput createOutput(String name, IOContext context) throws IOException {
ensureOpen();
boolean success = false;
boolean outputLocked = false;
try {
assert name != null : "name must not be null";
if (entries.containsKey(name)) {
throw new IllegalArgumentException("File " + name + " already exists");
}
final FileEntry entry = new FileEntry();
entry.file = name;
entries.put(name, entry);
final String id = IndexFileNames.stripSegmentName(name);
assert !seenIDs.contains(id): "file=\"" + name + "\" maps to id=\"" + id + "\", which was already written";
seenIDs.add(id);
final DirectCFSIndexOutput out;
if ((outputLocked = outputTaken.compareAndSet(false, true))) {
out = new DirectCFSIndexOutput(getOutput(context), entry, false);
} else {
entry.dir = this.directory;
out = new DirectCFSIndexOutput(directory.createOutput(name, context), entry,
true);
}
success = true;
return out;
} finally {
if (!success) {
entries.remove(name);
if (outputLocked) { // release the output lock if not successful
assert outputTaken.get();
releaseOutputLock();
}
}
}
}
final void releaseOutputLock() {
outputTaken.compareAndSet(true, false);
}
private final void prunePendingEntries() throws IOException {
// claim the output and copy all pending files in
if (outputTaken.compareAndSet(false, true)) {
try {
while (!pendingEntries.isEmpty()) {
FileEntry entry = pendingEntries.poll();
copyFileEntry(getOutput(new IOContext(new FlushInfo(0, entry.length))), entry);
entries.put(entry.file, entry);
}
} finally {
final boolean compareAndSet = outputTaken.compareAndSet(true, false);
assert compareAndSet;
}
}
}
long fileLength(String name) throws IOException {
FileEntry fileEntry = entries.get(name);
if (fileEntry == null) {
throw new FileNotFoundException(name + " does not exist");
}
return fileEntry.length;
}
boolean fileExists(String name) {
return entries.containsKey(name);
}
String[] listAll() {
return entries.keySet().toArray(new String[0]);
}
private final class DirectCFSIndexOutput extends IndexOutput {
private final IndexOutput delegate;
private final long offset;
private boolean closed;
private FileEntry entry;
private long writtenBytes;
private final boolean isSeparate;
DirectCFSIndexOutput(IndexOutput delegate, FileEntry entry,
boolean isSeparate) {
super();
this.delegate = delegate;
this.entry = entry;
entry.offset = offset = delegate.getFilePointer();
this.isSeparate = isSeparate;
}
@Override
public void close() throws IOException {
if (!closed) {
closed = true;
entry.length = writtenBytes;
if (isSeparate) {
delegate.close();
// we are a separate file - push into the pending entries
pendingEntries.add(entry);
} else {
// we have been written into the CFS directly - release the lock
releaseOutputLock();
}
// now prune all pending entries and push them into the CFS
prunePendingEntries();
}
}
@Override
public long getFilePointer() {
return delegate.getFilePointer() - offset;
}
@Override
public void writeByte(byte b) throws IOException {
assert !closed;
writtenBytes++;
delegate.writeByte(b);
}
@Override
public void writeBytes(byte[] b, int offset, int length) throws IOException {
assert !closed;
writtenBytes += length;
delegate.writeBytes(b, offset, length);
}
@Override
public long getChecksum() throws IOException {
return delegate.getChecksum();
}
}
}

View File

@ -34,7 +34,6 @@ import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
@ -85,7 +84,7 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
Lucene40DocValuesReader(SegmentReadState state, String filename, String legacyKey) throws IOException {
this.state = state;
this.legacyKey = legacyKey;
this.dir = new CompoundFileDirectory(state.segmentInfo.getId(), state.directory, filename, state.context, false);
this.dir = new Lucene40CompoundReader(state.directory, filename, state.context, false);
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOf(getClass()));
merging = false;
}

View File

@ -18,6 +18,7 @@ package org.apache.lucene.codecs.lucene41;
*/
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.PostingsFormat;
@ -26,6 +27,7 @@ import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40CompoundFormat;
import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
@ -45,6 +47,7 @@ public class Lucene41Codec extends Codec {
private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
private final CompoundFormat compoundFormat = new Lucene40CompoundFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@Override
@ -87,6 +90,11 @@ public class Lucene41Codec extends Codec {
public final LiveDocsFormat liveDocsFormat() {
return liveDocsFormat;
}
@Override
public CompoundFormat compoundFormat() {
return compoundFormat;
}
/** Returns the postings format that should be used for writing
* new segments of <code>field</code>.

View File

@ -20,6 +20,7 @@ package org.apache.lucene.codecs.lucene410;
import java.io.IOException;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
@ -29,6 +30,7 @@ import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40CompoundFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
@ -50,6 +52,7 @@ public class Lucene410Codec extends Codec {
private final FieldInfosFormat fieldInfosFormat = new Lucene46FieldInfosFormat();
private final SegmentInfoFormat segmentInfosFormat = new Lucene46SegmentInfoFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
private final CompoundFormat compoundFormat = new Lucene40CompoundFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@Override
@ -99,6 +102,11 @@ public class Lucene410Codec extends Codec {
public final LiveDocsFormat liveDocsFormat() {
return liveDocsFormat;
}
@Override
public CompoundFormat compoundFormat() {
return compoundFormat;
}
/** Returns the postings format that should be used for writing
* new segments of <code>field</code>.

View File

@ -20,6 +20,7 @@ package org.apache.lucene.codecs.lucene42;
import java.io.IOException;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
@ -29,6 +30,7 @@ import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40CompoundFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
@ -47,6 +49,7 @@ public class Lucene42Codec extends Codec {
private final FieldInfosFormat fieldInfosFormat = new Lucene42FieldInfosFormat();
private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
private final CompoundFormat compoundFormat = new Lucene40CompoundFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@Override
@ -97,6 +100,11 @@ public class Lucene42Codec extends Codec {
public final LiveDocsFormat liveDocsFormat() {
return liveDocsFormat;
}
@Override
public CompoundFormat compoundFormat() {
return compoundFormat;
}
/** Returns the postings format that should be used for writing
* new segments of <code>field</code>.

View File

@ -20,6 +20,7 @@ package org.apache.lucene.codecs.lucene45;
import java.io.IOException;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
@ -29,6 +30,7 @@ import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40CompoundFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
@ -50,6 +52,7 @@ public class Lucene45Codec extends Codec {
private final FieldInfosFormat fieldInfosFormat = new Lucene42FieldInfosFormat();
private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
private final CompoundFormat compoundFormat = new Lucene40CompoundFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@Override
@ -100,6 +103,11 @@ public class Lucene45Codec extends Codec {
public final LiveDocsFormat liveDocsFormat() {
return liveDocsFormat;
}
@Override
public CompoundFormat compoundFormat() {
return compoundFormat;
}
/** Returns the postings format that should be used for writing
* new segments of <code>field</code>.

View File

@ -20,6 +20,7 @@ package org.apache.lucene.codecs.lucene46;
import java.io.IOException;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
@ -29,6 +30,7 @@ import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40CompoundFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
@ -48,6 +50,7 @@ public class Lucene46Codec extends Codec {
private final FieldInfosFormat fieldInfosFormat = new Lucene46FieldInfosFormat();
private final SegmentInfoFormat segmentInfosFormat = new Lucene46SegmentInfoFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
private final CompoundFormat compoundFormat = new Lucene40CompoundFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@Override
@ -97,6 +100,11 @@ public class Lucene46Codec extends Codec {
public final LiveDocsFormat liveDocsFormat() {
return liveDocsFormat;
}
@Override
public CompoundFormat compoundFormat() {
return compoundFormat;
}
/** Returns the postings format that should be used for writing
* new segments of <code>field</code>.

View File

@ -20,6 +20,7 @@ package org.apache.lucene.codecs.lucene49;
import java.io.IOException;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
@ -29,6 +30,7 @@ import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40CompoundFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
@ -49,6 +51,7 @@ public class Lucene49Codec extends Codec {
private final FieldInfosFormat fieldInfosFormat = new Lucene46FieldInfosFormat();
private final SegmentInfoFormat segmentInfosFormat = new Lucene46SegmentInfoFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
private final CompoundFormat compoundFormat = new Lucene40CompoundFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@Override
@ -98,6 +101,11 @@ public class Lucene49Codec extends Codec {
public final LiveDocsFormat liveDocsFormat() {
return liveDocsFormat;
}
@Override
public CompoundFormat compoundFormat() {
return compoundFormat;
}
/** Returns the postings format that should be used for writing
* new segments of <code>field</code>.

View File

@ -29,7 +29,6 @@ import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosReader.LegacyDocValue
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
@ -51,7 +50,7 @@ final class Lucene40DocValuesWriter extends DocValuesConsumer {
Lucene40DocValuesWriter(SegmentWriteState state, String filename, String legacyKey) throws IOException {
this.state = state;
this.legacyKey = legacyKey;
this.dir = new CompoundFileDirectory(state.segmentInfo.getId(), state.directory, filename, state.context, true);
this.dir = new Lucene40CompoundReader(state.directory, filename, state.context, true);
}
@Override

View File

@ -18,6 +18,7 @@ package org.apache.lucene.codecs.simpletext;
*/
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.PostingsFormat;
@ -26,6 +27,7 @@ import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
/**
* plain text index format.
@ -42,6 +44,8 @@ public final class SimpleTextCodec extends Codec {
private final NormsFormat normsFormat = new SimpleTextNormsFormat();
private final LiveDocsFormat liveDocs = new SimpleTextLiveDocsFormat();
private final DocValuesFormat dvFormat = new SimpleTextDocValuesFormat();
// nocommit
private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
public SimpleTextCodec() {
super("SimpleText");
@ -86,4 +90,9 @@ public final class SimpleTextCodec extends Codec {
public DocValuesFormat docValuesFormat() {
return dvFormat;
}
@Override
public CompoundFormat compoundFormat() {
return compoundFormat;
}
}

View File

@ -86,6 +86,9 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
/** Encodes/decodes live docs */
public abstract LiveDocsFormat liveDocsFormat();
/** Encodes/decodes compound files */
public abstract CompoundFormat compoundFormat();
/** looks up a codec by name */
public static Codec forName(String name) {
if (loader == null) {

View File

@ -0,0 +1,52 @@
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.MergeState.CheckAbort;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/**
* Encodes/decodes compound files
* @lucene.experimental
*/
public abstract class CompoundFormat {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
public CompoundFormat() {
}
// TODO: this is very minimal. If we need more methods,
// we can add 'producer' classes.
/**
* Returns a Directory view (read-only) for the compound files in this segment
*/
public abstract Directory getCompoundReader(Directory dir, SegmentInfo si, IOContext context) throws IOException;
/**
* Packs the provided files into a compound format.
*/
// TODO: get checkAbort out of here, and everywhere, and have iw do it at a higher level
public abstract void write(Directory dir, SegmentInfo si, Collection<String> files, CheckAbort checkAbort, IOContext context) throws IOException;
}

View File

@ -99,4 +99,8 @@ public abstract class FilterCodec extends Codec {
return delegate.termVectorsFormat();
}
@Override
public CompoundFormat compoundFormat() {
return delegate.compoundFormat();
}
}

View File

@ -1,4 +1,4 @@
package org.apache.lucene.store;
package org.apache.lucene.codecs.lucene50;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -22,7 +22,15 @@ import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.LiveDocsFormat; // javadocs
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.store.BaseDirectory;
import org.apache.lucene.store.BufferedIndexInput;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.DataOutput; // javadocs
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.Lock;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.StringHelper;
@ -74,7 +82,7 @@ import java.io.IOException;
*
* @lucene.experimental
*/
public final class CompoundFileDirectory extends BaseDirectory {
final class CompoundFileDirectory extends BaseDirectory {
/** Offset/Length for a slice inside of a compound file */
public static final class FileEntry {

View File

@ -1,4 +1,4 @@
package org.apache.lucene.store;
package org.apache.lucene.codecs.lucene50;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -31,6 +31,12 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FlushInfo;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.IOUtils;
/**

View File

@ -18,6 +18,7 @@ package org.apache.lucene.codecs.lucene50;
*/
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec;
@ -46,6 +47,7 @@ public class Lucene50Codec extends Codec {
private final FieldInfosFormat fieldInfosFormat = new Lucene50FieldInfosFormat();
private final SegmentInfoFormat segmentInfosFormat = new Lucene50SegmentInfoFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@Override
@ -96,6 +98,11 @@ public class Lucene50Codec extends Codec {
return liveDocsFormat;
}
@Override
public final CompoundFormat compoundFormat() {
return compoundFormat;
}
/** Returns the postings format that should be used for writing
* new segments of <code>field</code>.
*

View File

@ -0,0 +1,80 @@
package org.apache.lucene.codecs.lucene50;
/*
* 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.codecs.CodecUtil;
import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.MergeState.CheckAbort;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/**
* Lucene 5.0 compound file format
* <p>
* Files:
* <ul>
* <li><tt>.cfs</tt>: An optional "virtual" file consisting of all the other
* index files for systems that frequently run out of file handles.
* <li><tt>.cfe</tt>: The "virtual" compound file's entry table holding all
* entries in the corresponding .cfs file.
* </ul>
* <p>Description:</p>
* <ul>
* <li>Compound (.cfs) --&gt; Header, FileData <sup>FileCount</sup>, Footer</li>
* <li>Compound Entry Table (.cfe) --&gt; Header, FileCount, &lt;FileName,
* DataOffset, DataLength&gt; <sup>FileCount</sup></li>
* <li>Header --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
* <li>FileCount --&gt; {@link DataOutput#writeVInt VInt}</li>
* <li>DataOffset,DataLength,Checksum --&gt; {@link DataOutput#writeLong UInt64}</li>
* <li>FileName --&gt; {@link DataOutput#writeString String}</li>
* <li>FileData --&gt; raw file data</li>
* <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
* </ul>
* <p>Notes:</p>
* <ul>
* <li>FileCount indicates how many files are contained in this compound file.
* The entry table that follows has that many entries.
* <li>Each directory entry contains a long pointer to the start of this file's data
* section, the files length, and a String with that file's name.
* </ul>
*/
public final class Lucene50CompoundFormat extends CompoundFormat {
@Override
public Directory getCompoundReader(Directory dir, SegmentInfo si, IOContext context) throws IOException {
String fileName = IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
return new CompoundFileDirectory(si.getId(), dir, fileName, context, false);
}
@Override
public void write(Directory dir, SegmentInfo si, Collection<String> files, CheckAbort checkAbort, IOContext context) throws IOException {
String fileName = IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
try (CompoundFileDirectory cfs = new CompoundFileDirectory(si.getId(), dir, fileName, context, true)) {
for (String file : files) {
dir.copy(cfs, file, file, context);
checkAbort.work(dir.fileLength(file));
}
}
}
}

View File

@ -50,7 +50,6 @@ import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.index.MergeState.CheckAbort;
import org.apache.lucene.search.Query;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.Lock;
@ -4453,40 +4452,30 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
static final Collection<String> createCompoundFile(InfoStream infoStream, Directory directory, CheckAbort checkAbort, final SegmentInfo info, IOContext context)
throws IOException {
final String fileName = IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
// nocommit: use trackingdirectorywrapper instead to know which files to delete when things fail:
String cfsFileName = IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
String cfeFileName = IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "create compound file " + fileName);
infoStream.message("IW", "create compound file");
}
// Now merge all added files
Collection<String> files = info.files();
CompoundFileDirectory cfsDir = new CompoundFileDirectory(info.getId(), directory, fileName, context, true);
boolean success = false;
try {
for (String file : files) {
directory.copy(cfsDir, file, file, context);
checkAbort.work(directory.fileLength(file));
}
info.getCodec().compoundFormat().write(directory, info, files, checkAbort, context);
success = true;
} finally {
if (success) {
IOUtils.close(cfsDir);
} else {
IOUtils.closeWhileHandlingException(cfsDir);
try {
directory.deleteFile(fileName);
} catch (Throwable t) {
}
try {
directory.deleteFile(IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
} catch (Throwable t) {
}
if (!success) {
IOUtils.deleteFilesIgnoringExceptions(directory, cfsFileName, cfeFileName);
}
}
// Replace all previous files with the CFS/CFE files:
Set<String> siFiles = new HashSet<>();
siFiles.add(fileName);
siFiles.add(IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
siFiles.add(cfsFileName);
siFiles.add(cfeFileName);
info.setFiles(siFiles);
return files;

View File

@ -33,7 +33,6 @@ import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.index.LeafReader.CoreClosedListener;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Accountable;
@ -60,7 +59,7 @@ final class SegmentCoreReaders implements Accountable {
final StoredFieldsReader fieldsReaderOrig;
final TermVectorsReader termVectorsReaderOrig;
final CompoundFileDirectory cfsReader;
final Directory cfsReader;
// TODO: make a single thread local w/ a
// Thingy class holding fieldsReader, termVectorsReader,
@ -99,7 +98,7 @@ final class SegmentCoreReaders implements Accountable {
try {
if (si.info.getUseCompoundFile()) {
cfsDir = cfsReader = new CompoundFileDirectory(si.info.getId(), dir, IndexFileNames.segmentFileName(si.info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
cfsDir = cfsReader = codec.compoundFormat().getCompoundReader(dir, si.info, context);
} else {
cfsReader = null;
cfsDir = dir;

View File

@ -263,7 +263,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
}
/** Since Lucene 5.0, every commit (segments_N) writes a unique id. This will
* return that id, or null if this commit was 5.0. */
* return that id, or null if this commit was prior to 5.0. */
public byte[] getId() {
return id == null ? null : id.clone();
}

View File

@ -33,7 +33,6 @@ import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Accountable;
@ -202,10 +201,7 @@ public final class SegmentReader extends LeafReader implements Accountable {
final boolean closeDir;
if (info.getFieldInfosGen() == -1 && info.info.getUseCompoundFile()) {
// no fieldInfos gen and segment uses a compound file
dir = new CompoundFileDirectory(info.info.getId(), info.info.dir,
IndexFileNames.segmentFileName(info.info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION),
IOContext.READONCE,
false);
dir = info.info.getCodec().compoundFormat().getCompoundReader(info.info.dir, info.info, IOContext.READONCE);
closeDir = true;
} else {
// gen'd FIS are read outside CFS, or the segment doesn't use a compound file

View File

@ -100,12 +100,7 @@ public abstract class Directory implements Closeable {
*/
public abstract void renameFile(String source, String dest) throws IOException;
/** Returns a stream reading an existing file, with the
* specified read buffer size. The particular Directory
* implementation may ignore the buffer size. Currently
* the only Directory implementations that respect this
* parameter are {@link FSDirectory} and {@link
* CompoundFileDirectory}.
/** Returns a stream reading an existing file.
* <p>Throws {@link FileNotFoundException} or {@link NoSuchFileException}
* if the file does not exist.
*/

View File

@ -24,7 +24,6 @@ import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.MockDirectoryWrapper;
@ -76,12 +75,11 @@ public class TestAllFilesHaveChecksumFooter extends LuceneTestCase {
for (SegmentCommitInfo si : sis) {
for (String file : si.files()) {
checkFooter(dir, file);
if (file.endsWith(IndexFileNames.COMPOUND_FILE_EXTENSION)) {
// recurse into CFS
try (CompoundFileDirectory cfsDir = new CompoundFileDirectory(si.info.getId(), dir, file, newIOContext(random()), false)) {
for (String cfsFile : cfsDir.listAll()) {
checkFooter(cfsDir, cfsFile);
}
}
if (si.info.getUseCompoundFile()) {
try (Directory cfsDir = si.info.getCodec().compoundFormat().getCompoundReader(dir, si.info, newIOContext(random()))) {
for (String cfsFile : cfsDir.listAll()) {
checkFooter(cfsDir, cfsFile);
}
}
}

View File

@ -28,7 +28,6 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.MockDirectoryWrapper;
@ -89,12 +88,11 @@ public class TestAllFilesHaveCodecHeader extends LuceneTestCase {
for (SegmentCommitInfo si : sis) {
for (String file : si.files()) {
checkHeader(dir, file, namesToExtensions);
if (file.endsWith(IndexFileNames.COMPOUND_FILE_EXTENSION)) {
// recurse into CFS
try (CompoundFileDirectory cfsDir = new CompoundFileDirectory(si.info.getId(), dir, file, newIOContext(random()), false)) {
for (String cfsFile : cfsDir.listAll()) {
checkHeader(cfsDir, cfsFile, namesToExtensions);
}
}
if (si.info.getUseCompoundFile()) {
try (Directory cfsDir = si.info.getCodec().compoundFormat().getCompoundReader(dir, si.info, newIOContext(random()))) {
for (String cfsFile : cfsDir.listAll()) {
checkHeader(cfsDir, cfsFile, namesToExtensions);
}
}
}

View File

@ -19,7 +19,6 @@ package org.apache.lucene.index;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
@ -33,487 +32,6 @@ import java.io.IOException;
import java.nio.file.Path;
public class TestCompoundFile extends LuceneTestCase {
private Directory dir;
@Override
public void setUp() throws Exception {
super.setUp();
Path file = createTempDir("testIndex");
dir = newFSDirectory(file);
}
@Override
public void tearDown() throws Exception {
dir.close();
super.tearDown();
}
/** Creates a file of the specified size with random data. */
private void createRandomFile(Directory dir, String name, int size) throws IOException {
IndexOutput os = dir.createOutput(name, newIOContext(random()));
for (int i=0; i<size; i++) {
byte b = (byte) (Math.random() * 256);
os.writeByte(b);
}
os.close();
}
/** Creates a file of the specified size with sequential data. The first
* byte is written as the start byte provided. All subsequent bytes are
* computed as start + offset where offset is the number of the byte.
*/
static void createSequenceFile(Directory dir, String name, byte start, int size) throws IOException {
IndexOutput os = dir.createOutput(name, newIOContext(random()));
for (int i=0; i < size; i++) {
os.writeByte(start);
start ++;
}
os.close();
}
static void assertSameStreams(String msg, IndexInput expected, IndexInput test) throws IOException {
assertNotNull(msg + " null expected", expected);
assertNotNull(msg + " null test", test);
assertEquals(msg + " length", expected.length(), test.length());
assertEquals(msg + " position", expected.getFilePointer(), test.getFilePointer());
byte expectedBuffer[] = new byte[512];
byte testBuffer[] = new byte[expectedBuffer.length];
long remainder = expected.length() - expected.getFilePointer();
while (remainder > 0) {
int readLen = (int) Math.min(remainder, expectedBuffer.length);
expected.readBytes(expectedBuffer, 0, readLen);
test.readBytes(testBuffer, 0, readLen);
assertEqualArrays(msg + ", remainder " + remainder, expectedBuffer, testBuffer, 0, readLen);
remainder -= readLen;
}
}
static void assertSameStreams(String msg, IndexInput expected, IndexInput actual, long seekTo) throws IOException {
if (seekTo >= 0 && seekTo < expected.length()) {
expected.seek(seekTo);
actual.seek(seekTo);
assertSameStreams(msg + ", seek(mid)", expected, actual);
}
}
static void assertSameSeekBehavior(String msg, IndexInput expected, IndexInput actual) throws IOException {
// seek to 0
long point = 0;
assertSameStreams(msg + ", seek(0)", expected, actual, point);
// seek to middle
point = expected.length() / 2l;
assertSameStreams(msg + ", seek(mid)", expected, actual, point);
// seek to end - 2
point = expected.length() - 2;
assertSameStreams(msg + ", seek(end-2)", expected, actual, point);
// seek to end - 1
point = expected.length() - 1;
assertSameStreams(msg + ", seek(end-1)", expected, actual, point);
// seek to the end
point = expected.length();
assertSameStreams(msg + ", seek(end)", expected, actual, point);
// seek past end
point = expected.length() + 1;
assertSameStreams(msg + ", seek(end+1)", expected, actual, point);
}
static void assertEqualArrays(String msg, byte[] expected, byte[] test, int start, int len) {
assertNotNull(msg + " null expected", expected);
assertNotNull(msg + " null test", test);
for (int i=start; i<len; i++) {
assertEquals(msg + " " + i, expected[i], test[i]);
}
}
// ===========================================================
// Tests of the basic CompoundFile functionality
// ===========================================================
/**
* This test creates compound file based on a single file.
* Files of different sizes are tested: 0, 1, 10, 100 bytes.
*/
public void testSingleFile() throws IOException {
int data[] = new int[] { 0, 1, 10, 100 };
for (int i=0; i<data.length; i++) {
byte id[] = StringHelper.randomId();
String name = "t" + data[i];
createSequenceFile(dir, name, (byte) 0, data[i]);
CompoundFileDirectory csw = new CompoundFileDirectory(id, dir, name + ".cfs", newIOContext(random()), true);
dir.copy(csw, name, name, newIOContext(random()));
csw.close();
CompoundFileDirectory csr = new CompoundFileDirectory(id, dir, name + ".cfs", newIOContext(random()), false);
IndexInput expected = dir.openInput(name, newIOContext(random()));
IndexInput actual = csr.openInput(name, newIOContext(random()));
assertSameStreams(name, expected, actual);
assertSameSeekBehavior(name, expected, actual);
expected.close();
actual.close();
csr.close();
}
}
/**
* This test creates compound file based on two files.
*/
public void testTwoFiles() throws IOException {
createSequenceFile(dir, "d1", (byte) 0, 15);
createSequenceFile(dir, "d2", (byte) 0, 114);
byte id[] = StringHelper.randomId();
CompoundFileDirectory csw = new CompoundFileDirectory(id, dir, "d.cfs", newIOContext(random()), true);
dir.copy(csw, "d1", "d1", newIOContext(random()));
dir.copy(csw, "d2", "d2", newIOContext(random()));
csw.close();
CompoundFileDirectory csr = new CompoundFileDirectory(id, dir, "d.cfs", newIOContext(random()), false);
IndexInput expected = dir.openInput("d1", newIOContext(random()));
IndexInput actual = csr.openInput("d1", newIOContext(random()));
assertSameStreams("d1", expected, actual);
assertSameSeekBehavior("d1", expected, actual);
expected.close();
actual.close();
expected = dir.openInput("d2", newIOContext(random()));
actual = csr.openInput("d2", newIOContext(random()));
assertSameStreams("d2", expected, actual);
assertSameSeekBehavior("d2", expected, actual);
expected.close();
actual.close();
csr.close();
}
/**
* This test creates a compound file based on a large number of files of
* various length. The file content is generated randomly. The sizes range
* from 0 to 1Mb. Some of the sizes are selected to test the buffering
* logic in the file reading code. For this the chunk variable is set to
* the length of the buffer used internally by the compound file logic.
*/
public void testRandomFiles() throws IOException {
// Setup the test segment
String segment = "test";
int chunk = 1024; // internal buffer size used by the stream
createRandomFile(dir, segment + ".zero", 0);
createRandomFile(dir, segment + ".one", 1);
createRandomFile(dir, segment + ".ten", 10);
createRandomFile(dir, segment + ".hundred", 100);
createRandomFile(dir, segment + ".big1", chunk);
createRandomFile(dir, segment + ".big2", chunk - 1);
createRandomFile(dir, segment + ".big3", chunk + 1);
createRandomFile(dir, segment + ".big4", 3 * chunk);
createRandomFile(dir, segment + ".big5", 3 * chunk - 1);
createRandomFile(dir, segment + ".big6", 3 * chunk + 1);
createRandomFile(dir, segment + ".big7", 1000 * chunk);
// Setup extraneous files
createRandomFile(dir, "onetwothree", 100);
createRandomFile(dir, segment + ".notIn", 50);
createRandomFile(dir, segment + ".notIn2", 51);
byte id[] = StringHelper.randomId();
// Now test
CompoundFileDirectory csw = new CompoundFileDirectory(id, dir, "test.cfs", newIOContext(random()), true);
final String data[] = new String[] {
".zero", ".one", ".ten", ".hundred", ".big1", ".big2", ".big3",
".big4", ".big5", ".big6", ".big7"
};
for (int i=0; i<data.length; i++) {
String fileName = segment + data[i];
dir.copy(csw, fileName, fileName, newIOContext(random()));
}
csw.close();
CompoundFileDirectory csr = new CompoundFileDirectory(id, dir, "test.cfs", newIOContext(random()), false);
for (int i=0; i<data.length; i++) {
IndexInput check = dir.openInput(segment + data[i], newIOContext(random()));
IndexInput test = csr.openInput(segment + data[i], newIOContext(random()));
assertSameStreams(data[i], check, test);
assertSameSeekBehavior(data[i], check, test);
test.close();
check.close();
}
csr.close();
}
/**
* This test that writes larger than the size of the buffer output
* will correctly increment the file pointer.
*/
public void testLargeWrites() throws IOException {
IndexOutput os = dir.createOutput("testBufferStart.txt", newIOContext(random()));
byte[] largeBuf = new byte[2048];
for (int i=0; i<largeBuf.length; i++) {
largeBuf[i] = (byte) (Math.random() * 256);
}
long currentPos = os.getFilePointer();
os.writeBytes(largeBuf, largeBuf.length);
try {
assertEquals(currentPos + largeBuf.length, os.getFilePointer());
} finally {
os.close();
}
}
public void testAddExternalFile() throws IOException {
createSequenceFile(dir, "d1", (byte) 0, 15);
Directory newDir = newDirectory();
byte id[] = StringHelper.randomId();
CompoundFileDirectory csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), true);
dir.copy(csw, "d1", "d1", newIOContext(random()));
csw.close();
CompoundFileDirectory csr = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), false);
IndexInput expected = dir.openInput("d1", newIOContext(random()));
IndexInput actual = csr.openInput("d1", newIOContext(random()));
assertSameStreams("d1", expected, actual);
assertSameSeekBehavior("d1", expected, actual);
expected.close();
actual.close();
csr.close();
newDir.close();
}
public void testAppend() throws IOException {
Directory newDir = newDirectory();
byte id[] = StringHelper.randomId();
CompoundFileDirectory csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), true);
int size = 5 + random().nextInt(128);
for (int j = 0; j < 2; j++) {
IndexOutput os = csw.createOutput("seg_" + j + "_foo.txt", newIOContext(random()));
for (int i = 0; i < size; i++) {
os.writeInt(i*j);
}
os.close();
String[] listAll = newDir.listAll();
assertEquals(1, listAll.length);
assertEquals("d.cfs", listAll[0]);
}
createSequenceFile(dir, "d1", (byte) 0, 15);
dir.copy(csw, "d1", "d1", newIOContext(random()));
String[] listAll = newDir.listAll();
assertEquals(1, listAll.length);
assertEquals("d.cfs", listAll[0]);
csw.close();
CompoundFileDirectory csr = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), false);
for (int j = 0; j < 2; j++) {
IndexInput openInput = csr.openInput("seg_" + j + "_foo.txt", newIOContext(random()));
assertEquals(size * 4, openInput.length());
for (int i = 0; i < size; i++) {
assertEquals(i*j, openInput.readInt());
}
openInput.close();
}
IndexInput expected = dir.openInput("d1", newIOContext(random()));
IndexInput actual = csr.openInput("d1", newIOContext(random()));
assertSameStreams("d1", expected, actual);
assertSameSeekBehavior("d1", expected, actual);
expected.close();
actual.close();
csr.close();
newDir.close();
}
public void testAppendTwice() throws IOException {
Directory newDir = newDirectory();
byte id[] = StringHelper.randomId();
CompoundFileDirectory csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), true);
createSequenceFile(newDir, "d1", (byte) 0, 15);
IndexOutput out = csw.createOutput("d.xyz", newIOContext(random()));
out.writeInt(0);
out.close();
assertEquals(1, csw.listAll().length);
assertEquals("d.xyz", csw.listAll()[0]);
csw.close();
CompoundFileDirectory cfr = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), false);
assertEquals(1, cfr.listAll().length);
assertEquals("d.xyz", cfr.listAll()[0]);
cfr.close();
newDir.close();
}
public void testEmptyCFS() throws IOException {
Directory newDir = newDirectory();
byte id[] = StringHelper.randomId();
CompoundFileDirectory csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), true);
csw.close();
CompoundFileDirectory csr = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), false);
assertEquals(0, csr.listAll().length);
csr.close();
newDir.close();
}
public void testReadNestedCFP() throws IOException {
Directory newDir = newDirectory();
// manually manipulates directory
if (newDir instanceof MockDirectoryWrapper) {
((MockDirectoryWrapper)newDir).setEnableVirusScanner(false);
}
byte id[] = StringHelper.randomId();
CompoundFileDirectory csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), true);
CompoundFileDirectory nested = new CompoundFileDirectory(id, newDir, "b.cfs", newIOContext(random()), true);
IndexOutput out = nested.createOutput("b.xyz", newIOContext(random()));
IndexOutput out1 = nested.createOutput("b_1.xyz", newIOContext(random()));
out.writeInt(0);
out1.writeInt(1);
out.close();
out1.close();
nested.close();
newDir.copy(csw, "b.cfs", "b.cfs", newIOContext(random()));
newDir.copy(csw, "b.cfe", "b.cfe", newIOContext(random()));
newDir.deleteFile("b.cfs");
newDir.deleteFile("b.cfe");
csw.close();
assertEquals(2, newDir.listAll().length);
csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), false);
assertEquals(2, csw.listAll().length);
nested = new CompoundFileDirectory(id, csw, "b.cfs", newIOContext(random()), false);
assertEquals(2, nested.listAll().length);
IndexInput openInput = nested.openInput("b.xyz", newIOContext(random()));
assertEquals(0, openInput.readInt());
openInput.close();
openInput = nested.openInput("b_1.xyz", newIOContext(random()));
assertEquals(1, openInput.readInt());
openInput.close();
nested.close();
csw.close();
newDir.close();
}
public void testDoubleClose() throws IOException {
Directory newDir = newDirectory();
byte id[] = StringHelper.randomId();
CompoundFileDirectory csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), true);
IndexOutput out = csw.createOutput("d.xyz", newIOContext(random()));
out.writeInt(0);
out.close();
csw.close();
// close a second time - must have no effect according to Closeable
csw.close();
csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), false);
IndexInput openInput = csw.openInput("d.xyz", newIOContext(random()));
assertEquals(0, openInput.readInt());
openInput.close();
csw.close();
// close a second time - must have no effect according to Closeable
csw.close();
newDir.close();
}
// Make sure we don't somehow use more than 1 descriptor
// when reading a CFS with many subs:
public void testManySubFiles() throws IOException {
final Directory d = newFSDirectory(createTempDir("CFSManySubFiles"));
byte id[] = StringHelper.randomId();
final int FILE_COUNT = atLeast(500);
for(int fileIdx=0;fileIdx<FILE_COUNT;fileIdx++) {
IndexOutput out = d.createOutput("file." + fileIdx, newIOContext(random()));
out.writeByte((byte) fileIdx);
out.close();
}
final CompoundFileDirectory cfd = new CompoundFileDirectory(id, d, "c.cfs", newIOContext(random()), true);
for(int fileIdx=0;fileIdx<FILE_COUNT;fileIdx++) {
final String fileName = "file." + fileIdx;
d.copy(cfd, fileName, fileName, newIOContext(random()));
}
cfd.close();
final IndexInput[] ins = new IndexInput[FILE_COUNT];
final CompoundFileDirectory cfr = new CompoundFileDirectory(id, d, "c.cfs", newIOContext(random()), false);
for(int fileIdx=0;fileIdx<FILE_COUNT;fileIdx++) {
ins[fileIdx] = cfr.openInput("file." + fileIdx, newIOContext(random()));
}
for(int fileIdx=0;fileIdx<FILE_COUNT;fileIdx++) {
assertEquals((byte) fileIdx, ins[fileIdx].readByte());
}
for(int fileIdx=0;fileIdx<FILE_COUNT;fileIdx++) {
ins[fileIdx].close();
}
cfr.close();
d.close();
}
public void testListAll() throws Exception {
Directory dir = newDirectory();
if (dir instanceof MockDirectoryWrapper) {
// test lists files manually and tries to verify every .cfs it finds,
// but a virus scanner could leave some trash.
((MockDirectoryWrapper)dir).setEnableVirusScanner(false);
}
// riw should sometimes create docvalues fields, etc
RandomIndexWriter riw = new RandomIndexWriter(random(), dir);
Document doc = new Document();
// these fields should sometimes get term vectors, etc
Field idField = newStringField("id", "", Field.Store.NO);
Field bodyField = newTextField("body", "", Field.Store.NO);
doc.add(idField);
doc.add(bodyField);
for (int i = 0; i < 100; i++) {
idField.setStringValue(Integer.toString(i));
bodyField.setStringValue(TestUtil.randomUnicodeString(random()));
riw.addDocument(doc);
if (random().nextInt(7) == 0) {
riw.commit();
}
}
riw.close();
SegmentInfos infos = new SegmentInfos();
infos.read(dir);
for (String file : infos.files(dir, true)) {
try (IndexInput in = dir.openInput(file, IOContext.DEFAULT)) {}
if (file.endsWith(IndexFileNames.COMPOUND_FILE_EXTENSION)) {
String segment = IndexFileNames.parseSegmentName(file);
// warning: N^2
boolean found = false;
for (SegmentCommitInfo si : infos) {
if (si.info.name.equals(segment)) {
found = true;
try (CompoundFileDirectory cfs = new CompoundFileDirectory(si.info.getId(), dir, file, IOContext.DEFAULT, false)) {
for (String cfsFile : cfs.listAll()) {
try (IndexInput cfsIn = cfs.openInput(cfsFile, IOContext.DEFAULT)) {}
}
}
}
}
assertTrue(found);
}
}
dir.close();
}
/* nocommit: fold all these tests into BaseCompoundFormatTestCase */
}

View File

@ -19,7 +19,6 @@ package org.apache.lucene.index;
import java.io.IOException;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.LuceneTestCase;
@ -32,265 +31,6 @@ import org.apache.lucene.util.StringHelper;
* the size of each file is 1000 bytes.
*/
public class TestCompoundFile2 extends LuceneTestCase {
private Directory dir;
byte id[];
@Override
public void setUp() throws Exception {
super.setUp();
id = StringHelper.randomId();
dir = newDirectory();
CompoundFileDirectory cw = new CompoundFileDirectory(id, dir, "f.comp", newIOContext(random()), true);
for (int i=0; i<20; i++) {
TestCompoundFile.createSequenceFile(dir, "f" + i, (byte) 0, 2000);
String fileName = "f" + i;
dir.copy(cw, fileName, fileName, newIOContext(random()));
}
cw.close();
}
@Override
public void tearDown() throws Exception {
dir.close();
super.tearDown();
}
public void testClonedStreamsClosing() throws IOException {
CompoundFileDirectory cr = new CompoundFileDirectory(id, dir, "f.comp", newIOContext(random()), false);
// basic clone
IndexInput expected = dir.openInput("f11", newIOContext(random()));
IndexInput one = cr.openInput("f11", newIOContext(random()));
IndexInput two = one.clone();
TestCompoundFile.assertSameStreams("basic clone one", expected, one);
expected.seek(0);
TestCompoundFile.assertSameStreams("basic clone two", expected, two);
// Now close the first stream
one.close();
// The following should really fail since we couldn't expect to
// access a file once close has been called on it (regardless of
// buffering and/or clone magic)
expected.seek(0);
two.seek(0);
TestCompoundFile.assertSameStreams("basic clone two/2", expected, two);
// Now close the compound reader
cr.close();
// The following may also fail since the compound stream is closed
expected.seek(0);
two.seek(0);
//assertSameStreams("basic clone two/3", expected, two);
// Now close the second clone
two.close();
expected.seek(0);
//assertSameStreams("basic clone two/4", expected, two);
expected.close();
}
/** This test opens two files from a compound stream and verifies that
* their file positions are independent of each other.
*/
public void testRandomAccess() throws IOException {
CompoundFileDirectory cr = new CompoundFileDirectory(id, dir, "f.comp", newIOContext(random()), false);
// Open two files
IndexInput e1 = dir.openInput("f11", newIOContext(random()));
IndexInput e2 = dir.openInput("f3", newIOContext(random()));
IndexInput a1 = cr.openInput("f11", newIOContext(random()));
IndexInput a2 = dir.openInput("f3", newIOContext(random()));
// Seek the first pair
e1.seek(100);
a1.seek(100);
assertEquals(100, e1.getFilePointer());
assertEquals(100, a1.getFilePointer());
byte be1 = e1.readByte();
byte ba1 = a1.readByte();
assertEquals(be1, ba1);
// Now seek the second pair
e2.seek(1027);
a2.seek(1027);
assertEquals(1027, e2.getFilePointer());
assertEquals(1027, a2.getFilePointer());
byte be2 = e2.readByte();
byte ba2 = a2.readByte();
assertEquals(be2, ba2);
// Now make sure the first one didn't move
assertEquals(101, e1.getFilePointer());
assertEquals(101, a1.getFilePointer());
be1 = e1.readByte();
ba1 = a1.readByte();
assertEquals(be1, ba1);
// Now more the first one again, past the buffer length
e1.seek(1910);
a1.seek(1910);
assertEquals(1910, e1.getFilePointer());
assertEquals(1910, a1.getFilePointer());
be1 = e1.readByte();
ba1 = a1.readByte();
assertEquals(be1, ba1);
// Now make sure the second set didn't move
assertEquals(1028, e2.getFilePointer());
assertEquals(1028, a2.getFilePointer());
be2 = e2.readByte();
ba2 = a2.readByte();
assertEquals(be2, ba2);
// Move the second set back, again cross the buffer size
e2.seek(17);
a2.seek(17);
assertEquals(17, e2.getFilePointer());
assertEquals(17, a2.getFilePointer());
be2 = e2.readByte();
ba2 = a2.readByte();
assertEquals(be2, ba2);
// Finally, make sure the first set didn't move
// Now make sure the first one didn't move
assertEquals(1911, e1.getFilePointer());
assertEquals(1911, a1.getFilePointer());
be1 = e1.readByte();
ba1 = a1.readByte();
assertEquals(be1, ba1);
e1.close();
e2.close();
a1.close();
a2.close();
cr.close();
}
/** This test opens two files from a compound stream and verifies that
* their file positions are independent of each other.
*/
public void testRandomAccessClones() throws IOException {
CompoundFileDirectory cr = new CompoundFileDirectory(id, dir, "f.comp", newIOContext(random()), false);
// Open two files
IndexInput e1 = cr.openInput("f11", newIOContext(random()));
IndexInput e2 = cr.openInput("f3", newIOContext(random()));
IndexInput a1 = e1.clone();
IndexInput a2 = e2.clone();
// Seek the first pair
e1.seek(100);
a1.seek(100);
assertEquals(100, e1.getFilePointer());
assertEquals(100, a1.getFilePointer());
byte be1 = e1.readByte();
byte ba1 = a1.readByte();
assertEquals(be1, ba1);
// Now seek the second pair
e2.seek(1027);
a2.seek(1027);
assertEquals(1027, e2.getFilePointer());
assertEquals(1027, a2.getFilePointer());
byte be2 = e2.readByte();
byte ba2 = a2.readByte();
assertEquals(be2, ba2);
// Now make sure the first one didn't move
assertEquals(101, e1.getFilePointer());
assertEquals(101, a1.getFilePointer());
be1 = e1.readByte();
ba1 = a1.readByte();
assertEquals(be1, ba1);
// Now more the first one again, past the buffer length
e1.seek(1910);
a1.seek(1910);
assertEquals(1910, e1.getFilePointer());
assertEquals(1910, a1.getFilePointer());
be1 = e1.readByte();
ba1 = a1.readByte();
assertEquals(be1, ba1);
// Now make sure the second set didn't move
assertEquals(1028, e2.getFilePointer());
assertEquals(1028, a2.getFilePointer());
be2 = e2.readByte();
ba2 = a2.readByte();
assertEquals(be2, ba2);
// Move the second set back, again cross the buffer size
e2.seek(17);
a2.seek(17);
assertEquals(17, e2.getFilePointer());
assertEquals(17, a2.getFilePointer());
be2 = e2.readByte();
ba2 = a2.readByte();
assertEquals(be2, ba2);
// Finally, make sure the first set didn't move
// Now make sure the first one didn't move
assertEquals(1911, e1.getFilePointer());
assertEquals(1911, a1.getFilePointer());
be1 = e1.readByte();
ba1 = a1.readByte();
assertEquals(be1, ba1);
e1.close();
e2.close();
a1.close();
a2.close();
cr.close();
}
public void testFileNotFound() throws IOException {
CompoundFileDirectory cr = new CompoundFileDirectory(id, dir, "f.comp", newIOContext(random()), false);
// Open two files
try {
cr.openInput("bogus", newIOContext(random()));
fail("File not found");
} catch (IOException e) {
/* success */
//System.out.println("SUCCESS: File Not Found: " + e);
}
cr.close();
}
public void testReadPastEOF() throws IOException {
CompoundFileDirectory cr = new CompoundFileDirectory(id, dir, "f.comp", newIOContext(random()), false);
IndexInput is = cr.openInput("f2", newIOContext(random()));
is.seek(is.length() - 10);
byte b[] = new byte[100];
is.readBytes(b, 0, 10);
try {
is.readByte();
fail("Single byte read past end of file");
} catch (IOException e) {
/* success */
//System.out.println("SUCCESS: single byte read past end of file: " + e);
}
is.seek(is.length() - 10);
try {
is.readBytes(b, 0, 50);
fail("Block read past end of file");
} catch (IOException e) {
/* success */
//System.out.println("SUCCESS: block read past end of file: " + e);
}
is.close();
cr.close();
}
/* nocommit: fold all these tests into BaseCompoundFormatTestCase */
}

View File

@ -126,6 +126,8 @@ public class TestNRTCachingDirectory extends BaseDirectoryTestCase {
// LUCENE-5724
public void testLargeCFS() throws IOException {
// nocommit: make a fake .si etc
/*
Directory dir = new NRTCachingDirectory(newFSDirectory(createTempDir()), 2.0, 25.0);
IOContext context = new IOContext(new FlushInfo(0, 512*1024*1024));
IndexOutput out = dir.createOutput("big.bin", context);
@ -139,5 +141,6 @@ public class TestNRTCachingDirectory extends BaseDirectoryTestCase {
dir.copy(cfsDir, "big.bin", "big.bin", context);
cfsDir.close();
dir.close();
*/
}
}

View File

@ -587,6 +587,8 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
// LUCENE-3382 test that delegate compound files correctly.
public void testCompoundFileAppendTwice() throws IOException {
// nocommit: what is this testing? does it belong here?
/*
Directory newDir = getDirectory(createTempDir("testCompoundFileAppendTwice"));
byte id[] = StringHelper.randomId();
CompoundFileDirectory csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), true);
@ -604,6 +606,7 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
assertEquals("d.xyz", cfr.listAll()[0]);
cfr.close();
newDir.close();
*/
}
/** Creates a file of the specified size with sequential data. The first