You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/12/01 22:24:21 UTC

svn commit: r1642762 - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/codecs/lucene50/ lucene/core/src/java/org/apache/lucene/index/ lucene/core/src/java/org/apache/lucene/store/ lucene/facet/ lucene/f...

Author: mikemccand
Date: Mon Dec  1 21:24:20 2014
New Revision: 1642762

URL: http://svn.apache.org/r1642762
Log:
LUCENE-6084: add reasonable IndexOutput.toString

Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/lucene/   (props changed)
    lucene/dev/branches/branch_5x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/lucene/core/   (props changed)
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsWriter.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CorruptIndexException.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java
    lucene/dev/branches/branch_5x/lucene/facet/   (props changed)
    lucene/dev/branches/branch_5x/lucene/facet/src/test/org/apache/lucene/facet/SlowRAMDirectory.java
    lucene/dev/branches/branch_5x/lucene/misc/   (props changed)
    lucene/dev/branches/branch_5x/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java
    lucene/dev/branches/branch_5x/lucene/test-framework/   (props changed)
    lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
    lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java
    lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java
    lucene/dev/branches/branch_5x/solr/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/store/blockcache/CachedIndexOutput.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/store/blockcache/ReusedBufferedIndexOutput.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java

Modified: lucene/dev/branches/branch_5x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/CHANGES.txt?rev=1642762&r1=1642761&r2=1642762&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/CHANGES.txt Mon Dec  1 21:24:20 2014
@@ -204,6 +204,10 @@ API Changes
 
 * LUCENE-6082: Remove abort() from codec apis. (Robert Muir)
 
+* LUCENE-6084: IndexOutput's constructor now requires a String
+  resourceDescription so its toString is sane (Robert Muir, Mike
+  McCandless)
+
 Bug Fixes
 
 * LUCENE-5650: Enforce read-only access to any path outside the temporary

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsWriter.java?rev=1642762&r1=1642761&r2=1642762&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsWriter.java Mon Dec  1 21:24:20 2014
@@ -221,7 +221,7 @@ public final class Lucene50PostingsWrite
     final int docDelta = docID - lastDocID;
 
     if (docID < 0 || (docCount > 0 && docDelta <= 0)) {
-      throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " )", docOut.toString());
+      throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " )", docOut);
     }
 
     docDeltaBuffer[docBufferUpto] = docDelta;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CorruptIndexException.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CorruptIndexException.java?rev=1642762&r1=1642761&r2=1642762&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CorruptIndexException.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CorruptIndexException.java Mon Dec  1 21:24:20 2014
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.Objects;
 
 import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
 
 /**
  * This exception is thrown when Lucene detects
@@ -31,11 +32,21 @@ public class CorruptIndexException exten
   public CorruptIndexException(String message, DataInput input) {
     this(message, input, null);
   }
+
+  /** Create exception with a message only */
+  public CorruptIndexException(String message, DataOutput output) {
+    this(message, output, null);
+  }
   
   /** Create exception with message and root cause. */
   public CorruptIndexException(String message, DataInput input, Throwable cause) {
     this(message, Objects.toString(input), cause);
   }
