You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by mi...@apache.org on 2009/01/30 12:12:48 UTC

svn commit: r739238 [1/2] - in /lucene/java/trunk: ./ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/store/ src/java/org/apache/lucene/util/ src/test/org/apache/lucene/index/

Author: mikemccand
Date: Fri Jan 30 11:12:48 2009
New Revision: 739238

URL: http://svn.apache.org/viewvc?rev=739238&view=rev
Log:
LUCENE-1314: add IndexReader.clone(boolean readOnly) and reopen(boolean readOnly)

Added:
    lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderClone.java   (with props)
    lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderCloneNorms.java   (with props)
Modified:
    lucene/java/trunk/CHANGES.txt
    lucene/java/trunk/src/java/org/apache/lucene/index/DirectoryIndexReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/MultiReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/MultiSegmentReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/ParallelReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/ReadOnlyMultiSegmentReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/SegmentReader.java
    lucene/java/trunk/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java
    lucene/java/trunk/src/java/org/apache/lucene/util/BitVector.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderReopen.java

Modified: lucene/java/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/java/trunk/CHANGES.txt?rev=739238&r1=739237&r2=739238&view=diff
==============================================================================
--- lucene/java/trunk/CHANGES.txt (original)
+++ lucene/java/trunk/CHANGES.txt Fri Jan 30 11:12:48 2009
@@ -142,6 +142,13 @@
     that field, even with different Terms in the filter, are fast.
     (Tim Sturge, Shalin Shekhar Mangar via Mike McCandless).
 
+13. LUCENE-1314: Add clone(), clone(boolean readOnly) and
+    reopen(boolean readOnly) to IndexReader.  Cloning an IndexReader
+    gives you a new reader which you can make changes to (deletions,
+    norms) without affecting the original reader.  Now, with clone or
+    reopen you can change the readOnly of the original reader.  (Jason
+    Rutherglen, Mike McCandless)
+
 Optimizations
 
  1. LUCENE-1427: Fixed QueryWrapperFilter to not waste time computing

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/DirectoryIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DirectoryIndexReader.java?rev=739238&r1=739237&r2=739238&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DirectoryIndexReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DirectoryIndexReader.java Fri Jan 30 11:12:48 2009
@@ -37,7 +37,7 @@
  * "own" the directory, which means that they try to acquire a write lock
  * whenever index modifications are performed.
  */
-abstract class DirectoryIndexReader extends IndexReader {
+abstract class DirectoryIndexReader extends IndexReader implements Cloneable {
   protected Directory directory;
   protected boolean closeDirectory;
   private IndexDeletionPolicy deletionPolicy;
@@ -62,12 +62,24 @@
     this.closeDirectory = closeDirectory;
     this.readOnly = readOnly;
 
+    if (readOnly) {
+      assert this instanceof ReadOnlySegmentReader ||
+        this instanceof ReadOnlyMultiSegmentReader;
+    } else {
+      assert !(this instanceof ReadOnlySegmentReader) &&
+        !(this instanceof ReadOnlyMultiSegmentReader);
+    }
+
     if (!readOnly && segmentInfos != null) {
       // We assume that this segments_N was previously
       // properly sync'd:
       synced.addAll(segmentInfos.files(directory, true));
     }
   }
+
+  boolean hasSegmentInfos() {
+    return segmentInfos != null;
+  }
   
   protected DirectoryIndexReader() {}
   
@@ -134,14 +146,73 @@
 
     return reader;
   }
