You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2014/10/02 18:09:12 UTC

svn commit: r1629008 - in /lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50: CompoundFileDirectory.java CompoundFileWriter.java Lucene50CompoundFormat.java Lucene50CompoundReader.java

Author: rmuir
Date: Thu Oct  2 16:09:12 2014
New Revision: 1629008

URL: http://svn.apache.org/r1629008
Log:
LUCENE-5969: simplify cfs for 5.0

Added:
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java
      - copied, changed from r1629001, lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/CompoundFileDirectory.java
Removed:
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/CompoundFileDirectory.java
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/CompoundFileWriter.java
Modified:
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java?rev=1629008&r1=1629007&r2=1629008&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java Thu Oct  2 16:09:12 2014
@@ -28,6 +28,8 @@ import org.apache.lucene.index.SegmentIn
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
 
 /**
  * Lucene 5.0 compound file format
@@ -64,17 +66,47 @@ public final class Lucene50CompoundForma
   @Override
   public Directory getCompoundReader(Directory dir, SegmentInfo si, IOContext context) throws IOException {
     String fileName = IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
-    return new CompoundFileDirectory(si.getId(), dir, fileName, context, false);
+    return new Lucene50CompoundReader(si.getId(), dir, fileName, context);
   }
 
   @Override
   public void write(Directory dir, SegmentInfo si, Collection<String> files, CheckAbort checkAbort, IOContext context) throws IOException {
-    String fileName = IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
-    try (CompoundFileDirectory cfs = new CompoundFileDirectory(si.getId(), dir, fileName, context, true)) {
+    String dataFile = IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
+    String entriesFile = IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
+    
+    try (IndexOutput data =    dir.createOutput(dataFile, context);
+         IndexOutput entries = dir.createOutput(entriesFile, context)) {
+      CodecUtil.writeSegmentHeader(data,    DATA_CODEC, VERSION_CURRENT, si.getId(), "");
+      CodecUtil.writeSegmentHeader(entries, ENTRY_CODEC, VERSION_CURRENT, si.getId(), "");
+      
+      // write number of files
+      entries.writeVInt(files.size());
       for (String file : files) {
-        dir.copy(cfs, file, file, context);
-        checkAbort.work(dir.fileLength(file));
+        
+        // write bytes for file
+        long startOffset = data.getFilePointer();
+        try (IndexInput in = dir.openInput(file, IOContext.READONCE)) {
+          data.copyBytes(in, in.length());
+        }
+        long endOffset = data.getFilePointer();
+        
+        long length = endOffset - startOffset;
+        
+        // write entry for file
+        entries.writeString(IndexFileNames.stripSegmentName(file));
+        entries.writeLong(startOffset);
+        entries.writeLong(length);
+        
+        checkAbort.work(length);
       }
+      
+      CodecUtil.writeFooter(data);
+      CodecUtil.writeFooter(entries);
     }
   }
+  
+  static final String DATA_CODEC = "Lucene50CompoundData";
+  static final String ENTRY_CODEC = "Lucene50CompoundEntries";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
 }

Copied: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java (from r1629001, lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/CompoundFileDirectory.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java?p2=lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java&p1=lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/CompoundFileDirectory.java&r1=1629001&r2=1629008&rev=1629008&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/CompoundFileDirectory.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java Thu Oct  2 16:09:12 2014
@@ -17,15 +17,11 @@ package org.apache.lucene.codecs.lucene5
  * limitations under the License.
  */
 
-import org.apache.lucene.codecs.Codec; // javadocs
 import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.LiveDocsFormat; // javadocs
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.store.BaseDirectory;
-import org.apache.lucene.store.BufferedIndexInput;
 import org.apache.lucene.store.ChecksumIndexInput;
-import org.apache.lucene.store.DataOutput; // javadocs
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -44,43 +40,9 @@ import java.io.IOException;
  * Class for accessing a compound stream.
  * This class implements a directory, but is limited to only read operations.
  * Directory methods that would normally modify data throw an exception.
