checked patch for SOLR-65

git-svn-id: https://svn.apache.org/repos/asf/incubator/solr/trunk@472720 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Mike Klaas 2006-11-09 00:43:54 +00:00
parent a84079798d
commit 5cd72ec45a
6 changed files with 272 additions and 146 deletions

View File

@ -62,6 +62,7 @@ New Features
28. json.wrf parameter adds a wrapper-function around the JSON response, 28. json.wrf parameter adds a wrapper-function around the JSON response,
useful in AJAX with dynamic script tags for specifying a JavaScript useful in AJAX with dynamic script tags for specifying a JavaScript
callback function. (Bertrand Delacretaz via yonik, SOLR-56) callback function. (Bertrand Delacretaz via yonik, SOLR-56)
29. autoCommit can be specified every so many documents added (klaas, SOLR-65)
Changes in runtime behavior Changes in runtime behavior
1. classes reorganized into different packages, package names changed to Apache 1. classes reorganized into different packages, package names changed to Apache
@ -80,6 +81,8 @@ Changes in runtime behavior
7. FunctionQuery.explain now uses ComplexExplanation to provide more 7. FunctionQuery.explain now uses ComplexExplanation to provide more
accurate score explanations when composed in a BooleanQuery. accurate score explanations when composed in a BooleanQuery.
(hossman, SOLR-25) (hossman, SOLR-25)
8. Document update handling locking is much sparser, allowing performance gains
through multiple threads. Large commits also might be faster (klaas, SOLR-65)
Optimizations Optimizations
1. getDocListAndSet can now generate both a DocList and a DocSet from a 1. getDocListAndSet can now generate both a DocList and a DocSet from a
@ -113,6 +116,7 @@ Bug Fixes
generate and catenate off. (Mike Klaas via yonik, SOLR-34) generate and catenate off. (Mike Klaas via yonik, SOLR-34)
8. Escape '>' in XML output (because ]]> is illegal in CharData) 8. Escape '>' in XML output (because ]]> is illegal in CharData)
9. field boosts weren't being applied and doc boosts were being applied to fields (klaas) 9. field boosts weren't being applied and doc boosts were being applied to fields (klaas)
10. Multiple-doc update generates well-formed xml (klaas, SOLR-65)
Other Changes Other Changes
1. Upgrade to Lucene 2.0 nightly build 2006-06-22, lucene SVN revision 416224, 1. Upgrade to Lucene 2.0 nightly build 2006-06-22, lucene SVN revision 416224,

View File

@ -38,6 +38,12 @@
<!-- the default high-performance update handler --> <!-- the default high-performance update handler -->
<updateHandler class="solr.DirectUpdateHandler2"> <updateHandler class="solr.DirectUpdateHandler2">
<!-- autocommit pending docs if certain criteria are met
<autoCommit>
<maxDocs>10000</maxDocs>
</autoCommit>
-->
<!-- The RunExecutableListener executes an external command. <!-- The RunExecutableListener executes an external command.
exe - the name of the executable to run exe - the name of the executable to run
dir - dir to use as the current working directory. default="." dir - dir to use as the current working directory. default="."

View File

