mirror of
https://github.com/apache/lucene.git
synced 2025-02-06 01:58:44 +00:00
LUCENE-5969: start improving CFSDir
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5969@1628889 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
5a9cba4598
commit
6456948a47
@ -85,7 +85,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.directory, filename, state.context, false);
|
||||
this.dir = new CompoundFileDirectory(state.segmentInfo.getId(), state.directory, filename, state.context, false);
|
||||
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOf(getClass()));
|
||||
merging = false;
|
||||
}
|
||||
|
@ -51,7 +51,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.directory, filename, state.context, true);
|
||||
this.dir = new CompoundFileDirectory(state.segmentInfo.getId(), state.directory, filename, state.context, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -4459,7 +4459,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||
}
|
||||
// Now merge all added files
|
||||
Collection<String> files = info.files();
|
||||
CompoundFileDirectory cfsDir = new CompoundFileDirectory(directory, fileName, context, true);
|
||||
CompoundFileDirectory cfsDir = new CompoundFileDirectory(info.getId(), directory, fileName, context, true);
|
||||
boolean success = false;
|
||||
try {
|
||||
for (String file : files) {
|
||||
|
@ -99,7 +99,7 @@ final class SegmentCoreReaders implements Accountable {
|
||||
|
||||
try {
|
||||
if (si.info.getUseCompoundFile()) {
|
||||
cfsDir = cfsReader = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(si.info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
|
||||
cfsDir = cfsReader = new CompoundFileDirectory(si.info.getId(), dir, IndexFileNames.segmentFileName(si.info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
|
||||
} else {
|
||||
cfsReader = null;
|
||||
cfsDir = dir;
|
||||
|
@ -202,7 +202,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.dir,
|
||||
dir = new CompoundFileDirectory(info.info.getId(), info.info.dir,
|
||||
IndexFileNames.segmentFileName(info.info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION),
|
||||
IOContext.READONCE,
|
||||
false);
|
||||
|
@ -24,7 +24,9 @@ import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.store.DataOutput; // javadocs
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
@ -55,7 +57,7 @@ import java.io.IOException;
|
||||
* <li>Compound (.cfs) --> Header, FileData <sup>FileCount</sup>, Footer</li>
|
||||
* <li>Compound Entry Table (.cfe) --> Header, FileCount, <FileName,
|
||||
* DataOffset, DataLength> <sup>FileCount</sup></li>
|
||||
* <li>Header --> {@link CodecUtil#writeHeader CodecHeader}</li>
|
||||
* <li>Header --> {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
|
||||
* <li>FileCount --> {@link DataOutput#writeVInt VInt}</li>
|
||||
* <li>DataOffset,DataLength,Checksum --> {@link DataOutput#writeLong UInt64}</li>
|
||||
* <li>FileName --> {@link DataOutput#writeString String}</li>
|
||||
@ -89,12 +91,14 @@ public final class CompoundFileDirectory extends BaseDirectory {
|
||||
private final CompoundFileWriter writer;
|
||||
private final IndexInput handle;
|
||||
private int version;
|
||||
private final byte[] segmentID;
|
||||
|
||||
/**
|
||||
* Create a new CompoundFileDirectory.
|
||||
*/
|
||||
public CompoundFileDirectory(Directory directory, String fileName, IOContext context, boolean openForWrite) throws IOException {
|
||||
public CompoundFileDirectory(byte[] segmentID, Directory directory, String fileName, IOContext context, boolean openForWrite) throws IOException {
|
||||
this.directory = directory;
|
||||
this.segmentID = segmentID;
|
||||
this.fileName = fileName;
|
||||
this.readBufferSize = BufferedIndexInput.bufferSize(context);
|
||||
this.isOpen = false;
|
||||
@ -105,7 +109,17 @@ public final class CompoundFileDirectory extends BaseDirectory {
|
||||
try {
|
||||
this.entries = readEntries(directory, fileName);
|
||||
if (version >= CompoundFileWriter.VERSION_CHECKSUM) {
|
||||
CodecUtil.checkHeader(handle, CompoundFileWriter.DATA_CODEC, version, version);
|
||||
if (version >= CompoundFileWriter.VERSION_SEGMENTHEADER) {
|
||||
// nocommit: remove this null "hack", its because old rw test codecs cant properly impersonate
|
||||
if (segmentID == null) {
|
||||
CodecUtil.checkHeader(handle, CompoundFileWriter.DATA_CODEC, version, version);
|
||||
handle.skipBytes(StringHelper.ID_LENGTH);
|
||||
} else {
|
||||
CodecUtil.checkSegmentHeader(handle, CompoundFileWriter.DATA_CODEC, version, version, segmentID, "");
|
||||
}
|
||||
} else {
|
||||
CodecUtil.checkHeader(handle, CompoundFileWriter.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
|
||||
@ -124,7 +138,7 @@ public final class CompoundFileDirectory extends BaseDirectory {
|
||||
assert !(directory instanceof CompoundFileDirectory) : "compound file inside of compound file: " + fileName;
|
||||
this.entries = SENTINEL;
|
||||
this.isOpen = true;
|
||||
writer = new CompoundFileWriter(directory, fileName);
|
||||
writer = new CompoundFileWriter(segmentID, directory, fileName);
|
||||
handle = null;
|
||||
}
|
||||
}
|
||||
@ -140,6 +154,19 @@ public final class CompoundFileDirectory extends BaseDirectory {
|
||||
IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
|
||||
entriesStream = dir.openChecksumInput(entriesFileName, IOContext.READONCE);
|
||||
version = CodecUtil.checkHeader(entriesStream, CompoundFileWriter.ENTRY_CODEC, CompoundFileWriter.VERSION_START, CompoundFileWriter.VERSION_CURRENT);
|
||||
if (version >= CompoundFileWriter.VERSION_SEGMENTHEADER) {
|
||||
byte id[] = new byte[StringHelper.ID_LENGTH];
|
||||
entriesStream.readBytes(id, 0, id.length);
|
||||
// nocommit: remove this null "hack", its because old rw test codecs cant properly impersonate
|
||||
if (segmentID != null && !Arrays.equals(id, segmentID)) {
|
||||
throw new CorruptIndexException("file mismatch, expected segment id=" + StringHelper.idToString(segmentID)
|
||||
+ ", got=" + StringHelper.idToString(id), entriesStream);
|
||||
}
|
||||
byte suffixLength = entriesStream.readByte();
|
||||
if (suffixLength != 0) {
|
||||
throw new CorruptIndexException("unexpected segment suffix, expected zero-length, got=" + (suffixLength & 0xFF), entriesStream);
|
||||
}
|
||||
}
|
||||
final int numEntries = entriesStream.readVInt();
|
||||
mapping = new HashMap<>(numEntries);
|
||||
for (int i = 0; i < numEntries; i++) {
|
||||
|
@ -55,7 +55,8 @@ final class CompoundFileWriter implements Closeable{
|
||||
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;
|
||||
static final int VERSION_SEGMENTHEADER = 2;
|
||||
static final int VERSION_CURRENT = VERSION_SEGMENTHEADER;
|
||||
|
||||
// versioning for the .cfe file
|
||||
static final String ENTRY_CODEC = "CompoundFileWriterEntries";
|
||||
@ -70,6 +71,7 @@ final class CompoundFileWriter implements Closeable{
|
||||
private final AtomicBoolean outputTaken = new AtomicBoolean(false);
|
||||
final String entryTableName;
|
||||
final String dataFileName;
|
||||
final byte[] segmentID;
|
||||
|
||||
/**
|
||||
* Create the compound stream in the specified file. The file name is the
|
||||
@ -78,11 +80,17 @@ final class CompoundFileWriter implements Closeable{
|
||||
* @throws NullPointerException
|
||||
* if <code>dir</code> or <code>name</code> is null
|
||||
*/
|
||||
CompoundFileWriter(Directory dir, String name) {
|
||||
if (dir == null)
|
||||
CompoundFileWriter(byte segmentID[], Directory dir, String name) {
|
||||
if (dir == null) {
|
||||
throw new NullPointerException("directory cannot be null");
|
||||
if (name == null)
|
||||
}
|
||||
if (name == null) {
|
||||
throw new NullPointerException("name cannot be null");
|
||||
}
|
||||
if (segmentID == null) {
|
||||
throw new NullPointerException("segmentID cannot be null");
|
||||
}
|
||||
this.segmentID = segmentID;
|
||||
directory = dir;
|
||||
entryTableName = IndexFileNames.segmentFileName(
|
||||
IndexFileNames.stripExtension(name), "",
|
||||
@ -96,7 +104,7 @@ final class CompoundFileWriter implements Closeable{
|
||||
boolean success = false;
|
||||
try {
|
||||
dataOut = directory.createOutput(dataFileName, context);
|
||||
CodecUtil.writeHeader(dataOut, DATA_CODEC, VERSION_CURRENT);
|
||||
CodecUtil.writeSegmentHeader(dataOut, DATA_CODEC, VERSION_CURRENT, segmentID, "");
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
@ -207,7 +215,7 @@ final class CompoundFileWriter implements Closeable{
|
||||
|
||||
protected void writeEntryTable(Collection<FileEntry> entries,
|
||||
IndexOutput entryOut) throws IOException {
|
||||
CodecUtil.writeHeader(entryOut, ENTRY_CODEC, VERSION_CURRENT);
|
||||
CodecUtil.writeSegmentHeader(entryOut, ENTRY_CODEC, VERSION_CURRENT, segmentID, "");
|
||||
entryOut.writeVInt(entries.size());
|
||||
for (FileEntry fe : entries) {
|
||||
entryOut.writeString(IndexFileNames.stripSegmentName(fe.file));
|
||||
|
@ -28,7 +28,6 @@ import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
@ -65,33 +64,33 @@ public class TestAllFilesHaveChecksumFooter extends LuceneTestCase {
|
||||
}
|
||||
}
|
||||
riw.close();
|
||||
checkHeaders(dir);
|
||||
checkFooters(dir);
|
||||
dir.close();
|
||||
}
|
||||
|
||||
private void checkHeaders(Directory dir) throws IOException {
|
||||
for (String file : dir.listAll()) {
|
||||
if (file.equals(IndexWriter.WRITE_LOCK_NAME)) {
|
||||
continue; // write.lock has no footer, thats ok
|
||||
}
|
||||
if (file.endsWith(IndexFileNames.COMPOUND_FILE_EXTENSION)) {
|
||||
CompoundFileDirectory cfsDir = new CompoundFileDirectory(dir, file, newIOContext(random()), false);
|
||||
checkHeaders(cfsDir); // recurse into cfs
|
||||
cfsDir.close();
|
||||
}
|
||||
IndexInput in = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
in = dir.openInput(file, newIOContext(random()));
|
||||
CodecUtil.checksumEntireFile(in);
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(in);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(in);
|
||||
private void checkFooters(Directory dir) throws IOException {
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
sis.read(dir);
|
||||
checkFooter(dir, sis.getSegmentsFileName());
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void checkFooter(Directory dir, String file) throws IOException {
|
||||
try (IndexInput in = dir.openInput(file, newIOContext(random()))) {
|
||||
CodecUtil.checksumEntireFile(in);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -32,7 +32,6 @@ import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
@ -83,40 +82,40 @@ public class TestAllFilesHaveCodecHeader extends LuceneTestCase {
|
||||
}
|
||||
|
||||
private void checkHeaders(Directory dir, Map<String,String> namesToExtensions) throws IOException {
|
||||
for (String file : dir.listAll()) {
|
||||
if (file.equals(IndexWriter.WRITE_LOCK_NAME)) {
|
||||
continue; // write.lock has no header, thats ok
|
||||
}
|
||||
if (file.endsWith(IndexFileNames.COMPOUND_FILE_EXTENSION)) {
|
||||
CompoundFileDirectory cfsDir = new CompoundFileDirectory(dir, file, newIOContext(random()), false);
|
||||
checkHeaders(cfsDir, namesToExtensions); // recurse into cfs
|
||||
cfsDir.close();
|
||||
}
|
||||
IndexInput in = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
in = dir.openInput(file, newIOContext(random()));
|
||||
int val = in.readInt();
|
||||
assertEquals(file + " has no codec header, instead found: " + val, CodecUtil.CODEC_MAGIC, val);
|
||||
String codecName = in.readString();
|
||||
assertFalse(codecName.isEmpty());
|
||||
String extension = IndexFileNames.getExtension(file);
|
||||
if (extension == null) {
|
||||
assertTrue(file.startsWith(IndexFileNames.SEGMENTS));
|
||||
extension = "<segments> (not a real extension, designates segments file)";
|
||||
}
|
||||
String previous = namesToExtensions.put(codecName, extension);
|
||||
if (previous != null && !previous.equals(extension)) {
|
||||
fail("extensions " + previous + " and " + extension + " share same codecName " + codecName);
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(in);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(in);
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
sis.read(dir);
|
||||
checkHeader(dir, sis.getSegmentsFileName(), namesToExtensions);
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void checkHeader(Directory dir, String file, Map<String,String> namesToExtensions) throws IOException {
|
||||
try (IndexInput in = dir.openInput(file, newIOContext(random()))) {
|
||||
int val = in.readInt();
|
||||
assertEquals(file + " has no codec header, instead found: " + val, CodecUtil.CODEC_MAGIC, val);
|
||||
String codecName = in.readString();
|
||||
assertFalse(codecName.isEmpty());
|
||||
String extension = IndexFileNames.getExtension(file);
|
||||
if (extension == null) {
|
||||
assertTrue(file.startsWith(IndexFileNames.SEGMENTS));
|
||||
extension = "<segments> (not a real extension, designates segments file)";
|
||||
}
|
||||
String previous = namesToExtensions.put(codecName, extension);
|
||||
if (previous != null && !previous.equals(extension)) {
|
||||
fail("extensions " + previous + " and " + extension + " share same codecName " + codecName);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -247,10 +247,10 @@ public class TestCodecs extends LuceneTestCase {
|
||||
final FieldData[] fields = new FieldData[] {field};
|
||||
final FieldInfos fieldInfos = builder.finish();
|
||||
final Directory dir = newDirectory();
|
||||
this.write(fieldInfos, dir, fields);
|
||||
Codec codec = Codec.getDefault();
|
||||
final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, null, StringHelper.randomId());
|
||||
|
||||
this.write(si, fieldInfos, dir, fields);
|
||||
final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random())));
|
||||
|
||||
final Iterator<String> fieldsEnum = reader.iterator();
|
||||
@ -304,9 +304,9 @@ public class TestCodecs extends LuceneTestCase {
|
||||
System.out.println("TEST: now write postings");
|
||||
}
|
||||
|
||||
this.write(fieldInfos, dir, fields);
|
||||
Codec codec = Codec.getDefault();
|
||||
final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, null, StringHelper.randomId());
|
||||
this.write(si, fieldInfos, dir, fields);
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now read postings");
|
||||
@ -798,10 +798,9 @@ public class TestCodecs extends LuceneTestCase {
|
||||
}
|
||||
}
|
||||
|
||||
private void write(final FieldInfos fieldInfos, final Directory dir, final FieldData[] fields) throws Throwable {
|
||||
private void write(SegmentInfo si, final FieldInfos fieldInfos, final Directory dir, final FieldData[] fields) throws Throwable {
|
||||
|
||||
final Codec codec = Codec.getDefault();
|
||||
final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, null, StringHelper.randomId());
|
||||
final Codec codec = si.getCodec();
|
||||
final SegmentWriteState state = new SegmentWriteState(InfoStream.getDefault(), dir, si, fieldInfos, null, newIOContext(random()));
|
||||
|
||||
Arrays.sort(fields);
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -0,0 +1,296 @@
|
||||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
/**
|
||||
* Setup a large compound file with a number of components, each of
|
||||
* which is a sequential file (so that we can easily tell that we are
|
||||
* reading in the right byte). The methods sets up 20 files - f0 to f19,
|
||||
* 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();
|
||||
}
|
||||
}
|
@ -35,6 +35,7 @@ import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LineFileDocs;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
public class TestNRTCachingDirectory extends BaseDirectoryTestCase {
|
||||
@ -134,7 +135,7 @@ public class TestNRTCachingDirectory extends BaseDirectoryTestCase {
|
||||
}
|
||||
out.close();
|
||||
|
||||
Directory cfsDir = new CompoundFileDirectory(dir, "big.cfs", context, true);
|
||||
Directory cfsDir = new CompoundFileDirectory(StringHelper.randomId(), dir, "big.cfs", context, true);
|
||||
dir.copy(cfsDir, "big.bin", "big.bin", context);
|
||||
cfsDir.close();
|
||||
dir.close();
|
||||
|
@ -1,134 +0,0 @@
|
||||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Prints the filename and size of each file within a given compound file.
|
||||
* Add the -extract flag to extract files to the current working directory.
|
||||
* In order to make the extracted version of the index work, you have to copy
|
||||
* the segments file from the compound index into the directory where the extracted files are stored.
|
||||
* @param args Usage: org.apache.lucene.index.IndexReader [-extract] <cfsfile>
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.CommandLineUtil;
|
||||
|
||||
/**
|
||||
* Command-line tool for extracting sub-files out of a compound file.
|
||||
*/
|
||||
public class CompoundFileExtractor {
|
||||
|
||||
public static void main(String [] args) {
|
||||
String filename = null;
|
||||
boolean extract = false;
|
||||
String dirImpl = null;
|
||||
|
||||
int j = 0;
|
||||
while(j < args.length) {
|
||||
String arg = args[j];
|
||||
if ("-extract".equals(arg)) {
|
||||
extract = true;
|
||||
} else if ("-dir-impl".equals(arg)) {
|
||||
if (j == args.length - 1) {
|
||||
System.out.println("ERROR: missing value for -dir-impl option");
|
||||
System.exit(1);
|
||||
}
|
||||
j++;
|
||||
dirImpl = args[j];
|
||||
} else if (filename == null) {
|
||||
filename = arg;
|
||||
}
|
||||
j++;
|
||||
}
|
||||
|
||||
if (filename == null) {
|
||||
System.out.println("Usage: org.apache.lucene.index.CompoundFileExtractor [-extract] [-dir-impl X] <cfsfile>");
|
||||
return;
|
||||
}
|
||||
|
||||
Directory dir = null;
|
||||
CompoundFileDirectory cfr = null;
|
||||
IOContext context = IOContext.READ;
|
||||
|
||||
try {
|
||||
Path file = Paths.get(filename);
|
||||
Path directory = file.toAbsolutePath().getParent();
|
||||
filename = file.getFileName().toString();
|
||||
if (dirImpl == null) {
|
||||
dir = FSDirectory.open(directory);
|
||||
} else {
|
||||
dir = CommandLineUtil.newFSDirectory(dirImpl, directory);
|
||||
}
|
||||
|
||||
cfr = new CompoundFileDirectory(dir, filename, IOContext.DEFAULT, false);
|
||||
|
||||
String [] files = cfr.listAll();
|
||||
ArrayUtil.timSort(files); // sort the array of filename so that the output is more readable
|
||||
|
||||
for (int i = 0; i < files.length; ++i) {
|
||||
long len = cfr.fileLength(files[i]);
|
||||
|
||||
if (extract) {
|
||||
System.out.println("extract " + files[i] + " with " + len + " bytes to local directory...");
|
||||
IndexInput ii = cfr.openInput(files[i], context);
|
||||
|
||||
OutputStream f = Files.newOutputStream(Paths.get(files[i]));
|
||||
|
||||
// read and write with a small buffer, which is more effective than reading byte by byte
|
||||
byte[] buffer = new byte[1024];
|
||||
int chunk = buffer.length;
|
||||
while(len > 0) {
|
||||
final int bufLen = (int) Math.min(chunk, len);
|
||||
ii.readBytes(buffer, 0, bufLen);
|
||||
f.write(buffer, 0, bufLen);
|
||||
len -= bufLen;
|
||||
}
|
||||
|
||||
f.close();
|
||||
ii.close();
|
||||
}
|
||||
else
|
||||
System.out.println(files[i] + ": " + len + " bytes");
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
ioe.printStackTrace();
|
||||
}
|
||||
finally {
|
||||
try {
|
||||
if (dir != null)
|
||||
dir.close();
|
||||
if (cfr != null)
|
||||
cfr.close();
|
||||
}
|
||||
catch (IOException ioe) {
|
||||
ioe.printStackTrace();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -34,6 +34,7 @@ import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexNotFoundException;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
/** Base class for per-Directory tests. */
|
||||
@ -587,7 +588,8 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
|
||||
// LUCENE-3382 test that delegate compound files correctly.
|
||||
public void testCompoundFileAppendTwice() throws IOException {
|
||||
Directory newDir = getDirectory(createTempDir("testCompoundFileAppendTwice"));
|
||||
CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random()), true);
|
||||
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);
|
||||
@ -597,7 +599,7 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
|
||||
|
||||
csw.close();
|
||||
|
||||
CompoundFileDirectory cfr = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random()), false);
|
||||
CompoundFileDirectory cfr = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()), false);
|
||||
assertEquals(1, cfr.listAll().length);
|
||||
assertEquals("d.xyz", cfr.listAll()[0]);
|
||||
cfr.close();
|
||||
|
Loading…
x
Reference in New Issue
Block a user