LUCENE-3218: Detach CompoundFileDirectory from Directory to prevent traps for delegating Directory implementations

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1161183 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Simon Willnauer 2011-08-24 16:05:25 +00:00
parent defe907de2
commit 28f59a2e05
26 changed files with 419 additions and 619 deletions

View File

@ -228,22 +228,21 @@ public class NRTCachingDirectory extends Directory {
}
}
// final due to LUCENE-3382: currently CFS backdoors the directory to create CFE
// by using the basic implementation and not delegating, we ensure that all
// openInput/createOutput requests come thru NRTCachingDirectory.
@Override
public final CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
return super.openCompoundInput(name, context);
public IndexInputSlicer createSlicer(final String name, final IOContext context) throws IOException {
ensureOpen();
if (VERBOSE) {
System.out.println("nrtdir.openInput name=" + name);
}
if (cache.fileExists(name)) {
if (VERBOSE) {
System.out.println(" from cache");
}
return cache.createSlicer(name, context);
} else {
return delegate.createSlicer(name, context);
}
}
// final due to LUCENE-3382: currently CFS backdoors the directory to create CFE
// by using the basic implementation and not delegating, we ensure that all
// openInput/createOutput requests come thru NRTCachingDirectory.
@Override
public final CompoundFileDirectory createCompoundOutput(String name, IOContext context) throws IOException {
return super.createCompoundOutput(name, context);
}
/** Close this directory, which flushes any cached files
* to the delegate and then closes the delegate. */
@Override

View File

@ -148,7 +148,7 @@ public class TestNRTCachingDirectory extends LuceneTestCase {
// LUCENE-3382 test that delegate compound files correctly.
public void testCompoundFileAppendTwice() throws IOException {
Directory newDir = new NRTCachingDirectory(newDirectory(), 2.0, 25.0);
CompoundFileDirectory csw = newDir.createCompoundOutput("d.cfs", newIOContext(random));
CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), true);
createSequenceFile(newDir, "d1", (byte) 0, 15);
IndexOutput out = csw.createOutput("d.xyz", newIOContext(random));
out.writeInt(0);
@ -164,7 +164,7 @@ public class TestNRTCachingDirectory extends LuceneTestCase {
csw.close();
CompoundFileDirectory cfr = newDir.openCompoundInput("d.cfs", newIOContext(random));
CompoundFileDirectory cfr = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), false);
assertEquals(1, cfr.listAll().length);
assertEquals("d.xyz", cfr.listAll()[0]);
cfr.close();

View File