@ -666,7 +666,10 @@ public final class SolrCore {
if (!committedAttr) cmd.overwriteCommitted=!cmd.allowDups; if (!committedAttr) cmd.overwriteCommitted=!cmd.allowDups;
DocumentBuilder builder = new DocumentBuilder(schema); DocumentBuilder builder = new DocumentBuilder(schema);
SchemaField uniqueKeyField = schema.getUniqueKeyField();
int eventType=0; int eventType=0;
// accumulate responses
List<String> added = new ArrayList<String>(10);
while(true) { while(true) {
// this may be our second time through the loop in the case // this may be our second time through the loop in the case
// that there are multiple docs in the add... so make sure that // that there are multiple docs in the add... so make sure that
@ -682,47 +685,27 @@ public final class SolrCore {
eventType = xpp.nextTag(); eventType = xpp.nextTag();
if (eventType == XmlPullParser.END_TAG || eventType == XmlPullParser.END_DOCUMENT) break; // should match </add> if (eventType == XmlPullParser.END_TAG || eventType == XmlPullParser.END_DOCUMENT) break; // should match </add>
try { readDoc(builder,xpp);
readDoc(builder,xpp); builder.endDoc();
builder.endDoc(); cmd.doc = builder.getDoc();
cmd.doc = builder.getDoc(); log.finest("adding doc...");
log.finest("adding doc..."); updateHandler.addDoc(cmd);
updateHandler.addDoc(cmd); String docId = null;
SchemaField uniqueKeyField = schema.getUniqueKeyField(); if (uniqueKeyField!=null)
StringBuilder addMsg = new StringBuilder("add "); docId = schema.printableUniqueKey(cmd.doc);
if (uniqueKeyField!=null) addMsg.append("(") added.add(docId);
.append(uniqueKeyField.getName())
.append("=") } // end while
.append(schema.printableUniqueKey(cmd.doc)) // write log and result
.append(") "); StringBuilder out = new StringBuilder();
log.info(addMsg.toString()+status+" "+(System.currentTimeMillis()-startTime)); for (String docId: added)
writer.write("<result status=\"" + status + "\"></result>"); if(docId != null)
} catch (SolrException e) { out.append(docId + ",");
log(e); String outMsg = out.toString();
log.info("add "+e.code+" "+(System.currentTimeMillis()-startTime)); if(outMsg.length() > 0)
writeResult(writer,e); outMsg = outMsg.substring(0, outMsg.length() - 1);
// we may not have finised reading the XML for this cmd, log.info("added id={" + outMsg + "} in " + (System.currentTimeMillis()-startTime) + "ms");
// so eat any unused input up till "</add>" writer.write("<result status=\"0\"></result>");
eventType = xpp.getEventType();
while (true) {
if ( eventType == XmlPullParser.END_DOCUMENT
|| (eventType == XmlPullParser.END_TAG && "add".equals(xpp.getName())))
{
break;
}
eventType = xpp.next();
}
}
}
/***
while (findNextTag(xpp,"doc") != XmlPullParser.END_DOCUMENT) {
readDoc(builder,xpp);
Document doc = builder.endDoc();
indexWriter.addDocument(doc);
docsAdded++;
}
***/
} // end add } // end add
@ -1014,8 +997,6 @@ public final class SolrCore {
public final QueryResponseWriter getQueryResponseWriter(SolrQueryRequest request) { public final QueryResponseWriter getQueryResponseWriter(SolrQueryRequest request) {
return getQueryResponseWriter(request.getParam("wt")); return getQueryResponseWriter(request.getParam("wt"));
} }
} }

View File

