You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ja...@apache.org on 2012/01/24 16:39:17 UTC

svn commit: r1235304 - in /lucene/dev/branches/branch_3x/solr: ./ core/src/java/org/apache/solr/handler/ core/src/java/org/apache/solr/update/ core/src/test/org/apache/solr/handler/ core/src/test/org/apache/solr/update/ solrj/src/java/org/apache/solr/c...

Author: janhoy
Date: Tue Jan 24 15:39:17 2012
New Revision: 1235304

URL: http://svn.apache.org/viewvc?rev=1235304&view=rev
Log:
SOLR-2280: commitWithin ignored for a delete query

Modified:
    lucene/dev/branches/branch_3x/solr/CHANGES.txt
    lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/BinaryUpdateRequestHandler.java
    lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/JsonLoader.java
    lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/XMLLoader.java
    lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/XmlUpdateRequestHandler.java
    lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/update/CommitTracker.java
    lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/update/DeleteUpdateCommand.java
    lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
    lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/handler/JsonLoaderTest.java
    lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/handler/XmlUpdateRequestHandlerTest.java
    lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/update/AutoCommitTest.java
    lucene/dev/branches/branch_3x/solr/solrj/src/java/org/apache/solr/client/solrj/SolrServer.java
    lucene/dev/branches/branch_3x/solr/solrj/src/java/org/apache/solr/client/solrj/request/JavaBinUpdateRequestCodec.java
    lucene/dev/branches/branch_3x/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java
    lucene/dev/branches/branch_3x/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java
    lucene/dev/branches/branch_3x/solr/test-framework/src/java/org/apache/solr/util/AbstractSolrTestCase.java
    lucene/dev/branches/branch_3x/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java

Modified: lucene/dev/branches/branch_3x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/CHANGES.txt?rev=1235304&r1=1235303&r2=1235304&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_3x/solr/CHANGES.txt Tue Jan 24 15:39:17 2012
@@ -123,6 +123,8 @@ Bug Fixes
   HyphenatedWordsFilter where they would create invalid offsets in
   some situations, leading to problems in highlighting.  (Robert Muir)
 
+* SOLR-2280: commitWithin ignored for a delete query (Juan Grande via janhoy)
+
 Other Changes
 ----------------------
 * SOLR-2922: Upgrade commons-io and commons-lang to 2.1 and 2.6, respectively. (koji)

