You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2014/06/03 15:59:07 UTC

svn commit: r1599548 - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/store/ lucene/facet/src/test/org/apache/lucene/facet/ lucene/misc/src/java/org/apache/lucene/store/ lucene/test-framework/ lucene/t...

Author: uschindler
Date: Tue Jun  3 13:59:06 2014
New Revision: 1599548

URL: http://svn.apache.org/r1599548
Log:
Merged revision(s) 1596057 from lucene/dev/trunk:
LUCENE-5678: Use FileOutputStream instead of RandomAccessFile to write index data. BufferedIndexOutput and related APIs were removed.

Added:
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java
      - copied, changed from r1596057, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java
Removed:
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/BufferedIndexOutput.java
Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_4x/lucene/core/   (props changed)
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java
    lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/SlowRAMDirectory.java
    lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java
    lucene/dev/branches/branch_4x/lucene/test-framework/   (props changed)
    lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
    lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java
    lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java
    lucene/dev/branches/branch_4x/solr/   (props changed)
    lucene/dev/branches/branch_4x/solr/core/   (props changed)
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/blockcache/ReusedBufferedIndexOutput.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/hdfs/NullIndexOutput.java

Modified: lucene/dev/branches/branch_4x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/CHANGES.txt?rev=1599548&r1=1599547&r2=1599548&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/lucene/CHANGES.txt Tue Jun  3 13:59:06 2014
@@ -46,7 +46,8 @@ Changes in Backwards Compatibility Polic
 * LUCENE-4371: Removed IndexInputSlicer and Directory.createSlicer() and replaced
   with IndexInput.slice(). (Robert Muir)
 
-* LUCENE-5727: Remove IndexOutput.seek (Robert Muir)
+* LUCENE-5727, LUCENE-5678: Remove IndexOutput.seek, IndexOutput.setLength().
+  (Robert Muir, Uwe Schindler)
 
 API Changes
 
@@ -102,6 +103,12 @@ API Changes
 * LUCENE-5701: Core closed listeners are now available in the AtomicReader API,
   they used to sit only in SegmentReader. (Adrien Grand, Robert Muir)
 
+* LUCENE-5678: IndexOutput no longer allows seeking, so it is no longer required
+  to use RandomAccessFile to write Indexes. Lucene now uses standard FileOutputStream
+  wrapped with OutputStreamIndexOutput to write index data. BufferedIndexOutput was
+  removed, because buffering and checksumming is provided by FilterOutputStreams,
+  provided by the JDK.  (Uwe Schindler, Mike McCandless)
+
 Optimizations
 
 * LUCENE-5603: hunspell stemmer more efficiently strips prefixes

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java?rev=1599548&r1=1599547&r2=1599548&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java Tue Jun  3 13:59:06 2014
@@ -342,12 +342,6 @@ final class CompoundFileWriter implement
     }
 
     @Override
-    public long length() throws IOException {
-      assert !closed;
-      return delegate.length() - offset;
-    }
-
-    @Override
     public void writeByte(byte b) throws IOException {
       assert !closed;
       writtenBytes++;

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java?rev=1599548&r1=1599547&r2=1599548&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java Tue Jun  3 13:59:06 2014
@@ -22,9 +22,10 @@ import org.apache.lucene.util.IOUtils;
 
 import java.io.File;
 import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
 import java.io.FilenameFilter;
+import java.io.FilterOutputStream;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.Set;
@@ -279,7 +280,7 @@ public abstract class FSDirectory extend
     ensureOpen();
 
     ensureCanWrite(name);
-    return new FSIndexOutput(this, name);
+    return new FSIndexOutput(name);
   }
 
   protected void ensureCanWrite(String name) throws IOException {
@@ -292,8 +293,12 @@ public abstract class FSDirectory extend
       throw new IOException("Cannot overwrite: " + file);
   }
 
