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

svn commit: r1522661 - in /lucene/dev/trunk: lucene/ lucene/core/src/java/org/apache/lucene/store/ lucene/core/src/test/org/apache/lucene/index/ lucene/core/src/test/org/apache/lucene/store/ lucene/test-framework/src/java/org/apache/lucene/store/ solr/...

Author: jpountz
Date: Thu Sep 12 16:59:34 2013
New Revision: 1522661

URL: http://svn.apache.org/r1522661
Log:
LUCENE-5204: Make Directory easier to wrap.

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/BaseDirectory.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/Directory.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RateLimitedDirectoryWrapper.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/TrackingDirectoryWrapper.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1522661&r1=1522660&r2=1522661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Thu Sep 12 16:59:34 2013
@@ -53,6 +53,13 @@ New Features
   for advanced use cases where String is too restrictive (Luca
   Cavanna, Robert Muir, Mike McCandless)
 
+Changes in backwards compatibility policy
+
+* LUCENE-5204: Directory doesn't have default implementations for
+  LockFactory-related methods, which have been moved to BaseDirectory. If you
+  had a custom Directory implementation that extended Directory, you need to
+  extend BaseDirectory instead. (Adrien Grand)
+
 ======================= Lucene 4.5.0 =======================
 
 New features
@@ -310,7 +317,7 @@ Changes in backwards compatibility polic
 * LUCENE-5101: CachingWrapperFilter doesn't always return FixedBitSet instances
   anymore. Users of the join module can use
   oal.search.join.FixedBitSetCachingWrapperFilter instead. (Adrien Grand)
