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);
}
}