-  protected void onIndexOutputClosed(FSIndexOutput io) {
-    staleFiles.add(io.name);
+  /**
+   * Sub classes should call this method on closing an open {@link IndexOutput}, reporting the name of the file
+   * that was closed. {@code FSDirectory} needs this information to take care of syncing stale files.
+   */
+  protected void onIndexOutputClosed(String name) {
+    staleFiles.add(name);
   }
 
   @Override
@@ -372,70 +377,39 @@ public abstract class FSDirectory extend
     return chunkSize;
   }
 
-  /**
-   * Writes output with {@link RandomAccessFile#write(byte[], int, int)}
-   */
-  protected static class FSIndexOutput extends BufferedIndexOutput {
+  final class FSIndexOutput extends OutputStreamIndexOutput {
     /**
-     * The maximum chunk size is 8192 bytes, because {@link RandomAccessFile} mallocs
+     * The maximum chunk size is 8192 bytes, because {@link FileOutputStream} mallocs
      * a native buffer outside of stack if the write buffer size is larger.
      */
-    private static final int CHUNK_SIZE = 8192;
+    static final int CHUNK_SIZE = 8192;
     
-    private final FSDirectory parent;
     private final String name;
-    private final RandomAccessFile file;
-    private volatile boolean isOpen; // remember if the file is open, so that we don't try to close it more than once
-    
-    public FSIndexOutput(FSDirectory parent, String name) throws IOException {
-      super(CHUNK_SIZE);
-      this.parent = parent;
-      this.name = name;
-      file = new RandomAccessFile(new File(parent.directory, name), "rw");
-      isOpen = true;
-    }
 
-    @Override
-    protected void flushBuffer(byte[] b, int offset, int size) throws IOException {
-      assert isOpen;
-      while (size > 0) {
-        final int toWrite = Math.min(CHUNK_SIZE, size);
-        file.write(b, offset, toWrite);
-        offset += toWrite;
-        size -= toWrite;
-      }
-      assert size == 0;
+    public FSIndexOutput(String name) throws IOException {
+      super(new FilterOutputStream(new FileOutputStream(new File(directory, 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 {
+          while (length > 0) {
+            final int chunk = Math.min(length, CHUNK_SIZE);
+            out.write(b, offset, chunk);
+            length -= chunk;
+            offset += chunk;
+          }
+        }
+      }, CHUNK_SIZE);
+      this.name = name;
     }
     
     @Override
     public void close() throws IOException {
-      parent.onIndexOutputClosed(this);
-      // only close the file if it has not been closed yet
-      if (isOpen) {
-        boolean success = false;
-        try {
-          super.close();
-          success = true;
-        } finally {
-          isOpen = false;
-          if (success) {
-            IOUtils.close(file);
-          } else {
-            IOUtils.closeWhileHandlingException(file);
-          }
-        }
+      try {
+        onIndexOutputClosed(name);
+      } finally {
+        super.close();
       }
     }
-
-    @Override
-    public long length() throws IOException {
-      return file.length();
-    }
-
-    @Override
-    public void setLength(long length) throws IOException {
-      file.setLength(length);
-    }
   }
 
   protected void fsync(String name) throws IOException {

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java?rev=1599548&r1=1599547&r2=1599548&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java Tue Jun  3 13:59:06 2014
@@ -55,20 +55,8 @@ public abstract class IndexOutput extend
    * method will be removed in Lucene5.0.
    */
   @Deprecated
-  public abstract long length() throws IOException;
-
-  /** Set the file length. By default, this method does
-   * nothing (it's optional for a Directory to implement
-   * it).  But, certain Directory implementations (for
-   * example @see FSDirectory) can use this to inform the
-   * underlying IO system to pre-allocate the file to the
-   * specified size.  If the length is longer than the
-   * current file length, the bytes added to the file are
-   * undefined.  Otherwise the file is truncated.
-   * @param length file length
-   * @deprecated (4.1) This method will be removed in Lucene 5.0
-   */
-  @Deprecated
-  public void setLength(long length) throws IOException {}
+  public long length() throws IOException {
+    return getFilePointer();
+  }
 
 }

Copied: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java (from r1596057, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java?p2=lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java&p1=lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java&r1=1596057&r2=1599548&rev=1599548&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java Tue Jun  3 13:59:06 2014
@@ -73,4 +73,9 @@ public class OutputStreamIndexOutput ext
     os.flush();
     return crc.getValue();
   }
+
+  @Override
+  public void flush() throws IOException {
+    os.flush();
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java?rev=1599548&r1=1599547&r2=1599548&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java Tue Jun  3 13:59:06 2014
@@ -117,11 +117,6 @@ public class RAMOutputStream extends Ind
   }
 
   @Override
-  public long length() {
-    return file.length;
-  }
-
-  @Override
   public void writeByte(byte b) throws IOException {
     if (bufferPosition == bufferLength) {
       currentBufferIndex++;

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java?rev=1599548&r1=1599547&r2=1599548&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/RateLimitedIndexOutput.java Tue Jun  3 13:59:06 2014
@@ -1,4 +1,5 @@
 package org.apache.lucene.store;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -15,6 +16,7 @@ package org.apache.lucene.store;
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 import java.io.IOException;
 
 /**
@@ -22,62 +24,63 @@ import java.io.IOException;
  * 
  * @lucene.internal
  */
-final class RateLimitedIndexOutput extends BufferedIndexOutput {
+final class RateLimitedIndexOutput extends IndexOutput {
   
   private final IndexOutput delegate;
-  private final BufferedIndexOutput bufferedDelegate;
   private final RateLimiter rateLimiter;
 
   /** How many bytes we've written since we last called rateLimiter.pause. */
   private long bytesSinceLastPause;
+  
+  /** Cached here not not always have to call RateLimiter#getMinPauseCheckBytes()
+   * which does volatile read. */
+  private long currentMinPauseCheckBytes;
 
   RateLimitedIndexOutput(final RateLimiter rateLimiter, final IndexOutput delegate) {
-    // TODO should we make buffer size configurable
-    if (delegate instanceof BufferedIndexOutput) {
-      bufferedDelegate = (BufferedIndexOutput) delegate;
-      this.delegate = delegate;
-    } else {
-      this.delegate = delegate;
-      bufferedDelegate = null;
-    }
+    this.delegate = delegate;
     this.rateLimiter = rateLimiter;
+    this.currentMinPauseCheckBytes = rateLimiter.getMinPauseCheckBytes();
   }
   
   @Override
-  protected void flushBuffer(byte[] b, int offset, int len) throws IOException {
-    bytesSinceLastPause += len;
-    if (bytesSinceLastPause > rateLimiter.getMinPauseCheckBytes()) {
-      rateLimiter.pause(bytesSinceLastPause);
-      bytesSinceLastPause = 0;
-    }
-    if (bufferedDelegate != null) {
-      bufferedDelegate.flushBuffer(b, offset, len);
-    } else {
-      delegate.writeBytes(b, offset, len);
-    }
-    
+  public void close() throws IOException {
+    delegate.close();
   }
-  
+
   @Override
-  public long length() throws IOException {
-    return delegate.length();
+  public long getFilePointer() {
+    return delegate.getFilePointer();
   }
 
   @Override
-  public void flush() throws IOException {
-    try {
-      super.flush();
-    } finally { 
-      delegate.flush();
-    }
+  public long getChecksum() throws IOException {
+    return delegate.getChecksum();
   }
 
   @Override
-  public void close() throws IOException {
-    try {
-      super.close();
-    } finally {
-      delegate.close();
-    }
+  public void writeByte(byte b) throws IOException {
+    bytesSinceLastPause++;
+    checkRate();
+    delegate.writeByte(b);
+  }
+
+  @Override
+  public void writeBytes(byte[] b, int offset, int length) throws IOException {
+    bytesSinceLastPause += length;
+    checkRate();
+    delegate.writeBytes(b, offset, length);
+  }
+  
+  private void checkRate() {
+    if (bytesSinceLastPause > currentMinPauseCheckBytes) {
+      rateLimiter.pause(bytesSinceLastPause);
+      bytesSinceLastPause = 0;
+      currentMinPauseCheckBytes = rateLimiter.getMinPauseCheckBytes();
+    }    
+  }
+
+  @Override
+  public void flush() throws IOException {
+    delegate.flush();
   }
 }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/SlowRAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/SlowRAMDirectory.java?rev=1599548&r1=1599547&r2=1599548&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/SlowRAMDirectory.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/SlowRAMDirectory.java Tue Jun  3 13:59:06 2014
@@ -170,7 +170,6 @@ public class SlowRAMDirectory extends RA
     @Override public void close() throws IOException { io.close(); }
     @Override public void flush() throws IOException { io.flush(); }
     @Override public long getFilePointer() { return io.getFilePointer(); }
-    @Override public long length() throws IOException { return io.length(); }
     @Override public long getChecksum() throws IOException { return io.getChecksum(); }
   }
   

Modified: lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java?rev=1599548&r1=1599547&r2=1599548&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java (original)
+++ lucene/dev/branches/branch_4x/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java Tue Jun  3 13:59:06 2014
@@ -237,11 +237,6 @@ public class NativeUnixDirectory extends
     }
 
     @Override
-    public long length() {
-      return fileLength + bufferPos;
-    }
-    
-    @Override
     public long getChecksum() throws IOException {
       throw new UnsupportedOperationException("this directory currently does not work at all!");
     }

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1599548&r1=1599547&r2=1599548&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java Tue Jun  3 13:59:06 2014
@@ -304,7 +304,6 @@ public class MockDirectoryWrapper extend
         // Totally truncate the file to zero bytes
         deleteFile(name, true);
         IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
-        out.setLength(0);
         out.close();
       }
       if (LuceneTestCase.VERBOSE) {
@@ -525,10 +524,6 @@ public class MockDirectoryWrapper extend
     }
     //System.out.println(Thread.currentThread().getName() + ": MDW: create " + name);
     IndexOutput delegateOutput = in.createOutput(name, LuceneTestCase.newIOContext(randomState, context));
-    if (randomState.nextInt(10) == 0){
-      // once in a while wrap the IO in a Buffered IO with random buffer sizes
-      delegateOutput = new BufferedIndexOutputWrapper(1+randomState.nextInt(BufferedIndexOutput.DEFAULT_BUFFER_SIZE), delegateOutput);
-    } 
     final IndexOutput io = new MockIndexOutputWrapper(this, delegateOutput, name);
     addFileHandle(io, name, Handle.Output);
     openFilesForWrite.add(name);
@@ -954,43 +949,6 @@ public class MockDirectoryWrapper extend
     in.copy(to, src, dest, context);
   }
   
-  final class BufferedIndexOutputWrapper extends BufferedIndexOutput {
-    private final IndexOutput io;
-    
-    public BufferedIndexOutputWrapper(int bufferSize, IndexOutput io) {
-      super(bufferSize);
-      this.io = io;
-    }
-    
-    @Override
-    public long length() throws IOException {
-      return io.length();
-    }
-    
-    @Override
-    protected void flushBuffer(byte[] b, int offset, int len) throws IOException {
-      io.writeBytes(b, offset, len);
-    }
-
-    @Override
-    public void flush() throws IOException {
-      try {
-        super.flush();
-      } finally { 
-        io.flush();
-      }
-    }
-    
-    @Override
-    public void close() throws IOException {
-      try {
-        super.close();
-      } finally {
-        io.close();
-      }
-    }
-  }
-
   /** Use this when throwing fake {@code IOException},
    *  e.g. from {@link MockDirectoryWrapper.Failure}. */
   public static class FakeIOException extends IOException {

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java?rev=1599548&r1=1599547&r2=1599548&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java Tue Jun  3 13:59:06 2014
@@ -74,7 +74,7 @@ public class MockIndexOutputWrapper exte
       if (realUsage > dir.maxUsedSize) {
         dir.maxUsedSize = realUsage;
       }
-      String message = "fake disk full at " + dir.getRecomputedActualSizeInBytes() + " bytes when writing " + name + " (file length=" + delegate.length();
+      String message = "fake disk full at " + dir.getRecomputedActualSizeInBytes() + " bytes when writing " + name + " (file length=" + delegate.getFilePointer();
       if (freeSpace > 0) {
         message += "; wrote " + freeSpace + " of " + len + " bytes";
       }
@@ -147,16 +147,6 @@ public class MockIndexOutputWrapper exte
   }
 
   @Override
-  public long length() throws IOException {
-    return delegate.length();
-  }
-
-  @Override
-  public void setLength(long length) throws IOException {
-    delegate.setLength(length);
-  }
-
-  @Override
   public void copyBytes(DataInput input, long numBytes) throws IOException {
     checkCrashed();
     checkDiskFull(null, 0, input, numBytes);

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java?rev=1599548&r1=1599547&r2=1599548&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java Tue Jun  3 13:59:06 2014
@@ -89,11 +89,6 @@ public class ThrottledIndexOutput extend
   }
 
   @Override
-  public long length() throws IOException {
-    return delegate.length();
-  }
-
-  @Override
   public void writeByte(byte b) throws IOException {
     bytes[0] = b;
     writeBytes(bytes, 0, 1);
@@ -137,11 +132,6 @@ public class ThrottledIndexOutput extend
   }
   
   @Override
-  public void setLength(long length) throws IOException {
-    delegate.setLength(length);
-  }
-
-  @Override
   public void copyBytes(DataInput input, long numBytes) throws IOException {
     delegate.copyBytes(input, numBytes);
   }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/blockcache/ReusedBufferedIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/blockcache/ReusedBufferedIndexOutput.java?rev=1599548&r1=1599547&r2=1599548&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/blockcache/ReusedBufferedIndexOutput.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/blockcache/ReusedBufferedIndexOutput.java Tue Jun  3 13:59:06 2014
@@ -98,11 +98,6 @@ public abstract class ReusedBufferedInde
   protected abstract void seekInternal(long pos) throws IOException;
   
   @Override
-  public long length() throws IOException {
-    return fileLength;
-  }
-  
-  @Override
   public void writeByte(byte b) throws IOException {
     if (bufferPosition >= bufferSize) {
       flushBufferToCache();

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java?rev=1599548&r1=1599547&r2=1599548&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java Tue Jun  3 13:59:06 2014
@@ -30,12 +30,10 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.lucene.store.BaseDirectory;
-import org.apache.lucene.store.BufferedIndexOutput;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.NoLockFactory;
-import org.apache.lucene.util.IOUtils;
 import org.apache.solr.store.blockcache.CustomBufferedIndexInput;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -99,14 +97,11 @@ public class HdfsDirectory extends BaseD
   }
   
   @Override
-  public IndexOutput createOutput(String name, IOContext context)
-      throws IOException {
+  public IndexOutput createOutput(String name, IOContext context) throws IOException {
     if (SEGMENTS_GEN.equals(name)) {
       return new NullIndexOutput();
     }
-    HdfsFileWriter writer = new HdfsFileWriter(getFileSystem(), new Path(
-        hdfsDirPath, name));
-    return new HdfsIndexOutput(writer);
+    return new HdfsFileWriter(getFileSystem(), new Path(hdfsDirPath, name));
   }
   
   private String[] getNormalNames(List<String> files) {
@@ -238,41 +233,6 @@ public class HdfsDirectory extends BaseD
     }
   }
   
-  static class HdfsIndexOutput extends BufferedIndexOutput {
-    
-    private HdfsFileWriter writer;
-    
-    public HdfsIndexOutput(HdfsFileWriter writer) {
-      this.writer = writer;
-    }
-    
-    @Override
-    public void close() throws IOException {
-      boolean success = false;
-      try {
-        super.close();
-        success = true;
-      } finally {
-        if (success) {
-          IOUtils.close(writer);
-        } else {
-          IOUtils.closeWhileHandlingException(writer);
-        }
-      }
-    }
-
-    @Override
-    protected void flushBuffer(byte[] b, int offset, int len)
-        throws IOException {
-      writer.writeBytes(b, offset, len);
-    }
-
-    @Override
-    public long length() throws IOException {
-      return writer.length();
-    }
-  }
-  
   @Override
   public void sync(Collection<String> names) throws IOException {
     LOG.debug("Sync called on {}", Arrays.toString(names.toArray()));

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java?rev=1599548&r1=1599547&r2=1599548&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java Tue Jun  3 13:59:06 2014
@@ -17,37 +17,31 @@ package org.apache.solr.store.hdfs;
  * limitations under the License.
  */
 
-import java.io.Closeable;
 import java.io.IOException;
+import java.io.OutputStream;
 import java.util.EnumSet;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.lucene.store.DataOutput;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.lucene.store.OutputStreamIndexOutput;
 
 /**
  * @lucene.experimental
  */
-public class HdfsFileWriter extends DataOutput implements Closeable {
-  public static Logger LOG = LoggerFactory.getLogger(HdfsFileWriter.class);
+public class HdfsFileWriter extends OutputStreamIndexOutput {
   
   public static final String HDFS_SYNC_BLOCK = "solr.hdfs.sync.block";
-  
-  private final Path path;
-  private FSDataOutputStream outputStream;
-  private long currentPosition;
+  public static final int BUFFER_SIZE = 16384;
   
   public HdfsFileWriter(FileSystem fileSystem, Path path) throws IOException {
-    LOG.debug("Creating writer on {}", path);
-    this.path = path;
-    
+    super(getOutputStream(fileSystem, path), BUFFER_SIZE);
+  }
+  
+  private static final OutputStream getOutputStream(FileSystem fileSystem, Path path) throws IOException {
     Configuration conf = fileSystem.getConf();
     FsServerDefaults fsDefaults = fileSystem.getServerDefaults(path);
     EnumSet<CreateFlag> flags = EnumSet.of(CreateFlag.CREATE,
@@ -55,45 +49,9 @@ public class HdfsFileWriter extends Data
     if (Boolean.getBoolean(HDFS_SYNC_BLOCK)) {
       flags.add(CreateFlag.SYNC_BLOCK);
     }
-    outputStream = fileSystem.create(path, FsPermission.getDefault()
+    return fileSystem.create(path, FsPermission.getDefault()
         .applyUMask(FsPermission.getUMask(conf)), flags, fsDefaults
         .getFileBufferSize(), fsDefaults.getReplication(), fsDefaults
         .getBlockSize(), null);
   }
-  
-  public long length() {
-    return currentPosition;
-  }
-  
-  public void seek(long pos) throws IOException {
-    LOG.error("Invalid seek called on {}", path);
-    throw new IOException("Seek not supported");
-  }
-  
-  public void flush() throws IOException {
-    // flush to the network, not guarantees it makes it to the DN (vs hflush)
-    outputStream.flush();
-    LOG.debug("Flushed file {}", path);
-  }
-  
-  public void close() throws IOException {
-    outputStream.close();
-    LOG.debug("Closed writer on {}", path);
-  }
-  
-  @Override
-  public void writeByte(byte b) throws IOException {
-    outputStream.write(b & 0xFF);
-    currentPosition++;
-  }
-  
-  @Override
-  public void writeBytes(byte[] b, int offset, int length) throws IOException {
-    outputStream.write(b, offset, length);
-    currentPosition += length;
-  }
-  
-  public long getPosition() {
-    return currentPosition;
-  }
 }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/hdfs/NullIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/hdfs/NullIndexOutput.java?rev=1599548&r1=1599547&r2=1599548&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/hdfs/NullIndexOutput.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/store/hdfs/NullIndexOutput.java Tue Jun  3 13:59:06 2014
@@ -45,11 +45,6 @@ public class NullIndexOutput extends Ind
   }
   
   @Override
-  public long length() throws IOException {
-    return length;
-  }
-  
-  @Override
   public void writeByte(byte b) throws IOException {
     pos++;
     updateLength();