-  
+
 Build
 
 * SOLR-5159: Manifest includes non-parsed maven variables. 

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/BaseDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/BaseDirectory.java?rev=1522661&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/BaseDirectory.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/BaseDirectory.java Thu Sep 12 16:59:34 2013
@@ -0,0 +1,69 @@
+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
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+/**
+ * Base implementation for a concrete {@link Directory}.
+ * @lucene.experimental
+ */
+public abstract class BaseDirectory extends Directory {
+
+  volatile protected boolean isOpen = true;
+
+  /** Holds the LockFactory instance (implements locking for
+   * this Directory instance). */
+  protected LockFactory lockFactory;
+
+  /** Sole constructor. */
+  protected BaseDirectory() {
+    super();
+  }
+
+  @Override
+  public Lock makeLock(String name) {
+      return lockFactory.makeLock(name);
+  }
+
+  @Override
+  public void clearLock(String name) throws IOException {
+    if (lockFactory != null) {
+      lockFactory.clearLock(name);
+    }
+  }
+
+  @Override
+  public void setLockFactory(LockFactory lockFactory) throws IOException {
+    assert lockFactory != null;
+    this.lockFactory = lockFactory;
+    lockFactory.setLockPrefix(this.getLockID());
+  }
+
+  @Override
+  public LockFactory getLockFactory() {
+    return this.lockFactory;
+  }
+
+  @Override
+  protected final void ensureOpen() throws AlreadyClosedException {
+    if (!isOpen)
+      throw new AlreadyClosedException("this Directory is closed");
+  }
+
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java?rev=1522661&r1=1522660&r2=1522661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java Thu Sep 12 16:59:34 2013
@@ -71,7 +71,7 @@ import java.io.IOException;
  * 
  * @lucene.experimental
  */
-public final class CompoundFileDirectory extends Directory {
+public final class CompoundFileDirectory extends BaseDirectory {
   
   /** Offset/Length for a slice inside of a compound file */
   public static final class FileEntry {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/Directory.java?rev=1522661&r1=1522660&r2=1522661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/Directory.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/Directory.java Thu Sep 12 16:59:34 2013
@@ -44,12 +44,6 @@ import org.apache.lucene.util.IOUtils;
  */
 public abstract class Directory implements Closeable {
 
-  volatile protected boolean isOpen = true;
-
-  /** Holds the LockFactory instance (implements locking for
-   * this Directory instance). */
-  protected LockFactory lockFactory;
-
   /**
    * Returns an array of strings, one for each file in the directory.
    * 
@@ -115,20 +109,15 @@ public abstract class Directory implemen
   /** Construct a {@link Lock}.
    * @param name the name of the lock file
    */
-  public Lock makeLock(String name) {
-      return lockFactory.makeLock(name);
-  }
+  public abstract Lock makeLock(String name);
+
   /**
    * Attempt to clear (forcefully unlock and remove) the
    * specified lock.  Only call this at a time when you are
    * certain this lock is no longer in use.
    * @param name name of the lock to be cleared.
    */
-  public void clearLock(String name) throws IOException {
-    if (lockFactory != null) {
-      lockFactory.clearLock(name);
-    }
-  }
+  public abstract void clearLock(String name) throws IOException;
 
   /** Closes the store. */
   @Override
@@ -144,11 +133,7 @@ public abstract class Directory implemen
    *
    * @param lockFactory instance of {@link LockFactory}.
    */
-  public void setLockFactory(LockFactory lockFactory) throws IOException {
-    assert lockFactory != null;
-    this.lockFactory = lockFactory;
-    lockFactory.setLockPrefix(this.getLockID());
-  }
+  public abstract void setLockFactory(LockFactory lockFactory) throws IOException;
 
   /**
    * Get the LockFactory that this Directory instance is
@@ -156,9 +141,7 @@ public abstract class Directory implemen
    * may be null for Directory implementations that provide
    * their own locking implementation.
    */
-  public LockFactory getLockFactory() {
-    return this.lockFactory;
-  }
+  public abstract LockFactory getLockFactory();
 
   /**
    * Return a string identifier that uniquely differentiates
@@ -252,10 +235,7 @@ public abstract class Directory implemen
   /**
    * @throws AlreadyClosedException if this Directory is closed
    */
-  protected final void ensureOpen() throws AlreadyClosedException {
-    if (!isOpen)
-      throw new AlreadyClosedException("this Directory is closed");
-  }
+  protected void ensureOpen() throws AlreadyClosedException {}
   
   /**
    * Allows to create one or more sliced {@link IndexInput} instances from a single 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java?rev=1522661&r1=1522660&r2=1522661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java Thu Sep 12 16:59:34 2013
@@ -110,7 +110,7 @@ import org.apache.lucene.util.IOUtils;
  *
  * @see Directory
  */
-public abstract class FSDirectory extends Directory {
+public abstract class FSDirectory extends BaseDirectory {
 
   protected final File directory; // The underlying filesystem directory
   protected final Set<String> staleFiles = synchronizedSet(new HashSet<String>()); // Files written, but not yet sync'ed

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java?rev=1522661&r1=1522660&r2=1522661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java Thu Sep 12 16:59:34 2013
@@ -39,7 +39,7 @@ import java.util.HashSet;
  * @lucene.experimental
  */
 
-public class FileSwitchDirectory extends Directory {
+public class FileSwitchDirectory extends BaseDirectory {
   private final Directory secondaryDir;
   private final Directory primaryDir;
   private final Set<String> primaryExtensions;

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java?rev=1522661&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java Thu Sep 12 16:59:34 2013
@@ -0,0 +1,113 @@
+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
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Collection;
+
+/** Directory implementation that delegates calls to another directory.
+ *  This class can be used to add limitations on top of an existing
+ *  {@link Directory} implementation such as
+ *  {@link RateLimitedDirectoryWrapper rate limiting} or to add additional
+ *  sanity checks for tests. However, if you plan to write your own
+ *  {@link Directory} implementation, you should consider extending directly
+ *  {@link Directory} or {@link BaseDirectory} rather than try to reuse
+ *  functionality of existing {@link Directory}s by extending this class.
+ *  @lucene.internal */
+public class FilterDirectory extends Directory {
+
+  protected final Directory in;
+
+  /** Sole constructor, typically called from sub-classes. */
+  protected FilterDirectory(Directory in) {
+    this.in = in;
+  }
+
+  /** Return the wrapped {@link Directory}. */
+  public final Directory getDelegate() {
+    return in;
+  }
+
+  @Override
+  public String[] listAll() throws IOException {
+    return in.listAll();
+  }
+
+  @Override
+  public boolean fileExists(String name) throws IOException {
+    return in.fileExists(name);
+  }
+
+  @Override
+  public void deleteFile(String name) throws IOException {
+    in.deleteFile(name);
+  }
+
+  @Override
+  public long fileLength(String name) throws IOException {
+    return in.fileLength(name);
+  }
+
+  @Override
+  public IndexOutput createOutput(String name, IOContext context)
+      throws IOException {
+    return in.createOutput(name, context);
+  }
+
+  @Override
+  public void sync(Collection<String> names) throws IOException {
+    in.sync(names);
+  }
+
+  @Override
+  public IndexInput openInput(String name, IOContext context)
+      throws IOException {
+    return in.openInput(name, context);
+  }
+
+  @Override
+  public Lock makeLock(String name) {
+    return in.makeLock(name);
+  }
+
+  @Override
+  public void clearLock(String name) throws IOException {
+    in.clearLock(name);
+  }
+
+  @Override
+  public void close() throws IOException {
+    in.close();
+  }
+
+  @Override
+  public void setLockFactory(LockFactory lockFactory) throws IOException {
+    in.setLockFactory(lockFactory);
+  }
+
+  @Override
+  public LockFactory getLockFactory() {
+    return in.getLockFactory();
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(" + in.toString() + ")";
+  }
+
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java?rev=1522661&r1=1522660&r2=1522661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java Thu Sep 12 16:59:34 2013
@@ -45,7 +45,7 @@ import java.util.concurrent.atomic.Atomi
  * implementation working directly on the file system cache of the
  * operating system, so copying data to Java heap space is not useful.
  */
-public class RAMDirectory extends Directory {
+public class RAMDirectory extends BaseDirectory {
   protected final Map<String,RAMFile> fileMap = new ConcurrentHashMap<String,RAMFile>();
   protected final AtomicLong sizeInBytes = new AtomicLong();
   

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RateLimitedDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RateLimitedDirectoryWrapper.java?rev=1522661&r1=1522660&r2=1522661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RateLimitedDirectoryWrapper.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/RateLimitedDirectoryWrapper.java Thu Sep 12 16:59:34 2013
@@ -16,7 +16,6 @@ package org.apache.lucene.store;
  * limitations under the License.
  */
 import java.io.IOException;
-import java.util.Collection;
 
 import org.apache.lucene.store.IOContext.Context;
 
@@ -28,123 +27,40 @@ import org.apache.lucene.store.IOContext
  *  @see #setRateLimiter(RateLimiter, IOContext.Context)
  * @lucene.experimental
  */
-public final class RateLimitedDirectoryWrapper extends Directory {
-  
-  private final Directory delegate;
+public final class RateLimitedDirectoryWrapper extends FilterDirectory {
+
   // we need to be volatile here to make sure we see all the values that are set
   // / modified concurrently
   private volatile RateLimiter[] contextRateLimiters = new RateLimiter[IOContext.Context
       .values().length];
   
   public RateLimitedDirectoryWrapper(Directory wrapped) {
-    this.delegate = wrapped;
-  }
-  
-  public Directory getDelegate() {
-    return delegate;
-  }
-  
-  @Override
-  public String[] listAll() throws IOException {
-    ensureOpen();
-    return delegate.listAll();
-  }
-  
-  @Override
-  public boolean fileExists(String name) throws IOException {
-    ensureOpen();
-    return delegate.fileExists(name);
-  }
-  
-  @Override
-  public void deleteFile(String name) throws IOException {
-    ensureOpen();
-    delegate.deleteFile(name);
-  }
-  
-  @Override
-  public long fileLength(String name) throws IOException {
-    ensureOpen();
-    return delegate.fileLength(name);
+    super(wrapped);
   }
   
   @Override
   public IndexOutput createOutput(String name, IOContext context)
       throws IOException {
     ensureOpen();
-    final IndexOutput output = delegate.createOutput(name, context);
+    final IndexOutput output = super.createOutput(name, context);
     final RateLimiter limiter = getRateLimiter(context.context);
     if (limiter != null) {
       return new RateLimitedIndexOutput(limiter, output);
     }
     return output;
   }
-  
-  @Override
-  public void sync(Collection<String> names) throws IOException {
-    ensureOpen();
-    delegate.sync(names);
-  }
-  
-  @Override
-  public IndexInput openInput(String name, IOContext context)
-      throws IOException {
-    ensureOpen();
-    return delegate.openInput(name, context);
-  }
-  
-  @Override
-  public void close() throws IOException {
-    isOpen = false;
-    delegate.close();
-  }
-  
+
   @Override
   public IndexInputSlicer createSlicer(String name, IOContext context)
       throws IOException {
     ensureOpen();
-    return delegate.createSlicer(name, context);
-  }
-  
-  @Override
-  public Lock makeLock(String name) {
-    ensureOpen();
-    return delegate.makeLock(name);
-  }
-
-  @Override
-  public void clearLock(String name) throws IOException {
-    ensureOpen();
-    delegate.clearLock(name);
-  }
-
-  @Override
-  public void setLockFactory(LockFactory lockFactory) throws IOException {
-    ensureOpen();
-    delegate.setLockFactory(lockFactory);
-  }
-
-  @Override
-  public LockFactory getLockFactory() {
-    ensureOpen();
-    return delegate.getLockFactory();
-  }
-
-  @Override
-  public String getLockID() {
-    ensureOpen();
-    return delegate.getLockID();
-  }
-
-  @Override
-  public String toString() {
-    return "RateLimitedDirectoryWrapper(" + delegate.toString() + ")";
+    return in.createSlicer(name, context);
   }
 
   @Override
   public void copy(Directory to, String src, String dest, IOContext context) throws IOException {
     ensureOpen();
-    delegate.copy(to, src, dest, context);
+    in.copy(to, src, dest, context);
   }
   
   private RateLimiter getRateLimiter(IOContext.Context context) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/TrackingDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/TrackingDirectoryWrapper.java?rev=1522661&r1=1522660&r2=1522661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/TrackingDirectoryWrapper.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/TrackingDirectoryWrapper.java Thu Sep 12 16:59:34 2013
@@ -17,105 +17,42 @@ package org.apache.lucene.store;
  * limitations under the License.
  */
 
-import java.io.Closeable;
 import java.io.IOException;
 import java.util.Collections;
-import java.util.Collection;
 import java.util.HashSet;
 import java.util.Set;
 
 /** A delegating Directory that records which files were
  *  written to and deleted. */
-public final class TrackingDirectoryWrapper extends Directory implements Closeable {
+public final class TrackingDirectoryWrapper extends FilterDirectory {
 
-  private final Directory other;
   private final Set<String> createdFileNames = Collections.synchronizedSet(new HashSet<String>());
 
-  public TrackingDirectoryWrapper(Directory other) {
-    this.other = other;
-  }
-
-  @Override
-  public String[] listAll() throws IOException {
-    return other.listAll();
-  }
-
-  @Override
-  public boolean fileExists(String name) throws IOException {
-    return other.fileExists(name);
+  public TrackingDirectoryWrapper(Directory in) {
+    super(in);
   }
 
   @Override
   public void deleteFile(String name) throws IOException {
     createdFileNames.remove(name);
-    other.deleteFile(name);
-  }
-
-  @Override
-  public long fileLength(String name) throws IOException {
-    return other.fileLength(name);
+    in.deleteFile(name);
   }
 
   @Override
   public IndexOutput createOutput(String name, IOContext context) throws IOException {
     createdFileNames.add(name);
-    return other.createOutput(name, context);
-  }
-
-  @Override
-  public void sync(Collection<String> names) throws IOException {
-    other.sync(names);
-  }
-
-  @Override
-  public IndexInput openInput(String name, IOContext context) throws IOException {
-    return other.openInput(name, context);
-  }
-
-  @Override
-  public Lock makeLock(String name) {
-    return other.makeLock(name);
-  }
-
-  @Override
-  public void clearLock(String name) throws IOException {
-    other.clearLock(name);
-  }
-
-  @Override
-  public void close() throws IOException {
-    other.close();
-  }
-
-  @Override
-  public void setLockFactory(LockFactory lockFactory) throws IOException {
-    other.setLockFactory(lockFactory);
-  }
-
-  @Override
-  public LockFactory getLockFactory() {
-    return other.getLockFactory();
-  }
-
-  @Override
-  public String getLockID() {
-    return other.getLockID();
-  }
-
-  @Override
-  public String toString() {
-    return "TrackingDirectoryWrapper(" + other.toString() + ")";
+    return in.createOutput(name, context);
   }
 
   @Override
   public void copy(Directory to, String src, String dest, IOContext context) throws IOException {
     createdFileNames.add(dest);
-    other.copy(to, src, dest, context);
+    in.copy(to, src, dest, context);
   }
 
   @Override
   public Directory.IndexInputSlicer createSlicer(final String name, final IOContext context) throws IOException {
-    return other.createSlicer(name, context);
+    return in.createSlicer(name, context);
   }
 
   // maybe clone before returning.... all callers are
@@ -124,7 +61,4 @@ public final class TrackingDirectoryWrap
     return createdFileNames;
   }
 
-  public Directory getDelegate() {
-    return other;
-  }
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java?rev=1522661&r1=1522660&r2=1522661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java Thu Sep 12 16:59:34 2013
@@ -27,6 +27,7 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.BaseDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.store.IOContext;
@@ -146,7 +147,7 @@ public class TestCrashCausesCorruptIndex
    * This test class provides direct access to "simulating" a crash right after 
    * realDirectory.createOutput(..) has been called on a certain specified name.
    */
-  private static class CrashAfterCreateOutput extends Directory {
+  private static class CrashAfterCreateOutput extends BaseDirectory {
         
     private Directory realDirectory;
     private String crashAfterCreateOutput;

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java?rev=1522661&r1=1522660&r2=1522661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java Thu Sep 12 16:59:34 2013
@@ -36,6 +36,7 @@ import org.apache.lucene.document.String
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.search.FieldCache;
+import org.apache.lucene.store.BaseDirectory;
 import org.apache.lucene.store.BufferedIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -112,7 +113,7 @@ public class TestFieldsReader extends Lu
   }
 
 
-  public static class FaultyFSDirectory extends Directory {
+  public static class FaultyFSDirectory extends BaseDirectory {
 
     Directory fsDir;
     

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java?rev=1522661&r1=1522660&r2=1522661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java Thu Sep 12 16:59:34 2013
@@ -283,7 +283,7 @@ public class TestBufferedIndexInput exte
       }
     }
 
-    private static class MockFSDirectory extends Directory {
+    private static class MockFSDirectory extends BaseDirectory {
 
       List<IndexInput> allIndexInputs = new ArrayList<IndexInput>();
 

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java?rev=1522661&r1=1522660&r2=1522661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryWrapper.java Thu Sep 12 16:59:34 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.store;
  */
 
 import java.io.IOException;
-import java.util.Collection;
 
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.util._TestUtil;
@@ -29,15 +28,14 @@ import org.apache.lucene.util._TestUtil;
 // do NOT make any methods in this class synchronized, volatile
 // do NOT import anything from the concurrency package.
 // no randoms, no nothing.
-public class BaseDirectoryWrapper extends Directory {
-  /** our in directory */
-  protected final Directory delegate;
+public class BaseDirectoryWrapper extends FilterDirectory {
   
   private boolean checkIndexOnClose = true;
   private boolean crossCheckTermVectorsOnClose = true;
+  protected volatile boolean isOpen = true;
 
   public BaseDirectoryWrapper(Directory delegate) {
-    this.delegate = delegate;
+    super(delegate);
   }
 
   @Override
@@ -46,7 +44,7 @@ public class BaseDirectoryWrapper extend
     if (checkIndexOnClose && DirectoryReader.indexExists(this)) {
       _TestUtil.checkIndex(this, crossCheckTermVectorsOnClose);
     }
-    delegate.close();
+    super.close();
   }
   
   public boolean isOpen() {
@@ -73,80 +71,13 @@ public class BaseDirectoryWrapper extend
     return crossCheckTermVectorsOnClose;
   }
 
-  // directory methods: delegate
-
-  @Override
-  public String[] listAll() throws IOException {
-    return delegate.listAll();
-  }
-
-  @Override
-  public boolean fileExists(String name) throws IOException {
-    return delegate.fileExists(name);
-  }
-
-  @Override
-  public void deleteFile(String name) throws IOException {
-    delegate.deleteFile(name);
-  }
-
-  @Override
-  public long fileLength(String name) throws IOException {
-    return delegate.fileLength(name);
-  }
-
-  @Override
-  public IndexOutput createOutput(String name, IOContext context) throws IOException {
-    return delegate.createOutput(name, context);
-  }
-
-  @Override
-  public void sync(Collection<String> names) throws IOException {
-    delegate.sync(names);
-  }
-
-  @Override
-  public IndexInput openInput(String name, IOContext context) throws IOException {
-    return delegate.openInput(name, context);
-  }
-
-  @Override
-  public Lock makeLock(String name) {
-    return delegate.makeLock(name);
-  }
-
-  @Override
-  public void clearLock(String name) throws IOException {
-    delegate.clearLock(name);
-  }
-
-  @Override
-  public void setLockFactory(LockFactory lockFactory) throws IOException {
-    delegate.setLockFactory(lockFactory);
-  }
-
-  @Override
-  public LockFactory getLockFactory() {
-    return delegate.getLockFactory();
-  }
-
-  @Override
-  public String getLockID() {
-    return delegate.getLockID();
-  }
-
-  @Override
-  public String toString() {
-    return "BaseDirectoryWrapper(" + delegate.toString() + ")";
-  }
-
   @Override
   public void copy(Directory to, String src, String dest, IOContext context) throws IOException {
-    delegate.copy(to, src, dest, context);
+    in.copy(to, src, dest, context);
   }
 
   @Override
   public IndexInputSlicer createSlicer(String name, IOContext context) throws IOException {
-    return delegate.createSlicer(name, context);
+    return in.createSlicer(name, context);
   }  
 }

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1522661&r1=1522660&r2=1522661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java Thu Sep 12 16:59:34 2013
@@ -80,6 +80,7 @@ public class MockDirectoryWrapper extend
   volatile boolean crashed;
   private ThrottledIndexOutput throttledOutput;
   private Throttling throttling = Throttling.SOMETIMES;
+  protected LockFactory lockFactory;
 
   final AtomicInteger inputCloneCount = new AtomicInteger();
 
@@ -121,10 +122,6 @@ public class MockDirectoryWrapper extend
     this.lockFactory = new MockLockFactoryWrapper(this, delegate.getLockFactory());
     init();
   }
-  
-  public Directory getDelegate() {
-    return this.delegate;
-  }
 
   public int getInputCloneCount() {
     return inputCloneCount.get();
@@ -160,21 +157,15 @@ public class MockDirectoryWrapper extend
   }
 
   /**
-   * Returns true if {@link #getDelegate() delegate} must sync its files.
+   * Returns true if {@link #in} must sync its files.
    * Currently, only {@link NRTCachingDirectory} requires sync'ing its files
    * because otherwise they are cached in an internal {@link RAMDirectory}. If
    * other directories require that too, they should be added to this method.
    */
   private boolean mustSync() {
-    Directory delegate = this.delegate;
-    while (true) {
-      if (delegate instanceof RateLimitedDirectoryWrapper) {
-        delegate = ((RateLimitedDirectoryWrapper) delegate).getDelegate();
-      } else if (delegate instanceof TrackingDirectoryWrapper) {
-        delegate = ((TrackingDirectoryWrapper) delegate).getDelegate();
-      } else {
-        break;
-      }
+    Directory delegate = in;
+    while (delegate instanceof FilterDirectory) {
+      delegate = ((FilterDirectory) delegate).getDelegate();
     }
     return delegate instanceof NRTCachingDirectory;
   }
@@ -191,31 +182,22 @@ public class MockDirectoryWrapper extend
       for (String name : names) {
         // randomly fail with IOE on any file
         maybeThrowIOException(name);
-        delegate.sync(Collections.singleton(name));
+        in.sync(Collections.singleton(name));
         unSyncedFiles.remove(name);
       }
     } else {
       unSyncedFiles.removeAll(names);
     }
   }
-  
-  @Override
-  public String toString() {
-    // NOTE: do not maybeYield here, since it consumes
-    // randomness and can thus (unexpectedly during
-    // debugging) change the behavior of a seed
-    // maybeYield();
-    return "MockDirWrapper(" + delegate + ")";
-  }
 
   public synchronized final long sizeInBytes() throws IOException {
-    if (delegate instanceof RAMDirectory)
-      return ((RAMDirectory) delegate).sizeInBytes();
+    if (in instanceof RAMDirectory)
+      return ((RAMDirectory) in).sizeInBytes();
     else {
       // hack
       long size = 0;
-      for (String file : delegate.listAll())
-        size += delegate.fileLength(file);
+      for (String file : in.listAll())
+        size += in.fileLength(file);
       return size;
     }
   }
@@ -252,7 +234,7 @@ public class MockDirectoryWrapper extend
         long length = fileLength(name);
         byte[] zeroes = new byte[256];
         long upto = 0;
-        IndexOutput out = delegate.createOutput(name, LuceneTestCase.newIOContext(randomState));
+        IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
         while(upto < length) {
           final int limit = (int) Math.min(length-upto, zeroes.length);
           out.writeBytes(zeroes, 0, limit);
@@ -268,12 +250,12 @@ public class MockDirectoryWrapper extend
         String tempFileName;
         while (true) {
           tempFileName = ""+randomState.nextInt();
-          if (!delegate.fileExists(tempFileName)) {
+          if (!in.fileExists(tempFileName)) {
             break;
           }
         }
-        final IndexOutput tempOut = delegate.createOutput(tempFileName, LuceneTestCase.newIOContext(randomState));
-        IndexInput ii = delegate.openInput(name, LuceneTestCase.newIOContext(randomState));
+        final IndexOutput tempOut = in.createOutput(tempFileName, LuceneTestCase.newIOContext(randomState));
+        IndexInput ii = in.openInput(name, LuceneTestCase.newIOContext(randomState));
         tempOut.copyBytes(ii, ii.length()/2);
         tempOut.close();
         ii.close();
@@ -281,8 +263,8 @@ public class MockDirectoryWrapper extend
         // Delete original and copy bytes back:
         deleteFile(name, true);
         
-        final IndexOutput out = delegate.createOutput(name, LuceneTestCase.newIOContext(randomState));
-        ii = delegate.openInput(tempFileName, LuceneTestCase.newIOContext(randomState));
+        final IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
+        ii = in.openInput(tempFileName, LuceneTestCase.newIOContext(randomState));
         out.copyBytes(ii, ii.length());
         out.close();
         ii.close();
@@ -294,7 +276,7 @@ public class MockDirectoryWrapper extend
         action = "fully truncated";
         // Totally truncate the file to zero bytes
         deleteFile(name, true);
-        IndexOutput out = delegate.createOutput(name, LuceneTestCase.newIOContext(randomState));
+        IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
         out.setLength(0);
         out.close();
       }
@@ -436,7 +418,7 @@ public class MockDirectoryWrapper extend
         openFilesDeleted.remove(name);
       }
     }
-    delegate.deleteFile(name);
+    in.deleteFile(name);
   }
 
   public synchronized Set<String> getOpenDeletedFiles() {
@@ -476,8 +458,8 @@ public class MockDirectoryWrapper extend
     unSyncedFiles.add(name);
     createdFiles.add(name);
     
-    if (delegate instanceof RAMDirectory) {
-      RAMDirectory ramdir = (RAMDirectory) delegate;
+    if (in instanceof RAMDirectory) {
+      RAMDirectory ramdir = (RAMDirectory) in;
       RAMFile file = new RAMFile(ramdir);
       RAMFile existing = ramdir.fileMap.get(name);
     
@@ -493,7 +475,7 @@ public class MockDirectoryWrapper extend
       }
     }
     //System.out.println(Thread.currentThread().getName() + ": MDW: create " + name);
-    IndexOutput delegateOutput = delegate.createOutput(name, LuceneTestCase.newIOContext(randomState, context));
+    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);
@@ -504,7 +486,7 @@ public class MockDirectoryWrapper extend
     
     // throttling REALLY slows down tests, so don't do it very often for SOMETIMES.
     if (throttling == Throttling.ALWAYS || 
-        (throttling == Throttling.SOMETIMES && randomState.nextInt(50) == 0) && !(delegate instanceof RateLimitedDirectoryWrapper)) {
+        (throttling == Throttling.SOMETIMES && randomState.nextInt(50) == 0) && !(in instanceof RateLimitedDirectoryWrapper)) {
       if (LuceneTestCase.VERBOSE) {
         System.out.println("MockDirectoryWrapper: throttling indexOutput (" + name + ")");
       }
@@ -544,8 +526,8 @@ public class MockDirectoryWrapper extend
     if (failOnOpenInput) {
       maybeThrowDeterministicException();
     }
-    if (!delegate.fileExists(name)) {
-      throw randomState.nextBoolean() ? new FileNotFoundException(name + " in dir=" + delegate) : new NoSuchFileException(name + " in dir=" + delegate);
+    if (!in.fileExists(name)) {
+      throw randomState.nextBoolean() ? new FileNotFoundException(name + " in dir=" + in) : new NoSuchFileException(name + " in dir=" + in);
     }
 
     // cannot open a file for input if it's still open for
@@ -554,7 +536,7 @@ public class MockDirectoryWrapper extend
       throw fillOpenTrace(new IOException("MockDirectoryWrapper: file \"" + name + "\" is still open for writing"), name, false);
     }
 
-    IndexInput delegateInput = delegate.openInput(name, LuceneTestCase.newIOContext(randomState, context));
+    IndexInput delegateInput = in.openInput(name, LuceneTestCase.newIOContext(randomState, context));
 
     final IndexInput ii;
     int randomInt = randomState.nextInt(500);
@@ -577,10 +559,10 @@ public class MockDirectoryWrapper extend
   
   /** Provided for testing purposes.  Use sizeInBytes() instead. */
   public synchronized final long getRecomputedSizeInBytes() throws IOException {
-    if (!(delegate instanceof RAMDirectory))
+    if (!(in instanceof RAMDirectory))
       return sizeInBytes();
     long size = 0;
-    for(final RAMFile file: ((RAMDirectory)delegate).fileMap.values()) {
+    for(final RAMFile file: ((RAMDirectory)in).fileMap.values()) {
       size += file.getSizeInBytes();
     }
     return size;
@@ -593,10 +575,10 @@ public class MockDirectoryWrapper extend
    */
 
   public final synchronized long getRecomputedActualSizeInBytes() throws IOException {
-    if (!(delegate instanceof RAMDirectory))
+    if (!(in instanceof RAMDirectory))
       return sizeInBytes();
     long size = 0;
-    for (final RAMFile file : ((RAMDirectory)delegate).fileMap.values())
+    for (final RAMFile file : ((RAMDirectory)in).fileMap.values())
       size += file.length;
     return size;
   }
@@ -666,8 +648,8 @@ public class MockDirectoryWrapper extend
           String[] startFiles = allFiles.toArray(new String[0]);
           IndexWriterConfig iwc = new IndexWriterConfig(LuceneTestCase.TEST_VERSION_CURRENT, null);
           iwc.setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE);
-          new IndexWriter(delegate, iwc).rollback();
-          String[] endFiles = delegate.listAll();
+          new IndexWriter(in, iwc).rollback();
+          String[] endFiles = in.listAll();
 
           Set<String> startSet = new TreeSet<String>(Arrays.asList(startFiles));
           Set<String> endSet = new TreeSet<String>(Arrays.asList(endFiles));
@@ -692,13 +674,13 @@ public class MockDirectoryWrapper extend
               }
               SegmentInfos sis = new SegmentInfos();
               try {
-                sis.read(delegate, file);
+                sis.read(in, file);
               } catch (IOException ioe) {
                 // OK: likely some of the .si files were deleted
               }
 
               try {
-                Set<String> ghosts = new HashSet<String>(sis.files(delegate, false));
+                Set<String> ghosts = new HashSet<String>(sis.files(in, false));
                 for (String s : ghosts) {
                   if (endSet.contains(s) && !startSet.contains(s)) {
                     assert pendingDeletions.contains(s);
@@ -763,7 +745,7 @@ public class MockDirectoryWrapper extend
         }
       }
     }
-    delegate.close();
+    in.close();
   }
 
   synchronized void removeOpenFile(Closeable c, String name) {
@@ -856,19 +838,19 @@ public class MockDirectoryWrapper extend
   @Override
   public synchronized String[] listAll() throws IOException {
     maybeYield();
-    return delegate.listAll();
+    return in.listAll();
   }
 
   @Override
   public synchronized boolean fileExists(String name) throws IOException {
     maybeYield();
-    return delegate.fileExists(name);
+    return in.fileExists(name);
   }
 
   @Override
   public synchronized long fileLength(String name) throws IOException {
     maybeYield();
-    return delegate.fileLength(name);
+    return in.fileLength(name);
   }
 
   @Override
@@ -888,7 +870,7 @@ public class MockDirectoryWrapper extend
     maybeYield();
     // sneaky: we must pass the original this way to the dir, because
     // some impls (e.g. FSDir) do instanceof here.
-    delegate.setLockFactory(lockFactory);
+    in.setLockFactory(lockFactory);
     // now set our wrapped factory here
     this.lockFactory = new MockLockFactoryWrapper(this, lockFactory);
   }
@@ -899,28 +881,28 @@ public class MockDirectoryWrapper extend
     if (wrapLockFactory) {
       return lockFactory;
     } else {
-      return delegate.getLockFactory();
+      return in.getLockFactory();
     }
   }
 
   @Override
   public synchronized String getLockID() {
     maybeYield();
-    return delegate.getLockID();
+    return in.getLockID();
   }
 
   @Override
   public synchronized void copy(Directory to, String src, String dest, IOContext context) throws IOException {
     maybeYield();
     // randomize the IOContext here?
-    delegate.copy(to, src, dest, context);
+    in.copy(to, src, dest, context);
   }
 
   @Override
   public IndexInputSlicer createSlicer(final String name, IOContext context)
       throws IOException {
     maybeYield();
-    if (!delegate.fileExists(name)) {
+    if (!in.fileExists(name)) {
       throw randomState.nextBoolean() ? new FileNotFoundException(name) : new NoSuchFileException(name);
     }
     // cannot open a file for input if it's still open for
@@ -929,7 +911,7 @@ public class MockDirectoryWrapper extend
       throw fillOpenTrace(new IOException("MockDirectoryWrapper: file \"" + name + "\" is still open for writing"), name, false);
     }
     
-    final IndexInputSlicer delegateHandle = delegate.createSlicer(name, context);
+    final IndexInputSlicer delegateHandle = in.createSlicer(name, context);
     final IndexInputSlicer handle = new IndexInputSlicer() {
       
       private boolean isClosed;

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java?rev=1522661&r1=1522660&r2=1522661&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java Thu Sep 12 16:59:34 2013
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FSDataInputS
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.lucene.store.BaseDirectory;
 import org.apache.lucene.store.BufferedIndexOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -39,7 +40,7 @@ import org.apache.solr.store.blockcache.
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class HdfsDirectory extends Directory {
+public class HdfsDirectory extends BaseDirectory {
   public static Logger LOG = LoggerFactory.getLogger(HdfsDirectory.class);
   
   public static final int BUFFER_SIZE = 8192;