You are viewing a plain text version of this content. The canonical link for it is here.
Posted to solr-commits@lucene.apache.org by yo...@apache.org on 2008/05/10 23:16:37 UTC

svn commit: r655190 - in /lucene/solr/trunk: CHANGES.txt example/solr/conf/solrconfig.xml src/java/org/apache/solr/update/DirectUpdateHandler2.java src/java/org/apache/solr/update/UpdateHandler.java src/test/org/apache/solr/update/AutoCommitTest.java

Author: yonik
Date: Sat May 10 14:16:37 2008
New Revision: 655190

URL: http://svn.apache.org/viewvc?rev=655190&view=rev
Log:
SOLR-559: use Lucene updateDocument, deleteDocuments

Modified:
    lucene/solr/trunk/CHANGES.txt
    lucene/solr/trunk/example/solr/conf/solrconfig.xml
    lucene/solr/trunk/src/java/org/apache/solr/update/DirectUpdateHandler2.java
    lucene/solr/trunk/src/java/org/apache/solr/update/UpdateHandler.java
    lucene/solr/trunk/src/test/org/apache/solr/update/AutoCommitTest.java

Modified: lucene/solr/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/CHANGES.txt?rev=655190&r1=655189&r2=655190&view=diff
==============================================================================
--- lucene/solr/trunk/CHANGES.txt (original)
+++ lucene/solr/trunk/CHANGES.txt Sat May 10 14:16:37 2008
@@ -253,6 +253,11 @@
 46. SOLR-557: Added SolrCore.getSearchComponents() to return an unmodifiable Map. (gsingers)
     
 Changes in runtime behavior
+ 1. SOLR-559: use Lucene updateDocument, deleteDocuments methods.  This
+   removes the maxBufferedDeletes parameter added by SOLR-310 as Lucene
+   now manages the deletes.  This provides slightly better indexing 
+   performance and makes overwrites atomic, eliminating the possibility of
+   a crash causing duplicates. (yonik)
 
 Optimizations
  1. SOLR-276: improve JSON writer speed. (yonik)

Modified: lucene/solr/trunk/example/solr/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/example/solr/conf/solrconfig.xml?rev=655190&r1=655189&r2=655190&view=diff
==============================================================================
--- lucene/solr/trunk/example/solr/conf/solrconfig.xml (original)
+++ lucene/solr/trunk/example/solr/conf/solrconfig.xml Sat May 10 14:16:37 2008
@@ -125,14 +125,7 @@
          org.apache.solr.(search|update|request|core|analysis)
      -->
 
-    <!-- Limit the number of deletions Solr will buffer during doc updating.
-        
-        Setting this lower can help bound memory use during indexing.
-    -->
-    <maxPendingDeletes>100000</maxPendingDeletes>
-
     <!-- Perform a <commit/> automatically under certain conditions:
-
          maxDocs - number of updates since last commit is greater than this
          maxTime - oldest uncommited update (in ms) is this long ago
     <autoCommit> 

Modified: lucene/solr/trunk/src/java/org/apache/solr/update/DirectUpdateHandler2.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/update/DirectUpdateHandler2.java?rev=655190&r1=655189&r2=655190&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/update/DirectUpdateHandler2.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/update/DirectUpdateHandler2.java Sat May 10 14:16:37 2008
@@ -21,14 +21,9 @@
 package org.apache.solr.update;
 
 import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.TermDocs;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.MatchAllDocsQuery;
 