Modified: lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/BinaryUpdateRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/BinaryUpdateRequestHandler.java?rev=1235304&r1=1235303&r2=1235304&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/BinaryUpdateRequestHandler.java (original)
+++ lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/BinaryUpdateRequestHandler.java Tue Jan 24 15:39:17 2012
@@ -34,7 +34,6 @@ import org.apache.solr.update.processor.
 import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.List;
 
 /**
  * Update handler which uses the JavaBin format
@@ -104,11 +103,8 @@ public class BinaryUpdateRequestHandler 
         log.error("Exception while processing update request", e);
         break;
       }
-      if (update.getDeleteById() != null) {
-        delete(update.getDeleteById(), processor, true);
-      }
-      if (update.getDeleteQuery() != null) {
-        delete(update.getDeleteQuery(), processor, false);
+      if (update.getDeleteById() != null || update.getDeleteQuery() != null) {
+        delete(update, processor);
       }
     }
   }
@@ -117,37 +113,37 @@ public class BinaryUpdateRequestHandler 
     AddUpdateCommand addCmd = new AddUpdateCommand();
     boolean overwrite = true;  // the default
 
-    Boolean overwritePending = null;
-    Boolean overwriteCommitted = null;
-
-
     overwrite = params.getBool(UpdateParams.OVERWRITE, overwrite);
     addCmd.commitWithin = params.getInt(UpdateParams.COMMIT_WITHIN, -1);
     // check if these flags are set
-    if (overwritePending != null && overwriteCommitted != null) {
-      if (overwritePending != overwriteCommitted) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                "can't have different values for 'overwritePending' and 'overwriteCommitted'");
-      }
-      overwrite = overwritePending;
-    }
     addCmd.overwriteCommitted = overwrite;
     addCmd.overwritePending = overwrite;
     addCmd.allowDups = !overwrite;
     return addCmd;
   }
 
-  private void delete(List<String> l, UpdateRequestProcessor processor, boolean isId) throws IOException {
-    for (String s : l) {
-      DeleteUpdateCommand delcmd = new DeleteUpdateCommand();
-      if (isId) {
+  private void delete(UpdateRequest update, UpdateRequestProcessor processor) throws IOException {
+    SolrParams params = update.getParams();
+    DeleteUpdateCommand delcmd = new DeleteUpdateCommand();
+    delcmd.fromCommitted = true;
+    delcmd.fromPending = true;
+    if(params != null) {
+      delcmd.commitWithin = params.getInt(UpdateParams.COMMIT_WITHIN, -1);
+    }
+    
+    if(update.getDeleteById() != null) {
+      for (String s : update.getDeleteById()) {
         delcmd.id = s;
-      } else {
+        processor.processDelete(delcmd);
+      }
+      delcmd.id = null;
+    }
+    
+    if(update.getDeleteQuery() != null) {
+      for (String s : update.getDeleteQuery()) {
         delcmd.query = s;
+        processor.processDelete(delcmd);
       }
-      delcmd.fromCommitted = true;
-      delcmd.fromPending = true;
-      processor.processDelete(delcmd);
     }
   }
 

Modified: lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/JsonLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/JsonLoader.java?rev=1235304&r1=1235303&r2=1235304&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/JsonLoader.java (original)
+++ lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/JsonLoader.java Tue Jan 24 15:39:17 2012
@@ -156,7 +156,8 @@ class JsonLoader extends ContentStreamLo
 
     DeleteUpdateCommand cmd = new DeleteUpdateCommand();
     cmd.fromCommitted = cmd.fromPending = true;
-
+    cmd.commitWithin = commitWithin;
+    
     while( true ) {
       int ev = parser.nextEvent();
       if( ev == JSONParser.STRING ) {
@@ -168,7 +169,9 @@ class JsonLoader extends ContentStreamLo
           else if( "query".equals(key) ) {
             cmd.query = parser.getString();
           }
-          else {
+          else if( "commitWithin".equals(key) ) { 
+            cmd.commitWithin = Integer.parseInt(parser.getString());
+          } else {
             throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown key: "+key+" ["+parser.getPosition()+"]" );
           }
         }

Modified: lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/XMLLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/XMLLoader.java?rev=1235304&r1=1235303&r2=1235304&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/XMLLoader.java (original)
+++ lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/XMLLoader.java Tue Jan 24 15:39:17 2012
@@ -198,28 +198,43 @@ class XMLLoader extends ContentStreamLoa
           } // end rollback
           else if (XmlUpdateRequestHandler.DELETE.equals(currTag)) {
             XmlUpdateRequestHandler.log.trace("parsing delete");
-            processDelete(processor, parser);
+            processDelete(req, processor, parser);
           } // end delete
           break;
       }
     }
   }
 
+  /*
+   * Signature for backward compat
+   */
+  void processDelete(UpdateRequestProcessor processor, XMLStreamReader parser) throws XMLStreamException, IOException {
+    processDelete(null, processor, parser);
+  }
+
   /**
    * @since solr 1.3
    */