- * <p>
- * All files belonging to a segment have the same name with varying extensions.
- * The extensions correspond to the different file formats used by the {@link Codec}. 
- * When using the Compound File format these files are collapsed into a 
- * single <tt>.cfs</tt> file (except for the {@link LiveDocsFormat}, with a 
- * corresponding <tt>.cfe</tt> file indexing its sub-files.
- * <p>
- * Files:
- * <ul>
- *    <li><tt>.cfs</tt>: An optional "virtual" file consisting of all the other 
- *    index files for systems that frequently run out of file handles.
- *    <li><tt>.cfe</tt>: The "virtual" compound file's entry table holding all 
- *    entries in the corresponding .cfs file.
- * </ul>
- * <p>Description:</p>
- * <ul>
- *   <li>Compound (.cfs) --&gt; Header, FileData <sup>FileCount</sup>, Footer</li>
- *   <li>Compound Entry Table (.cfe) --&gt; Header, FileCount, &lt;FileName,
- *       DataOffset, DataLength&gt; <sup>FileCount</sup></li>
- *   <li>Header --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
- *   <li>FileCount --&gt; {@link DataOutput#writeVInt VInt}</li>
- *   <li>DataOffset,DataLength,Checksum --&gt; {@link DataOutput#writeLong UInt64}</li>
- *   <li>FileName --&gt; {@link DataOutput#writeString String}</li>
- *   <li>FileData --&gt; raw file data</li>
- *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- * </ul>
- * <p>Notes:</p>
- * <ul>
- *   <li>FileCount indicates how many files are contained in this compound file. 
- *       The entry table that follows has that many entries. 
- *   <li>Each directory entry contains a long pointer to the start of this file's data
- *       section, the files length, and a String with that file's name.
- * </ul>
- * 
  * @lucene.experimental
  */
-final class CompoundFileDirectory extends BaseDirectory {
+final class Lucene50CompoundReader extends BaseDirectory {
   
   /** Offset/Length for a slice inside of a compound file */
   public static final class FileEntry {
@@ -90,65 +52,47 @@ final class CompoundFileDirectory extend
   
   private final Directory directory;
   private final String fileName;
-  protected final int readBufferSize;  
   private final Map<String,FileEntry> entries;
-  private final boolean openForWrite;
-  private static final Map<String,FileEntry> SENTINEL = Collections.emptyMap();
-  private final CompoundFileWriter writer;
   private final IndexInput handle;
   private int version;
-  private final byte[] segmentID;
   
   /**
    * Create a new CompoundFileDirectory.
    */
-  public CompoundFileDirectory(byte[] segmentID, Directory directory, String fileName, IOContext context, boolean openForWrite) throws IOException {
+  public Lucene50CompoundReader(byte[] segmentID, Directory directory, String fileName, IOContext context) throws IOException {
     this.directory = directory;
-    this.segmentID = segmentID;
     this.fileName = fileName;
-    this.readBufferSize = BufferedIndexInput.bufferSize(context);
-    this.isOpen = false;
-    this.openForWrite = openForWrite;
-    if (!openForWrite) {
-      boolean success = false;
-      handle = directory.openInput(fileName, context);
-      try {
-        this.entries = readEntries(directory, fileName);
-        CodecUtil.checkSegmentHeader(handle, CompoundFileWriter.DATA_CODEC, version, version, segmentID, "");
-
-        // NOTE: data file is too costly to verify checksum against all the bytes on open,
-        // but for now we at least verify proper structure of the checksum footer: which looks
-        // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
-        // such as file truncation.
-        CodecUtil.retrieveChecksum(handle);
-        success = true;
-      } finally {
-        if (!success) {
-          IOUtils.closeWhileHandlingException(handle);
-        }
+    this.entries = readEntries(segmentID, directory, fileName);
+    boolean success = false;
+    handle = directory.openInput(fileName, context);
+    try {
+      CodecUtil.checkSegmentHeader(handle, Lucene50CompoundFormat.DATA_CODEC, version, version, segmentID, "");
+      
+      // NOTE: data file is too costly to verify checksum against all the bytes on open,
+      // but for now we at least verify proper structure of the checksum footer: which looks
+      // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
+      // such as file truncation.
+      CodecUtil.retrieveChecksum(handle);
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(handle);
       }
-      this.isOpen = true;
-      writer = null;
-    } else {
-      assert !(directory instanceof CompoundFileDirectory) : "compound file inside of compound file: " + fileName;
-      this.entries = SENTINEL;
-      this.isOpen = true;
-      writer = new CompoundFileWriter(segmentID, directory, fileName);
-      handle = null;
     }
+    this.isOpen = true;
   }
 
   /** Helper method that reads CFS entries from an input stream */
-  private final Map<String, FileEntry> readEntries(Directory dir, String name) throws IOException {
+  private final Map<String, FileEntry> readEntries(byte[] segmentID, Directory dir, String name) throws IOException {
     Map<String,FileEntry> mapping = null;
     final String entriesFileName = IndexFileNames.segmentFileName(IndexFileNames.stripExtension(name), "",
                                                                   IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
     try (ChecksumIndexInput entriesStream = dir.openChecksumInput(entriesFileName, IOContext.READONCE)) {
       Throwable priorE = null;
       try {
-        version = CodecUtil.checkSegmentHeader(entriesStream, CompoundFileWriter.ENTRY_CODEC, 
-                                                              CompoundFileWriter.VERSION_START, 
-                                                              CompoundFileWriter.VERSION_CURRENT, segmentID, "");
+        version = CodecUtil.checkSegmentHeader(entriesStream, Lucene50CompoundFormat.ENTRY_CODEC, 
+                                                              Lucene50CompoundFormat.VERSION_START, 
+                                                              Lucene50CompoundFormat.VERSION_CURRENT, segmentID, "");
         final int numEntries = entriesStream.readVInt();
         mapping = new HashMap<>(numEntries);
         for (int i = 0; i < numEntries; i++) {
@@ -167,36 +111,18 @@ final class CompoundFileDirectory extend
         CodecUtil.checkFooter(entriesStream, priorE);
       }
     }
-    return mapping;
-  }
-  
-  public Directory getDirectory() {
-    return directory;
-  }
-  
-  public String getName() {
-    return fileName;
+    return Collections.unmodifiableMap(mapping);
   }
   
   @Override
-  public synchronized void close() throws IOException {
-    if (!isOpen) {
-      // allow double close - usually to be consistent with other closeables
-      return; // already closed
-     }
+  public void close() throws IOException {
     isOpen = false;
-    if (writer != null) {
-      assert openForWrite;
-      writer.close();
-    } else {
-      IOUtils.close(handle);
-    }
+    IOUtils.close(handle);
   }
   
   @Override
-  public synchronized IndexInput openInput(String name, IOContext context) throws IOException {
+  public IndexInput openInput(String name, IOContext context) throws IOException {
     ensureOpen();
-    assert !openForWrite;
     final String id = IndexFileNames.stripSegmentName(name);
     final FileEntry entry = entries.get(id);
     if (entry == null) {
@@ -209,16 +135,12 @@ final class CompoundFileDirectory extend
   @Override
   public String[] listAll() {
     ensureOpen();
-    String[] res;
-    if (writer != null) {
-      res = writer.listAll(); 
-    } else {
-      res = entries.keySet().toArray(new String[entries.size()]);
-      // Add the segment name
-      String seg = IndexFileNames.parseSegmentName(fileName);
-      for (int i = 0; i < res.length; i++) {
-        res[i] = seg + res[i];
-      }
+    String[] res = entries.keySet().toArray(new String[entries.size()]);
+    
+    // Add the segment name
+    String seg = IndexFileNames.parseSegmentName(fileName);
+    for (int i = 0; i < res.length; i++) {
+      res[i] = seg + res[i];
     }
     return res;
   }
@@ -241,9 +163,6 @@ final class CompoundFileDirectory extend
   @Override
   public long fileLength(String name) throws IOException {
     ensureOpen();
-    if (this.writer != null) {
-      return writer.fileLength(name);
-    }
     FileEntry e = entries.get(IndexFileNames.stripSegmentName(name));
     if (e == null)
       throw new FileNotFoundException(name);
@@ -252,8 +171,7 @@ final class CompoundFileDirectory extend
   
   @Override
   public IndexOutput createOutput(String name, IOContext context) throws IOException {
-    ensureOpen();
-    return writer.createOutput(name, context);
+    throw new UnsupportedOperationException();
   }
   
   @Override