-import java.util.TreeMap;
-import java.util.Map;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.ExecutionException;
@@ -49,7 +44,6 @@
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.core.SolrCore;
-import org.apache.solr.core.SolrConfig;
 
 /**
  * <code>DirectUpdateHandler2</code> implements an UpdateHandler where documents are added
@@ -141,14 +135,7 @@
   // tracks when auto-commit should occur
   protected final CommitTracker tracker;
 
-  // The key is the id, the value (Integer) is the number
-  // of docs to save (delete all except the last "n" added)
-  protected final Map<String,Integer> pset;
   protected int maxPendingDeletes = -1;
-  
-  // commonly used constants for the count in the pset
-  protected final static Integer ZERO = 0;
-  protected final static Integer ONE = 1;
 
   // iwCommit protects internal data and open/close of the IndexWriter and
   // is a mutex. Any use of the index writer should be protected by iwAccess, 
@@ -161,10 +148,6 @@
 
   public DirectUpdateHandler2(SolrCore core) throws IOException {
     super(core);
-    /* A TreeMap is used to maintain the natural ordering of the document ids,
-       which makes commits more efficient
-     */
-    pset = new TreeMap<String,Integer>(); 
     maxPendingDeletes = core.getSolrConfig().getInt("updateHandler/maxPendingDeletes", -1);
 
     ReadWriteLock rwl = new ReentrantReadWriteLock();
@@ -179,7 +162,6 @@
     core.log.info(core.getLogId()+"REMOVING ALL DOCUMENTS FROM INDEX");
     closeWriter();
     closeSearcher();
-    pset.clear(); // ignore docs marked for deletion since we are removing all
     writer = createMainIndexWriter("DirectUpdateHandler2", true);
   }
 
@@ -238,45 +220,26 @@
       // state.  This is safe as all other state-changing operations are
       // protected with iwCommit (which iwAccess excludes from this block).
       synchronized (this) {
-        if (!cmd.allowDups && !cmd.overwritePending && !cmd.overwriteCommitted) {
-          throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,"unsupported param combo:" + cmd);
-          // this would need a reader to implement (to be able to check committed
-          // before adding.)
-          // return addNoOverwriteNoDups(cmd);
-        } else if (!cmd.allowDups && !cmd.overwritePending && cmd.overwriteCommitted) {
-          rc = addConditionally(cmd);
-      } else if (!cmd.allowDups && cmd.overwritePending && !cmd.overwriteCommitted) {
-          throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,"unsupported param combo:" + cmd);
-        } else if (!cmd.allowDups && cmd.overwritePending && cmd.overwriteCommitted) {
-          rc = overwriteBoth(cmd);
-        } else if (cmd.allowDups && !cmd.overwritePending && !cmd.overwriteCommitted) {
-          rc = allowDups(cmd);
-        } else if (cmd.allowDups && !cmd.overwritePending && cmd.overwriteCommitted) {
-          throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,"unsupported param combo:" + cmd);
-        } else if (cmd.allowDups && cmd.overwritePending && !cmd.overwriteCommitted) {
-          throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,"unsupported param combo:" + cmd);
-        } else if (cmd.allowDups && cmd.overwritePending && cmd.overwriteCommitted) {
-          rc = overwriteBoth(cmd);
-        }
-        if (rc == -1)
-          throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,"unsupported param combo:" + cmd);
-        
-        if (rc == 1) {
-          // adding document -- prep writer
-          closeSearcher();
-          openWriter();
-          tracker.addedDocument();          
-        } else {
-          // exit prematurely
-          return rc;
-        }
+        // adding document -- prep writer
+        closeSearcher();
+        openWriter();
+        tracker.addedDocument();
       } // end synchronized block
 
       // this is the only unsynchronized code in the iwAccess block, which
       // should account for most of the time
-      assert(rc == 1);
-      writer.addDocument(cmd.doc);
-      
+
+      if (cmd.overwriteCommitted || cmd.overwritePending) {
+        if (cmd.indexedId == null) {
+          cmd.indexedId = getIndexedId(cmd.doc);
+        }
+        writer.updateDocument(idTerm.createTerm(cmd.indexedId), cmd.getLuceneDocument(schema));
+      } else {
+        // allow duplicates
+        writer.addDocument(cmd.getLuceneDocument(schema));
+      }
+
+      rc = 1;
     } finally {
       iwAccess.unlock();
       if (rc!=1) {
@@ -286,17 +249,7 @@
         numDocsPending.incrementAndGet();
       }
     }
