lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From rm...@apache.org
Subject svn commit: r1628996 [2/2] - in /lucene/dev/branches/lucene5969/lucene: backward-codecs/src/java/org/apache/lucene/codecs/lucene40/ backward-codecs/src/java/org/apache/lucene/codecs/lucene41/ backward-codecs/src/java/org/apache/lucene/codecs/lucene410/...
Date Thu, 02 Oct 2014 15:27:14 GMT
Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1628996&r1=1628995&r2=1628996&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
(original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
Thu Oct  2 15:27:13 2014
@@ -50,7 +50,6 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.index.MergeState.CheckAbort;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.Lock;
@@ -4453,40 +4452,30 @@ public class IndexWriter implements Clos
   static final Collection<String> createCompoundFile(InfoStream infoStream, Directory
directory, CheckAbort checkAbort, final SegmentInfo info, IOContext context)
           throws IOException {
 
-    final String fileName = IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
+    // nocommit: use trackingdirectorywrapper instead to know which files to delete when
things fail:
+    String cfsFileName = IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
+    String cfeFileName = IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
+
     if (infoStream.isEnabled("IW")) {
-      infoStream.message("IW", "create compound file " + fileName);
+      infoStream.message("IW", "create compound file");
     }
     // Now merge all added files
     Collection<String> files = info.files();
-    CompoundFileDirectory cfsDir = new CompoundFileDirectory(info.getId(), directory, fileName,
context, true);
+    
     boolean success = false;
     try {
-      for (String file : files) {
-        directory.copy(cfsDir, file, file, context);
-        checkAbort.work(directory.fileLength(file));
-      }
+      info.getCodec().compoundFormat().write(directory, info, files, checkAbort, context);
       success = true;
     } finally {
-      if (success) {
-        IOUtils.close(cfsDir);
-      } else {
-        IOUtils.closeWhileHandlingException(cfsDir);
-        try {
-          directory.deleteFile(fileName);
-        } catch (Throwable t) {
-        }
-        try {
-          directory.deleteFile(IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
-        } catch (Throwable t) {
-        }
+      if (!success) {
+        IOUtils.deleteFilesIgnoringExceptions(directory, cfsFileName, cfeFileName);
       }
     }
 
     // Replace all previous files with the CFS/CFE files:
     Set<String> siFiles = new HashSet<>();
-    siFiles.add(fileName);
-    siFiles.add(IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
+    siFiles.add(cfsFileName);
+    siFiles.add(cfeFileName);
     info.setFiles(siFiles);
 
     return files;

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1628996&r1=1628995&r2=1628996&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
(original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
Thu Oct  2 15:27:13 2014
@@ -33,7 +33,6 @@ import org.apache.lucene.codecs.StoredFi
 import org.apache.lucene.codecs.TermVectorsReader;
 import org.apache.lucene.index.LeafReader.CoreClosedListener;
 import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.Accountable;
@@ -60,7 +59,7 @@ final class SegmentCoreReaders implement
 
   final StoredFieldsReader fieldsReaderOrig;
   final TermVectorsReader termVectorsReaderOrig;
-  final CompoundFileDirectory cfsReader;
+  final Directory cfsReader;
 
   // TODO: make a single thread local w/ a
   // Thingy class holding fieldsReader, termVectorsReader,
@@ -99,7 +98,7 @@ final class SegmentCoreReaders implement
     
     try {
       if (si.info.getUseCompoundFile()) {
-        cfsDir = cfsReader = new CompoundFileDirectory(si.info.getId(), dir, IndexFileNames.segmentFileName(si.info.name,
"", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
+        cfsDir = cfsReader = codec.compoundFormat().getCompoundReader(dir, si.info, context);
       } else {
         cfsReader = null;
         cfsDir = dir;

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1628996&r1=1628995&r2=1628996&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
(original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
Thu Oct  2 15:27:13 2014
@@ -263,7 +263,7 @@ public final class SegmentInfos implemen
   }
 
   /** Since Lucene 5.0, every commit (segments_N) writes a unique id.  This will
-   *  return that id, or null if this commit was 5.0. */
+   *  return that id, or null if this commit was prior to 5.0. */
   public byte[] getId() {
     return id == null ? null : id.clone();
   }

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java?rev=1628996&r1=1628995&r2=1628996&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
(original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
Thu Oct  2 15:27:13 2014
@@ -33,7 +33,6 @@ import org.apache.lucene.codecs.NormsPro
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.TermVectorsReader;
 import org.apache.lucene.index.FieldInfo.DocValuesType;
-import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.Accountable;
@@ -202,10 +201,7 @@ public final class SegmentReader extends
     final boolean closeDir;
     if (info.getFieldInfosGen() == -1 && info.info.getUseCompoundFile()) {
       // no fieldInfos gen and segment uses a compound file
-      dir = new CompoundFileDirectory(info.info.getId(), info.info.dir,
-          IndexFileNames.segmentFileName(info.info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION),
-          IOContext.READONCE,
-          false);
+      dir = info.info.getCodec().compoundFormat().getCompoundReader(info.info.dir, info.info,
IOContext.READONCE);
       closeDir = true;
     } else {
       // gen'd FIS are read outside CFS, or the segment doesn't use a compound file

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/store/Directory.java?rev=1628996&r1=1628995&r2=1628996&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/store/Directory.java
(original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/store/Directory.java
Thu Oct  2 15:27:13 2014
@@ -100,12 +100,7 @@ public abstract class Directory implemen
    */
   public abstract void renameFile(String source, String dest) throws IOException;
   
-  /** Returns a stream reading an existing file, with the
-   * specified read buffer size.  The particular Directory
-   * implementation may ignore the buffer size.  Currently
-   * the only Directory implementations that respect this
-   * parameter are {@link FSDirectory} and {@link
-   * CompoundFileDirectory}.
+  /** Returns a stream reading an existing file.
    * <p>Throws {@link FileNotFoundException} or {@link NoSuchFileException}
    * if the file does not exist.
    */

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java?rev=1628996&r1=1628995&r2=1628996&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java
(original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java
Thu Oct  2 15:27:13 2014
@@ -24,7 +24,6 @@ import org.apache.lucene.codecs.CodecUti
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.NumericDocValuesField;
-import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.MockDirectoryWrapper;
@@ -76,12 +75,11 @@ public class TestAllFilesHaveChecksumFoo
     for (SegmentCommitInfo si : sis) {
       for (String file : si.files()) {
         checkFooter(dir, file);
-        if (file.endsWith(IndexFileNames.COMPOUND_FILE_EXTENSION)) {
-          // recurse into CFS
-          try (CompoundFileDirectory cfsDir = new CompoundFileDirectory(si.info.getId(),
dir, file, newIOContext(random()), false)) {
-            for (String cfsFile : cfsDir.listAll()) {
-              checkFooter(cfsDir, cfsFile);
-            }
+      }
+      if (si.info.getUseCompoundFile()) {
+        try (Directory cfsDir = si.info.getCodec().compoundFormat().getCompoundReader(dir,
si.info, newIOContext(random()))) {
+          for (String cfsFile : cfsDir.listAll()) {
+            checkFooter(cfsDir, cfsFile);
           }
         }
       }

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java?rev=1628996&r1=1628995&r2=1628996&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
(original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
Thu Oct  2 15:27:13 2014
@@ -28,7 +28,6 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.MockDirectoryWrapper;
@@ -89,12 +88,11 @@ public class TestAllFilesHaveCodecHeader
     for (SegmentCommitInfo si : sis) {
       for (String file : si.files()) {
         checkHeader(dir, file, namesToExtensions);
-        if (file.endsWith(IndexFileNames.COMPOUND_FILE_EXTENSION)) {
-          // recurse into CFS
-          try (CompoundFileDirectory cfsDir = new CompoundFileDirectory(si.info.getId(),
dir, file, newIOContext(random()), false)) {
-            for (String cfsFile : cfsDir.listAll()) {
-              checkHeader(cfsDir, cfsFile, namesToExtensions);
-            }
+      }
+      if (si.info.getUseCompoundFile()) {
+        try (Directory cfsDir = si.info.getCodec().compoundFormat().getCompoundReader(dir,
si.info, newIOContext(random()))) {
+          for (String cfsFile : cfsDir.listAll()) {
+            checkHeader(cfsDir, cfsFile, namesToExtensions);
           }
         }
       }

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

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

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java?rev=1628996&r1=1628995&r2=1628996&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java
(original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java
Thu Oct  2 15:27:13 2014
@@ -126,6 +126,8 @@ public class TestNRTCachingDirectory ext
 
   // LUCENE-5724
   public void testLargeCFS() throws IOException {
+    // nocommit: make a fake .si etc
+    /*
     Directory dir = new NRTCachingDirectory(newFSDirectory(createTempDir()), 2.0, 25.0);
     IOContext context = new IOContext(new FlushInfo(0, 512*1024*1024));
     IndexOutput out = dir.createOutput("big.bin", context);
@@ -139,5 +141,6 @@ public class TestNRTCachingDirectory ext
     dir.copy(cfsDir, "big.bin", "big.bin", context);
     cfsDir.close();
     dir.close();
+    */
   }
 }

Modified: lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java?rev=1628996&r1=1628995&r2=1628996&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
(original)
+++ lucene/dev/branches/lucene5969/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
Thu Oct  2 15:27:13 2014
@@ -587,6 +587,8 @@ public abstract class BaseDirectoryTestC
 
   // LUCENE-3382 test that delegate compound files correctly.
   public void testCompoundFileAppendTwice() throws IOException {
+    // nocommit: what is this testing? does it belong here?
+    /*
     Directory newDir = getDirectory(createTempDir("testCompoundFileAppendTwice"));
     byte id[] = StringHelper.randomId();
     CompoundFileDirectory csw = new CompoundFileDirectory(id, newDir, "d.cfs", newIOContext(random()),
true);
@@ -604,6 +606,7 @@ public abstract class BaseDirectoryTestC
     assertEquals("d.xyz", cfr.listAll()[0]);
     cfr.close();
     newDir.close();
+    */
   }
 
   /** Creates a file of the specified size with sequential data. The first



Mime
View raw message