@ -29,9 +29,13 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
import java.util.HashMap; import java.util.HashMap;
import java.util.TreeMap;
import java.util.Map; import java.util.Map;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import java.util.logging.Level; import java.util.logging.Level;
import java.io.IOException; import java.io.IOException;
@ -41,6 +45,7 @@ import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.search.QueryParsing; import org.apache.solr.search.QueryParsing;
import org.apache.solr.util.NamedList; import org.apache.solr.util.NamedList;
import org.apache.solr.core.SolrCore; import org.apache.solr.core.SolrCore;
import org.apache.solr.core.SolrConfig;
import org.apache.solr.core.SolrException; import org.apache.solr.core.SolrException;
/** /**
@ -131,30 +136,48 @@ public class DirectUpdateHandler2 extends UpdateHandler {
AtomicLong numErrors = new AtomicLong(); AtomicLong numErrors = new AtomicLong();
AtomicLong numErrorsCumulative = new AtomicLong(); AtomicLong numErrorsCumulative = new AtomicLong();
// tracks when auto-commit should occur
protected final CommitTracker tracker;
// The key is the id, the value (Integer) is the number // The key is the id, the value (Integer) is the number
// of docs to save (delete all except the last "n" added) // of docs to save (delete all except the last "n" added)
protected final HashMap<String,Integer> pset; protected final Map<String,Integer> pset;
// commonly used constants for the count in the pset // commonly used constants for the count in the pset
protected final static Integer ZERO = 0; protected final static Integer ZERO = 0;
protected final static Integer ONE = 1; 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,
// which admits multiple simultaneous acquisitions. iwAccess is
// mutually-exclusive with the iwCommit lock.
protected final Lock iwAccess, iwCommit;
protected IndexWriter writer; protected IndexWriter writer;
protected SolrIndexSearcher searcher; protected SolrIndexSearcher searcher;
public DirectUpdateHandler2(SolrCore core) throws IOException { public DirectUpdateHandler2(SolrCore core) throws IOException {
super(core); super(core);
pset = new HashMap<String,Integer>(256); // 256 is just an optional head-start /* A TreeMap is used to maintain the natural ordering of the document ids,
which makes commits more efficient
*/
pset = new TreeMap<String,Integer>();
ReadWriteLock rwl = new ReentrantReadWriteLock();
iwAccess = rwl.readLock();
iwCommit = rwl.writeLock();
tracker = new CommitTracker();
} }
// must only be called when iwCommit lock held
protected void openWriter() throws IOException { protected void openWriter() throws IOException {
if (writer==null) { if (writer==null) {
writer = createMainIndexWriter("DirectUpdateHandler2"); writer = createMainIndexWriter("DirectUpdateHandler2");
} }
} }
// must only be called when iwCommit lock held
protected void closeWriter() throws IOException { protected void closeWriter() throws IOException {
try { try {
numDocsPending.set(0); numDocsPending.set(0);
@ -182,44 +205,60 @@ public class DirectUpdateHandler2 extends UpdateHandler {
} }
} }
protected void doAdd(Document doc) throws IOException {
closeSearcher(); openWriter();
writer.addDocument(doc);
}
public int addDoc(AddUpdateCommand cmd) throws IOException { public int addDoc(AddUpdateCommand cmd) throws IOException {
addCommands.incrementAndGet(); addCommands.incrementAndGet();
addCommandsCumulative.incrementAndGet(); addCommandsCumulative.incrementAndGet();
int rc=-1; int rc=-1;
iwAccess.lock();
try { try {
if (!cmd.allowDups && !cmd.overwritePending && !cmd.overwriteCommitted) {
throw new SolrException(400,"unsupported param combo:" + cmd); // We can't using iwCommit to protect internal data here, since it would
// this would need a reader to implement (to be able to check committed // block other addDoc calls. Hence, we synchronize to protect internal
// before adding.) // state. This is safe as all other state-changing operations are
// return addNoOverwriteNoDups(cmd); // protected with iwCommit (which iwAccess excludes from this block).
} else if (!cmd.allowDups && !cmd.overwritePending && cmd.overwriteCommitted) { synchronized (this) {
rc = addConditionally(cmd); if (!cmd.allowDups && !cmd.overwritePending && !cmd.overwriteCommitted) {
return rc; throw new SolrException(400,"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) { } else if (!cmd.allowDups && cmd.overwritePending && !cmd.overwriteCommitted) {
throw new SolrException(400,"unsupported param combo:" + cmd); throw new SolrException(400,"unsupported param combo:" + cmd);
} else if (!cmd.allowDups && cmd.overwritePending && cmd.overwriteCommitted) { } else if (!cmd.allowDups && cmd.overwritePending && cmd.overwriteCommitted) {
rc = overwriteBoth(cmd); rc = overwriteBoth(cmd);
return rc; } else if (cmd.allowDups && !cmd.overwritePending && !cmd.overwriteCommitted) {
} else if (cmd.allowDups && !cmd.overwritePending && !cmd.overwriteCommitted) { rc = allowDups(cmd);
rc = allowDups(cmd); } else if (cmd.allowDups && !cmd.overwritePending && cmd.overwriteCommitted) {
return rc; throw new SolrException(400,"unsupported param combo:" + cmd);
} else if (cmd.allowDups && !cmd.overwritePending && cmd.overwriteCommitted) { } else if (cmd.allowDups && cmd.overwritePending && !cmd.overwriteCommitted) {
throw new SolrException(400,"unsupported param combo:" + cmd); throw new SolrException(400,"unsupported param combo:" + cmd);
} else if (cmd.allowDups && cmd.overwritePending && !cmd.overwriteCommitted) { } else if (cmd.allowDups && cmd.overwritePending && cmd.overwriteCommitted) {
throw new SolrException(400,"unsupported param combo:" + cmd); rc = overwriteBoth(cmd);
} else if (cmd.allowDups && cmd.overwritePending && cmd.overwriteCommitted) { }
rc = overwriteBoth(cmd); if (rc == -1)
return rc; throw new SolrException(400,"unsupported param combo:" + cmd);
}
throw new SolrException(400,"unsupported param combo:" + cmd); if (rc == 1) {
// adding document -- prep writer
closeSearcher();
openWriter();
tracker.increment(1);
} else {
// exit prematurely
return rc;
}
} // 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);
} finally { } finally {
iwAccess.unlock();
if (rc!=1) { if (rc!=1) {
numErrors.incrementAndGet(); numErrors.incrementAndGet();
numErrorsCumulative.incrementAndGet(); numErrorsCumulative.incrementAndGet();
@ -227,6 +266,10 @@ public class DirectUpdateHandler2 extends UpdateHandler {
numDocsPending.incrementAndGet(); numDocsPending.incrementAndGet();
} }
} }
// might need to commit (wait for searcher if so)
checkCommit(true);
return rc;
} }
@ -246,8 +289,11 @@ public class DirectUpdateHandler2 extends UpdateHandler {
throw new SolrException(400,"operation not supported" + cmd); throw new SolrException(400,"operation not supported" + cmd);
} }
synchronized(this) { iwCommit.lock();
try {
pset.put(idFieldType.toInternal(cmd.id), ZERO); pset.put(idFieldType.toInternal(cmd.id), ZERO);
} finally {
iwCommit.unlock();
} }
} }
@ -273,11 +319,12 @@ public class DirectUpdateHandler2 extends UpdateHandler {
Query q = QueryParsing.parseQuery(cmd.query, schema); Query q = QueryParsing.parseQuery(cmd.query, schema);
int totDeleted = 0; int totDeleted = 0;
synchronized(this) { iwCommit.lock();
try {
// we need to do much of the commit logic (mainly doing queued // we need to do much of the commit logic (mainly doing queued
// deletes since deleteByQuery can throw off our counts. // deletes since deleteByQuery can throw off our counts.
doDeletions(); doDeletions();
closeWriter(); closeWriter();
openSearcher(); openSearcher();
@ -286,6 +333,8 @@ public class DirectUpdateHandler2 extends UpdateHandler {
final DeleteHitCollector deleter = new DeleteHitCollector(searcher); final DeleteHitCollector deleter = new DeleteHitCollector(searcher);
searcher.search(q, null, deleter); searcher.search(q, null, deleter);
totDeleted = deleter.deleted; totDeleted = deleter.deleted;
} finally {
iwCommit.unlock();
} }
if (SolrCore.log.isLoggable(Level.FINE)) { if (SolrCore.log.isLoggable(Level.FINE)) {
@ -306,73 +355,63 @@ public class DirectUpdateHandler2 extends UpdateHandler {
/////////////////// helper method for each add type /////////////// /////////////////// 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 { protected int addConditionally(AddUpdateCommand cmd) throws IOException {
if (cmd.indexedId ==null) { if (cmd.indexedId ==null) {
cmd.indexedId =getIndexedId(cmd.doc); cmd.indexedId =getIndexedId(cmd.doc);
} }
synchronized(this) { Integer saveCount = pset.get(cmd.indexedId);
Integer saveCount = pset.get(cmd.indexedId); if (saveCount!=null && saveCount!=0) {
if (saveCount!=null && saveCount!=0) { // a doc with this id already exists in the pending set
// a doc with this id already exists in the pending set return 0;
return 0;
}
pset.put(cmd.indexedId, ONE);
doAdd(cmd.doc);
return 1;
} }
pset.put(cmd.indexedId, ONE);
return 1;
} }
// overwrite both pending and committed // overwrite both pending and committed
protected synchronized int overwriteBoth(AddUpdateCommand cmd) throws IOException { protected int overwriteBoth(AddUpdateCommand cmd) throws IOException {
if (cmd.indexedId ==null) { if (cmd.indexedId ==null) {
cmd.indexedId =getIndexedId(cmd.doc); cmd.indexedId =getIndexedId(cmd.doc);
} }
synchronized (this) { pset.put(cmd.indexedId, ONE);
pset.put(cmd.indexedId, ONE);
doAdd(cmd.doc);
}
return 1; return 1;
} }
// add without checking // add without checking
protected synchronized int allowDups(AddUpdateCommand cmd) throws IOException { protected int allowDups(AddUpdateCommand cmd) throws IOException {
if (cmd.indexedId ==null) { if (cmd.indexedId ==null) {
cmd.indexedId =getIndexedIdOptional(cmd.doc); cmd.indexedId =getIndexedIdOptional(cmd.doc);
} }
synchronized(this) { if (cmd.indexedId != null) {
doAdd(cmd.doc); 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 (cmd.indexedId != null) { // If there were docs marked for deletion, then increment the number of
Integer saveCount = pset.get(cmd.indexedId); // docs to save at the end.
// if there weren't any docs marked for deletion before, then don't mark // the following line is optional, but it saves an allocation in the common case.
// any for deletion now. if (saveCount == ZERO) saveCount=ONE;
if (saveCount == null) return 1; else saveCount++;
// If there were docs marked for deletion, then increment the number of pset.put(cmd.indexedId, saveCount);
// 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; return 1;
} }
// NOT FOR USE OUTSIDE OF A "synchronized(this)" BLOCK
private int[] docnums;
// //
// do all needed deletions. // do all needed deletions.
// call in a synchronized context. // call with iwCommit lock held
// //
protected void doDeletions() throws IOException { protected void doDeletions() throws IOException {
int[] docnums = new int[0];
if (pset.size() > 0) { // optimization: only open searcher if there is something to delete... 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"); log.info("DirectUpdateHandler2 deleting and removing dups for " + pset.size() +" ids");
@ -449,35 +488,38 @@ public class DirectUpdateHandler2 extends UpdateHandler {
} }
boolean error=true; boolean error=true;
iwCommit.lock();
try { try {
synchronized (this) { log.info("start "+cmd);
log.info("start "+cmd); doDeletions();
doDeletions();
if (cmd.optimize) {
if (cmd.optimize) {
closeSearcher();
openWriter();
writer.optimize();
}
closeSearcher(); closeSearcher();
closeWriter(); openWriter();
writer.optimize();
}
callPostCommitCallbacks(); closeSearcher();
if (cmd.optimize) { closeWriter();
callPostOptimizeCallbacks();
}
// open a new searcher in the sync block to avoid opening it
// after a deleteByQuery changed the index, or in between deletes
// and adds of another commit being done.
core.getSearcher(true,false,waitSearcher);
log.info("end_commit_flush"); callPostCommitCallbacks();
} // end synchronized block if (cmd.optimize) {
callPostOptimizeCallbacks();
}
// open a new searcher in the sync block to avoid opening it
// after a deleteByQuery changed the index, or in between deletes
// and adds of another commit being done.
core.getSearcher(true,false,waitSearcher);
// reset commit tracking
tracker.didCommit();
log.info("end_commit_flush");
error=false; error=false;
} }
finally { finally {
iwCommit.unlock();
addCommands.set(0); addCommands.set(0);
deleteByIdCommands.set(0); deleteByIdCommands.set(0);
deleteByQueryCommands.set(0); deleteByQueryCommands.set(0);
@ -502,14 +544,105 @@ public class DirectUpdateHandler2 extends UpdateHandler {
public void close() throws IOException { public void close() throws IOException {
log.info("closing " + this); log.info("closing " + this);
synchronized(this) { iwCommit.lock();
try{
doDeletions(); doDeletions();
closeSearcher(); closeSearcher();
closeWriter(); closeWriter();
} finally {
iwCommit.unlock();
} }
log.info("closed " + this); log.info("closed " + this);
} }
/** Inform tracker that <code>docs</code> docs have been added. Will
* perform commit and/or optimize if constraints are satisfied.
*/
protected void checkCommit() throws IOException {
checkCommit(false);
}
protected void checkCommit(boolean waitSearcher) throws IOException {
synchronized (tracker) {
if (tracker.needCommit()) {
CommitUpdateCommand cmd = new CommitUpdateCommand(false);
cmd.waitSearcher = waitSearcher;
log.info("autocommitting: " + cmd);
commit(cmd);
}
}
}
/** Helper class for tracking autoCommit state.
*
* Note: This is purely an implementation detail of autoCommit and will
* definitely change in the future, so the interface should not be
* relied-upon
*
* Note: all access must be synchronized.
*/
class CommitTracker {
// settings
private final ConstraintTester commitTester;
// state
private long timeOfCommit;
private long docsSinceCommit;
private boolean needCommit;
public CommitTracker() {
timeOfCommit = timestamp();
docsSinceCommit = 0;
needCommit = false;
commitTester = new ConstraintTester(
SolrConfig.config.getInt("updateHandler/autoCommit/maxDocs", -1));
SolrCore.log.info("autocommit if " + commitTester);
}
/** Indicate that <code>count</code> docs have been added. May set
* <code>needCommit()</code> and perhaps also <code>needOptimize</code>
*/
public void increment(int count) {
docsSinceCommit += count;
if (docsSinceCommit > 0) {
needCommit = commitTester.testConstraints(docsSinceCommit);
}
}
/** @return true if commit is needed */
public boolean needCommit() { return needCommit; }
/** Inform tracker that a commit has occurred */
public void didCommit() {
didCommit(docsSinceCommit);
}
public void didCommit(long docsCommitted) {
timeOfCommit = timestamp();
docsSinceCommit -= docsCommitted;
needCommit = false;
}
/** @return milliseconds since epoch */
private long timestamp() { return System.currentTimeMillis();}
class ConstraintTester {
private long docsUpperBound = -1;
public ConstraintTester(long docsUpperBound) {
this.docsUpperBound = docsUpperBound;
}
private boolean checkDocsUpper(long docs) {
return docsUpperBound == -1 ? false : docsUpperBound <= docs;
}
public boolean testConstraints(long docs) {
return checkDocsUpper(docs);
}
public String toString() {
return docsUpperBound != -1 ? "docs >= " + docsUpperBound : "{no doc limit}" ;
}
}
}
///////////////////////////////////////////////////////////////////// /////////////////////////////////////////////////////////////////////
// SolrInfoMBean stuff: Statistics and Module Info // SolrInfoMBean stuff: Statistics and Module Info
///////////////////////////////////////////////////////////////////// /////////////////////////////////////////////////////////////////////

View File

@ -150,7 +150,7 @@ public class BasicFunctionalityTest extends AbstractSolrTestCase {
// big freaking kludge since the response is currently not well formed. // big freaking kludge since the response is currently not well formed.
String res = h.update("<add><doc><field name=\"id\">1</field></doc><doc><field name=\"id\">2</field></doc></add>"); String res = h.update("<add><doc><field name=\"id\">1</field></doc><doc><field name=\"id\">2</field></doc></add>");
assertEquals("<result status=\"0\"></result><result status=\"0\"></result>", res); assertEquals("<result status=\"0\"></result>", res);
assertU("<commit/>"); assertU("<commit/>");
assertQ(req("id:[0 TO 99]") assertQ(req("id:[0 TO 99]")
,"//*[@numFound='2']" ,"//*[@numFound='2']"
@ -165,7 +165,7 @@ public class BasicFunctionalityTest extends AbstractSolrTestCase {
"<field name=\"text\">hello</field></doc>" + "<field name=\"text\">hello</field></doc>" +
"</add>"); "</add>");
assertEquals("<result status=\"0\"></result><result status=\"0\"></result>", res); assertEquals("<result status=\"0\"></result>", res);
assertU("<commit/>"); assertU("<commit/>");
assertQ(req("text:hello") assertQ(req("text:hello")
,"//*[@numFound='2']" ,"//*[@numFound='2']"
@ -183,7 +183,7 @@ public class BasicFunctionalityTest extends AbstractSolrTestCase {
"<field boost=\"2.0\" name=\"text\">hello</field></doc>" + "<field boost=\"2.0\" name=\"text\">hello</field></doc>" +
"</add>"); "</add>");
assertEquals("<result status=\"0\"></result><result status=\"0\"></result>", res); assertEquals("<result status=\"0\"></result>", res);
assertU("<commit/>"); assertU("<commit/>");
assertQ(req("text:hello"), assertQ(req("text:hello"),
"//*[@numFound='2']" "//*[@numFound='2']"
@ -194,7 +194,6 @@ public class BasicFunctionalityTest extends AbstractSolrTestCase {
assertTrue( resp.indexOf("id=2") < resp.indexOf("id=1") ); assertTrue( resp.indexOf("id=2") < resp.indexOf("id=1") );
} }
public void testXMLWriter() throws Exception { public void testXMLWriter() throws Exception {
SolrQueryResponse rsp = new SolrQueryResponse(); SolrQueryResponse rsp = new SolrQueryResponse();

View File

@ -42,16 +42,19 @@
<updateHandler class="solr.DirectUpdateHandler2"> <updateHandler class="solr.DirectUpdateHandler2">
<!-- autocommit pending docs if certain criteria are met --> <!-- autocommit pending docs if certain criteria are met
<autocommit> <!-- NOTE: autocommit not implemented yet --> NOTE: maxSecs not implemented yet
<autoCommit>
<maxDocs>10000</maxDocs> <maxDocs>10000</maxDocs>
<maxSec>3600</maxSec> <maxSec>3600</maxSec>
</autocommit> </autoCommit>
-->
<!-- represents a lower bound on the frequency that commits may <!-- represents a lower bound on the frequency that commits may
occur (in seconds). NOTE: not yet implemented occur (in seconds). NOTE: not yet implemented
-->
<commitIntervalLowerBound>0</commitIntervalLowerBound> <commitIntervalLowerBound>0</commitIntervalLowerBound>
-->
<!-- The RunExecutableListener executes an external command. <!-- The RunExecutableListener executes an external command.
exe - the name of the executable to run exe - the name of the executable to run