-  void processDelete(UpdateRequestProcessor processor, XMLStreamReader parser) throws XMLStreamException, IOException {
+  void processDelete(SolrQueryRequest req, UpdateRequestProcessor processor, XMLStreamReader parser) throws XMLStreamException, IOException {
     // Parse the command
     DeleteUpdateCommand deleteCmd = new DeleteUpdateCommand();
     deleteCmd.fromPending = true;
     deleteCmd.fromCommitted = true;
+
+    // Need to instansiate a SolrParams, even if req is null, for backward compat with legacyUpdate
+    SolrParams params = (req != null) ? req.getParams() : new ModifiableSolrParams();
+
+    // First look for commitWithin parameter on the request, will be overwritten for individual <delete>'s
+    deleteCmd.commitWithin = params.getInt(UpdateParams.COMMIT_WITHIN, -1);
     for (int i = 0; i < parser.getAttributeCount(); i++) {
       String attrName = parser.getAttributeLocalName(i);
       String attrVal = parser.getAttributeValue(i);
-      if ("fromPending".equals(attrName)) {
+      if (XmlUpdateRequestHandler.FROM_PENDING.equals(attrName)) {
         deleteCmd.fromPending = StrUtils.parseBoolean(attrVal);
-      } else if ("fromCommitted".equals(attrName)) {
+      } else if (XmlUpdateRequestHandler.FROM_COMMITTED.equals(attrName)) {
         deleteCmd.fromCommitted = StrUtils.parseBoolean(attrVal);
+      } else if (XmlUpdateRequestHandler.COMMIT_WITHIN.equals(attrName)) {
+        deleteCmd.commitWithin = Integer.parseInt(attrVal);
       } else {
         XmlUpdateRequestHandler.log.warn("unexpected attribute delete/@" + attrName);
       }

Modified: lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/XmlUpdateRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/XmlUpdateRequestHandler.java?rev=1235304&r1=1235303&r2=1235304&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/XmlUpdateRequestHandler.java (original)
+++ lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/handler/XmlUpdateRequestHandler.java Tue Jan 24 15:39:17 2012
@@ -60,6 +60,9 @@ public class XmlUpdateRequestHandler ext
   //NOTE: This constant is for use with the <add> XML tag, not the HTTP param with same name
   public static final String COMMIT_WITHIN = "commitWithin";
   
+  public static final String FROM_COMMITTED = "fromCommitted";
+  public static final String FROM_PENDING = "fromPending";
+  
   /**
    * @deprecated use {@link #OVERWRITE}
    */

Modified: lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/update/CommitTracker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/update/CommitTracker.java?rev=1235304&r1=1235303&r2=1235304&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/update/CommitTracker.java (original)
+++ lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/update/CommitTracker.java Tue Jan 24 15:39:17 2012
@@ -91,6 +91,14 @@ final class CommitTracker implements Run
   public void scheduleCommitWithin(long commitMaxTime) {
     _scheduleCommitWithin(commitMaxTime);
   }
+  
+  private void _scheduleCommitWithinIfNeeded(long commitWithin) {
+    long ctime = (commitWithin > 0) ? commitWithin : timeUpperBound;
+
+    if (ctime > 0) {
+      _scheduleCommitWithin(ctime);
+    }
+  }
 
   private void _scheduleCommitWithin(long commitMaxTime) {
     if (commitMaxTime <= 0) return;
@@ -139,11 +147,14 @@ final class CommitTracker implements Run
     }
     
     // maxTime-triggered autoCommit
-    long ctime = (commitWithin > 0) ? commitWithin : timeUpperBound;
-
-    if (ctime > 0) {
-      _scheduleCommitWithin(ctime);
-    }
+    _scheduleCommitWithinIfNeeded(commitWithin);
+  }
+  
+  /** 
+   * Indicate that documents have been deleted
+   */
+  public void deletedDocument( int commitWithin ) {
+    _scheduleCommitWithinIfNeeded(commitWithin);
   }
   
   /** Inform tracker that a commit has occurred */

Modified: lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/update/DeleteUpdateCommand.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/update/DeleteUpdateCommand.java?rev=1235304&r1=1235303&r2=1235304&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/update/DeleteUpdateCommand.java (original)
+++ lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/update/DeleteUpdateCommand.java Tue Jan 24 15:39:17 2012
@@ -24,6 +24,7 @@ public class DeleteUpdateCommand extends
   public String query; // query string for delete-by-query
   public boolean fromPending;
   public boolean fromCommitted;
+  public int commitWithin = -1;
 
   public DeleteUpdateCommand() {
     super("delete");
@@ -37,6 +38,7 @@ public class DeleteUpdateCommand extends
     else sb.append("query=`").append(query).append('`');
     sb.append(",fromPending=").append(fromPending);
     sb.append(",fromCommitted=").append(fromCommitted);
+    sb.append(",commitWithin=").append(commitWithin);
     return sb.toString();
   }
 }

Modified: lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java?rev=1235304&r1=1235303&r2=1235304&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java (original)
+++ lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java Tue Jan 24 15:39:17 2012
@@ -284,6 +284,7 @@ public class DirectUpdateHandler2 extend
     iwCommit.lock();
     try {
       openWriter();
+      commitTracker.deletedDocument( cmd.commitWithin );
       writer.deleteDocuments(idTerm.createTerm(idFieldType.toInternal(cmd.id)));
     } finally {
       iwCommit.unlock();
@@ -318,6 +319,7 @@ public class DirectUpdateHandler2 extend
 
      iwCommit.lock();
      try {
+       commitTracker.deletedDocument(cmd.commitWithin);
        if (delAll) {
          deleteAll();
        } else {

Modified: lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/handler/JsonLoaderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/handler/JsonLoaderTest.java?rev=1235304&r1=1235303&r2=1235304&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/handler/JsonLoaderTest.java (original)
+++ lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/handler/JsonLoaderTest.java Tue Jan 24 15:39:17 2012
@@ -68,7 +68,9 @@ public class JsonLoaderTest extends Solr
       "'optimize': { 'waitFlush':false, 'waitSearcher':false },\n" +
       "\n" +
       "'delete': { 'id':'ID' },\n" +
+      "'delete': { 'id':'ID', 'commitWithin':'500' },\n" +
       "'delete': { 'query':'QUERY' },\n" +
+      "'delete': { 'query':'QUERY', 'commitWithin':'500' },\n" +
       "'rollback': {}\n" +
       "\n" +
       "}\n" +
@@ -115,16 +117,30 @@ public class JsonLoaderTest extends Solr
     
 
     // DELETE COMMANDS
-    assertEquals( 2, p.deleteCommands.size() );
+    assertEquals( 4, p.deleteCommands.size() );
     DeleteUpdateCommand delete = p.deleteCommands.get( 0 );
     assertEquals( delete.id, "ID" );
     assertEquals( delete.query, null );
     assertTrue(delete.fromPending && delete.fromCommitted);
-
+    assertEquals( delete.commitWithin, -1);
+    
     delete = p.deleteCommands.get( 1 );
+    assertEquals( delete.id, "ID" );
+    assertEquals( delete.query, null );
+    assertTrue(delete.fromPending && delete.fromCommitted);
+    assertEquals( delete.commitWithin, 500);
+    
+    delete = p.deleteCommands.get( 2 );
+    assertEquals( delete.id, null );
+    assertEquals( delete.query, "QUERY" );
+    assertTrue(delete.fromPending && delete.fromCommitted);
+    assertEquals( delete.commitWithin, -1);
+    
+    delete = p.deleteCommands.get( 3 );
     assertEquals( delete.id, null );
     assertEquals( delete.query, "QUERY" );
     assertTrue(delete.fromPending && delete.fromCommitted);
+    assertEquals( delete.commitWithin, 500);
 
     // ROLLBACK COMMANDS
     assertEquals( 1, p.rollbackCommands.size() );

Modified: lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/handler/XmlUpdateRequestHandlerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/handler/XmlUpdateRequestHandlerTest.java?rev=1235304&r1=1235303&r2=1235304&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/handler/XmlUpdateRequestHandlerTest.java (original)
+++ lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/handler/XmlUpdateRequestHandlerTest.java Tue Jan 24 15:39:17 2012
@@ -16,13 +16,20 @@
  */
 package org.apache.solr.handler;
 
+import org.apache.commons.lang.ObjectUtils;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.DeleteUpdateCommand;
 import org.apache.solr.update.processor.BufferingRequestProcessor;
+import org.apache.solr.update.processor.UpdateRequestProcessor;
 import org.apache.solr.util.AbstractSolrTestCase;
+
+import java.io.IOException;
 import java.io.StringReader;
 import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Queue;
 
 import javax.xml.stream.XMLInputFactory;
 import javax.xml.stream.XMLStreamReader;
@@ -109,5 +116,76 @@ public class XmlUpdateRequestHandlerTest
     assertEquals(true, add.allowDups);
     req.close();
   }
+  
+  @Test
+  public void testReadDelete() throws Exception {
+	    String xml =
+	      "<update>" +
+	      " <delete>" +
+	      "   <query>id:150</query>" +
+	      "   <id>150</id>" +
+	      "   <id>200</id>" +
+	      "   <query>id:200</query>" +
+	      " </delete>" +
+	      " <delete commitWithin=\"500\">" +
+	      "   <query>id:150</query>" +
+	      " </delete>" +
+	      " <delete fromPending=\"false\">" +
+	      "   <id>150</id>" +
+	      " </delete>" +
+	      " <delete fromCommitted=\"false\">" +
+	      "   <id>150</id>" +
+	      " </delete>" +
+	      "</update>";
+	    
+	    MockUpdateRequestProcessor p = new MockUpdateRequestProcessor(null);
+	    p.expectDelete(null, "id:150", true, true, -1);
+	    p.expectDelete("150", null, true, true, -1);
+	    p.expectDelete("200", null, true, true, -1);
+	    p.expectDelete(null, "id:200", true, true, -1);
+	    p.expectDelete(null, "id:150", true, true, 500);
+	    p.expectDelete("150", null, false, true, -1);
+	    p.expectDelete("150", null, true, false, -1);
+
+	    XMLLoader loader = new XMLLoader(p, inputFactory);
+	    loader.load(req(), new SolrQueryResponse(), new ContentStreamBase.StringStream(xml));
+	    
+	    p.assertNoCommandsPending();
+	  }
+	  
+	  private class MockUpdateRequestProcessor extends UpdateRequestProcessor {
+	    
+	    private Queue<DeleteUpdateCommand> deleteCommands = new LinkedList<DeleteUpdateCommand>();
+	    
+	    public MockUpdateRequestProcessor(UpdateRequestProcessor next) {
+	      super(next);
+	    }
+	    
+	    public void expectDelete(String id, String query, boolean fromPending, boolean fromCommitted, int commitWithin) {
+	      DeleteUpdateCommand cmd = new DeleteUpdateCommand();
+	      cmd.id = id;
+	      cmd.query = query;
+	      cmd.fromCommitted = fromCommitted;
+	      cmd.fromPending = fromPending;
+	      cmd.commitWithin = commitWithin;
+	      deleteCommands.add(cmd);
+	    }
+	    
+	    public void assertNoCommandsPending() {
+	      assertTrue(deleteCommands.isEmpty());
+	    }
+	    
+	    @Override
+	    public void processDelete(DeleteUpdateCommand cmd) throws IOException {
+	      DeleteUpdateCommand expected = deleteCommands.poll();
+	      assertNotNull("Unexpected delete command: [" + cmd + "]", expected);
+	      assertTrue("Expected [" + expected + "] but found [" + cmd + "]",
+	          ObjectUtils.equals(expected.id, cmd.id) &&
+	          ObjectUtils.equals(expected.query, cmd.query) &&
+	          expected.fromPending==cmd.fromPending &&
+	          expected.fromCommitted==cmd.fromCommitted &&
+	          expected.commitWithin==cmd.commitWithin);
+	    }
+	  }
 
 }

Modified: lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/update/AutoCommitTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/update/AutoCommitTest.java?rev=1235304&r1=1235303&r2=1235304&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/update/AutoCommitTest.java (original)
+++ lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/update/AutoCommitTest.java Tue Jan 24 15:39:17 2012
@@ -254,5 +254,85 @@ public class AutoCommitTest extends Abst
 
     assertQ("now it should", req("id:500") ,"//result[@numFound=1]" );
   }
+  
+  public void testCommitWithin() throws Exception {
+    SolrCore core = h.getCore();
+    NewSearcherListener trigger = new NewSearcherListener();    
+    core.registerNewSearcherListener(trigger);
+    DirectUpdateHandler2 updater = (DirectUpdateHandler2) core.getUpdateHandler();
+    CommitTracker tracker = updater.commitTracker;
+    tracker.setTimeUpperBound(0);
+    tracker.setDocsUpperBound(-1);
+    
+    XmlUpdateRequestHandler handler = new XmlUpdateRequestHandler();
+    handler.init( null );
+    
+    MapSolrParams params = new MapSolrParams( new HashMap<String, String>() );
+    
+    // Add a single document with commitWithin == 1 second
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    SolrQueryRequestBase req = new SolrQueryRequestBase( core, params ) {};
+    req.setContentStreams( toContentStreams(
+      adoc(1000, "id", "529", "field_t", "what's inside?", "subject", "info"), null ) );
+    trigger.reset();
+    handler.handleRequest( req, rsp );
+
+    // Check it isn't in the index
+    assertQ("shouldn't find any", req("id:529") ,"//result[@numFound=0]" );
+    
+    // Wait longer than the commitWithin time
+    assertTrue("commitWithin failed to commit", trigger.waitForNewSearcher(30000));
+
+    // Add one document without commitWithin
+    req.setContentStreams( toContentStreams(
+        adoc("id", "530", "field_t", "what's inside?", "subject", "info"), null ) );
+      trigger.reset();
+      handler.handleRequest( req, rsp );
+      
+    // Check it isn't in the index
+    assertQ("shouldn't find any", req("id:530") ,"//result[@numFound=0]" );
+    
+    // Delete one document with commitWithin
+    req.setContentStreams( toContentStreams(
+      delI("529", "commitWithin", "1000"), null ) );
+    trigger.reset();
+    handler.handleRequest( req, rsp );
+      
+    // Now make sure we can find it
+    assertQ("should find one", req("id:529") ,"//result[@numFound=1]" );
+    
+    // Wait for the commit to happen
+    assertTrue("commitWithin failed to commit", trigger.waitForNewSearcher(30000));
+    
+    // Now we shouldn't find it
+    assertQ("should find none", req("id:529") ,"//result[@numFound=0]" );
+    // ... but we should find the new one
+    assertQ("should find one", req("id:530") ,"//result[@numFound=1]" );
+    
+    trigger.reset();
+    
+    // now make the call 10 times really fast and make sure it 
+    // only commits once
+    req.setContentStreams( toContentStreams(
+        adoc(1000, "id", "500" ), null ) );
+    for( int i=0;i<10; i++ ) {
+      handler.handleRequest( req, rsp );
+    }
+    assertQ("should not be there yet", req("id:500") ,"//result[@numFound=0]" );
+    
+    // the same for the delete
+    req.setContentStreams( toContentStreams(
+        delI("530", "commitWithin", "1000"), null ) );
+    for( int i=0;i<10; i++ ) {
+      handler.handleRequest( req, rsp );
+    }
+    assertQ("should be there", req("id:530") ,"//result[@numFound=1]" );
+    
+    assertTrue("commitWithin failed to commit", trigger.waitForNewSearcher(30000));
+    assertQ("should be there", req("id:500") ,"//result[@numFound=1]" );
+    assertQ("should not be there", req("id:530") ,"//result[@numFound=0]" );
+    
+    assertEquals(3, tracker.getCommitCount());
+  }
 
 }

Modified: lucene/dev/branches/branch_3x/solr/solrj/src/java/org/apache/solr/client/solrj/SolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/solrj/src/java/org/apache/solr/client/solrj/SolrServer.java?rev=1235304&r1=1235303&r2=1235304&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/solrj/src/java/org/apache/solr/client/solrj/SolrServer.java (original)
+++ lucene/dev/branches/branch_3x/solr/solrj/src/java/org/apache/solr/client/solrj/SolrServer.java Tue Jan 24 15:39:17 2012
@@ -225,7 +225,22 @@ public abstract class SolrServer impleme
    * @throws IOException
    */
   public UpdateResponse deleteById(String id) throws SolrServerException, IOException {
-    return new UpdateRequest().deleteById( id ).process( this );
+    return deleteById(id, -1);
+  }
+
+  /**
+   * Deletes a single document by unique ID, specifying max time before commit
+   * @param id  the ID of the document to delete
+   * @param commitWithinMs  max time (in ms) before a commit will happen 
+   * @throws SolrServerException
+   * @throws IOException
+   * @since 3.6
+   */
+  public UpdateResponse deleteById(String id, int commitWithinMs) throws SolrServerException, IOException {
+    UpdateRequest req = new UpdateRequest();
+    req.deleteById(id);
+    req.setCommitWithin(commitWithinMs);
+    return req.process(this);
   }
 
   /**
@@ -235,7 +250,22 @@ public abstract class SolrServer impleme
    * @throws IOException
    */
   public UpdateResponse deleteById(List<String> ids) throws SolrServerException, IOException {
-    return new UpdateRequest().deleteById( ids ).process( this );
+    return deleteById(ids, -1);
+  }
+
+  /**
+   * Deletes a list of documents by unique ID, specifying max time before commit
+   * @param ids  the list of document IDs to delete 
+   * @param commitWithinMs  max time (in ms) before a commit will happen 
+   * @throws SolrServerException
+   * @throws IOException
+   * @since 3.6
+   */
+  public UpdateResponse deleteById(List<String> ids, int commitWithinMs) throws SolrServerException, IOException {
+    UpdateRequest req = new UpdateRequest();
+    req.deleteById(ids);
+    req.setCommitWithin(commitWithinMs);
+    return req.process(this);
   }
 
   /**
@@ -245,7 +275,22 @@ public abstract class SolrServer impleme
    * @throws IOException
    */
   public UpdateResponse deleteByQuery(String query) throws SolrServerException, IOException {
-    return new UpdateRequest().deleteByQuery( query ).process( this );
+    return deleteByQuery(query, -1);
+  }
+
+  /**
+   * Deletes documents from the index based on a query, specifying max time before commit
+   * @param query  the query expressing what documents to delete
+   * @param commitWithinMs  max time (in ms) before a commit will happen 
+   * @throws SolrServerException
+   * @throws IOException
+   * @since 3.6
+   */
+  public UpdateResponse deleteByQuery(String query, int commitWithinMs) throws SolrServerException, IOException {
+    UpdateRequest req = new UpdateRequest();
+    req.deleteByQuery(query);
+    req.setCommitWithin(commitWithinMs);
+    return req.process(this);
   }
 
   /**

Modified: lucene/dev/branches/branch_3x/solr/solrj/src/java/org/apache/solr/client/solrj/request/JavaBinUpdateRequestCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/solrj/src/java/org/apache/solr/client/solrj/request/JavaBinUpdateRequestCodec.java?rev=1235304&r1=1235303&r2=1235304&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/solrj/src/java/org/apache/solr/client/solrj/request/JavaBinUpdateRequestCodec.java (original)
+++ lucene/dev/branches/branch_3x/solr/solrj/src/java/org/apache/solr/client/solrj/request/JavaBinUpdateRequestCodec.java Tue Jan 24 15:39:17 2012
@@ -148,6 +148,15 @@ public class JavaBinUpdateRequestCodec {
       }
     };
     codec.unmarshal(is);
+    
+    // NOTE: if the update request contains only delete commands the params
+    // must be loaded now
+    if(updateRequest.getParams()==null) {
+      NamedList params = (NamedList) namedList[0].get("params");
+      if(params!=null) {
+        updateRequest.setParams(new ModifiableSolrParams(SolrParams.toSolrParams(params)));
+      }
+    }
     delById = (List<String>) namedList[0].get("delById");
     delByQ = (List<String>) namedList[0].get("delByQ");
     doclist = (List<List<NamedList>>) namedList[0].get("docs");

Modified: lucene/dev/branches/branch_3x/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java?rev=1235304&r1=1235303&r2=1235304&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java (original)
+++ lucene/dev/branches/branch_3x/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java Tue Jan 24 15:39:17 2012
@@ -229,7 +229,11 @@ public class UpdateRequest extends Abstr
     boolean deleteI = deleteById != null && deleteById.size() > 0;
     boolean deleteQ = deleteQuery != null && deleteQuery.size() > 0;
     if( deleteI || deleteQ ) {
-      writer.append( "<delete>" );
+      if(commitWithin>0) {
+        writer.append( "<delete commitWithin=\"" + commitWithin + "\">" );
+      } else {
+        writer.append( "<delete>" );
+      }
       if( deleteI ) {
         for( String id : deleteById ) {
           writer.append( "<id>" );

Modified: lucene/dev/branches/branch_3x/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java?rev=1235304&r1=1235303&r2=1235304&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java (original)
+++ lucene/dev/branches/branch_3x/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java Tue Jan 24 15:39:17 2012
@@ -304,7 +304,7 @@ abstract public class SolrExampleTests e
    * query the example
    */
  @Test
- public void testCommitWithin() throws Exception
+ public void testCommitWithinOnAdd() throws Exception
   {    
     // make sure it is empty...
     SolrServer server = getSolrServer();
@@ -340,7 +340,6 @@ abstract public class SolrExampleTests e
     
     Assert.assertEquals( 1, rsp.getResults().getNumFound() );
     
-
     // Now test the new convenience parameter on the add() for commitWithin
     SolrInputDocument doc4 = new SolrInputDocument();
     doc4.addField( "id", "id4", 1.0f );
@@ -361,7 +360,52 @@ abstract public class SolrExampleTests e
     }
     
     Assert.assertEquals( 1, rsp.getResults().getNumFound() );
+  }
+ 
+ @Test
+ public void testCommitWithinOnDelete() throws Exception
+  {    
+    // make sure it is empty...
+    SolrServer server = getSolrServer();
+    server.deleteByQuery( "*:*" );// delete everything!
+    server.commit();
+    QueryResponse rsp = server.query( new SolrQuery( "*:*") );
+    Assert.assertEquals( 0, rsp.getResults().getNumFound() );
 
+    // Now add one document...
+    SolrInputDocument doc3 = new SolrInputDocument();
+    doc3.addField( "id", "id3", 1.0f );
+    doc3.addField( "name", "doc3", 1.0f );
+    doc3.addField( "price", 10 );
+    server.add(doc3);
+    server.commit();
+
+    // now check that it comes out...
+    rsp = server.query( new SolrQuery( "id:id3") );    
+    Assert.assertEquals( 1, rsp.getResults().getNumFound() );
+    
+    // now test commitWithin on a delete
+    UpdateRequest up = new UpdateRequest();
+    up.setCommitWithin(1000);
+    up.deleteById("id3");
+    up.process( server );
+    
+    // the document should still be there
+    rsp = server.query( new SolrQuery( "id:id3") );
+    Assert.assertEquals( 1, rsp.getResults().getNumFound() );
+    
+    // check if the doc has been deleted every 250 ms for 30 seconds
+    long timeout = System.currentTimeMillis() + 30000;
+    do {
+      Thread.sleep( 250 ); // wait 250 ms
+      
+      rsp = server.query( new SolrQuery( "id:id3") );
+      if(rsp.getResults().getNumFound()==0) {
+        return;
+      }
+    } while(System.currentTimeMillis()<timeout);
+    
+    Assert.fail("commitWithin failed to commit");
   }
 
 

