You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by do...@apache.org on 2011/11/17 13:30:29 UTC

svn commit: r1203165 - in /lucene/dev/branches/branch_3x: ./ lucene/ lucene/contrib/ lucene/contrib/facet/docs/ lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/ lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/directory/ lu...

Author: doronc
Date: Thu Nov 17 12:30:29 2011
New Revision: 1203165

URL: http://svn.apache.org/viewvc?rev=1203165&view=rev
Log:
LUCENE-3573: TaxonomyReader.refresh() was broken if taxonomy index recreated since taxo reader last opened or refreshed. (merge from trunk)

Added:
    lucene/dev/branches/branch_3x/lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/InconsistentTaxonomyException.java
      - copied unchanged from r1202754, lucene/dev/trunk/modules/facet/src/java/org/apache/lucene/facet/taxonomy/InconsistentTaxonomyException.java
Modified:
    lucene/dev/branches/branch_3x/   (props changed)
    lucene/dev/branches/branch_3x/lucene/   (props changed)
    lucene/dev/branches/branch_3x/lucene/contrib/CHANGES.txt
    lucene/dev/branches/branch_3x/lucene/contrib/facet/docs/userguide.html
    lucene/dev/branches/branch_3x/lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyReader.java
    lucene/dev/branches/branch_3x/lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java
    lucene/dev/branches/branch_3x/lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
    lucene/dev/branches/branch_3x/lucene/contrib/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyReader.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java
    lucene/dev/branches/branch_3x/solr/   (props changed)