@ -1435,7 +1435,7 @@ public abstract class IndexReader implements Cloneable,Closeable {
String dirname = file.getAbsoluteFile().getParent();
filename = file.getName();
dir = FSDirectory.open(new File(dirname));
cfr = dir.openCompoundInput(filename, IOContext.DEFAULT);
cfr = new CompoundFileDirectory(dir, filename, IOContext.DEFAULT, false);
String [] files = cfr.listAll();
ArrayUtil.mergeSort(files); // sort the array of filename so that the output is more readable

View File

@ -2262,7 +2262,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
String compoundFileName = IndexFileNames.segmentFileName(newSegment.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
message("creating compound file " + compoundFileName);
// Now build compound file
final Directory cfsDir = directory.createCompoundOutput(compoundFileName, context);
final Directory cfsDir = new CompoundFileDirectory(directory, compoundFileName, context, true);
IOException prior = null;
try {
for(String fileName : newSegment.files()) {
@ -2594,7 +2594,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
private void copySegmentIntoCFS(SegmentInfo info, String segName, IOContext context) throws IOException {
String segFileName = IndexFileNames.segmentFileName(segName, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
Collection<String> files = info.files();
final CompoundFileDirectory cfsdir = directory.createCompoundOutput(segFileName, context);
final CompoundFileDirectory cfsdir = new CompoundFileDirectory(directory, segFileName, context, true);
try {
for (String file : files) {
String newFileName = segName + IndexFileNames.stripSegmentName(file);

View File

@ -76,7 +76,7 @@ final class SegmentCoreReaders {
try {
Directory dir0 = dir;
if (si.getUseCompoundFile()) {
cfsReader = dir.openCompoundInput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
cfsReader = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
dir0 = cfsReader;
}
cfsDir = dir0;
@ -140,9 +140,9 @@ final class SegmentCoreReaders {
if (si.getDocStoreOffset() != -1) {
if (si.getDocStoreIsCompoundFile()) {
assert storeCFSReader == null;
storeCFSReader = dir.openCompoundInput(
storeCFSReader = new CompoundFileDirectory(dir,
IndexFileNames.segmentFileName(si.getDocStoreSegment(), "", IndexFileNames.COMPOUND_FILE_STORE_EXTENSION),
context);
context, false);
storeDir = storeCFSReader;
assert storeDir != null;
} else {
@ -154,7 +154,7 @@ final class SegmentCoreReaders {
// was not used, but then we are asked to open doc
// stores after the segment has switched to CFS
if (cfsReader == null) {
cfsReader = dir.openCompoundInput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
cfsReader = new CompoundFileDirectory(dir,IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
}
storeDir = cfsReader;
assert storeDir != null;

View File

@ -30,6 +30,7 @@ import java.util.Set;
import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
@ -247,7 +248,7 @@ public final class SegmentInfo implements Cloneable {
}
final Directory dirToTest;
if (isCompoundFile) {
dirToTest = dir.openCompoundInput(IndexFileNames.segmentFileName(storesSegment, "", ext), IOContext.READONCE);
dirToTest = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(storesSegment, "", ext), IOContext.READONCE, false);
} else {
dirToTest = dir;
}
@ -265,8 +266,8 @@ public final class SegmentInfo implements Cloneable {
if (fieldInfos == null) {
Directory dir0 = dir;
if (isCompoundFile && checkCompoundFile) {
dir0 = dir.openCompoundInput(IndexFileNames.segmentFileName(name,
"", IndexFileNames.COMPOUND_FILE_EXTENSION), IOContext.READONCE);
dir0 = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(name,
"", IndexFileNames.COMPOUND_FILE_EXTENSION), IOContext.READONCE, false);
}
try {
fieldInfos = new FieldInfos(dir0, IndexFileNames.segmentFileName(name,
@ -619,7 +620,7 @@ public final class SegmentInfo implements Cloneable {
if (useCompoundFile) {
fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.COMPOUND_FILE_EXTENSION));
if (version != null && StringHelper.getVersionComparator().compare("3.4", version) <= 0) {
if (version != null && StringHelper.getVersionComparator().compare("4.0", version) <= 0) {
fileSet.add(IndexFileNames.segmentFileName(name, "",
IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
}

View File

@ -153,7 +153,7 @@ final class SegmentMerger {
// Now merge all added files
Collection<String> files = info.files();
CompoundFileDirectory cfsDir = directory.createCompoundOutput(fileName, context);
CompoundFileDirectory cfsDir = new CompoundFileDirectory(directory, fileName, context, true);
try {
for (String file : files) {
assert !IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION)

View File

@ -28,6 +28,7 @@ import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.values.Writer;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BytesRef;
@ -51,9 +52,9 @@ public class DefaultDocValuesConsumer extends PerDocConsumer {
this.bytesUsed = state.bytesUsed;
this.context = state.context;
//TODO maybe we should enable a global CFS that all codecs can pull on demand to further reduce the number of files?
this.directory = useCompoundFile ? state.directory.createCompoundOutput(
this.directory = useCompoundFile ? new CompoundFileDirectory(state.directory,
IndexFileNames.segmentFileName(segmentName, codecId,
IndexFileNames.COMPOUND_FILE_EXTENSION), context) : state.directory;
IndexFileNames.COMPOUND_FILE_EXTENSION), context, true) : state.directory;
this.comparator = comparator;
this.useCompoundFile = useCompoundFile;
}

View File

@ -32,6 +32,7 @@ import org.apache.lucene.index.values.IndexDocValues;
import org.apache.lucene.index.values.Floats;
import org.apache.lucene.index.values.Ints;
import org.apache.lucene.index.values.ValueType;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BytesRef;
@ -78,7 +79,7 @@ public class DefaultDocValuesProducer extends PerDocValues {
this.sortComparator = sortComparator;
final Directory directory;
if (useCompoundFile) {
cfs = directory = dir.openCompoundInput(IndexFileNames.segmentFileName(si.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION), context);
cfs = directory = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(si.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
} else {
cfs = null;
directory = dir;

View File

@ -27,6 +27,7 @@ import org.apache.lucene.index.IndexFormatTooNewException;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfos;
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;
@ -68,13 +69,13 @@ public class DefaultSegmentInfosReader extends SegmentInfosReader {
Directory dir = directory;
if (si.getDocStoreOffset() != -1) {
if (si.getDocStoreIsCompoundFile()) {
dir = dir.openCompoundInput(IndexFileNames.segmentFileName(
dir = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(
si.getDocStoreSegment(), "",
IndexFileNames.COMPOUND_FILE_STORE_EXTENSION), context);
IndexFileNames.COMPOUND_FILE_STORE_EXTENSION), context, false);
}
} else if (si.getUseCompoundFile()) {
dir = dir.openCompoundInput(IndexFileNames.segmentFileName(
si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
dir = new CompoundFileDirectory(dir,IndexFileNames.segmentFileName(
si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
}
try {

View File

@ -34,7 +34,7 @@ import java.io.IOException;
* Directory methods that would normally modify data throw an exception.
* @lucene.experimental
*/
public abstract class CompoundFileDirectory extends Directory {
public final class CompoundFileDirectory extends Directory {
/** Offset/Length for a slice inside of a compound file */
public static final class FileEntry {
@ -45,68 +45,86 @@ public abstract class CompoundFileDirectory extends Directory {
private final Directory directory;
private final String fileName;
protected final int readBufferSize;
private Map<String,FileEntry> entries;
private boolean openForWrite;
private final Map<String,FileEntry> entries;
private final boolean openForWrite;
private static final Map<String,FileEntry> SENTINEL = Collections.emptyMap();
private CompoundFileWriter writer;
private final CompoundFileWriter writer;
private final IndexInputSlicer handle;
/**
* Create a new CompoundFileDirectory.
* <p>
* NOTE: subclasses must call {@link #initForRead(Map)} before the directory can be used.
*/
public CompoundFileDirectory(Directory directory, String fileName, IOContext context) throws IOException {
public CompoundFileDirectory(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.createSlicer(fileName, context);
try {
this.entries = readEntries(handle, directory, fileName);
success = true;
} finally {
if (!success) {
IOUtils.closeSafely(true, handle);
}
}
this.isOpen = true;
writer = null;
} else {
assert !(directory instanceof CompoundFileDirectory) : "compound file inside of compound file: " + fileName;
this.entries = SENTINEL;
this.isOpen = true;
writer = new CompoundFileWriter(directory, fileName);
handle = null;
}
}
/** Initialize with a map of filename->slices */
protected final void initForRead(Map<String,FileEntry> entries) {
this.entries = entries;
this.isOpen = true;
this.openForWrite = false;
}
protected final void initForWrite() throws IOException {
assert !(directory instanceof CompoundFileDirectory) : "compound file inside of compound file: " + fileName;
this.entries = SENTINEL;
this.openForWrite = true;
this.isOpen = true;
writer = new CompoundFileWriter(directory, fileName);
}
/** Helper method that reads CFS entries from an input stream */
public static final Map<String,FileEntry> readEntries(IndexInput stream, Directory dir, String name) throws IOException {
private static final Map<String, FileEntry> readEntries(
IndexInputSlicer handle, Directory dir, String name) throws IOException {
// read the first VInt. If it is negative, it's the version number
// otherwise it's the count (pre-3.1 indexes)
final int firstInt = stream.readVInt();
if (firstInt == CompoundFileWriter.FORMAT_CURRENT) {
IndexInput input = null;
try {
input = dir.openInput(IndexFileNames.segmentFileName(IndexFileNames.stripExtension(name), "",
IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION), IOContext.READONCE);
final int readInt = input.readInt(); // unused right now
assert readInt == CompoundFileWriter.ENTRY_FORMAT_CURRENT;
final int numEntries = input.readVInt();
final Map<String, FileEntry> mapping = new HashMap<String, CompoundFileDirectory.FileEntry>(
numEntries);
for (int i = 0; i < numEntries; i++) {
final FileEntry fileEntry = new FileEntry();
mapping.put(input.readString(), fileEntry);
fileEntry.offset = input.readLong();
fileEntry.length = input.readLong();
final IndexInput stream = handle.openFullSlice();
final Map<String, FileEntry> mapping;
boolean success = false;
try {
final int firstInt = stream.readVInt();
if (firstInt == CompoundFileWriter.FORMAT_CURRENT) {
IndexInput input = null;
try {
input = dir.openInput(IndexFileNames.segmentFileName(
IndexFileNames.stripExtension(name), "",
IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION),
IOContext.READONCE);
final int readInt = input.readInt(); // unused right now
assert readInt == CompoundFileWriter.ENTRY_FORMAT_CURRENT;
final int numEntries = input.readVInt();
mapping = new HashMap<String, CompoundFileDirectory.FileEntry>(
numEntries);
for (int i = 0; i < numEntries; i++) {
final FileEntry fileEntry = new FileEntry();
mapping.put(input.readString(), fileEntry);
fileEntry.offset = input.readLong();
fileEntry.length = input.readLong();
}
return mapping;
} finally {
IOUtils.closeSafely(true, input);
}
return mapping;
} finally {
IOUtils.closeSafely(true, input);
} else {
// TODO remove once 3.x is not supported anymore
mapping = readLegacyEntries(stream, firstInt);
}
success = true;
return mapping;
} finally {
IOUtils.closeSafely(!success, stream);
}
// TODO remove once 3.x is not supported anymore
return readLegacyEntries(stream, firstInt);
}
private static Map<String, FileEntry> readLegacyEntries(IndexInput stream,
@ -173,32 +191,29 @@ public abstract class CompoundFileDirectory extends Directory {
public synchronized void close() throws IOException {
if (!isOpen) {
// allow double close - usually to be consistent with other closeables
assert entries == null;
return; // already closed
}
entries = null;
isOpen = false;
if (writer != null) {
assert openForWrite;
writer.close();
} else {
IOUtils.closeSafely(false, handle);
}
}
@Override
public synchronized IndexInput openInput(String fileName, IOContext context) throws IOException {
public synchronized IndexInput openInput(String name, IOContext context) throws IOException {
ensureOpen();
assert !openForWrite;
final String id = IndexFileNames.stripSegmentName(fileName);
final String id = IndexFileNames.stripSegmentName(name);
final FileEntry entry = entries.get(id);
if (entry == null) {
throw new IOException("No sub-file with id " + id + " found (fileName=" + fileName + " files: " + entries.keySet() + ")");
throw new IOException("No sub-file with id " + id + " found (fileName=" + name + " files: " + entries.keySet() + ")");
}
return openInputSlice(id, entry.offset, entry.length, readBufferSize);
return handle.openSlice(entry.offset, entry.length);
}
/** Return an IndexInput that represents a "slice" or portion of the CFS file. */
public abstract IndexInput openInputSlice(String id, long offset, long length, int readBufferSize) throws IOException;
/** Returns an array of strings, one for each file in the directory. */
@Override
public String[] listAll() {
@ -279,51 +294,31 @@ public abstract class CompoundFileDirectory extends Directory {
public Lock makeLock(String name) {
throw new UnsupportedOperationException();
}
@Override
public CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
FileEntry fileEntry = this.entries.get(IndexFileNames.stripSegmentName(name));
if (fileEntry == null) {
throw new FileNotFoundException("file " + name + " does not exists in this CFS");
}
return new NestedCompoundFileDirectory(name, context, fileEntry.offset, fileEntry.length);
}
/** Not implemented
* @throws UnsupportedOperationException */
@Override
public CompoundFileDirectory createCompoundOutput(String name, IOContext context)
public IndexInputSlicer createSlicer(final String name, IOContext context)
throws IOException {
throw new UnsupportedOperationException("can not create nested CFS, create seperately and use Directory.copy instead");
}
private class NestedCompoundFileDirectory extends CompoundFileDirectory {
private final long cfsOffset;
private final long cfsLength;
public NestedCompoundFileDirectory(String fileName, IOContext context, long offset, long length)
throws IOException {
super(directory, fileName, context);
this.cfsOffset = offset;
this.cfsLength = length;
IndexInput input = null;
try {
input = CompoundFileDirectory.this.openInput(fileName, IOContext.READONCE);
initForRead(CompoundFileDirectory.readEntries(input,
CompoundFileDirectory.this, fileName));
} finally {
IOUtils.closeSafely(false, input);
ensureOpen();
assert !openForWrite;
final String id = IndexFileNames.stripSegmentName(name);
final FileEntry entry = entries.get(id);
if (entry == null) {
throw new IOException("No sub-file with id " + id + " found (fileName=" + name + " files: " + entries.keySet() + ")");
}
return new IndexInputSlicer() {
@Override
public void close() throws IOException {
}
@Override
public IndexInput openSlice(long offset, long length) throws IOException {
return handle.openSlice(entry.offset + offset, length);
}
}
@Override
public IndexInput openInputSlice(String id, long offset, long length,
int readBufferSize) throws IOException {
assert offset + length <= cfsLength;
return CompoundFileDirectory.this.openInputSlice(id, cfsOffset + offset, length, readBufferSize);
}
@Override
public IndexInput openFullSlice() throws IOException {
return openSlice(0, entry.length);
}
};
}
}

View File

@ -190,6 +190,7 @@ final class CompoundFileWriter implements Closeable{
private final long copyFileEntry(IndexOutput dataOut, FileEntry fileEntry)
throws IOException, MergeAbortedException {
final IndexInput is = fileEntry.dir.openInput(fileEntry.file, IOContext.READONCE);
boolean success = false;
try {
final long startPtr = dataOut.getFilePointer();
final long length = fileEntry.length;
@ -201,11 +202,14 @@ final class CompoundFileWriter implements Closeable{
throw new IOException("Difference in the output file offsets " + diff
+ " does not match the original file length " + length);
fileEntry.offset = startPtr;
// copy successful - delete file
fileEntry.dir.deleteFile(fileEntry.file);
success = true;
return length;
} finally {
is.close();
IOUtils.closeSafely(!success, is);
if (success) {
// copy successful - delete file
fileEntry.dir.deleteFile(fileEntry.file);
}
}
}

View File

@ -1,140 +0,0 @@
package org.apache.lucene.store;
/**
* 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 org.apache.lucene.util.IOUtils;
/**
* Default implementation of {@link CompoundFileDirectory}.
* <p>
* This implementation returns a BufferedIndexInput that wraps the underlying
* Directory's IndexInput for the compound file (using unbuffered reads).
* @lucene.experimental
*/
public class DefaultCompoundFileDirectory extends CompoundFileDirectory {
protected IndexInput stream;
public DefaultCompoundFileDirectory(Directory directory, String fileName, IOContext context, boolean writeable) throws IOException {
super(directory, fileName, context);
if (!writeable) {
try {
stream = directory.openInput(fileName, context);
initForRead(CompoundFileDirectory.readEntries(stream, directory, fileName));
} catch (IOException e) {
IOUtils.closeSafely(e, stream);
}
} else {
initForWrite();
}
}
@Override
public IndexInput openInputSlice(String id, long offset, long length, int readBufferSize) throws IOException {
return new CSIndexInput(stream, offset, length, readBufferSize);
}
@Override
public synchronized void close() throws IOException {
try {
IOUtils.closeSafely(false, stream);
} finally {
super.close();
}
}
/** Implementation of an IndexInput that reads from a portion of the
* compound file.
*/
static final class CSIndexInput extends BufferedIndexInput {
IndexInput base;
long fileOffset;
long length;
CSIndexInput(final IndexInput base, final long fileOffset, final long length) {
this(base, fileOffset, length, BufferedIndexInput.BUFFER_SIZE);
}
CSIndexInput(final IndexInput base, final long fileOffset, final long length, int readBufferSize) {
super(readBufferSize);
this.base = (IndexInput)base.clone();
this.fileOffset = fileOffset;
this.length = length;
}
@Override
public Object clone() {
CSIndexInput clone = (CSIndexInput)super.clone();
clone.base = (IndexInput)base.clone();
clone.fileOffset = fileOffset;
clone.length = length;
return clone;
}
/** Expert: implements buffer refill. Reads bytes from the current
* position in the input.
* @param b the array to read bytes into
* @param offset the offset in the array to start storing bytes
* @param len the number of bytes to read
*/
@Override
protected void readInternal(byte[] b, int offset, int len) throws IOException {
long start = getFilePointer();
if(start + len > length)
throw new IOException("read past EOF");
base.seek(fileOffset + start);
base.readBytes(b, offset, len, false);
}
/** Expert: implements seek. Sets current position in this file, where
* the next {@link #readInternal(byte[],int,int)} will occur.
* @see #readInternal(byte[],int,int)
*/
@Override
protected void seekInternal(long pos) {}
/** Closes the stream to further operations. */
@Override
public void close() throws IOException {
base.close();
}
@Override
public long length() {
return length;
}
@Override
public void copyBytes(IndexOutput out, long numBytes) throws IOException {
// Copy first whatever is in the buffer
numBytes -= flushBuffer(out, numBytes);
// If there are more bytes left to copy, delegate the copy task to the
// base IndexInput, in case it can do an optimized copy.
if (numBytes > 0) {
long start = getFilePointer();
if (start + numBytes > length) {
throw new IOException("read past EOF");
}
base.seek(fileOffset + start);
base.copyBytes(out, numBytes);
}
}
}
}

View File

@ -112,30 +112,6 @@ public abstract class Directory implements Closeable {
*/
public abstract IndexInput openInput(String name, IOContext context) throws IOException;
/**
* Returns a {@link CompoundFileDirectory} capable of
* reading the Lucene compound file format.
* <p>
* The default implementation returns
* {@link DefaultCompoundFileDirectory}.
* @lucene.experimental
*/
public CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
return new DefaultCompoundFileDirectory(this, name, context, false);
}
/**
* Returns a {@link CompoundFileDirectory} capable of
* writing the Lucene compound file format.
* <p>
* The default implementation returns
* {@link DefaultCompoundFileDirectory}.
* @lucene.experimental
*/
public CompoundFileDirectory createCompoundOutput(String name, IOContext context) throws IOException {
return new DefaultCompoundFileDirectory(this, name, context, true);
}
/** Construct a {@link Lock}.
* @param name the name of the lock file
*/
@ -232,6 +208,37 @@ public abstract class Directory implements Closeable {
}
}
/**
* Creates an {@link IndexInputSlicer} for the given file name.
* IndexInputSlicer allows other {@link Directory} implementations to
* efficiently open one or more sliced {@link IndexInput} instances from a
* single file handle. The underlying file handle is kept open until the
* {@link IndexInputSlicer} is closed.
*
* @throws IOException
* if an {@link IOException} occurs
* @lucene.internal
* @lucene.experimental
*/
public IndexInputSlicer createSlicer(final String name, final IOContext context) throws IOException {
ensureOpen();
return new IndexInputSlicer() {
private final IndexInput base = Directory.this.openInput(name, context);
@Override
public IndexInput openSlice(long offset, long length) {
return new SlicedIndexInput(base, offset, length);
}
@Override
public void close() throws IOException {
base.close();
}
@Override
public IndexInput openFullSlice() throws IOException {
return (IndexInput) base.clone();
}
};
}
/**
* @throws AlreadyClosedException if this Directory is closed
*/
@ -239,4 +246,103 @@ public abstract class Directory implements Closeable {
if (!isOpen)
throw new AlreadyClosedException("this Directory is closed");
}
/**
* Allows to create one or more sliced {@link IndexInput} instances from a single
* file handle. Some {@link Directory} implementations may be able to efficiently map slices of a file
* into memory when only certain parts of a file are required.
* @lucene.internal
* @lucene.experimental
*/
public abstract class IndexInputSlicer implements Closeable {
/**
* Returns an {@link IndexInput} slice starting at the given offset with the given length.
*/
public abstract IndexInput openSlice(long offset, long length) throws IOException;
/**
* Returns an {@link IndexInput} slice starting at offset <i>0</i> with a
* length equal to the length of the underlying file
*/
public abstract IndexInput openFullSlice() throws IOException;
}
/** Implementation of an IndexInput that reads from a portion of
* a file.
*/
private static final class SlicedIndexInput extends BufferedIndexInput {
IndexInput base;
long fileOffset;
long length;
SlicedIndexInput(final IndexInput base, final long fileOffset, final long length) {
this(base, fileOffset, length, BufferedIndexInput.BUFFER_SIZE);
}
SlicedIndexInput(final IndexInput base, final long fileOffset, final long length, int readBufferSize) {
super(readBufferSize);
this.base = (IndexInput) base.clone();
this.fileOffset = fileOffset;
this.length = length;
}
@Override
public Object clone() {
SlicedIndexInput clone = (SlicedIndexInput)super.clone();
clone.base = (IndexInput)base.clone();
clone.fileOffset = fileOffset;
clone.length = length;
return clone;
}
/** Expert: implements buffer refill. Reads bytes from the current
* position in the input.
* @param b the array to read bytes into
* @param offset the offset in the array to start storing bytes
* @param len the number of bytes to read
*/
@Override
protected void readInternal(byte[] b, int offset, int len) throws IOException {
long start = getFilePointer();
if(start + len > length)
throw new IOException("read past EOF");
base.seek(fileOffset + start);
base.readBytes(b, offset, len, false);
}
/** Expert: implements seek. Sets current position in this file, where
* the next {@link #readInternal(byte[],int,int)} will occur.
* @see #readInternal(byte[],int,int)
*/
@Override
protected void seekInternal(long pos) {}
/** Closes the stream to further operations. */
@Override
public void close() throws IOException {
base.close();
}
@Override
public long length() {
return length;
}
@Override
public void copyBytes(IndexOutput out, long numBytes) throws IOException {
// Copy first whatever is in the buffer
numBytes -= flushBuffer(out, numBytes);
// If there are more bytes left to copy, delegate the copy task to the
// base IndexInput, in case it can do an optimized copy.
if (numBytes > 0) {
long start = getFilePointer();
if (start + numBytes > length) {
throw new IOException("read past EOF");
}
base.seek(fileOffset + start);
base.copyBytes(out, numBytes);
}
}
}
}

View File

@ -177,19 +177,9 @@ public class FileSwitchDirectory extends Directory {
return getDirectory(name).openInput(name, context);
}
// final due to LUCENE-3380: currently CFS backdoors the directory to create CFE
// by using the basic implementation and not delegating, we ensure that all
// openInput/createOutput requests come thru NRTCachingDirectory.
@Override
public final CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
return super.openCompoundInput(name, context);
}
// final due to LUCENE-3380: currently CFS backdoors the directory to create CFE
// by using the basic implementation and not delegating, we ensure that all
// openInput/createOutput requests come thru NRTCachingDirectory.
@Override
public final CompoundFileDirectory createCompoundOutput(String name, IOContext context) throws IOException {
return super.createCompoundOutput(name, context);
public IndexInputSlicer createSlicer(String name, IOContext context)
throws IOException {
return getDirectory(name).createSlicer(name, context);
}
}

View File

@ -32,7 +32,6 @@ import java.security.PrivilegedActionException;
import java.lang.reflect.Method;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.IOUtils;
/** File-based {@link Directory} implementation that uses
* mmap for reading, and {@link
@ -220,42 +219,26 @@ public class MMapDirectory extends FSDirectory {
}
}
@Override
public CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
return new MMapCompoundFileDirectory(name, context);
}
private final class MMapCompoundFileDirectory extends CompoundFileDirectory {
private RandomAccessFile raf = null;
public MMapCompoundFileDirectory(String fileName, IOContext context) throws IOException {
super(MMapDirectory.this, fileName, context);
IndexInput stream = null;
try {
File f = new File(MMapDirectory.this.getDirectory(), fileName);
raf = new RandomAccessFile(f, "r");
stream = new MMapIndexInput(raf, 0, raf.length(), chunkSizePower);
initForRead(CompoundFileDirectory.readEntries(stream, MMapDirectory.this, fileName));
stream.close();
} catch (IOException e) {
// throw our original exception
IOUtils.closeSafely(e, raf, stream);
}
}
@Override
public IndexInput openInputSlice(String id, long offset, long length, int readBufferSize) throws IOException {
return new MMapIndexInput(raf, offset, length, chunkSizePower);
}
@Override
public synchronized void close() throws IOException {
try {
public IndexInputSlicer createSlicer(final String name, final IOContext context) throws IOException {
ensureOpen();
File f = new File(getDirectory(), name);
final RandomAccessFile raf = new RandomAccessFile(f, "r");
return new IndexInputSlicer() {
@Override
public void close() throws IOException {
raf.close();
} finally {
super.close();
}
}
@Override
public IndexInput openSlice(long offset, long length) throws IOException {
return new MMapIndexInput(raf, offset, length, chunkSizePower);
}
@Override
public IndexInput openFullSlice() throws IOException {
return openSlice(0, raf.length());
}
};
}
// Because Java's ByteBuffer uses an int to address the

View File

@ -24,8 +24,7 @@ import java.nio.channels.ClosedChannelException; // javadoc @link
import java.nio.channels.FileChannel;
import java.util.concurrent.Future; // javadoc
import org.apache.lucene.store.SimpleFSDirectory.SimpleFSIndexInput;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.store.SimpleFSDirectory.SimpleFSIndexInput.Descriptor;
/**
* An {@link FSDirectory} implementation that uses java.nio's FileChannel's
@ -81,45 +80,29 @@ public class NIOFSDirectory extends FSDirectory {
return new NIOFSIndexInput(new File(getDirectory(), name), context, getReadChunkSize());
}
@Override
public CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
return new NIOFSCompoundFileDirectory(name, context);
}
public IndexInputSlicer createSlicer(final String name,
final IOContext context) throws IOException {
ensureOpen();
final File file = new File(getDirectory(), name);
final Descriptor descriptor = new Descriptor(file, "r");
return new Directory.IndexInputSlicer() {
private final class NIOFSCompoundFileDirectory extends CompoundFileDirectory {
private SimpleFSIndexInput.Descriptor fd;
private FileChannel fc;
public NIOFSCompoundFileDirectory(String fileName, IOContext context) throws IOException {
super(NIOFSDirectory.this, fileName, context);
IndexInput stream = null;
try {
File f = new File(NIOFSDirectory.this.getDirectory(), fileName);
fd = new SimpleFSIndexInput.Descriptor(f, "r");
fc = fd.getChannel();
stream = new NIOFSIndexInput(fd, fc, 0, fd.length, readBufferSize,
getReadChunkSize());
initForRead(CompoundFileDirectory.readEntries(stream, NIOFSDirectory.this, fileName));
stream.close();
} catch (IOException e) {
// throw our original exception
IOUtils.closeSafely(e, fc, fd, stream);
@Override
public void close() throws IOException {
descriptor.close();
}
}
@Override
public IndexInput openInputSlice(String id, long offset, long length, int readBufferSize) throws IOException {
return new NIOFSIndexInput(fd, fc, offset, length, readBufferSize, getReadChunkSize());
}
@Override
public synchronized void close() throws IOException {
try {
IOUtils.closeSafely(false, fc, fd);
} finally {
super.close();
@Override
public IndexInput openSlice(long offset, long length) throws IOException {
return new NIOFSIndexInput(descriptor, descriptor.getChannel(), offset,
length, BufferedIndexInput.bufferSize(context), getReadChunkSize());
}
}
@Override
public IndexInput openFullSlice() throws IOException {
return openSlice(0, descriptor.length);
}
};
}
protected static class NIOFSIndexInput extends SimpleFSDirectory.SimpleFSIndexInput {

View File

@ -21,8 +21,7 @@ import java.io.File;
import java.io.IOException;
import java.io.RandomAccessFile;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.store.SimpleFSDirectory.SimpleFSIndexInput.Descriptor;
/** A straightforward implementation of {@link FSDirectory}
* using java.io.RandomAccessFile. However, this class has
@ -59,44 +58,33 @@ public class SimpleFSDirectory extends FSDirectory {
return new SimpleFSIndexInput(new File(directory, name), context, getReadChunkSize());
}
@Override
public CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
return new SimpleFSCompoundFileDirectory(name, context);
public IndexInputSlicer createSlicer(final String name,
final IOContext context) throws IOException {
ensureOpen();
final File file = new File(getDirectory(), name);
final Descriptor descriptor = new Descriptor(file, "r");
return new IndexInputSlicer() {
@Override
public void close() throws IOException {
descriptor.close();
}
@Override
public IndexInput openSlice(long offset, long length) throws IOException {
return new SimpleFSIndexInput(descriptor, offset,
length, BufferedIndexInput.bufferSize(context), getReadChunkSize());
}
@Override
public IndexInput openFullSlice() throws IOException {
return openSlice(0, descriptor.length);
}
};
}
private final class SimpleFSCompoundFileDirectory extends CompoundFileDirectory {
private SimpleFSIndexInput.Descriptor fd;
public SimpleFSCompoundFileDirectory(String fileName, IOContext context) throws IOException {
super(SimpleFSDirectory.this, fileName, context);
IndexInput stream = null;
try {
final File f = new File(SimpleFSDirectory.this.getDirectory(), fileName);
fd = new SimpleFSIndexInput.Descriptor(f, "r");
stream = new SimpleFSIndexInput(fd, 0, fd.length, readBufferSize,
getReadChunkSize());
initForRead(CompoundFileDirectory.readEntries(stream, SimpleFSDirectory.this, fileName));
stream.close();
} catch (IOException e) {
// throw our original exception
IOUtils.closeSafely(e, fd, stream);
}
}
@Override
public IndexInput openInputSlice(String id, long offset, long length, int readBufferSize) throws IOException {
return new SimpleFSIndexInput(fd, offset, length, readBufferSize, getReadChunkSize());
}
@Override
public synchronized void close() throws IOException {
try {
fd.close();
} finally {
super.close();
}
}
}
protected static class SimpleFSIndexInput extends BufferedIndexInput {

View File

@ -1,149 +0,0 @@
package org.apache.lucene.store;
/**
* 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;
public class MockCompoundFileDirectoryWrapper extends CompoundFileDirectory {
private final MockDirectoryWrapper parent;
private final CompoundFileDirectory delegate;
private final String name;
public MockCompoundFileDirectoryWrapper(String name, MockDirectoryWrapper parent, CompoundFileDirectory delegate, boolean forWrite) throws IOException {
super(parent, name, IOContext.DEFAULT);
this.name = name;
this.parent = parent;
this.delegate = delegate;
// don't initialize here since we delegate everything - if not initialized a direct call will cause an assert to fail!
parent.addFileHandle(this, name, !forWrite);
}
@Override
public Directory getDirectory() {
return delegate.getDirectory();
}
@Override
public String getName() {
return delegate.getName();
}
@Override
public synchronized void close() throws IOException {
delegate.close();
parent.removeOpenFile(this, name);
}
@Override
public synchronized IndexInput openInput(String id, IOContext context) throws IOException {
return delegate.openInput(id, context);
}
@Override
public String[] listAll() {
return delegate.listAll();
}
@Override
public boolean fileExists(String name) {
return delegate.fileExists(name);
}
@Override
public long fileModified(String name) throws IOException {
return delegate.fileModified(name);
}
@Override
public void deleteFile(String name) {
delegate.deleteFile(name);
}
@Override
public void renameFile(String from, String to) {
delegate.renameFile(from, to);
}
@Override
public long fileLength(String name) throws IOException {
return delegate.fileLength(name);
}
@Override
public IndexOutput createOutput(String name, IOContext context) throws IOException {
return delegate.createOutput(name, context);
}
@Override
public void sync(Collection<String> names) throws IOException {
delegate.sync(names);
}
@Override
public Lock makeLock(String name) {
return delegate.makeLock(name);
}
@Override
public void clearLock(String name) throws IOException {
delegate.clearLock(name);
}
@Override
public void setLockFactory(LockFactory lockFactory) throws IOException {
delegate.setLockFactory(lockFactory);
}
@Override
public LockFactory getLockFactory() {
return delegate.getLockFactory();
}
@Override
public String getLockID() {
return delegate.getLockID();
}
@Override
public String toString() {
return "MockCompoundFileDirectoryWrapper(" + super.toString() + ")";
}
@Override
public void copy(Directory to, String src, String dest, IOContext context) throws IOException {
delegate.copy(to, src, dest, context);
}
@Override
public IndexInput openInputSlice(String id, long offset, long length, int readBufferSize) throws IOException {
return delegate.openInputSlice(id, offset, length, readBufferSize);
}
@Override
public CompoundFileDirectory createCompoundOutput(String name, IOContext context) throws IOException {
return delegate.createCompoundOutput(name, context);
}
@Override
public CompoundFileDirectory openCompoundInput(String name, IOContext context)
throws IOException {
return delegate.openCompoundInput(name, context);
}
}

View File

@ -377,7 +377,7 @@ public class MockDirectoryWrapper extends Directory {
//System.out.println(Thread.currentThread().getName() + ": MDW: create " + name);
IndexOutput io = new MockIndexOutputWrapper(this, delegate.createOutput(name, LuceneTestCase.newIOContext(randomState)), name);
addFileHandle(io, name, false);
addFileHandle(io, name, Handle.Output);
openFilesForWrite.add(name);
// throttling REALLY slows down tests, so don't do it very often for SOMETIMES.
@ -391,8 +391,12 @@ public class MockDirectoryWrapper extends Directory {
return io;
}
}
private static enum Handle {
Input, Output, Slice
}
synchronized void addFileHandle(Closeable c, String name, boolean input) {
synchronized void addFileHandle(Closeable c, String name, Handle handle) {
Integer v = openFiles.get(name);
if (v != null) {
v = Integer.valueOf(v.intValue()+1);
@ -401,7 +405,7 @@ public class MockDirectoryWrapper extends Directory {
openFiles.put(name, Integer.valueOf(1));
}
openFileHandles.put(c, new RuntimeException("unclosed Index" + (input ? "Input" : "Output") + ": " + name));
openFileHandles.put(c, new RuntimeException("unclosed Index" + handle.name() + ": " + name));
}
@Override
@ -417,22 +421,10 @@ public class MockDirectoryWrapper extends Directory {
}
IndexInput ii = new MockIndexInputWrapper(this, name, delegate.openInput(name, LuceneTestCase.newIOContext(randomState)));
addFileHandle(ii, name, true);
addFileHandle(ii, name, Handle.Input);
return ii;
}
@Override
public synchronized CompoundFileDirectory openCompoundInput(String name, IOContext context) throws IOException {
maybeYield();
return new MockCompoundFileDirectoryWrapper(name, this, delegate.openCompoundInput(name, context), false);
}
@Override
public CompoundFileDirectory createCompoundOutput(String name, IOContext context) throws IOException {
maybeYield();
return new MockCompoundFileDirectoryWrapper(name, this, delegate.createCompoundOutput(name, context), true);
}
/** Provided for testing purposes. Use sizeInBytes() instead. */
public synchronized final long getRecomputedSizeInBytes() throws IOException {
if (!(delegate instanceof RAMDirectory))
@ -658,5 +650,50 @@ public class MockDirectoryWrapper extends Directory {
// randomize the IOContext here?
delegate.copy(to, src, dest, context);
}
@Override
public IndexInputSlicer createSlicer(final String name, IOContext context)
throws IOException {
maybeYield();
if (!delegate.fileExists(name))
throw new FileNotFoundException(name);
// cannot open a file for input if it's still open for
// output, except for segments.gen and segments_N
if (openFilesForWrite.contains(name) && !name.startsWith("segments")) {
throw fillOpenTrace(new IOException("MockDirectoryWrapper: file \"" + name + "\" is still open for writing"), name, false);
}
final IndexInputSlicer delegateHandle = delegate.createSlicer(name, context);
final IndexInputSlicer handle = new IndexInputSlicer() {
private boolean isClosed;
@Override
public void close() throws IOException {
if (!isClosed) {
delegateHandle.close();
MockDirectoryWrapper.this.removeOpenFile(this, name);
isClosed = true;
}
}
@Override
public IndexInput openSlice(long offset, long length) throws IOException {
maybeYield();
IndexInput ii = new MockIndexInputWrapper(MockDirectoryWrapper.this, name, delegateHandle.openSlice(offset, length));
addFileHandle(ii, name, Handle.Input);
return ii;
}
@Override
public IndexInput openFullSlice() throws IOException {
maybeYield();
IndexInput ii = new MockIndexInputWrapper(MockDirectoryWrapper.this, name, delegateHandle.openFullSlice());
addFileHandle(ii, name, Handle.Input);
return ii;
}
};
addFileHandle(handle, name, Handle.Slice);
return handle;
}
}

View File

@ -548,7 +548,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
// figure out which field number corresponds to
// "content", and then set our expected file names below
// accordingly:
CompoundFileDirectory cfsReader = dir.openCompoundInput("_0.cfs", newIOContext(random));
CompoundFileDirectory cfsReader = new CompoundFileDirectory(dir, "_0.cfs", newIOContext(random), false);
FieldInfos fieldInfos = new FieldInfos(cfsReader, "_0.fnm");
int contentFieldIndex = -1;
for (FieldInfo fi : fieldInfos) {

View File

@ -182,11 +182,11 @@ public class TestCompoundFile extends LuceneTestCase
for (int i=0; i<data.length; i++) {
String name = "t" + data[i];
createSequenceFile(dir, name, (byte) 0, data[i]);
CompoundFileDirectory csw = dir.createCompoundOutput(name + ".cfs", newIOContext(random));
CompoundFileDirectory csw = new CompoundFileDirectory(dir, name + ".cfs", newIOContext(random), true);
dir.copy(csw, name, name, newIOContext(random));
csw.close();
CompoundFileDirectory csr = dir.openCompoundInput(name + ".cfs", newIOContext(random));
CompoundFileDirectory csr = new CompoundFileDirectory(dir, name + ".cfs", newIOContext(random), false);
IndexInput expected = dir.openInput(name, newIOContext(random));
IndexInput actual = csr.openInput(name, newIOContext(random));
assertSameStreams(name, expected, actual);
@ -205,12 +205,12 @@ public class TestCompoundFile extends LuceneTestCase
createSequenceFile(dir, "d1", (byte) 0, 15);
createSequenceFile(dir, "d2", (byte) 0, 114);
CompoundFileDirectory csw = dir.createCompoundOutput("d.cfs", newIOContext(random));
CompoundFileDirectory csw = new CompoundFileDirectory(dir, "d.cfs", newIOContext(random), true);
dir.copy(csw, "d1", "d1", newIOContext(random));
dir.copy(csw, "d2", "d2", newIOContext(random));
csw.close();
CompoundFileDirectory csr = dir.openCompoundInput("d.cfs", newIOContext(random));
CompoundFileDirectory csr = new CompoundFileDirectory(dir, "d.cfs", newIOContext(random), false);
IndexInput expected = dir.openInput("d1", newIOContext(random));
IndexInput actual = csr.openInput("d1", newIOContext(random));
assertSameStreams("d1", expected, actual);
@ -255,7 +255,7 @@ public class TestCompoundFile extends LuceneTestCase
createRandomFile(dir, segment + ".notIn2", 51);
// Now test
CompoundFileDirectory csw = dir.createCompoundOutput("test.cfs", newIOContext(random));
CompoundFileDirectory csw = new CompoundFileDirectory(dir, "test.cfs", newIOContext(random), true);
final String data[] = new String[] {
".zero", ".one", ".ten", ".hundred", ".big1", ".big2", ".big3",
".big4", ".big5", ".big6", ".big7"
@ -266,7 +266,7 @@ public class TestCompoundFile extends LuceneTestCase
}
csw.close();
CompoundFileDirectory csr = dir.openCompoundInput("test.cfs", newIOContext(random));
CompoundFileDirectory csr = new CompoundFileDirectory(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));
@ -285,7 +285,7 @@ public class TestCompoundFile extends LuceneTestCase
* the size of each file is 1000 bytes.
*/
private void setUp_2() throws IOException {
CompoundFileDirectory cw = dir.createCompoundOutput("f.comp", newIOContext(random));
CompoundFileDirectory cw = new CompoundFileDirectory(dir, "f.comp", newIOContext(random), true);
for (int i=0; i<20; i++) {
createSequenceFile(dir, "f" + i, (byte) 0, 2000);
String fileName = "f" + i;
@ -336,7 +336,7 @@ public class TestCompoundFile extends LuceneTestCase
public void testClonedStreamsClosing() throws IOException {
setUp_2();
CompoundFileDirectory cr = dir.openCompoundInput("f.comp", newIOContext(random));
CompoundFileDirectory cr = new CompoundFileDirectory(dir, "f.comp", newIOContext(random), false);
// basic clone
IndexInput expected = dir.openInput("f11", newIOContext(random));
@ -388,7 +388,7 @@ public class TestCompoundFile extends LuceneTestCase
*/
public void testRandomAccess() throws IOException {
setUp_2();
CompoundFileDirectory cr = dir.openCompoundInput("f.comp", newIOContext(random));
CompoundFileDirectory cr = new CompoundFileDirectory(dir, "f.comp", newIOContext(random), false);
// Open two files
IndexInput e1 = dir.openInput("f11", newIOContext(random));
@ -467,7 +467,7 @@ public class TestCompoundFile extends LuceneTestCase
*/
public void testRandomAccessClones() throws IOException {
setUp_2();
CompoundFileDirectory cr = dir.openCompoundInput("f.comp", newIOContext(random));
CompoundFileDirectory cr = new CompoundFileDirectory(dir, "f.comp", newIOContext(random), false);
// Open two files
IndexInput e1 = cr.openInput("f11", newIOContext(random));
@ -544,7 +544,7 @@ public class TestCompoundFile extends LuceneTestCase
public void testFileNotFound() throws IOException {
setUp_2();
CompoundFileDirectory cr = dir.openCompoundInput("f.comp", newIOContext(random));
CompoundFileDirectory cr = new CompoundFileDirectory(dir, "f.comp", newIOContext(random), false);
// Open two files
try {
@ -562,7 +562,7 @@ public class TestCompoundFile extends LuceneTestCase
public void testReadPastEOF() throws IOException {
setUp_2();
CompoundFileDirectory cr = dir.openCompoundInput("f.comp", newIOContext(random));
CompoundFileDirectory cr = new CompoundFileDirectory(dir, "f.comp", newIOContext(random), false);
IndexInput is = cr.openInput("f2", newIOContext(random));
is.seek(is.length() - 10);
byte b[] = new byte[100];
@ -615,11 +615,11 @@ public class TestCompoundFile extends LuceneTestCase
createSequenceFile(dir, "d1", (byte) 0, 15);
Directory newDir = newDirectory();
CompoundFileDirectory csw = newDir.createCompoundOutput("d.cfs", newIOContext(random));
CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), true);
dir.copy(csw, "d1", "d1", newIOContext(random));
csw.close();
CompoundFileDirectory csr = newDir.openCompoundInput("d.cfs", newIOContext(random));
CompoundFileDirectory csr = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), false);
IndexInput expected = dir.openInput("d1", newIOContext(random));
IndexInput actual = csr.openInput("d1", newIOContext(random));
assertSameStreams("d1", expected, actual);
@ -634,7 +634,7 @@ public class TestCompoundFile extends LuceneTestCase
public void testAppend() throws IOException {
Directory newDir = newDirectory();
CompoundFileDirectory csw = newDir.createCompoundOutput("d.cfs", newIOContext(random));
CompoundFileDirectory csw = new CompoundFileDirectory(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));
@ -652,7 +652,7 @@ public class TestCompoundFile extends LuceneTestCase
assertEquals(1, listAll.length);
assertEquals("d.cfs", listAll[0]);
csw.close();
CompoundFileDirectory csr = newDir.openCompoundInput("d.cfs", newIOContext(random));
CompoundFileDirectory csr = new CompoundFileDirectory(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());
@ -675,7 +675,7 @@ public class TestCompoundFile extends LuceneTestCase
public void testAppendTwice() throws IOException {
Directory newDir = newDirectory();
CompoundFileDirectory csw = newDir.createCompoundOutput("d.cfs", newIOContext(random));
CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), true);
createSequenceFile(newDir, "d1", (byte) 0, 15);
IndexOutput out = csw.createOutput("d.xyz", newIOContext(random));
out.writeInt(0);
@ -691,7 +691,7 @@ public class TestCompoundFile extends LuceneTestCase
csw.close();
CompoundFileDirectory cfr = newDir.openCompoundInput("d.cfs", newIOContext(random));
CompoundFileDirectory cfr = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), false);
assertEquals(1, cfr.listAll().length);
assertEquals("d.xyz", cfr.listAll()[0]);
cfr.close();
@ -700,10 +700,10 @@ public class TestCompoundFile extends LuceneTestCase
public void testEmptyCFS() throws IOException {
Directory newDir = newDirectory();
CompoundFileDirectory csw = newDir.createCompoundOutput("d.cfs", newIOContext(random));
CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), true);
csw.close();
CompoundFileDirectory csr = newDir.openCompoundInput("d.cfs", newIOContext(random));
CompoundFileDirectory csr = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), false);
assertEquals(0, csr.listAll().length);
csr.close();
@ -712,8 +712,8 @@ public class TestCompoundFile extends LuceneTestCase
public void testReadNestedCFP() throws IOException {
Directory newDir = newDirectory();
CompoundFileDirectory csw = newDir.createCompoundOutput("d.cfs", newIOContext(random));
CompoundFileDirectory nested = newDir.createCompoundOutput("b.cfs", newIOContext(random));
CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), true);
CompoundFileDirectory nested = new CompoundFileDirectory(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);
@ -728,10 +728,10 @@ public class TestCompoundFile extends LuceneTestCase
csw.close();
assertEquals(2, newDir.listAll().length);
csw = newDir.openCompoundInput("d.cfs", newIOContext(random));
csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), false);
assertEquals(2, csw.listAll().length);
nested = csw.openCompoundInput("b.cfs", newIOContext(random));
nested = new CompoundFileDirectory(csw, "b.cfs", newIOContext(random), false);
assertEquals(2, nested.listAll().length);
IndexInput openInput = nested.openInput("b.xyz", newIOContext(random));
@ -747,7 +747,7 @@ public class TestCompoundFile extends LuceneTestCase
public void testDoubleClose() throws IOException {
Directory newDir = newDirectory();
CompoundFileDirectory csw = newDir.createCompoundOutput("d.cfs", newIOContext(random));
CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), true);
IndexOutput out = csw.createOutput("d.xyz", newIOContext(random));
out.writeInt(0);
out.close();
@ -756,7 +756,7 @@ public class TestCompoundFile extends LuceneTestCase
// close a second time - must have no effect according to Closeable
csw.close();
csw = newDir.openCompoundInput("d.cfs", newIOContext(random));
csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), false);
IndexInput openInput = csw.openInput("d.xyz", newIOContext(random));
assertEquals(0, openInput.readInt());
openInput.close();
@ -781,7 +781,7 @@ public class TestCompoundFile extends LuceneTestCase
out.close();
}
final CompoundFileDirectory cfd = d.createCompoundOutput("c.cfs", newIOContext(random));
final CompoundFileDirectory cfd = new CompoundFileDirectory(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));
@ -789,7 +789,7 @@ public class TestCompoundFile extends LuceneTestCase
cfd.close();
final IndexInput[] ins = new IndexInput[FILE_COUNT];
final CompoundFileDirectory cfr = d.openCompoundInput("c.cfs", newIOContext(random));
final CompoundFileDirectory cfr = new CompoundFileDirectory(d, "c.cfs", newIOContext(random), false);
for(int fileIdx=0;fileIdx<FILE_COUNT;fileIdx++) {
ins[fileIdx] = cfr.openInput("file." + fileIdx, newIOContext(random));
}

View File

@ -91,7 +91,7 @@ public class TestIndexFileDeleter extends LuceneTestCase {
// figure out which field number corresponds to
// "content", and then set our expected file names below
// accordingly:
CompoundFileDirectory cfsReader = dir.openCompoundInput("_2.cfs", newIOContext(random));
CompoundFileDirectory cfsReader = new CompoundFileDirectory(dir, "_2.cfs", newIOContext(random), false);
FieldInfos fieldInfos = new FieldInfos(cfsReader, "_2.fnm");
int contentFieldIndex = -1;
for (FieldInfo fi : fieldInfos) {

View File

@ -116,7 +116,7 @@ public class TestFileSwitchDirectory extends LuceneTestCase {
// LUCENE-3380 test that delegate compound files correctly.
public void testCompoundFileAppendTwice() throws IOException {
Directory newDir = newFSSwitchDirectory(Collections.singleton("cfs"));
CompoundFileDirectory csw = newDir.createCompoundOutput("d.cfs", newIOContext(random));
CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), true);
createSequenceFile(newDir, "d1", (byte) 0, 15);
IndexOutput out = csw.createOutput("d.xyz", newIOContext(random));
out.writeInt(0);
@ -132,7 +132,7 @@ public class TestFileSwitchDirectory extends LuceneTestCase {
csw.close();
CompoundFileDirectory cfr = newDir.openCompoundInput("d.cfs", newIOContext(random));
CompoundFileDirectory cfr = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), false);
assertEquals(1, cfr.listAll().length);
assertEquals("d.xyz", cfr.listAll()[0]);
cfr.close();