+
+  /** Create exception with message and root cause. */
+  public CorruptIndexException(String message, DataOutput output, Throwable cause) {
+    this(message, Objects.toString(output), cause);
+  }
   
   /** Create exception with a message only */
   public CorruptIndexException(String message, String resourceDescription) {

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java?rev=1642762&r1=1642761&r2=1642762&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java Mon Dec  1 21:24:20 2014
@@ -282,7 +282,7 @@ public abstract class FSDirectory extend
     private final String name;
 
     public FSIndexOutput(String name) throws IOException {
-      super(new FilterOutputStream(Files.newOutputStream(directory.resolve(name))) {
+      super("FSIndexOutput(path=\"" + directory.resolve(name) + "\")", new FilterOutputStream(Files.newOutputStream(directory.resolve(name))) {
         // This implementation ensures, that we never write more than CHUNK_SIZE bytes:
         @Override
         public void write(byte[] b, int offset, int length) throws IOException {

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java?rev=1642762&r1=1642761&r2=1642762&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java Mon Dec  1 21:24:20 2014
@@ -31,6 +31,17 @@ import java.io.IOException;
  */
 public abstract class IndexOutput extends DataOutput implements Closeable {
 
+  private final String resourceDescription;
+
+  /** Sole constructor.  resourceDescription should be non-null, opaque string
+   *  describing this resource; it's returned from {@link #toString}. */
+  protected IndexOutput(String resourceDescription) {
+    if (resourceDescription == null) {
+      throw new IllegalArgumentException("resourceDescription must not be null");
+    }
+    this.resourceDescription = resourceDescription;
+  }
+
   /** Closes this stream to further operations. */
   @Override
   public abstract void close() throws IOException;
@@ -42,4 +53,9 @@ public abstract class IndexOutput extend
 
   /** Returns the current checksum of bytes written so far */
   public abstract long getChecksum() throws IOException;
+
+  @Override
+  public String toString() {
+    return resourceDescription;
+  }
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java?rev=1642762&r1=1642761&r2=1642762&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java Mon Dec  1 21:24:20 2014
@@ -36,7 +36,8 @@ public class OutputStreamIndexOutput ext
    * @param bufferSize the buffer size in bytes used to buffer writes internally.
    * @throws IllegalArgumentException if the given buffer size is less or equal to <tt>0</tt>
    */
-  public OutputStreamIndexOutput(OutputStream out, int bufferSize) {
+  public OutputStreamIndexOutput(String resourceDescription, OutputStream out, int bufferSize) {
+    super(resourceDescription);
     this.os = new BufferedOutputStream(new CheckedOutputStream(out, crc), bufferSize);
   }
 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java?rev=1642762&r1=1642761&r2=1642762&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java Mon Dec  1 21:24:20 2014
@@ -173,7 +173,7 @@ public class RAMDirectory extends BaseDi
       existing.directory = null;
     }
     fileMap.put(name, file);
-    return new RAMOutputStream(file, true);
+    return new RAMOutputStream(name, file, true);
   }
 
   /**

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java?rev=1642762&r1=1642761&r2=1642762&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java Mon Dec  1 21:24:20 2014
@@ -46,10 +46,17 @@ public class RAMOutputStream extends Ind
 
   /** Construct an empty output buffer. */
   public RAMOutputStream() {
-    this(new RAMFile(), false);
+    this("noname", new RAMFile(), false);
   }
 
+  /** Creates this, with no name. */
   public RAMOutputStream(RAMFile f, boolean checksum) {
+    this("noname", f, checksum);
+  }
+
+  /** Creates this, with specified name. */
+  public RAMOutputStream(String name, RAMFile f, boolean checksum) {
+    super("RAMOutputStream(name=\"" + name + "\")");
     file = f;
 
     // make sure that we switch to the

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java?rev=1642762&r1=1642761&r2=1642762&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java Mon Dec  1 21:24:20 2014
@@ -37,6 +37,7 @@ final class RateLimitedIndexOutput exten
   private long currentMinPauseCheckBytes;
 
   RateLimitedIndexOutput(final RateLimiter rateLimiter, final IndexOutput delegate) {
+    super("RateLimitedIndexOutput(" + delegate + ")");
     this.delegate = delegate;
     this.rateLimiter = rateLimiter;
     this.currentMinPauseCheckBytes = rateLimiter.getMinPauseCheckBytes();

Modified: lucene/dev/branches/branch_5x/lucene/facet/src/test/org/apache/lucene/facet/SlowRAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/facet/src/test/org/apache/lucene/facet/SlowRAMDirectory.java?rev=1642762&r1=1642761&r2=1642762&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/facet/src/test/org/apache/lucene/facet/SlowRAMDirectory.java (original)
+++ lucene/dev/branches/branch_5x/lucene/facet/src/test/org/apache/lucene/facet/SlowRAMDirectory.java Mon Dec  1 21:24:20 2014
@@ -143,6 +143,7 @@ public class SlowRAMDirectory extends RA
     private final Random rand;
     
     public SlowIndexOutput(IndexOutput io) {
+      super("SlowIndexOutput(" + io + ")");
       this.io = io;
       this.rand = forkRandom();
     }

Modified: lucene/dev/branches/branch_5x/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java?rev=1642762&r1=1642761&r2=1642762&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java (original)
+++ lucene/dev/branches/branch_5x/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java Mon Dec  1 21:24:20 2014
@@ -166,6 +166,7 @@ public class NativeUnixDirectory extends
     private boolean isOpen;
 
     public NativeUnixIndexOutput(Path path, int bufferSize) throws IOException {
+      super("NativeUnixIndexOutput(path=\"" + path.toString() + "\")");
       //this.path = path;
       final FileDescriptor fd = NativePosixUtil.open_direct(path.toString(), false);
       fos = new FileOutputStream(fd);

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java?rev=1642762&r1=1642761&r2=1642762&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java Mon Dec  1 21:24:20 2014
@@ -1031,5 +1031,14 @@ public abstract class BaseDirectoryTestC
     }
     dir.close();
   }
+
+  // LUCENE-6084
+  public void testIndexOutputToString() throws Throwable {
+    Directory dir = getDirectory(createTempDir());
+    IndexOutput out = dir.createOutput("camelCase.txt", newIOContext(random()));
+    assertTrue(out.toString(), out.toString().contains("camelCase.txt"));
+    out.close();
+    dir.close();
+  }
 }
 

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java?rev=1642762&r1=1642761&r2=1642762&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java Mon Dec  1 21:24:20 2014
@@ -38,6 +38,7 @@ public class MockIndexOutputWrapper exte
 
   /** Construct an empty output buffer. */
   public MockIndexOutputWrapper(MockDirectoryWrapper dir, IndexOutput delegate, String name) {
+    super("MockIndexOutputWrapper(" + delegate + ")");
     this.dir = dir;
     this.name = name;
     this.delegate = delegate;

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java?rev=1642762&r1=1642761&r2=1642762&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java Mon Dec  1 21:24:20 2014
@@ -59,6 +59,7 @@ public class ThrottledIndexOutput extend
   public ThrottledIndexOutput(int bytesPerSecond, long flushDelayMillis,
       long closeDelayMillis, long seekDelayMillis, long minBytesWritten,
       IndexOutput delegate) {
+    super("ThrottledIndexOutput(" + delegate + ")");
     assert bytesPerSecond > 0;
     this.delegate = delegate;
     this.bytesPerSecond = bytesPerSecond;

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/store/blockcache/CachedIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/store/blockcache/CachedIndexOutput.java?rev=1642762&r1=1642761&r2=1642762&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/store/blockcache/CachedIndexOutput.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/store/blockcache/CachedIndexOutput.java Mon Dec  1 21:24:20 2014
@@ -37,7 +37,7 @@ public class CachedIndexOutput extends R
   
   public CachedIndexOutput(BlockDirectory directory, IndexOutput dest,
       int blockSize, String name, Cache cache, int bufferSize) {
-    super(bufferSize);
+    super("dest=" + dest + " name=" + name, bufferSize);
     this.directory = directory;
     this.dest = dest;
     this.blockSize = blockSize;

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/store/blockcache/ReusedBufferedIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/store/blockcache/ReusedBufferedIndexOutput.java?rev=1642762&r1=1642761&r2=1642762&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/store/blockcache/ReusedBufferedIndexOutput.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/store/blockcache/ReusedBufferedIndexOutput.java Mon Dec  1 21:24:20 2014
@@ -43,11 +43,12 @@ public abstract class ReusedBufferedInde
   
   private final Store store;
   
-  public ReusedBufferedIndexOutput() {
-    this(BUFFER_SIZE);
+  public ReusedBufferedIndexOutput(String resourceDescription) {
+    this(resourceDescription, BUFFER_SIZE);
   }
   
-  public ReusedBufferedIndexOutput(int bufferSize) {
+  public ReusedBufferedIndexOutput(String resourceDescription, int bufferSize) {
+    super(resourceDescription);
     checkBufferSize(bufferSize);
     this.bufferSize = bufferSize;
     store = BufferStore.instance(bufferSize);

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java?rev=1642762&r1=1642761&r2=1642762&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java Mon Dec  1 21:24:20 2014
@@ -38,7 +38,7 @@ public class HdfsFileWriter extends Outp
   public static final int BUFFER_SIZE = 16384;
   
   public HdfsFileWriter(FileSystem fileSystem, Path path) throws IOException {
-    super(getOutputStream(fileSystem, path), BUFFER_SIZE);
+    super("fileSystem=" + fileSystem + " path=" + path, getOutputStream(fileSystem, path), BUFFER_SIZE);
   }
   
   private static final OutputStream getOutputStream(FileSystem fileSystem, Path path) throws IOException {