Modified: lucene/dev/branches/branch_3x/solr/test-framework/src/java/org/apache/solr/util/AbstractSolrTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/test-framework/src/java/org/apache/solr/util/AbstractSolrTestCase.java?rev=1235304&r1=1235303&r2=1235304&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/test-framework/src/java/org/apache/solr/util/AbstractSolrTestCase.java (original)
+++ lucene/dev/branches/branch_3x/solr/test-framework/src/java/org/apache/solr/util/AbstractSolrTestCase.java Tue Jan 24 15:39:17 2012
@@ -316,6 +316,20 @@ public abstract class AbstractSolrTestCa
     Doc d = doc(fieldsAndValues);
     return add(d);
   }
+  
+  /**
+   * Generates a simple &lt;add&gt;&lt;doc&gt;... XML String with the
+   * commitWithin attribute.
+   *
+   * @param commitWithin the value of the commitWithin attribute 
+   * @param fieldsAndValues 0th and Even numbered args are fields names odds are field values.
+   * @see #add
+   * @see #doc
+   */
+  public String adoc(int commitWithin, String... fieldsAndValues) {
+    Doc d = doc(fieldsAndValues);
+    return add(d, "commitWithin", String.valueOf(commitWithin));
+  }
 
   /**
    * Generates a simple &lt;add&gt;&lt;doc&gt;... XML String with no options
@@ -366,16 +380,17 @@ public abstract class AbstractSolrTestCa
    *
    * @see TestHarness#deleteById
    */