-
+  
   public final synchronized IndexReader reopen() throws CorruptIndexException, IOException {
+    // Preserve current readOnly
+    return doReopen(readOnly);
+  }
+
+  public final synchronized IndexReader reopen(boolean openReadOnly) throws CorruptIndexException, IOException {
+    return doReopen(openReadOnly);
+  }
+
+  public final synchronized Object clone() {
+    try { 
+      // Preserve current readOnly
+      return clone(readOnly);
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+  
+  public final synchronized IndexReader clone(boolean openReadOnly) throws CorruptIndexException, IOException {
+
+    final SegmentInfos infos = (SegmentInfos) segmentInfos.clone();
+    DirectoryIndexReader newReader = doReopen(infos, true, openReadOnly);
+    
+    if (this != newReader) {
+      newReader.init(directory, infos, closeDirectory, openReadOnly);
+      newReader.deletionPolicy = deletionPolicy;
+    }
+
+    // If we're cloning a non-readOnly reader, move the
+    // writeLock (if there is one) to the new reader:
+    if (!openReadOnly && writeLock != null) {
+      newReader.writeLock = writeLock;
+      writeLock = null;
+      hasChanges = false;
+    }
+    
+    return newReader;
+  }
+  
+  // If there are no changes to the index, simply return
+  // ourself.  If there are changes, load the latest
+  // SegmentInfos and reopen based on that
+  protected final synchronized IndexReader doReopen(final boolean openReadOnly) throws CorruptIndexException, IOException {
     ensureOpen();
 
-    if (this.hasChanges || this.isCurrent()) {
-      // this has changes, therefore we have the lock and don't need to reopen
-      // OR: the index in the directory hasn't changed - nothing to do here
-      return this;
+    if (hasChanges) {
+      // We have changes, which means we are not readOnly:
+      assert readOnly == false;
+      // and we hold the write lock:
+      assert writeLock != null;
+      // so no other writer holds the write lock, which
+      // means no changes could have been done to the index:
+      assert isCurrent();
+
+      if (openReadOnly) {
+        return (IndexReader) clone(openReadOnly);
+      } else {
+        return this;
+      }
+    } else if (isCurrent()) {
+      if (openReadOnly != readOnly) {
+        // Just fallback to clone
+        return (IndexReader) clone(openReadOnly);
+      } else {
+        return this;
+      }
     }
 
     final SegmentInfos.FindSegmentsFile finder = new SegmentInfos.FindSegmentsFile(directory) {
@@ -149,11 +220,10 @@
       protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
         SegmentInfos infos = new SegmentInfos();
         infos.read(directory, segmentFileName);
-
-        DirectoryIndexReader newReader = doReopen(infos);
+        DirectoryIndexReader newReader = doReopen(infos, false, openReadOnly);
         
         if (DirectoryIndexReader.this != newReader) {
-          newReader.init(directory, infos, closeDirectory, readOnly);
+          newReader.init(directory, infos, closeDirectory, openReadOnly);
           newReader.deletionPolicy = deletionPolicy;
         }
 
@@ -193,7 +263,7 @@
   /**
    * Re-opens the index using the passed-in SegmentInfos 
    */
-  protected abstract DirectoryIndexReader doReopen(SegmentInfos infos) throws CorruptIndexException, IOException;
+  protected abstract DirectoryIndexReader doReopen(SegmentInfos infos, boolean doClone, boolean openReadOnly) throws CorruptIndexException, IOException;
   
   public void setDeletionPolicy(IndexDeletionPolicy deletionPolicy) {
     this.deletionPolicy = deletionPolicy;
@@ -341,6 +411,14 @@
    * @throws IOException if there is a low-level IO error
    */
   protected void acquireWriteLock() throws StaleReaderException, CorruptIndexException, LockObtainFailedException, IOException {
+
+    if (readOnly) {
+      // NOTE: we should not reach this code w/ the core
+      // IndexReader classes; however, an external subclass
+      // of IndexReader could reach this.
+      ReadOnlySegmentReader.noWrite();
+    }
+
     if (segmentInfos != null) {
       ensureOpen();
       if (stale)

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=739238&r1=739237&r2=739238&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java Fri Jan 30 11:12:48 2009
@@ -919,9 +919,13 @@
         if (reader != null) {
           try {
             if (success)
-              reader.doCommit();
+              reader.commit();
           } finally {
-            reader.doClose();
+            // Force reader to not have changes; if we hit
+            // an exception during commit, we don't want
+            // close to retry the commit:
+            reader.hasChanges = false;
+            reader.close();
           }
         }
       }

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexReader.java?rev=739238&r1=739237&r2=739238&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexReader.java Fri Jan 30 11:12:48 2009
@@ -67,7 +67,7 @@
 
  @version $Id$
 */
-public abstract class IndexReader {
+public abstract class IndexReader implements Cloneable {
 
   // NOTE: in 3.0 this will change to true
   final static boolean READ_ONLY_DEFAULT = false;
@@ -328,12 +328,13 @@
    * If the index has not changed since this instance was (re)opened, then this
    * call is a NOOP and returns this instance. Otherwise, a new instance is 
    * returned. The old instance is <b>not</b> closed and remains usable.<br>
-   * <b>Note:</b> The re-opened reader instance and the old instance might share
-   * the same resources. For this reason no index modification operations 
-   * (e. g. {@link #deleteDocument(int)}, {@link #setNorm(int, String, byte)}) 
-   * should be performed using one of the readers until the old reader instance
-   * is closed. <b>Otherwise, the behavior of the readers is undefined.</b> 
    * <p>   
+   * If the reader is reopened, even though they share
+   * resources internally, it's safe to make changes
+   * (deletions, norms) with the new reader.  All shared
+   * mutable state obeys "copy on write" semantics to ensure
+   * the changes are not seen by other readers.
+   * <p>
    * You can determine whether a reader was actually reopened by comparing the
    * old instance with the instance returned by this method: 
    * <pre>
@@ -354,6 +355,48 @@
   public synchronized IndexReader reopen() throws CorruptIndexException, IOException {
     throw new UnsupportedOperationException("This reader does not support reopen().");
   }
+  
+
+  /** Just like {@link #reopen()}, except you can change the
+   *  readOnly of the original reader.  If the index is
+   *  unchanged but readOnly is different then a new reader
+   *  will be returned. */
+  public synchronized IndexReader reopen(boolean openReadOnly) throws CorruptIndexException, IOException {
+    throw new UnsupportedOperationException("This reader does not support reopen().");
+  }
+  
+  /**
+   * Efficiently clones the IndexReader (sharing most
+   * internal state).
+   * <p>
+   * On cloning a reader with pending changes (deletions,
+   * norms), the original reader transfers its write lock to
+   * the cloned reader.  This means only the cloned reader
+   * may make further changes to the index, and commit the
+   * changes to the index on close, but the old reader still
+   * reflects all changes made up until it was cloned.
+   * <p>
+   * Like {@link #reopen()}, it's safe to make changes to
+   * either the original or the cloned reader: all shared
+   * mutable state obeys "copy on write" semantics to ensure
+   * the changes are not seen by other readers.
+   * <p>
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
+   */
+  public synchronized Object clone() {
+    throw new UnsupportedOperationException("This reader does not implement clone()");
+  }
+  
+  /**
+   * Clones the IndexReader and optionally changes readOnly.  A readOnly 
+   * reader cannot open a writeable reader.  
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
+   */
+  public synchronized IndexReader clone(boolean openReadOnly) throws CorruptIndexException, IOException {
+    throw new UnsupportedOperationException("This reader does not implement clone()");
+  }
 
   /** 
    * Returns the directory associated with this index.  The Default 

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/MultiReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/MultiReader.java?rev=739238&r1=739237&r2=739238&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/MultiReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/MultiReader.java Fri Jan 30 11:12:48 2009
@@ -32,7 +32,7 @@
  *
  * @version $Id$
  */
-public class MultiReader extends IndexReader {
+public class MultiReader extends IndexReader implements Cloneable {
   protected IndexReader[] subReaders;
   private int[] starts;                           // 1st docno for each segment
   private boolean[] decrefOnClose;                // remember which subreaders to decRef on close
@@ -86,7 +86,7 @@
     }
     starts[subReaders.length] = maxDoc;
   }
-
+  
   /**
    * Tries to reopen the subreaders.
    * <br>
@@ -106,53 +106,61 @@
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error 
    */
-  public IndexReader reopen() throws CorruptIndexException, IOException {
+  public synchronized IndexReader reopen() throws CorruptIndexException, IOException {
+    return doReopen(false);
+  }
+  
+  /**
+   * Clones the subreaders.
+   * (see {@link IndexReader#clone()}).
+   * <br>
+   * <p>
+   * If subreaders are shared, then the reference count of those
+   * readers is increased to ensure that the subreaders remain open
+   * until the last referring reader is closed.
+   */
+  public synchronized Object clone() {
+    try {
+      return doReopen(true);
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+  
+  /**
+   * If clone is true then we clone each of the subreaders
+   * @param doClone
+   * @return New IndexReader, or same one (this) if
+   *   reopen/clone is not necessary
+   * @throws CorruptIndexException
+   * @throws IOException
+   */
+  protected IndexReader doReopen(boolean doClone) throws CorruptIndexException, IOException {
     ensureOpen();
     
     boolean reopened = false;
     IndexReader[] newSubReaders = new IndexReader[subReaders.length];
-    boolean[] newDecrefOnClose = new boolean[subReaders.length];
     
     boolean success = false;
     try {
       for (int i = 0; i < subReaders.length; i++) {
-        newSubReaders[i] = subReaders[i].reopen();
+        if (doClone)
+          newSubReaders[i] = (IndexReader) subReaders[i].clone();
+        else
+          newSubReaders[i] = subReaders[i].reopen();
         // if at least one of the subreaders was updated we remember that
         // and return a new MultiReader
         if (newSubReaders[i] != subReaders[i]) {
           reopened = true;
-          // this is a new subreader instance, so on close() we don't
-          // decRef but close it 
-          newDecrefOnClose[i] = false;
-        }
-      }
-
-      if (reopened) {
-        for (int i = 0; i < subReaders.length; i++) {
-          if (newSubReaders[i] == subReaders[i]) {
-            newSubReaders[i].incRef();
-            newDecrefOnClose[i] = true;
-          }
         }
-        
-        MultiReader mr = new MultiReader(newSubReaders);
-        mr.decrefOnClose = newDecrefOnClose;
-        success = true;
-        return mr;
-      } else {
-        success = true;
-        return this;
       }
+      success = true;
     } finally {
       if (!success && reopened) {
         for (int i = 0; i < newSubReaders.length; i++) {
-          if (newSubReaders[i] != null) {
+          if (newSubReaders[i] != subReaders[i]) {
             try {
-              if (newDecrefOnClose[i]) {
-                newSubReaders[i].decRef();
-              } else {
-                newSubReaders[i].close();
-              }
+              newSubReaders[i].close();
             } catch (IOException ignore) {
               // keep going - we want to clean up as much as possible
             }
@@ -160,6 +168,23 @@
         }
       }
     }
+
+    if (reopened) {
+      boolean[] newDecrefOnClose = new boolean[subReaders.length];
+      for (int i = 0; i < subReaders.length; i++) {
+        if (newSubReaders[i] == subReaders[i]) {
+          newSubReaders[i].incRef();
+          newDecrefOnClose[i] = true;
+        }
+      }
+      MultiReader mr = new MultiReader(newSubReaders);
+      mr.decrefOnClose = newDecrefOnClose;
+      success = true;
+      return mr;
+    } else {
+      success = true;
+      return this;
+    }
   }
 
   public TermFreqVector[] getTermFreqVectors(int n) throws IOException {

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/MultiSegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/MultiSegmentReader.java?rev=739238&r1=739237&r2=739238&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/MultiSegmentReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/MultiSegmentReader.java Fri Jan 30 11:12:48 2009
@@ -32,7 +32,7 @@
 /** 
  * An IndexReader which reads indexes with multiple segments.
  */
-class MultiSegmentReader extends DirectoryIndexReader {
+class MultiSegmentReader extends DirectoryIndexReader implements Cloneable {
   protected SegmentReader[] subReaders;
   private int[] starts;                           // 1st docno for each segment
   private Map normsCache = new HashMap();
@@ -70,7 +70,8 @@
   }
 
   /** This contructor is only used for {@link #reopen()} */
-  MultiSegmentReader(Directory directory, SegmentInfos infos, boolean closeDirectory, SegmentReader[] oldReaders, int[] oldStarts, Map oldNormsCache, boolean readOnly) throws IOException {
+  MultiSegmentReader(Directory directory, SegmentInfos infos, boolean closeDirectory, SegmentReader[] oldReaders, int[] oldStarts,
+                     Map oldNormsCache, boolean readOnly, boolean doClone) throws IOException {
     super(directory, infos, closeDirectory, readOnly);
 
     // we put the old SegmentReaders in a map, that allows us
@@ -108,7 +109,20 @@
           // this is a new reader; in case we hit an exception we can close it safely
           newReader = SegmentReader.get(readOnly, infos.info(i));
         } else {
-          newReader = (SegmentReader) newReaders[i].reopenSegment(infos.info(i));
+          newReader = (SegmentReader) newReaders[i].reopenSegment(infos.info(i), doClone, readOnly);
+          if (newReader == newReaders[i] && newReaders[i].hasSegmentInfos()) {
+            // Special case when a single-segment reader was
+            // reopened to a multi-segment reader -- we must
+            // get a private clone, to clear its
+            // SegmentInfos, so it does not attempt to
+            // obtain the write lock
+            newReader = (SegmentReader) newReaders[i].clone(readOnly);
+            newReader.init(directory, null, false, readOnly);
+          } 
+
+          // Make sure reopenSegment did not carry over a
+          // segmentInfos instance
+          assert !newReader.hasSegmentInfos();
         }
         if (newReader == newReaders[i]) {
           // this reader will be shared between the old and the new one,
@@ -194,15 +208,15 @@
     starts[subReaders.length] = maxDoc;
   }
 
-  protected synchronized DirectoryIndexReader doReopen(SegmentInfos infos) throws CorruptIndexException, IOException {
+  protected synchronized DirectoryIndexReader doReopen(SegmentInfos infos, boolean doClone, boolean openReadOnly) throws CorruptIndexException, IOException {
     if (infos.size() == 1) {
       // The index has only one segment now, so we can't refresh the MultiSegmentReader.
       // Return a new [ReadOnly]SegmentReader instead
-      return SegmentReader.get(readOnly, infos, infos.info(0), false);
-    } else if (readOnly) {
-      return new ReadOnlyMultiSegmentReader(directory, infos, closeDirectory, subReaders, starts, normsCache);
+      return SegmentReader.get(openReadOnly, infos, infos.info(0), false);
+    } else if (openReadOnly) {
+      return new ReadOnlyMultiSegmentReader(directory, infos, closeDirectory, subReaders, starts, normsCache, doClone);
     } else {
-      return new MultiSegmentReader(directory, infos, closeDirectory, subReaders, starts, normsCache, false);
+      return new MultiSegmentReader(directory, infos, closeDirectory, subReaders, starts, normsCache, false, doClone);
     }            
   }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/ParallelReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/ParallelReader.java?rev=739238&r1=739237&r2=739238&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/ParallelReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/ParallelReader.java Fri Jan 30 11:12:48 2009
@@ -122,7 +122,15 @@
     }
     decrefOnClose.add(Boolean.valueOf(incRefReaders));
   }
-
+  
+  public synchronized Object clone() {
+    try {
+      return doReopen(true);
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+  
   /**
    * Tries to reopen the subreaders.
    * <br>
@@ -142,63 +150,42 @@
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error 
    */
-  public IndexReader reopen() throws CorruptIndexException, IOException {
+  public synchronized IndexReader reopen() throws CorruptIndexException, IOException {
+    return doReopen(false);
+  }
+    
+  protected IndexReader doReopen(boolean doClone) throws CorruptIndexException, IOException {
     ensureOpen();
     
     boolean reopened = false;
     List newReaders = new ArrayList();
-    List newDecrefOnClose = new ArrayList();
     
     boolean success = false;
     
     try {
-    
       for (int i = 0; i < readers.size(); i++) {
         IndexReader oldReader = (IndexReader) readers.get(i);
-        IndexReader newReader = oldReader.reopen();
+        IndexReader newReader = null;
+        if (doClone) {
+          newReader = (IndexReader) oldReader.clone();
+        } else {
+          newReader = oldReader.reopen();
+        }
         newReaders.add(newReader);
         // if at least one of the subreaders was updated we remember that
-        // and return a new MultiReader
+        // and return a new ParallelReader
         if (newReader != oldReader) {
           reopened = true;
         }
       }
-  
-      if (reopened) {
-        ParallelReader pr = new ParallelReader();
-        for (int i = 0; i < readers.size(); i++) {
-          IndexReader oldReader = (IndexReader) readers.get(i);
-          IndexReader newReader = (IndexReader) newReaders.get(i);
-          if (newReader == oldReader) {
-            newDecrefOnClose.add(Boolean.TRUE);
-            newReader.incRef();
-          } else {
-            // this is a new subreader instance, so on close() we don't
-            // decRef but close it 
-            newDecrefOnClose.add(Boolean.FALSE);
-          }
-          pr.add(newReader, !storedFieldReaders.contains(oldReader));
-        }
-        pr.decrefOnClose = newDecrefOnClose;
-        pr.incRefReaders = incRefReaders;
-        success = true;
-        return pr;
-      } else {
-        success = true; 
-       // No subreader was refreshed
-        return this;
-      }
+      success = true;
     } finally {
       if (!success && reopened) {
         for (int i = 0; i < newReaders.size(); i++) {
           IndexReader r = (IndexReader) newReaders.get(i);
-          if (r != null) {
+          if (r != readers.get(i)) {
             try {
-              if (((Boolean) newDecrefOnClose.get(i)).booleanValue()) {
-                r.decRef();
-              } else {
-                r.close();
-              }
+              r.close();
             } catch (IOException ignore) {
               // keep going - we want to clean up as much as possible
             }
@@ -206,6 +193,30 @@
         }
       }
     }
+
+    if (reopened) {
+      List newDecrefOnClose = new ArrayList();
+      ParallelReader pr = new ParallelReader();
+      for (int i = 0; i < readers.size(); i++) {
+        IndexReader oldReader = (IndexReader) readers.get(i);
+        IndexReader newReader = (IndexReader) newReaders.get(i);
+        if (newReader == oldReader) {
+          newDecrefOnClose.add(Boolean.TRUE);
+          newReader.incRef();
+        } else {
+          // this is a new subreader instance, so on close() we don't
+          // decRef but close it 
+          newDecrefOnClose.add(Boolean.FALSE);
+        }
+        pr.add(newReader, !storedFieldReaders.contains(oldReader));
+      }
+      pr.decrefOnClose = newDecrefOnClose;
+      pr.incRefReaders = incRefReaders;
+      return pr;
+    } else {
+      // No subreader was refreshed
+      return this;
+    }
   }
 
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/ReadOnlyMultiSegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/ReadOnlyMultiSegmentReader.java?rev=739238&r1=739237&r2=739238&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/ReadOnlyMultiSegmentReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/ReadOnlyMultiSegmentReader.java Fri Jan 30 11:12:48 2009
@@ -27,8 +27,8 @@
     super(directory, sis, closeDirectory, true);
   }
 
-  ReadOnlyMultiSegmentReader(Directory directory, SegmentInfos infos, boolean closeDirectory, SegmentReader[] oldReaders, int[] oldStarts, Map oldNormsCache) throws IOException {
-    super(directory, infos, closeDirectory, oldReaders, oldStarts, oldNormsCache, true);
+  ReadOnlyMultiSegmentReader(Directory directory, SegmentInfos infos, boolean closeDirectory, SegmentReader[] oldReaders, int[] oldStarts, Map oldNormsCache, boolean doClone) throws IOException {
+    super(directory, infos, closeDirectory, oldReaders, oldStarts, oldNormsCache, true, doClone);
   }
 
   protected void acquireWriteLock() {

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/SegmentReader.java?rev=739238&r1=739237&r2=739238&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/SegmentReader.java Fri Jan 30 11:12:48 2009
@@ -48,12 +48,13 @@
 
   FieldInfos fieldInfos;
   private FieldsReader fieldsReaderOrig = null;
-
+  CloseableThreadLocal fieldsReaderLocal = new FieldsReaderLocal();
   TermInfosReader tis;
   TermVectorsReader termVectorsReaderOrig = null;
   CloseableThreadLocal termVectorsLocal = new CloseableThreadLocal();
 
   BitVector deletedDocs = null;
+  Ref deletedDocsRef = null;
   private boolean deletedDocsDirty = false;
   private boolean normsDirty = false;
   private boolean undeleteAll = false;
@@ -63,13 +64,12 @@
   private boolean rollbackNormsDirty = false;
   private boolean rollbackUndeleteAll = false;
   private int rollbackPendingDeleteCount;
-  private boolean readOnly;
-
   IndexInput freqStream;
   IndexInput proxStream;
 
   // optionally used for the .nrm file shared by multiple norms
   private IndexInput singleNormStream;
+  private Ref singleNormRef;
 
   // Compound File Reader when based on a compound file segment
   CompoundFileReader cfsReader = null;
@@ -87,43 +87,238 @@
       return (FieldsReader) fieldsReaderOrig.clone();
     }
   }
-  CloseableThreadLocal fieldsReaderLocal = new FieldsReaderLocal();
   
-  private class Norm {
-    volatile int refCount;
-    boolean useSingleNormStream;
+  static class Ref {
+    private int refCount = 1;
     
-    public synchronized void incRef() {
+    public String toString() {
+      return "refcount: "+refCount;
+    }
+    
+    public synchronized int refCount() {
+      return refCount;
+    }
+    
+    public synchronized int incRef() {
       assert refCount > 0;
       refCount++;
+      return refCount;
     }
 
-    public synchronized void decRef() throws IOException {
+    public synchronized int decRef() {
       assert refCount > 0;
-      if (refCount == 1) {
-        close();
-      }
       refCount--;
-
-    }
-    
-    public Norm(IndexInput in, boolean useSingleNormStream, int number, long normSeek)
-    {
-      refCount = 1;
-      this.in = in;
-      this.number = number;
-      this.normSeek = normSeek;
-      this.useSingleNormStream = useSingleNormStream;
+      return refCount;
     }
+  }
+  
+  /**
+   * Byte[] referencing is used because a new norm object needs 
+   * to be created for each clone, and the byte array is all 
+   * that is needed for sharing between cloned readers.  The 
+   * current norm referencing is for sharing between readers 
+   * whereas the byte[] referencing is for copy on write which 
+   * is independent of reader references (i.e. incRef, decRef).
+   */
+
+  final class Norm implements Cloneable {
+    private int refCount = 1;
+
+    // If this instance is a clone, the originalNorm
+    // references the Norm that has a real open IndexInput:
+    private Norm origNorm;
 
     private IndexInput in;
+    private long normSeek;
+
+    // null until bytes is set
+    private Ref bytesRef;
     private byte[] bytes;
     private boolean dirty;
     private int number;
-    private long normSeek;
     private boolean rollbackDirty;
+    
+    public Norm(IndexInput in, int number, long normSeek) {
+      this.in = in;
+      this.number = number;
+      this.normSeek = normSeek;
+    }
+
+    public synchronized void incRef() {
+      assert refCount > 0 && (origNorm == null || origNorm.refCount > 0);
+      refCount++;
+    }
+
+    private void closeInput() throws IOException {
+      if (in != null) {
+        if (in != singleNormStream) {
+          // It's private to us -- just close it
+          in.close();
+        } else {
+          // We are sharing this with others -- decRef and
+          // maybe close the shared norm stream
+          if (singleNormRef.decRef() == 0) {
+            singleNormStream.close();
+            singleNormStream = null;
+          }
+        }
+
+        in = null;
+      }
+    }
+
+    public synchronized void decRef() throws IOException {
+      assert refCount > 0 && (origNorm == null || origNorm.refCount > 0);
+
+      if (--refCount == 0) {
+        if (origNorm != null) {
+          origNorm.decRef();
+          origNorm = null;
+        } else {
+          closeInput();
+        }
+
+        if (bytes != null) {
+          assert bytesRef != null;
+          bytesRef.decRef();
+          bytes = null;
+          bytesRef = null;
+        } else {
+          assert bytesRef == null;
+        }
+      }
+    }
+
+    // Load bytes but do not cache them if they were not
+    // already cached
+    public synchronized void bytes(byte[] bytesOut, int offset, int len) throws IOException {
+      assert refCount > 0 && (origNorm == null || origNorm.refCount > 0);
+      if (bytes != null) {
+        // Already cached -- copy from cache:
+        assert len <= maxDoc();
+        System.arraycopy(bytes, 0, bytesOut, offset, len);
+      } else {
+        // Not cached
+        if (origNorm != null) {
+          // Ask origNorm to load
+          origNorm.bytes(bytesOut, offset, len);
+        } else {
+          // We are orig -- read ourselves from disk:
+          synchronized(in) {
+            in.seek(normSeek);
+            in.readBytes(bytesOut, offset, len, false);
+          }
+        }
+      }
+    }
+
+    // Load & cache full bytes array.  Returns bytes.
+    public synchronized byte[] bytes() throws IOException {
+      assert refCount > 0 && (origNorm == null || origNorm.refCount > 0);
+      if (bytes == null) {                     // value not yet read
+        assert bytesRef == null;
+        if (origNorm != null) {
+          // Ask origNorm to load so that for a series of
+          // reopened readers we share a single read-only
+          // byte[]
+          bytes = origNorm.bytes();
+          bytesRef = origNorm.bytesRef;
+          bytesRef.incRef();
+
+          // Once we've loaded the bytes we no longer need
+          // origNorm:
+          origNorm.decRef();
+          origNorm = null;
+
+        } else {
+          // We are the origNorm, so load the bytes for real
+          // ourself:
+          final int count = maxDoc();
+          bytes = new byte[count];
+
+          // Since we are orig, in must not be null
+          assert in != null;
+
+          // Read from disk.
+          synchronized(in) {
+            in.seek(normSeek);
+            in.readBytes(bytes, 0, count, false);
+          }
+
+          bytesRef = new Ref();
+          closeInput();
+        }
+      }
+
+      return bytes;
+    }
+
+    // Only for testing
+    Ref bytesRef() {
+      return bytesRef;
+    }
+
+    // Called if we intend to change a norm value.  We make a
+    // private copy of bytes if it's shared with others:
+    public synchronized byte[] copyOnWrite() throws IOException {
+      assert refCount > 0 && (origNorm == null || origNorm.refCount > 0);
+      bytes();
+      assert bytes != null;
+      assert bytesRef != null;
+      if (bytesRef.refCount() > 1) {
+        // I cannot be the origNorm for another norm
+        // instance if I'm being changed.  Ie, only the
+        // "head Norm" can be changed:
+        assert refCount == 1;
+        final Ref oldRef = bytesRef;
+        bytes = cloneNormBytes(bytes);
+        bytesRef = new Ref();
+        oldRef.decRef();
+      }
+      dirty = true;
+      return bytes;
+    }
+    
+    // Returns a copy of this Norm instance that shares
+    // IndexInput & bytes with the original one
+    public synchronized Object clone() {
+      assert refCount > 0 && (origNorm == null || origNorm.refCount > 0);
+        
+      Norm clone;
+      try {
+        clone = (Norm) super.clone();
+      } catch (CloneNotSupportedException cnse) {
+        // Cannot happen
+        throw new RuntimeException("unexpected CloneNotSupportedException", cnse);
+      }
+      clone.refCount = 1;
+
+      if (bytes != null) {
+        assert bytesRef != null;
+        assert origNorm == null;
+
+        // Clone holds a reference to my bytes:
+        clone.bytesRef.incRef();
+      } else {
+        assert bytesRef == null;
+        if (origNorm == null) {
+          // I become the origNorm for the clone:
+          clone.origNorm = this;
+        }
+        clone.origNorm.incRef();
+      }
+
+      // Only the origNorm will actually readBytes from in:
+      clone.in = null;
+
+      return clone;
+    }
+
+    // Flush all pending changes to the next generation
+    // separate norms file.
+    public void reWrite(SegmentInfo si) throws IOException {
+      assert refCount > 0 && (origNorm == null || origNorm.refCount > 0): "refCount=" + refCount + " origNorm=" + origNorm;
 
-    private void reWrite(SegmentInfo si) throws IOException {
       // NOTE: norms are re-written in regular directory, not cfs
       si.advanceNormGen(this.number);
       IndexOutput out = directory().createOutput(si.getNormFileName(this.number));
@@ -134,55 +329,39 @@
       }
       this.dirty = false;
     }
-    
-    /** Closes the underlying IndexInput for this norm.
-     * It is still valid to access all other norm properties after close is called.
-     * @throws IOException
-     */
-    private synchronized void close() throws IOException {
-      if (in != null && !useSingleNormStream) {
-        in.close();
-      }
-      in = null;
-    }
   }
-  
-  /**
-   * Increments the RC of this reader, as well as
-   * of all norms this reader is using
-   */
+
   public synchronized void incRef() {
     super.incRef();
     Iterator it = norms.values().iterator();
     while (it.hasNext()) {
-      Norm norm = (Norm) it.next();
-      norm.incRef();
+      ((Norm) it.next()).incRef();
+    }
+    if (deletedDocsRef != null) {
+      deletedDocsRef.incRef();
     }
   }
   
-  /**
-   * only increments the RC of this reader, not tof 
-   * he norms. This is important whenever a reopen()
-   * creates a new SegmentReader that doesn't share
-   * the norms with this one 
-   */
   private synchronized void incRefReaderNotNorms() {
     super.incRef();
   }
-
+  
   public synchronized void decRef() throws IOException {
     super.decRef();
     Iterator it = norms.values().iterator();
     while (it.hasNext()) {
-      Norm norm = (Norm) it.next();
-      norm.decRef();
+      ((Norm) it.next()).decRef();
+    }
+
+    if (deletedDocsRef != null) {
+      deletedDocsRef.decRef();
     }
   }
   
   private synchronized void decRefReaderNotNorms() throws IOException {
     super.decRef();
   }
-  
+
   Map norms = new HashMap();
   
   /** The class which implements SegmentReader. */
@@ -410,6 +589,7 @@
     // NOTE: the bitvector is stored using the regular directory, not cfs
     if (hasDeletions(si)) {
       deletedDocs = new BitVector(directory(), si.getDelFileName());
+      deletedDocsRef = new Ref();
      
       assert si.getDelCount() == deletedDocs.count() : 
         "delete count mismatch: info=" + si.getDelCount() + " vs BitVector=" + deletedDocs.count();
@@ -423,66 +603,88 @@
       assert si.getDelCount() == 0;
   }
   
-  protected synchronized DirectoryIndexReader doReopen(SegmentInfos infos) throws CorruptIndexException, IOException {
+  /**
+   * Clones the norm bytes.  May be overridden by subclasses.  New and experimental.
+   * @param bv Byte array to clone
+   * @return New BitVector
+   */
+  protected byte[] cloneNormBytes(byte[] bytes) {
+    byte[] cloneBytes = new byte[bytes.length];
+    System.arraycopy(bytes, 0, cloneBytes, 0, bytes.length);
+    return cloneBytes;
+  }
+  
+  /**
+   * Clones the deleteDocs BitVector.  May be overridden by subclasses. New and experimental.
+   * @param bv BitVector to clone
+   * @return New BitVector
+   */
+  protected BitVector cloneDeletedDocs(BitVector bv) {
+    return (BitVector)bv.clone();
+  }
+
+  protected synchronized DirectoryIndexReader doReopen(SegmentInfos infos, boolean doClone, boolean openReadOnly) throws CorruptIndexException, IOException {
     DirectoryIndexReader newReader;
     
     if (infos.size() == 1) {
       SegmentInfo si = infos.info(0);
       if (segment.equals(si.name) && si.getUseCompoundFile() == SegmentReader.this.si.getUseCompoundFile()) {
-        newReader = reopenSegment(si);
+        newReader = reopenSegment(si, doClone, openReadOnly);
       } else { 
         // segment not referenced anymore, reopen not possible
         // or segment format changed
-        newReader = SegmentReader.get(readOnly, infos, infos.info(0), false);
+        newReader = SegmentReader.get(openReadOnly, infos, infos.info(0), false);
       }
     } else {
-      if (readOnly)
-        return new ReadOnlyMultiSegmentReader(directory, infos, closeDirectory, new SegmentReader[] {this}, null, null);
+      if (openReadOnly)
+        return new ReadOnlyMultiSegmentReader(directory, infos, closeDirectory, new SegmentReader[] {this}, null, null, doClone);
       else
-        return new MultiSegmentReader(directory, infos, closeDirectory, new SegmentReader[] {this}, null, null, false);
+        return new MultiSegmentReader(directory, infos, closeDirectory, new SegmentReader[] {this}, null, null, false, doClone);
     }
     
     return newReader;
   }
   
-  synchronized SegmentReader reopenSegment(SegmentInfo si) throws CorruptIndexException, IOException {
+  synchronized SegmentReader reopenSegment(SegmentInfo si, boolean doClone, boolean openReadOnly) throws CorruptIndexException, IOException {
     boolean deletionsUpToDate = (this.si.hasDeletions() == si.hasDeletions()) 
                                   && (!si.hasDeletions() || this.si.getDelFileName().equals(si.getDelFileName()));
     boolean normsUpToDate = true;
-
     
     boolean[] fieldNormsChanged = new boolean[fieldInfos.size()];
-    if (normsUpToDate) {
-      for (int i = 0; i < fieldInfos.size(); i++) {
-        if (!this.si.getNormFileName(i).equals(si.getNormFileName(i))) {
-          normsUpToDate = false;
-          fieldNormsChanged[i] = true;
-        }
+    final int fieldCount = fieldInfos.size();
+    for (int i = 0; i < fieldCount; i++) {
+      if (!this.si.getNormFileName(i).equals(si.getNormFileName(i))) {
+        normsUpToDate = false;
+        fieldNormsChanged[i] = true;
       }
     }
 
-    if (normsUpToDate && deletionsUpToDate) {
+    // if we're cloning we need to run through the reopenSegment logic
+    if (normsUpToDate && deletionsUpToDate && !doClone && openReadOnly == readOnly) {
       return this;
     }    
-    
 
     // clone reader
     SegmentReader clone;
-    if (readOnly) 
-      clone = new ReadOnlySegmentReader();
-    else
-      clone = new SegmentReader();
+    try {
+      if (openReadOnly)
+        clone = (SegmentReader) READONLY_IMPL.newInstance();
+      else
+        clone = (SegmentReader) IMPL.newInstance();
+    } catch (Exception e) {
+      throw new RuntimeException("cannot load SegmentReader class: " + e, e);
+    }
 
     boolean success = false;
     try {
-      clone.readOnly = readOnly;
+      clone.readOnly = openReadOnly;
       clone.directory = directory;
       clone.si = si;
       clone.segment = segment;
       clone.readBufferSize = readBufferSize;
       clone.cfsReader = cfsReader;
       clone.storeCFSReader = storeCFSReader;
-  
+
       clone.fieldInfos = fieldInfos;
       clone.tis = tis;
       clone.freqStream = freqStream;
@@ -493,71 +695,52 @@
         clone.fieldsReaderOrig = (FieldsReader) fieldsReaderOrig.clone();
       }      
       
-      if (!deletionsUpToDate) {
-        // load deleted docs
-        clone.deletedDocs = null;
-        clone.loadDeletedDocs();
+      if (deletedDocsRef != null) {
+        deletedDocsRef.incRef();
+      }
+      if (doClone) {
+        clone.deletedDocs = deletedDocs;
+        clone.deletedDocsRef = deletedDocsRef;
       } else {
-        clone.deletedDocs = this.deletedDocs;
+        if (!deletionsUpToDate) {
+          // load deleted docs
+          clone.deletedDocs = null;
+          clone.deletedDocsRef = null;
+          clone.loadDeletedDocs();
+        } else {
+          clone.deletedDocs = deletedDocs;
+          clone.deletedDocsRef = deletedDocsRef;
+        }
       }
-  
+
       clone.norms = new HashMap();
-      if (!normsUpToDate) {
-        // load norms
-        for (int i = 0; i < fieldNormsChanged.length; i++) {
-          // copy unchanged norms to the cloned reader and incRef those norms
-          if (!fieldNormsChanged[i]) {
-            String curField = fieldInfos.fieldInfo(i).name;
-            Norm norm = (Norm) this.norms.get(curField);
-            norm.incRef();
-            clone.norms.put(curField, norm);
-          }
-        }
-        
-        clone.openNorms(si.getUseCompoundFile() ? cfsReader : directory(), readBufferSize);
-      } else {
-        Iterator it = norms.keySet().iterator();
-        while (it.hasNext()) {
-          String field = (String) it.next();
-          Norm norm = (Norm) norms.get(field);
-          norm.incRef();
-          clone.norms.put(field, norm);
+
+      // Clone norms
+      for (int i = 0; i < fieldNormsChanged.length; i++) {
+
+        // Clone unchanged norms to the cloned reader
+        if (doClone || !fieldNormsChanged[i]) {
+          final String curField = fieldInfos.fieldInfo(i).name;
+          Norm norm = (Norm) this.norms.get(curField);
+          clone.norms.put(curField, norm.clone());
         }
       }
-  
-      if (clone.singleNormStream == null) {
-        for (int i = 0; i < fieldInfos.size(); i++) {
-          FieldInfo fi = fieldInfos.fieldInfo(i);
-          if (fi.isIndexed && !fi.omitNorms) {
-            Directory d = si.getUseCompoundFile() ? cfsReader : directory();
-            String fileName = si.getNormFileName(fi.number);
-            if (si.hasSeparateNorms(fi.number)) {
-              continue;
-            }  
-  
-            if (fileName.endsWith("." + IndexFileNames.NORMS_EXTENSION)) {
-              clone.singleNormStream = d.openInput(fileName, readBufferSize);    
-              break;
-            }
-          }
-        }  
-      }    
-  
+      
+      // If we are not cloning, then this will open anew
+      // any norms that have changed:
+      clone.openNorms(si.getUseCompoundFile() ? cfsReader : directory(), readBufferSize);
+
       success = true;
     } finally {
       if (this.referencedSegmentReader != null) {
-        // this reader shares resources with another SegmentReader,
-        // so we increment the other readers refCount. We don't
-        // increment the refCount of the norms because we did
-        // that already for the shared norms
+        // This reader shares resources with another SegmentReader,
+        // so we increment the other reader's refCount.
         clone.referencedSegmentReader = this.referencedSegmentReader;
-        referencedSegmentReader.incRefReaderNotNorms();
       } else {
-        // this reader wasn't reopened, so we increment this
-        // readers refCount
+        // We are the original SegmentReader
         clone.referencedSegmentReader = this;
-        incRefReaderNotNorms();
       }
+      clone.referencedSegmentReader.incRefReaderNotNorms();
       
       if (!success) {
         // An exception occured during reopen, we have to decRef the norms
@@ -570,6 +753,7 @@
   }
 
   protected void commitChanges() throws IOException {
+
     if (deletedDocsDirty) {               // re-write deleted
       si.advanceDelGen();
 
@@ -603,35 +787,24 @@
   FieldsReader getFieldsReader() {
     return (FieldsReader) fieldsReaderLocal.get();
   }
-
+  
   protected void doClose() throws IOException {
     boolean hasReferencedReader = (referencedSegmentReader != null);
 
     termVectorsLocal.close();
     fieldsReaderLocal.close();
-
+    
     if (hasReferencedReader) {
       referencedSegmentReader.decRefReaderNotNorms();
       referencedSegmentReader = null;
     }
 
-    deletedDocs = null;
-
-    // close the single norms stream
-    if (singleNormStream != null) {
-      // we can close this stream, even if the norms
-      // are shared, because every reader has it's own 
-      // singleNormStream
-      singleNormStream.close();
-      singleNormStream = null;
-    }
-    
     if (!hasReferencedReader) { 
       // close everything, nothing is shared anymore with other readers
       if (tis != null) {
         tis.close();
       }
-  
+      
       if (freqStream != null)
         freqStream.close();
       if (proxStream != null)
@@ -676,8 +849,21 @@
   }
 
   protected void doDelete(int docNum) {
-    if (deletedDocs == null)
+    if (deletedDocs == null) {
       deletedDocs = new BitVector(maxDoc());
+      deletedDocsRef = new Ref();
+    }
+    // there is more than 1 SegmentReader with a reference to this
+    // deletedDocs BitVector so decRef the current deletedDocsRef,
+    // clone the BitVector, create a new deletedDocsRef
+    if (deletedDocsRef.refCount() > 1) {
+      synchronized (deletedDocsRef) {
+        Ref oldRef = deletedDocsRef;
+        deletedDocs = cloneDeletedDocs(deletedDocs);
+        deletedDocsRef = new Ref();
+        oldRef.decRef();
+      }
+    }
     deletedDocsDirty = true;
     undeleteAll = false;
     if (!deletedDocs.getAndSet(docNum))
@@ -685,9 +871,16 @@
   }
 
   protected void doUndeleteAll() {
+    deletedDocsDirty = false;
+    undeleteAll = true;
+    if (deletedDocs != null) {
+      assert deletedDocsRef != null;
+      deletedDocsRef.decRef();
       deletedDocs = null;
-      deletedDocsDirty = false;
-      undeleteAll = true;
+      deletedDocsRef = null;
+    } else {
+      assert deletedDocsRef == null;
+    }
   }
 
   List files() throws IOException {
@@ -844,17 +1037,7 @@
   protected synchronized byte[] getNorms(String field) throws IOException {
     Norm norm = (Norm) norms.get(field);
     if (norm == null) return null;  // not indexed, or norms not stored
-    synchronized(norm) {
-      if (norm.bytes == null) {                     // value not yet read
-        byte[] bytes = new byte[maxDoc()];
-        norms(field, bytes, 0);
-        norm.bytes = bytes;                         // cache it
-        // it's OK to close the underlying IndexInput as we have cached the
-        // norms and will never read them again.
-        norm.close();
-      }
-      return norm.bytes;
-    }
+    return norm.bytes();
   }
 
   // returns fake norms if norms aren't available
@@ -871,10 +1054,8 @@
     if (norm == null)                             // not an indexed field
       return;
 
-    norm.dirty = true;                            // mark it dirty
     normsDirty = true;
-
-    norms(field)[doc] = value;                    // set the value
+    norm.copyOnWrite()[doc] = value;                    // set the value
   }
 
   /** Read norms into a pre-allocated array. */
@@ -887,24 +1068,8 @@
       System.arraycopy(fakeNorms(), 0, bytes, offset, maxDoc());
       return;
     }
-    
-    synchronized(norm) {
-      if (norm.bytes != null) {                     // can copy from cache
-        System.arraycopy(norm.bytes, 0, bytes, offset, maxDoc());
-        return;
-      }
-
-    // Read from disk.  norm.in may be shared across  multiple norms and
-    // should only be used in a synchronized context.
-      IndexInput normStream;
-      if (norm.useSingleNormStream) {
-        normStream = singleNormStream;
-      } else {
-        normStream = norm.in;
-      }
-      normStream.seek(norm.normSeek);
-      normStream.readBytes(bytes, offset, maxDoc());
-    }
+  
+    norm.bytes(bytes, offset, maxDoc());
   }
 
 
@@ -932,8 +1097,11 @@
 
         if (singleNormFile) {
           normSeek = nextNormSeek;
-          if (singleNormStream==null) {
+          if (singleNormStream == null) {
             singleNormStream = d.openInput(fileName, readBufferSize);
+            singleNormRef = new Ref();
+          } else {
+            singleNormRef.incRef();
           }
           // All norms in the .nrm file can share a single IndexInput since
           // they are only used in a synchronized context.
@@ -944,7 +1112,7 @@
           normInput = d.openInput(fileName);
         }
 
-        norms.put(fi.name, new Norm(normInput, singleNormFile, fi.number, normSeek));
+        norms.put(fi.name, new Norm(normInput, fi.number, normSeek));
         nextNormSeek += maxDoc; // increment also if some norms are separate
       }
     }
@@ -964,11 +1132,11 @@
     }
     return true;
   }
-  
+
   // for testing only
   boolean normsClosed(String field) {
-      Norm norm = (Norm) norms.get(field);
-      return norm.refCount == 0;
+    Norm norm = (Norm) norms.get(field);
+    return norm.refCount == 0;
   }
 
   /**

Modified: lucene/java/trunk/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java?rev=739238&r1=739237&r2=739238&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java Fri Jan 30 11:12:48 2009
@@ -80,6 +80,6 @@
   }
 
   public String toString() {
-      return "SingleInstanceLock: " + lockName;
+    return super.toString() + ": " + lockName;
   }
 }

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/BitVector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/BitVector.java?rev=739238&r1=739237&r2=739238&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/BitVector.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/BitVector.java Fri Jan 30 11:12:48 2009
@@ -35,7 +35,7 @@
 
   @version $Id$
   */
-public final class BitVector {
+public final class BitVector implements Cloneable {
 
   private byte[] bits;
   private int size;
@@ -46,7 +46,19 @@
     size = n;
     bits = new byte[(size >> 3) + 1];
   }
-
+  
+  BitVector(byte[] bits, int size, int count) {
+    this.bits = bits;
+    this.size = size;
+    this.count = count;
+  }
+  
+  public Object clone() {
+    byte[] copyBits = new byte[bits.length];
+    System.arraycopy(bits, 0, copyBits, 0, bits.length);
+    return new BitVector(copyBits, size, count);
+  }
+  
   /** Sets the value of <code>bit</code> to one. */
   public final void set(int bit) {
     if (bit >= size) {

Added: lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderClone.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderClone.java?rev=739238&view=auto
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderClone.java (added)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderClone.java Fri Jan 30 11:12:48 2009
@@ -0,0 +1,406 @@
+package org.apache.lucene.index;
+
+/**
+ * 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 org.apache.lucene.index.SegmentReader.Norm;
+import org.apache.lucene.search.Similarity;
+import org.apache.lucene.analysis.SimpleAnalyzer;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.LockObtainFailedException;
+import org.apache.lucene.store.MockRAMDirectory;
+import org.apache.lucene.util.LuceneTestCase;
+
+/**
+ * Tests cloning multiple types of readers, modifying the deletedDocs and norms
+ * and verifies copy on write semantics of the deletedDocs and norms is
+ * implemented properly
+ */
+public class TestIndexReaderClone extends LuceneTestCase {
+
+  public void testCloneReadOnlySegmentReader() throws Exception {
+    final Directory dir1 = new MockRAMDirectory();
+
+    TestIndexReaderReopen.createIndex(dir1, false);
+    IndexReader reader = IndexReader.open(dir1);
+    IndexReader readOnlyReader = (IndexReader) reader.clone(true);
+    if (!isReadOnly(readOnlyReader)) {
+      fail("reader isn't read only");
+    }
+    if (deleteWorked(1, readOnlyReader)) {
+      fail("deleting from the original should not have worked");
+    }
+    reader.close();
+    readOnlyReader.close();
+    dir1.close();
+  }
+
+  // open non-readOnly reader1, clone to non-readOnly
+  // reader2, make sure we can change reader2
+  public void testCloneNoChangesStillReadOnly() throws Exception {
+    final Directory dir1 = new MockRAMDirectory();
+
+    TestIndexReaderReopen.createIndex(dir1, true);
+    IndexReader r1 = IndexReader.open(dir1, false);
+    DirectoryIndexReader r2 = (DirectoryIndexReader) r1.clone(false);
+    if (!deleteWorked(1, r2)) {
+      fail("deleting from the cloned should have worked");
+    }
+    r1.close();
+    r2.close();
+    dir1.close();
+  }
+  
+  // open non-readOnly reader1, clone to non-readOnly
+  // reader2, make sure we can change reader1
+  public void testCloneWriteToOrig() throws Exception {
+    final Directory dir1 = new MockRAMDirectory();
+
+    TestIndexReaderReopen.createIndex(dir1, true);
+    IndexReader r1 = IndexReader.open(dir1, false);
+    DirectoryIndexReader r2 = (DirectoryIndexReader) r1.clone(false);
+    if (!deleteWorked(1, r1)) {
+      fail("deleting from the original should have worked");
+    }
+    r1.close();
+    r2.close();
+    dir1.close();
+  }
+  
+  // open non-readOnly reader1, clone to non-readOnly
+  // reader2, make sure we can change reader2
+  public void testCloneWriteToClone() throws Exception {
+    final Directory dir1 = new MockRAMDirectory();
+
+    TestIndexReaderReopen.createIndex(dir1, true);
+    IndexReader r1 = IndexReader.open(dir1, false);
+    DirectoryIndexReader r2 = (DirectoryIndexReader) r1.clone(false);
+    if (!deleteWorked(1, r2)) {
+      fail("deleting from the original should have worked");
+    }
+    // should fail because reader1 holds the write lock
+    assertTrue("first reader should not be able to delete", !deleteWorked(1, r1));
+    r2.close();
+    // should fail because we are now stale (reader1
+    // committed changes)
+    assertTrue("first reader should not be able to delete", !deleteWorked(1, r1));
+    r1.close();
+
+    dir1.close();
+  }
+  
+  // create single-segment index, open non-readOnly
+  // SegmentReader, add docs, reopen to multireader, then do
+  // delete
+  public void testReopenSegmentReaderToMultiReader() throws Exception {
+    final Directory dir1 = new MockRAMDirectory();
+
+    TestIndexReaderReopen.createIndex(dir1, false);
+    IndexReader reader1 = IndexReader.open(dir1, false);
+
+    TestIndexReaderReopen.modifyIndex(5, dir1);
+    
+    IndexReader reader2 = reader1.reopen();
+    assertTrue(reader1 != reader2);
+
+    assertTrue(deleteWorked(1, reader2));
+    reader1.close();
+    reader2.close();
+    dir1.close();
+  }
+
+  // open non-readOnly reader1, clone to readOnly reader2
+  public void testCloneWriteableToReadOnly() throws Exception {
+    final Directory dir1 = new MockRAMDirectory();
+
+    TestIndexReaderReopen.createIndex(dir1, true);
+    IndexReader reader = IndexReader.open(dir1, false);
+    DirectoryIndexReader readOnlyReader = (DirectoryIndexReader) reader.clone(true);
+    if (!isReadOnly(readOnlyReader)) {
+      fail("reader isn't read only");
+    }
+    if (deleteWorked(1, readOnlyReader)) {
+      fail("deleting from the original should not have worked");
+    }
+    // this readonly reader shouldn't have a write lock
+    if (readOnlyReader.hasChanges) {
+      fail("readOnlyReader has a write lock");
+    }
+    reader.close();
+    readOnlyReader.close();
+    dir1.close();
+  }
+
+  // open non-readOnly reader1, reopen to readOnly reader2
+  public void testReopenWriteableToReadOnly() throws Exception {
+    final Directory dir1 = new MockRAMDirectory();
+
+    TestIndexReaderReopen.createIndex(dir1, true);
+    IndexReader reader = IndexReader.open(dir1, false);
+    final int docCount = reader.numDocs();
+    assertTrue(deleteWorked(1, reader));
+    assertEquals(docCount-1, reader.numDocs());
+
+    DirectoryIndexReader readOnlyReader = (DirectoryIndexReader) reader.reopen(true);
+    if (!isReadOnly(readOnlyReader)) {
+      fail("reader isn't read only");
+    }
+    assertFalse(deleteWorked(1, readOnlyReader));
+    assertEquals(docCount-1, readOnlyReader.numDocs());
+    reader.close();
+    readOnlyReader.close();
+    dir1.close();
+  }
+
+  // open readOnly reader1, clone to non-readOnly reader2
+  public void testCloneReadOnlyToWriteable() throws Exception {
+    final Directory dir1 = new MockRAMDirectory();
+
+    TestIndexReaderReopen.createIndex(dir1, true);
+    IndexReader reader1 = IndexReader.open(dir1, true);
+
+    DirectoryIndexReader reader2 = (DirectoryIndexReader) reader1.clone(false);
+    if (isReadOnly(reader2)) {
+      fail("reader should not be read only");
+    }
+    assertFalse("deleting from the original reader should not have worked", deleteWorked(1, reader1));
+    // this readonly reader shouldn't yet have a write lock
+    if (reader2.hasChanges) {
+      fail("cloned reader should not have write lock");
+    }
+    assertTrue("deleting from the cloned reader should have worked", deleteWorked(1, reader2));
+    reader1.close();
+    reader2.close();
+    dir1.close();
+  }
+
+  // open non-readOnly reader1 on multi-segment index, then
+  // optimize the index, then clone to readOnly reader2
+  public void testReadOnlyCloneAfterOptimize() throws Exception {
+    final Directory dir1 = new MockRAMDirectory();
+
+    TestIndexReaderReopen.createIndex(dir1, true);
+    IndexReader reader1 = IndexReader.open(dir1, false);
+    IndexWriter w = new IndexWriter(dir1, new SimpleAnalyzer(), IndexWriter.MaxFieldLength.LIMITED);
+    w.optimize();
+    w.close();
+    IndexReader reader2 = reader1.clone(true);
+    assertTrue(isReadOnly(reader2));
+    reader1.close();
+    reader2.close();
+    dir1.close();
+  }
+  
+  private static boolean deleteWorked(int doc, IndexReader r) {
+    boolean exception = false;
+    try {
+      // trying to delete from the original reader should throw an exception
+      r.deleteDocument(doc);
+    } catch (Exception ex) {
+      exception = true;
+    }
+    return !exception;
+  }
+  
+  public void testCloneReadOnlyMultiSegmentReader() throws Exception {
+    final Directory dir1 = new MockRAMDirectory();
+
+    TestIndexReaderReopen.createIndex(dir1, true);
+    IndexReader reader = IndexReader.open(dir1);
+    IndexReader readOnlyReader = (IndexReader) reader.clone(true);
+    if (!isReadOnly(readOnlyReader)) {
+      fail("reader isn't read only");
+    }
+    reader.close();
+    readOnlyReader.close();
+    dir1.close();
+  }
+
+  public static boolean isReadOnly(IndexReader r) {
+    if (r instanceof ReadOnlySegmentReader
+        || r instanceof ReadOnlyMultiSegmentReader)
+      return true;
+    return false;
+  }
+
+  public void testParallelReader() throws Exception {
+    final Directory dir1 = new MockRAMDirectory();
+    TestIndexReaderReopen.createIndex(dir1, true);
+    final Directory dir2 = new MockRAMDirectory();
+    TestIndexReaderReopen.createIndex(dir2, true);
+    IndexReader r1 = IndexReader.open(dir1);
+    IndexReader r2 = IndexReader.open(dir2);
+
+    ParallelReader pr1 = new ParallelReader();
+    pr1.add(r1);
+    pr1.add(r2);
+
+    performDefaultTests(pr1);
+    pr1.close();
+    dir1.close();
+    dir2.close();
+  }
+
+  /**
+   * 1. Get a norm from the original reader 2. Clone the original reader 3.
+   * Delete a document and set the norm of the cloned reader 4. Verify the norms
+   * are not the same on each reader 5. Verify the doc deleted is only in the
+   * cloned reader 6. Try to delete a document in the original reader, an
+   * exception should be thrown
+   * 
+   * @param r1 IndexReader to perform tests on
+   * @throws Exception
+   */
+  private void performDefaultTests(IndexReader r1) throws Exception {
+    float norm1 = Similarity.decodeNorm(r1.norms("field1")[4]);
+
+    IndexReader pr1Clone = (IndexReader) r1.clone();
+    pr1Clone.deleteDocument(10);
+    pr1Clone.setNorm(4, "field1", 0.5f);
+    assertTrue(Similarity.decodeNorm(r1.norms("field1")[4]) == norm1);
+    assertTrue(Similarity.decodeNorm(pr1Clone.norms("field1")[4]) != norm1);
+
+    assertTrue(!r1.isDeleted(10));
+    assertTrue(pr1Clone.isDeleted(10));
+
+    // try to update the original reader, which should throw an exception
+    try {
+      r1.deleteDocument(11);
+      fail("Tried to delete doc 11 and an exception should have been thrown");
+    } catch (Exception exception) {
+      // expectted
+    }
+    pr1Clone.close();
+  }
+
+  public void testMixedReaders() throws Exception {
+    final Directory dir1 = new MockRAMDirectory();
+    TestIndexReaderReopen.createIndex(dir1, true);
+    final Directory dir2 = new MockRAMDirectory();
+    TestIndexReaderReopen.createIndex(dir2, true);
+    IndexReader r1 = IndexReader.open(dir1);
+    IndexReader r2 = IndexReader.open(dir2);
+
+    MultiReader multiReader = new MultiReader(new IndexReader[] { r1, r2 });
+    performDefaultTests(multiReader);
+    multiReader.close();
+    dir1.close();
+    dir2.close();
+  }
+
+  public void testSegmentReaderUndeleteall() throws Exception {
+    final Directory dir1 = new MockRAMDirectory();
+    TestIndexReaderReopen.createIndex(dir1, false);
+    SegmentReader origSegmentReader = (SegmentReader) IndexReader.open(dir1);
+    origSegmentReader.deleteDocument(10);
+    assertDelDocsRefCountEquals(1, origSegmentReader);
+    origSegmentReader.undeleteAll();
+    assertNull(origSegmentReader.deletedDocsRef);
+    origSegmentReader.close();
+    // need to test norms?
+    dir1.close();
+  }
+  
+  public void testSegmentReaderCloseReferencing() throws Exception {
+    final Directory dir1 = new MockRAMDirectory();
+    TestIndexReaderReopen.createIndex(dir1, false);
+    SegmentReader origSegmentReader = (SegmentReader) IndexReader.open(dir1);
+    origSegmentReader.deleteDocument(1);
+    origSegmentReader.setNorm(4, "field1", 0.5f);
+
+    SegmentReader clonedSegmentReader = (SegmentReader) origSegmentReader
+        .clone();
+    assertDelDocsRefCountEquals(2, origSegmentReader);
+    origSegmentReader.close();
+    assertDelDocsRefCountEquals(1, origSegmentReader);
+    // check the norm refs
+    Norm norm = (Norm) clonedSegmentReader.norms.get("field1");
+    assertEquals(1, norm.bytesRef().refCount());
+    clonedSegmentReader.close();
+    dir1.close();
+  }
+  
+  public void testSegmentReaderDelDocsReferenceCounting() throws Exception {
+    final Directory dir1 = new MockRAMDirectory();
+    TestIndexReaderReopen.createIndex(dir1, false);
+    SegmentReader origSegmentReader = (SegmentReader) IndexReader.open(dir1);
+    // deletedDocsRef should be null because nothing has updated yet
+    assertNull(origSegmentReader.deletedDocsRef);
+
+    // we deleted a document, so there is now a deletedDocs bitvector and a
+    // reference to it
+    origSegmentReader.deleteDocument(1);
+    assertDelDocsRefCountEquals(1, origSegmentReader);
+
+    // the cloned segmentreader should have 2 references, 1 to itself, and 1 to
+    // the original segmentreader
+    SegmentReader clonedSegmentReader = (SegmentReader) origSegmentReader
+        .clone();
+    assertDelDocsRefCountEquals(2, origSegmentReader);
+    // deleting a document creates a new deletedDocs bitvector, the refs goes to
+    // 1
+    clonedSegmentReader.deleteDocument(2);
+    assertDelDocsRefCountEquals(1, origSegmentReader);
+    assertDelDocsRefCountEquals(1, clonedSegmentReader);
+
+    // make sure the deletedocs objects are different (copy
+    // on write)
+    assertTrue(origSegmentReader.deletedDocs != clonedSegmentReader.deletedDocs);
+
+    assertDocDeleted(origSegmentReader, clonedSegmentReader, 1);
+    assertTrue(!origSegmentReader.isDeleted(2)); // doc 2 should not be deleted
+                                                  // in original segmentreader
+    assertTrue(clonedSegmentReader.isDeleted(2)); // doc 2 should be deleted in
+                                                  // cloned segmentreader
+
+    // deleting a doc from the original segmentreader should throw an exception
+    try {
+      origSegmentReader.deleteDocument(4);
+      fail("expected exception");
+    } catch (LockObtainFailedException lbfe) {
+      // expected
+    }
+
+    origSegmentReader.close();
+    // try closing the original segment reader to see if it affects the
+    // clonedSegmentReader
+    clonedSegmentReader.deleteDocument(3);
+    clonedSegmentReader.flush();
+    assertDelDocsRefCountEquals(1, clonedSegmentReader);
+
+    // test a reopened reader
+    SegmentReader reopenedSegmentReader = (SegmentReader) clonedSegmentReader
+        .reopen();
+    SegmentReader cloneSegmentReader2 = (SegmentReader) reopenedSegmentReader
+        .clone();
+    assertDelDocsRefCountEquals(2, cloneSegmentReader2);
+    clonedSegmentReader.close();
+    reopenedSegmentReader.close();
+    cloneSegmentReader2.close();
+
+    dir1.close();
+  }
+
+  private void assertDocDeleted(SegmentReader reader, SegmentReader reader2,
+      int doc) {
+    assertEquals(reader.isDeleted(doc), reader2.isDeleted(doc));
+  }
+
+  private void assertDelDocsRefCountEquals(int refCount, SegmentReader reader) {
+    assertEquals(refCount, reader.deletedDocsRef.refCount());
+  }
+}

Propchange: lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderClone.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderCloneNorms.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderCloneNorms.java?rev=739238&view=auto
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderCloneNorms.java (added)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderCloneNorms.java Fri Jan 30 11:12:48 2009
@@ -0,0 +1,333 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.File;
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Index;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.index.SegmentReader.Norm;
+import org.apache.lucene.search.DefaultSimilarity;
+import org.apache.lucene.search.Similarity;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.store.MockRAMDirectory;
+import org.apache.lucene.util.LuceneTestCase;
+
+/**
+ * Tests cloning IndexReader norms
+ */
+public class TestIndexReaderCloneNorms extends LuceneTestCase {
+
+  private class SimilarityOne extends DefaultSimilarity {
+    public float lengthNorm(String fieldName, int numTerms) {
+      return 1;
+    }
+  }
+
+  private static final int NUM_FIELDS = 10;
+
+  private Similarity similarityOne;
+
+  private Analyzer anlzr;
+
+  private int numDocNorms;
+
+  private ArrayList norms;
+
+  private ArrayList modifiedNorms;
+
+  private float lastNorm = 0;
+
+  private float normDelta = (float) 0.001;
+
+  public TestIndexReaderCloneNorms(String s) {
+    super(s);
+  }
+
+  protected void setUp() throws Exception {
+    super.setUp();
+    similarityOne = new SimilarityOne();
+    anlzr = new StandardAnalyzer();
+  }
+  
+  /**
+   * Test that norms values are preserved as the index is maintained. Including
+   * separate norms. Including merging indexes with seprate norms. Including
+   * optimize.
+   */
+  public void testNorms() throws IOException {
+    // tmp dir
+    String tempDir = System.getProperty("java.io.tmpdir");
+    if (tempDir == null) {
+      throw new IOException("java.io.tmpdir undefined, cannot run test");
+    }
+
+    // test with a single index: index1
+    File indexDir1 = new File(tempDir, "lucenetestindex1");
+    Directory dir1 = FSDirectory.getDirectory(indexDir1);
+    IndexWriter.unlock(dir1);
+
+    norms = new ArrayList();
+    modifiedNorms = new ArrayList();
+
+    createIndex(dir1);
+    doTestNorms(dir1);
+
+    // test with a single index: index2
+    ArrayList norms1 = norms;
+    ArrayList modifiedNorms1 = modifiedNorms;
+    int numDocNorms1 = numDocNorms;
+
+    norms = new ArrayList();
+    modifiedNorms = new ArrayList();
+    numDocNorms = 0;
+
+    File indexDir2 = new File(tempDir, "lucenetestindex2");
+    Directory dir2 = FSDirectory.getDirectory(indexDir2);
+
+    createIndex(dir2);
+    doTestNorms(dir2);
+
+    // add index1 and index2 to a third index: index3
+    File indexDir3 = new File(tempDir, "lucenetestindex3");
+    Directory dir3 = FSDirectory.getDirectory(indexDir3);
+
+    createIndex(dir3);
+    IndexWriter iw = new IndexWriter(dir3, anlzr, false,
+        IndexWriter.MaxFieldLength.LIMITED);
+    iw.setMaxBufferedDocs(5);
+    iw.setMergeFactor(3);
+    iw.addIndexes(new Directory[] { dir1, dir2 });
+    iw.close();
+
+    norms1.addAll(norms);
+    norms = norms1;
+    modifiedNorms1.addAll(modifiedNorms);
+    modifiedNorms = modifiedNorms1;
+    numDocNorms += numDocNorms1;
+
+    // test with index3
+    verifyIndex(dir3);
+    doTestNorms(dir3);
+
+    // now with optimize
+    iw = new IndexWriter(dir3, anlzr, false, IndexWriter.MaxFieldLength.LIMITED);
+    iw.setMaxBufferedDocs(5);
+    iw.setMergeFactor(3);
+    iw.optimize();
+    iw.close();
+    verifyIndex(dir3);
+
+    dir1.close();
+    dir2.close();
+    dir3.close();
+  }
+
+  // try cloning and reopening the norms
+  private void doTestNorms(Directory dir) throws IOException {
+    addDocs(dir, 12, true);
+    IndexReader ir = IndexReader.open(dir);
+    verifyIndex(ir);
+    modifyNormsForF1(ir);
+    IndexReader irc = (IndexReader) ir.clone();// IndexReader.open(dir);//ir.clone();
+    verifyIndex(irc);
+
+    modifyNormsForF1(irc);
+
+    IndexReader irc3 = (IndexReader) irc.clone();
+    verifyIndex(irc3);
+    modifyNormsForF1(irc3);
+    verifyIndex(irc3);
+    irc3.flush();
+    irc3.close();
+  }
+  
+  public void testNormsClose() throws IOException { 
+    Directory dir1 = new MockRAMDirectory(); 
+    TestIndexReaderReopen.createIndex(dir1, false);
+    SegmentReader reader1 = (SegmentReader) IndexReader.open(dir1);
+    reader1.norms("field1");
+    Norm r1norm = (Norm)reader1.norms.get("field1");
+    SegmentReader.Ref r1BytesRef = r1norm.bytesRef();
+    SegmentReader reader2 = (SegmentReader)reader1.clone();
+    assertEquals(2, r1norm.bytesRef().refCount());
+    reader1.close();
+    assertEquals(1, r1BytesRef.refCount());
+    reader2.norms("field1");
+    reader2.close();
+    dir1.close();
+  }
+  
+  public void testNormsRefCounting() throws IOException { 
+    Directory dir1 = new MockRAMDirectory(); 
+    TestIndexReaderReopen.createIndex(dir1, false);
+    SegmentReader reader1 = (SegmentReader) IndexReader.open(dir1);
+        
+    SegmentReader reader2C = (SegmentReader)reader1.clone();
+    reader2C.norms("field1"); // load the norms for the field
+    Norm reader2CNorm = (Norm)reader2C.norms.get("field1");
+    assertTrue("reader2CNorm.bytesRef()=" + reader2CNorm.bytesRef(), reader2CNorm.bytesRef().refCount() == 2);
+    
+    
+    
+    SegmentReader reader3C = (SegmentReader)reader2C.clone();
+    Norm reader3CCNorm = (Norm)reader3C.norms.get("field1");
+    assertEquals(3, reader3CCNorm.bytesRef().refCount());
+    
+    // edit a norm and the refcount should be 1
+    SegmentReader reader4C = (SegmentReader)reader3C.clone();
+    assertEquals(4, reader3CCNorm.bytesRef().refCount());
+    reader4C.setNorm(5, "field1", 0.33f);
+    
+    // generate a cannot update exception in reader1
+    try {
+      reader3C.setNorm(1, "field1", 0.99f);
+      fail("did not hit expected exception");
+    } catch (Exception ex) {
+      // expected
+    }
+    
+    // norm values should be different 
+    assertTrue(Similarity.decodeNorm(reader3C.norms("field1")[5]) != Similarity.decodeNorm(reader4C.norms("field1")[5]));
+    Norm reader4CCNorm = (Norm)reader4C.norms.get("field1");
+    assertEquals(3, reader3CCNorm.bytesRef().refCount());
+    assertEquals(1, reader4CCNorm.bytesRef().refCount());
+        
+    SegmentReader reader5C = (SegmentReader)reader4C.clone();
+    Norm reader5CCNorm = (Norm)reader5C.norms.get("field1");
+    reader5C.setNorm(5, "field1", 0.7f);
+    assertEquals(1, reader5CCNorm.bytesRef().refCount());    
+
+    reader5C.close();
+    reader4C.close();
+    reader3C.close();
+    reader2C.close();
+    reader1.close();
+    dir1.close();
+  }
+  
+  private void createIndex(Directory dir) throws IOException {
+    IndexWriter iw = new IndexWriter(dir, anlzr, true,
+        IndexWriter.MaxFieldLength.LIMITED);
+    iw.setMaxBufferedDocs(5);
+    iw.setMergeFactor(3);
+    iw.setSimilarity(similarityOne);
+    iw.setUseCompoundFile(true);
+    iw.close();
+  }
+
+  private void modifyNormsForF1(Directory dir) throws IOException {
+    IndexReader ir = IndexReader.open(dir);
+    modifyNormsForF1(ir);
+  }
+
+  private void modifyNormsForF1(IndexReader ir) throws IOException {
+    int n = ir.maxDoc();
+    // System.out.println("modifyNormsForF1 maxDoc: "+n);
+    for (int i = 0; i < n; i += 3) { // modify for every third doc
+      int k = (i * 3) % modifiedNorms.size();
+      float origNorm = ((Float) modifiedNorms.get(i)).floatValue();
+      float newNorm = ((Float) modifiedNorms.get(k)).floatValue();
+      // System.out.println("Modifying: for "+i+" from "+origNorm+" to
+      // "+newNorm);
+      // System.out.println(" and: for "+k+" from "+newNorm+" to "+origNorm);
+      modifiedNorms.set(i, new Float(newNorm));
+      modifiedNorms.set(k, new Float(origNorm));
+      ir.setNorm(i, "f" + 1, newNorm);
+      ir.setNorm(k, "f" + 1, origNorm);
+      // System.out.println("setNorm i: "+i);
+      // break;
+    }
+    // ir.close();
+  }
+
+  private void verifyIndex(Directory dir) throws IOException {
+    IndexReader ir = IndexReader.open(dir);
+    verifyIndex(ir);
+    ir.close();
+  }
+
+  private void verifyIndex(IndexReader ir) throws IOException {
+    for (int i = 0; i < NUM_FIELDS; i++) {
+      String field = "f" + i;
+      byte b[] = ir.norms(field);
+      assertEquals("number of norms mismatches", numDocNorms, b.length);
+      ArrayList storedNorms = (i == 1 ? modifiedNorms : norms);
+      for (int j = 0; j < b.length; j++) {
+        float norm = Similarity.decodeNorm(b[j]);
+        float norm1 = ((Float) storedNorms.get(j)).floatValue();
+        assertEquals("stored norm value of " + field + " for doc " + j + " is "
+            + norm + " - a mismatch!", norm, norm1, 0.000001);
+      }
+    }
+  }
+
+  private void addDocs(Directory dir, int ndocs, boolean compound)
+      throws IOException {
+    IndexWriter iw = new IndexWriter(dir, anlzr, false,
+        IndexWriter.MaxFieldLength.LIMITED);
+    iw.setMaxBufferedDocs(5);
+    iw.setMergeFactor(3);
+    iw.setSimilarity(similarityOne);
+    iw.setUseCompoundFile(compound);
+    for (int i = 0; i < ndocs; i++) {
+      iw.addDocument(newDoc());
+    }
+    iw.close();
+  }
+
+  // create the next document
+  private Document newDoc() {
+    Document d = new Document();
+    float boost = nextNorm();
+    for (int i = 0; i < 10; i++) {
+      Field f = new Field("f" + i, "v" + i, Store.NO, Index.NOT_ANALYZED);
+      f.setBoost(boost);
+      d.add(f);
+    }
+    return d;
+  }
+
+  // return unique norm values that are unchanged by encoding/decoding
+  private float nextNorm() {
+    float norm = lastNorm + normDelta;
+    do {
+      float norm1 = Similarity.decodeNorm(Similarity.encodeNorm(norm));
+      if (norm1 > lastNorm) {
+        // System.out.println(norm1+" > "+lastNorm);
+        norm = norm1;
+        break;
+      }
+      norm += normDelta;
+    } while (true);
+    norms.add(numDocNorms, new Float(norm));
+    modifiedNorms.add(numDocNorms, new Float(norm));
+    // System.out.println("creating norm("+numDocNorms+"): "+norm);
+    numDocNorms++;
+    lastNorm = (norm > 10 ? 0 : norm); // there's a limit to how many distinct
+                                        // values can be stored in a ingle byte
+    return norm;
+  }
+}

Propchange: lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReaderCloneNorms.java
------------------------------------------------------------------------------
    svn:eol-style = native