-    if (maxPendingDeletes > 0 && pset.size() > maxPendingDeletes) {
-      iwCommit.lock();
-      try {
-        // note: this may be entered multiple times since the synchro is 
-        // inside the if(), but doDeletions() is a cheap no-op if it has
-        // already executed
-        doDeletions();
-      } finally {
-        iwCommit.unlock();
-      }
-    }    
+
     return rc;
   }
 
@@ -319,9 +272,11 @@
 
     iwCommit.lock();
     try {
-      pset.put(idFieldType.toInternal(cmd.id), ZERO);
-    } finally { 
-      iwCommit.unlock(); 
+      closeSearcher();
+      openWriter();
+      writer.deleteDocuments(idTerm.createTerm(idFieldType.toInternal(cmd.id)));
+    } finally {
+      iwCommit.unlock();
     }
     
     if( tracker.timeUpperBound > 0 ) {
@@ -358,10 +313,6 @@
        if (delAll) {
          deleteAll();
        } else {
-         // we need to do much of the commit logic (mainly doing queued
-         // deletes since deleteByQuery can throw off our counts.
-         doDeletions();
-
          closeWriter();
          openSearcher();
 
@@ -396,128 +347,6 @@
    }
 
 
-  ///////////////////////////////////////////////////////////////////
-  /////////////////// helper method for each add type ///////////////
-  ///////////////////////////////////////////////////////////////////
-
-  // methods return 1 if the document is to be added; 0 otherwise.
-  // methods must be called in synchronized context
-
-  protected int addConditionally(AddUpdateCommand cmd) throws IOException {
-    if (cmd.indexedId ==null) {
-      cmd.indexedId =getIndexedId(cmd.doc);
-    }
-    Integer saveCount = pset.get(cmd.indexedId);
-    if (saveCount!=null && saveCount!=0) {
-      // a doc with this id already exists in the pending set
-      return 0;
-    }
-    pset.put(cmd.indexedId, ONE);
-    return 1;
-  }
-
-
-  // overwrite both pending and committed
-  protected int overwriteBoth(AddUpdateCommand cmd) throws IOException {
-    if (cmd.indexedId ==null) {
-      cmd.indexedId =getIndexedId(cmd.doc);
-    }
-    pset.put(cmd.indexedId, ONE);
-    return 1;
-  }
-
-
-  // add without checking
-  protected int allowDups(AddUpdateCommand cmd) throws IOException {
-    if (cmd.indexedId ==null) {
-      cmd.indexedId =getIndexedIdOptional(cmd.doc);
-    }
-    if (cmd.indexedId != null) {
-      Integer saveCount = pset.get(cmd.indexedId);
-      
-      // if there weren't any docs marked for deletion before, then don't mark
-      // any for deletion now.
-      if (saveCount == null) return 1;
-
-      // If there were docs marked for deletion, then increment the number of
-      // docs to save at the end.
-
-      // the following line is optional, but it saves an allocation in the common case.
-      if (saveCount == ZERO) saveCount=ONE;
-      else saveCount++;
-
-      pset.put(cmd.indexedId, saveCount);
-    }
-    return 1;
-  }
-
-  //
-  // do all needed deletions.
-  // call with iwCommit lock held
-  //
-  protected void doDeletions() throws IOException {
-    int[] docnums = new int[0];
-
-    if (pset.size() > 0) { // optimization: only open searcher if there is something to delete...
-      log.info("DirectUpdateHandler2 deleting and removing dups for " + pset.size() +" ids");
-      int numDeletes=0;
-
-      closeWriter();
-      openSearcher();
-      IndexReader reader = searcher.getReader();
-      TermDocs tdocs = reader.termDocs();
-      String fieldname = idField.getName();
-
-      for (Map.Entry<String,Integer> entry : pset.entrySet()) {
-        String id = entry.getKey();
-        int saveLast = entry.getValue();  // save the last "saveLast" documents
-
-        //expand our array that keeps track of docs if needed.
-        if (docnums==null || saveLast > docnums.length) {
-          docnums = new int[saveLast];
-        }
-
-        // initialize all docnums in the list to -1 (unused)
-        for (int i=0; i<saveLast; i++) {
-          docnums[i] = -1;
-        }
-
-        tdocs.seek(new Term(fieldname,id));
-
-        //
-        // record the docs for this term in the "docnums" array and wrap around
-        // at size "saveLast".  If we reuse a slot in the array, then we delete
-        // the doc that was there from the index.
-        //
-        int pos=0;
-        while (tdocs.next()) {
-          if (saveLast==0) {
-            // special case - delete all the docs as we see them.
-            reader.deleteDocument(tdocs.doc());
-            numDeletes++;
-            continue;
-          }
-
-          int prev=docnums[pos];
-          docnums[pos]=tdocs.doc();
-          if (prev != -1) {
-            reader.deleteDocument(prev);
-            numDeletes++;
-          }
-
-          if (++pos >= saveLast) pos=0;
-        }
-      }
-
-      // should we ever shrink it again, or just clear it?
-      pset.clear();
-      log.info("DirectUpdateHandler2 docs deleted=" + numDeletes);
-      numDocsDeleted.addAndGet(numDeletes);
-    }
-
-  }
-
-
 
   public void commit(CommitUpdateCommand cmd) throws IOException {
 
@@ -536,8 +365,7 @@
     iwCommit.lock();
     try {
       log.info("start "+cmd);
-      doDeletions();
-        
+
       if (cmd.optimize) {
         closeSearcher();
         openWriter(); 
@@ -597,7 +425,6 @@
         tracker.pending = null;
       }
       tracker.scheduler.shutdown(); 
-      doDeletions();
       closeSearcher();
       closeWriter();
     } finally {
@@ -786,7 +613,7 @@
     lst.add("optimizes", optimizeCommands.get());
     lst.add("docsPending", numDocsPending.get());
     // pset.size() not synchronized, but it should be fine to access.
-    lst.add("deletesPending", pset.size());
+    // lst.add("deletesPending", pset.size());
     lst.add("adds", addCommands.get());
     lst.add("deletesById", deleteByIdCommands.get());
     lst.add("deletesByQuery", deleteByQueryCommands.get());

Modified: lucene/solr/trunk/src/java/org/apache/solr/update/UpdateHandler.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/update/UpdateHandler.java?rev=655190&r1=655189&r2=655190&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/update/UpdateHandler.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/update/UpdateHandler.java Sat May 10 14:16:37 2008
@@ -56,6 +56,7 @@
 
   protected final SchemaField idField;
   protected final FieldType idFieldType;
+  protected final Term idTerm; // prototype term to avoid interning fieldname
 
   protected Vector<SolrEventListener> commitCallbacks = new Vector<SolrEventListener>();
   protected Vector<SolrEventListener> optimizeCallbacks = new Vector<SolrEventListener>();
@@ -112,7 +113,7 @@
     schema = core.getSchema();
     idField = schema.getUniqueKeyField();
     idFieldType = idField!=null ? idField.getType() : null;
-
+    idTerm = idField!=null ? new Term(idField.getName(),"") : null;
     parseEventListeners();
     core.getInfoRegistry().put("updateHandler", this);
   }

Modified: lucene/solr/trunk/src/test/org/apache/solr/update/AutoCommitTest.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/test/org/apache/solr/update/AutoCommitTest.java?rev=655190&r1=655189&r2=655190&view=diff
==============================================================================
--- lucene/solr/trunk/src/test/org/apache/solr/update/AutoCommitTest.java (original)
+++ lucene/solr/trunk/src/test/org/apache/solr/update/AutoCommitTest.java Sat May 10 14:16:37 2008
@@ -215,8 +215,9 @@
         adoc("id", "A14", "subject", "info" ), null ) );
     handler.handleRequest( req, rsp );
 
-    assertEquals(updater.numDocsPending.get(), 0);
-    assertEquals(updater.commitCommands.get(), 0);
+    // Lucene now manages it's own deletes.
+    // assertEquals(updater.numDocsPending.get(), 0);
+    // assertEquals(updater.commitCommands.get(), 0);
   }
 
 }