-  public String delI(String id) {
-    return h.deleteById(id);
+  public String delI(String id, String... args) {
+    return h.deleteById(id, args);
   }
+  
   /**
    * Generates a &lt;delete&gt;... XML string for an query
    *
    * @see TestHarness#deleteByQuery
    */
-  public String delQ(String q) {
-    return h.deleteByQuery(q);
+  public String delQ(String q, String... args) {
+    return h.deleteByQuery(q, args);
   }
   
   /**

Modified: lucene/dev/branches/branch_3x/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java?rev=1235304&r1=1235303&r2=1235304&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java (original)
+++ lucene/dev/branches/branch_3x/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java Tue Jan 24 15:39:17 2012
@@ -451,32 +451,46 @@ public class TestHarness {
   /**
    * Generates a delete by query xml string
    * @param q Query that has not already been xml escaped
+   * @param args The attributes of the delete tag
    */
-  public static String deleteByQuery(String q) {
-    return delete("query", q);
+  public static String deleteByQuery(String q, String... args) {
+    try {
+      StringWriter r = new StringWriter();
+      XML.writeXML(r, "query", q);
+      return delete(r.getBuffer().toString(), args);
+    } catch(IOException e) {
+      throw new RuntimeException
+        ("this should never happen with a StringWriter", e);
+    }
   }
+  
   /**
    * Generates a delete by id xml string
    * @param id ID that has not already been xml escaped
+   * @param args The attributes of the delete tag
    */
-  public static String deleteById(String id) {
-    return delete("id", id);
+  public static String deleteById(String id, String... args) {
+    try {
+      StringWriter r = new StringWriter();
+      XML.writeXML(r, "id", id);
+      return delete(r.getBuffer().toString(), args);
+    } catch(IOException e) {
+      throw new RuntimeException
+        ("this should never happen with a StringWriter", e);
+    }
   }
         
   /**
    * Generates a delete xml string
    * @param val text that has not already been xml escaped
+   * @param args 0 and Even numbered args are params, Odd numbered args are XML escaped values.
    */
-  private static String delete(String deltype, String val) {
+  private static String delete(String val, String... args) {
     try {
       StringWriter r = new StringWriter();
-            
-      r.write("<delete>");
-      XML.writeXML(r, deltype, val);
-      r.write("</delete>");
-            
+      XML.writeUnescapedXML(r, "delete", val, (Object[])args);
       return r.getBuffer().toString();
-    } catch (IOException e) {
+    } catch(IOException e) {
       throw new RuntimeException
         ("this should never happen with a StringWriter", e);
     }