Modified: lucene/dev/branches/branch_3x/lucene/contrib/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/CHANGES.txt?rev=1203165&r1=1203164&r2=1203165&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/CHANGES.txt (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/CHANGES.txt Thu Nov 17 12:30:29 2011
@@ -73,6 +73,11 @@ API Changes
    
  * LUCENE-3579: DirectoryTaxonomyWriter throws AlreadyClosedException if it was
    closed, but any of its API methods are called. (Shai Erera)
+   
+ * LUCENE-3573: TaxonomyReader.refresh() signature was modified from void to 
+   boolean, now returning an indication if any change was detected. It 
+   throws a new InconsistentTaxonomyException if the taxonomy was recreated
+   since TaxonomyReader was last opened or refreshed. (Doron Cohen)     
 
 Bug Fixes
 
@@ -113,6 +118,12 @@ Bug Fixes
  * LUCENE-3542: Group expanded query terms to preserve parent boolean operator
    in StandartQueryParser. (Simon Willnauer) 
 
+ * LUCENE-3573: TaxonomyReader.refresh() was broken in case that the taxonomy was 
+   recreated since the taxonomy reader was last refreshed or opened. TR.refresh()
+   now detects this situation and throws an InconsistentTaxonomyException. 
+   When obtaining such an exception the application should open a new taxonomy 
+   reader. Old taxonomy reader should be closed, once not more used.  (Doron Cohen)
+
 API Changes
  
  * LUCENE-3436: Add SuggestMode to the spellchecker, so you can specify the strategy

Modified: lucene/dev/branches/branch_3x/lucene/contrib/facet/docs/userguide.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/facet/docs/userguide.html?rev=1203165&r1=1203164&r2=1203165&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/facet/docs/userguide.html (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/facet/docs/userguide.html Thu Nov 17 12:30:29 2011
@@ -779,6 +779,12 @@ example) so a thread which is in the mid
 <code>TaxonomyReader</code>, however, we are guaranteed that existing categories are never deleted or modified - 
 the only thing that can happen is that new categories are added. Since search threads do not care if new categories 
 are added in the middle of a search, there is no reason to keep around the old object, and the new one suffices.
+<br><b>However</b>, if the taxonomy index was recreated since the <code>TaxonomyReader</code> was opened or
+refreshed, this assumption (that categories are forevr) no longer holds, and <code>refresh()</code> will 
+throw an <code>InconsistentTaxonomyException</code>, guiding the application to open 
+a new <code>TaxonomyReader</code> for up-to-date taxonomy data. (Old one can
+be closed as soon as it is no more used.)
+
 
 </body>
 </html>

Modified: lucene/dev/branches/branch_3x/lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyReader.java?rev=1203165&r1=1203164&r2=1203165&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyReader.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyReader.java Thu Nov 17 12:30:29 2011
@@ -120,6 +120,14 @@ public interface TaxonomyReader extends 
    * faceted search, the taxonomy reader's refresh() should be called only after
    * a reopen() of the main index.
    * <P>
+   * Refreshing the taxonomy might fail in some cases, for example 
+   * if the taxonomy was recreated since this instance was opened or last refreshed.
+   * In this case an {@link InconsistentTaxonomyException} is thrown,
+   * suggesting that in order to obtain up-to-date taxonomy data a new
+   * {@link TaxonomyReader} should be opened. Note: This {@link TaxonomyReader} 
+   * instance remains unchanged and usable in this case, and the application can
+   * continue to use it, and should still {@link #close()} when no longer needed.  
+   * <P>
    * It should be noted that refresh() is similar in purpose to
    * IndexReader.reopen(), but the two methods behave differently. refresh()
    * refreshes the existing TaxonomyReader object, rather than opening a new one
@@ -129,8 +137,9 @@ public interface TaxonomyReader extends 
    * of the taxonomy open - refreshing the taxonomy to the newest data and using
    * this new snapshots in all threads (whether new or old) is fine. This saves
    * us needing to keep multiple copies of the taxonomy open in memory.
+   * @return true if anything has changed, false otherwise. 
    */
-  public void refresh() throws IOException;
+  public boolean refresh() throws IOException, InconsistentTaxonomyException;
   
   /**
    * getParent() returns the ordinal of the parent category of the category

Modified: lucene/dev/branches/branch_3x/lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java?rev=1203165&r1=1203164&r2=1203165&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyReader.java Thu Nov 17 12:30:29 2011
@@ -10,6 +10,7 @@ import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.apache.lucene.facet.taxonomy.InconsistentTaxonomyException;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexReader;
@@ -326,7 +327,7 @@ public class DirectoryTaxonomyReader imp
   // Note that refresh() is synchronized (it is the only synchronized
   // method in this class) to ensure that it never gets called concurrently
   // with itself.
-  public synchronized void refresh() throws IOException {
+  public synchronized boolean refresh() throws IOException, InconsistentTaxonomyException {
     ensureOpen();
     /*
      * Since refresh() can be a lengthy operation, it is very important that we
@@ -342,7 +343,24 @@ public class DirectoryTaxonomyReader imp
     // no other thread can be writing at this time (this method is the
     // only possible writer, and it is "synchronized" to avoid this case).
     IndexReader r2 = IndexReader.openIfChanged(indexReader);
-    if (r2 != null) {
+    if (r2 == null) {
+    	return false; // no changes, nothing to do
+    } 
+    
+    // validate that a refresh is valid at this point, i.e. that the taxonomy 
+    // was not recreated since this reader was last opened or refresshed.
+    String t1 = indexReader.getCommitUserData().get(DirectoryTaxonomyWriter.INDEX_CREATE_TIME);
+    String t2 = r2.getCommitUserData().get(DirectoryTaxonomyWriter.INDEX_CREATE_TIME);
+    if (t1==null) {
+    	if (t2!=null) {
+    		r2.close();
+    		throw new InconsistentTaxonomyException("Taxonomy was recreated at: "+t2);
+    	}
+    } else if (!t1.equals(t2)) {
+    	r2.close();
+    	throw new InconsistentTaxonomyException("Taxonomy was recreated at: "+t2+"  !=  "+t1);
+    }
+    
       IndexReader oldreader = indexReader;
       // we can close the old searcher, but need to synchronize this
       // so that we don't close it in the middle that another routine
@@ -385,8 +403,8 @@ public class DirectoryTaxonomyReader imp
           i.remove();
         }
       }
+      return true;
     }
-  }
 
   public void close() throws IOException {
     if (!closed) {

Modified: lucene/dev/branches/branch_3x/lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java?rev=1203165&r1=1203164&r2=1203165&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java Thu Nov 17 12:30:29 2011
@@ -9,6 +9,7 @@ import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.lucene.analysis.KeywordAnalyzer;
@@ -79,6 +80,14 @@ import org.apache.lucene.facet.taxonomy.
  */
 public class DirectoryTaxonomyWriter implements TaxonomyWriter {
 
+  /**
+   * Property name of user commit data that contains the creation time of a taxonomy index.
+   * <p>
+   * Applications making use of {@link TaxonomyWriter#commit(Map)} should not use this
+   * particular property name. 
+   */
+  public static final String INDEX_CREATE_TIME = "index.create.time";
+  
   private IndexWriter indexWriter;
   private int nextID;
   private char delimiter = Consts.DEFAULT_DELIMITER;
@@ -102,6 +111,12 @@ public class DirectoryTaxonomyWriter imp
   private int cacheMisses;
 
   /**
+   * When a taxonomy is created, we mark that its create time should be committed in the 
+   * next commit.
+   */
+  private String taxoIndexCreateTime = null;
+  
+  /**
    * setDelimiter changes the character that the taxonomy uses in its internal
    * storage as a delimiter between category components. Do not use this
    * method unless you really know what you are doing. It has nothing to do
@@ -168,6 +183,10 @@ public class DirectoryTaxonomyWriter imp
   throws CorruptIndexException, LockObtainFailedException,
   IOException {
 
+    if (!IndexReader.indexExists(directory) || openMode==OpenMode.CREATE) {
+      taxoIndexCreateTime = Long.toString(System.nanoTime());
+    }
+    
     indexWriter = openIndexWriter(directory, openMode);
     reader = null;
 
@@ -275,10 +294,17 @@ public class DirectoryTaxonomyWriter imp
    */
   public synchronized void close() throws CorruptIndexException, IOException {
     if (indexWriter != null) {
-      indexWriter.close();
-      indexWriter = null;
+      if (taxoIndexCreateTime != null) {
+        indexWriter.commit(combinedCommitData(null));
+        taxoIndexCreateTime = null;
+      }
+      doClose();
     }
-
+  }
+  
+  private void doClose() throws CorruptIndexException, IOException {
+    indexWriter.close();
+    indexWriter = null;
     closeResources();
   }
 
@@ -574,18 +600,40 @@ public class DirectoryTaxonomyWriter imp
    */ 
   public synchronized void commit() throws CorruptIndexException, IOException {
     ensureOpen();
-    indexWriter.commit();
+    if (taxoIndexCreateTime != null) {
+      indexWriter.commit(combinedCommitData(null));
+      taxoIndexCreateTime = null;
+    } else {
+      indexWriter.commit();
+    }
     refreshReader();
   }
 
   /**
+   * Combine original user data with that of the taxonomy creation time
+   */
+  private Map<String,String> combinedCommitData(Map<String,String> userData) {
+    Map<String,String> m = new HashMap<String, String>();
+    if (userData != null) {
+      m.putAll(userData);
+    }
+    m.put(INDEX_CREATE_TIME, taxoIndexCreateTime);
+    return m;
+  }
+  
+  /**
    * Like commit(), but also store properties with the index. These properties
    * are retrievable by {@link DirectoryTaxonomyReader#getCommitUserData}.
    * See {@link TaxonomyWriter#commit(Map)}. 
    */
   public synchronized void commit(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
     ensureOpen();
-    indexWriter.commit(commitUserData);
+    if (taxoIndexCreateTime != null) {
+      indexWriter.commit(combinedCommitData(commitUserData));
+      taxoIndexCreateTime = null;
+    } else {
+      indexWriter.commit(commitUserData);
+    }
     refreshReader();
   }
   
@@ -595,7 +643,12 @@ public class DirectoryTaxonomyWriter imp
    */
   public synchronized void prepareCommit() throws CorruptIndexException, IOException {
     ensureOpen();
-    indexWriter.prepareCommit();
+    if (taxoIndexCreateTime != null) {
+      indexWriter.prepareCommit(combinedCommitData(null));
+      taxoIndexCreateTime = null;
+    } else {
+      indexWriter.prepareCommit();
+    }
   }
 
   /**
@@ -604,7 +657,12 @@ public class DirectoryTaxonomyWriter imp
    */
   public synchronized void prepareCommit(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
     ensureOpen();
-    indexWriter.prepareCommit(commitUserData);
+    if (taxoIndexCreateTime != null) {
+      indexWriter.prepareCommit(combinedCommitData(commitUserData));
+      taxoIndexCreateTime = null;
+    } else {
+      indexWriter.prepareCommit(commitUserData);
+    }
   }
   
   /**
@@ -1017,7 +1075,7 @@ public class DirectoryTaxonomyWriter imp
     ensureOpen();
     indexWriter.rollback();
     // since IndexWriter.rollback() closes the IW instance, we should close too.
-    close();
+    doClose();
   }
   
 }

Modified: lucene/dev/branches/branch_3x/lucene/contrib/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyReader.java?rev=1203165&r1=1203164&r2=1203165&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyReader.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestDirectoryTaxonomyReader.java Thu Nov 17 12:30:29 2011
@@ -1,10 +1,17 @@
 package org.apache.lucene.facet.taxonomy.directory;
 
+import java.util.Random;
+
 import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.apache.lucene.facet.taxonomy.InconsistentTaxonomyException;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.junit.Test;
 
@@ -59,6 +66,36 @@ public class TestDirectoryTaxonomyReader
     dir.close();
   }
   
+  /**
+   * Test the boolean returned by TR.refresh
+   * @throws Exception
+   */
+  @Test
+  public void testReaderRefreshResult() throws Exception {
+    Directory dir = null;
+    DirectoryTaxonomyWriter ltw = null;
+    DirectoryTaxonomyReader ltr = null;
+    
+    try {
+      dir = newDirectory();
+      ltw = new DirectoryTaxonomyWriter(dir);
+      
+      ltw.addCategory(new CategoryPath("a"));
+      ltw.commit();
+      
+      ltr = new DirectoryTaxonomyReader(dir);
+      assertFalse("Nothing has changed",ltr.refresh());
+      
+      ltw.addCategory(new CategoryPath("b"));
+      ltw.commit();
+      
+      assertTrue("changes were committed",ltr.refresh());
+      assertFalse("Nothing has changed",ltr.refresh());
+    } finally {
+      IOUtils.close(ltw, ltr, dir);
+    }
+  }
+  
   @Test
   public void testAlreadyClosed() throws Exception {
     Directory dir = newDirectory();
@@ -77,4 +114,68 @@ public class TestDirectoryTaxonomyReader
     dir.close();
   }
   
+  /**
+   * recreating a taxonomy should work well with a freshly opened taxonomy reader 
+   */
+  @Test
+  public void testFreshReadRecreatedTaxonomy() throws Exception {
+    doTestReadRecreatedTaxono(random, true);
+  }
+  
+  /**
+   * recreating a taxonomy should work well with a refreshed taxonomy reader 
+   */
+  @Test
+  public void testRefreshReadRecreatedTaxonomy() throws Exception {
+    doTestReadRecreatedTaxono(random, false);
+  }
+  
+  private void doTestReadRecreatedTaxono(Random random, boolean closeReader) throws Exception {
+    Directory dir = null;
+    TaxonomyWriter tw = null;
+    TaxonomyReader tr = null;
+    
+    // prepare a few categories
+    int  n = 10;
+    CategoryPath[] cp = new CategoryPath[n];
+    for (int i=0; i<n; i++) {
+      cp[i] = new CategoryPath("a", Integer.toString(i));
+    }
+    
+    try {
+      dir = newDirectory();
+      
+      tw = new DirectoryTaxonomyWriter(dir);
+      tw.addCategory(new CategoryPath("a"));
+      tw.close();
+      
+      tr = new DirectoryTaxonomyReader(dir);
+      int baseNumcategories = tr.getSize();
+      
+      for (int i=0; i<n; i++) {
+        int k = random.nextInt(n);
+        tw = new DirectoryTaxonomyWriter(dir, OpenMode.CREATE);
+        for (int j=0; j<=k; j++) {
+          tw.addCategory(new CategoryPath(cp[j]));
+        }
+        tw.close();
+        if (closeReader) {
+          tr.close();
+          tr = new DirectoryTaxonomyReader(dir);
+        } else {
+          try {
+            tr.refresh();
+            fail("Expected InconsistentTaxonomyException");
+          } catch (InconsistentTaxonomyException e) {
+            tr.close();
+            tr = new DirectoryTaxonomyReader(dir);
+          }
+        }
+        assertEquals("Wrong #categories in taxonomy (i="+i+", k="+k+")", baseNumcategories + 1 + k, tr.getSize());
+      }
+    } finally {
+      IOUtils.close(tr, tw, dir);
+    }
+  }
+  
 }

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java?rev=1203165&r1=1203164&r2=1203165&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java Thu Nov 17 12:30:29 2011
@@ -280,6 +280,11 @@ public class FilterIndexReader extends I
     return in.getSequentialSubReaders();
   }
 
+  @Override
+  public Map<String, String> getCommitUserData() { 
+    return in.getCommitUserData();
+  }
+  
   /** If the subclass of FilteredIndexReader modifies the
    *  contents of the FieldCache, you must override this
    *  method to provide a different key */