You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2016/03/12 01:26:46 UTC

[05/50] [abbrv] lucene-solr git commit: SOLR-445: adding really basic CloudSolrClient support

SOLR-445: adding really basic CloudSolrClient support


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/50697eee
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/50697eee
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/50697eee

Branch: refs/heads/jira/SOLR-445
Commit: 50697eee9585d99df7eaccf643e3e3088a4172a3
Parents: 0691d47
Author: Chris Hostetter <ho...@apache.org>
Authored: Tue Mar 8 18:51:24 2016 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Tue Mar 8 18:51:24 2016 -0700

----------------------------------------------------------------------
 .../processor/TolerantUpdateProcessor.java      |  1 +
 .../cloud/TestTolerantUpdateProcessorCloud.java | 29 +++++++++---------
 .../solr/client/solrj/impl/CloudSolrClient.java | 31 +++++++++++++++++++-
 3 files changed, 45 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50697eee/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessor.java
index a325420..cbfa1e2 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/TolerantUpdateProcessor.java
@@ -270,6 +270,7 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
     } else {
       header.add("numErrors", 0); // nocommit: eliminate from response, client can count
     }
+    header.add("maxErrors", maxErrors);
 
     // annotate any error that might be thrown (or was already thrown)
     firstErrTracker.annotate(knownErrors);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50697eee/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
index 410b142..aef0385 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
@@ -51,7 +51,6 @@ import org.apache.solr.util.RevertDefaultThreadHandlerRule;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -234,6 +233,18 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
       assertQueryDocIds(c, true, S_ONE_PRE + "1",  S_TWO_PRE + "2");
       assertQueryDocIds(c, false, "id_not_exists");
 
+      // verify adding 2 broken docs causes a clint exception
+      try {
+        UpdateResponse rsp = update(params(),
+                                    doc(f("id", S_ONE_PRE + "X"), f("foo_i", "bogus_val_X")),
+                                    doc(f("id", S_TWO_PRE + "Y"), f("foo_i", "bogus_val_Y"))
+                                    ).process(c);
+        fail("did not get a top level exception when more then 10 docs failed: " + rsp.toString());
+      } catch (SolrException e) {
+        assertEquals("not the type of error we were expecting ("+e.code()+"): " + e.toString(),
+                     400, e.code());
+      }
+        
       // verify malformed deleteByQuerys fail
       try {
         UpdateResponse rsp = update(params()).deleteByQuery("foo_i:not_a_num").process(c);
@@ -260,7 +271,6 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
   }
 
   //
-  @Ignore("nocommit: need to implement tolerante response merging in cloud client")
   public void testVariousDeletesViaCloudClient() throws Exception {
     testVariousDeletes(CLOUD_CLIENT);
   }
@@ -372,7 +382,6 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
 
   
   //
-  @Ignore("nocommit: need to implement tolerante response merging in cloud client")
   public void testVariousAddsViaCloudClient() throws Exception {
     testVariousAdds(CLOUD_CLIENT);
   }
@@ -509,9 +518,6 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
                    doc(f("id", S_ONE_PRE + "10")), // may be skipped, more then 10 fails
                    doc(f("id", S_TWO_PRE + "20"))  // may be skipped, more then 10 fails
                    ).process(client);
-
-      // nocommit: should this really be a top level exception?
-      // nocommit: or should it be an HTTP:200 with the details of what faild in the body?
       
       fail("did not get a top level exception when more then 10 docs failed: " + rsp.toString());
     } catch (SolrException e) {
@@ -563,9 +569,6 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
                           "commit", "true"),
                    docs.toArray(new SolrInputDocument[docs.size()])).process(client);
       
-      // nocommit: should this really be a top level exception?
-      // nocommit: or should it be an HTTP:200 with the details of what faild in the body?
-      
       fail("did not get a top level exception when more then 10 docs failed: " + rsp.toString());
     } catch (SolrException e) {
       // we can't make any reliable assertions about the error message, because
@@ -599,7 +602,6 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
   }
 
   //
-  @Ignore("nocommit: need to implement tolerante response merging in cloud client")
   public void testAddsMixedWithDeletesViaCloudClient() throws Exception {
     testAddsMixedWithDeletes(CLOUD_CLIENT);
   }
@@ -623,6 +625,8 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
     assertNotNull("client not initialized", client);
 
     // nocommit: test adds & deletes mixed in a single UpdateRequest, w/ tolerated failures of both types
+
+    // nocommit: be sure to include DBQ mixed with other things.
   }
 
   /** Asserts that the UpdateResponse contains the specified expectedErrs and no others */
@@ -657,11 +661,6 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
       assertTrue(assertErrPre + " ... unexpected err, not found in: " + response.toString(), found);
 
     }
-    
-    // nocommit: retire numErrors, we've already checked errors.size()
-    assertEquals(assertionMsgPrefix + ": numErrors: " + response.toString(),
-                 expectedErrs.length, response.getResponseHeader().get("numErrors"));
- 
   }
   
   /** convinience method when the only type of errors you expect are 'add' errors */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50697eee/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index 59b37c5..0248a7d 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -72,6 +72,7 @@ import org.apache.solr.common.params.UpdateParams;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.Hash;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.SolrjNamedThreadFactory;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.zookeeper.KeeperException;
@@ -726,6 +727,11 @@ public class CloudSolrClient extends SolrClient {
     int status = 0;
     Integer rf = null;
     Integer minRf = null;
+    
+    // TolerantUpdateProcessor
+    List<SimpleOrderedMap<String>> toleratedErrors = null; 
+    int maxToleratedErrors = Integer.MAX_VALUE;
+      
     for(int i=0; i<response.size(); i++) {
       NamedList shardResponse = (NamedList)response.getVal(i);
       NamedList header = (NamedList)shardResponse.get("responseHeader");      
@@ -741,6 +747,23 @@ public class CloudSolrClient extends SolrClient {
           rf = routeRf;
       }
       minRf = (Integer)header.get(UpdateRequest.MIN_REPFACT);
+
+      List<SimpleOrderedMap<String>> shardTolerantErrors = 
+        (List<SimpleOrderedMap<String>>) header.get("errors");
+      if (null != shardTolerantErrors) {
+        Number shardMaxToleratedErrors = (Number) header.get("maxErrors");
+        assert null != shardMaxToleratedErrors : "TolerantUpdateProcessor reported errors but not maxErrors";
+        // if we get into some weird state where the nodes disagree about the effective maxErrors,
+        // assume the min value seen to decide if we should fail.
+        maxToleratedErrors = Math.min(maxToleratedErrors, shardMaxToleratedErrors.intValue());
+        
+        if (null == toleratedErrors) {
+          toleratedErrors = new ArrayList<SimpleOrderedMap<String>>(shardTolerantErrors.size());
+        }
+        for (SimpleOrderedMap<String> err : shardTolerantErrors) {
+          toleratedErrors.add(err);
+        }
+      }
     }
 
     NamedList cheader = new NamedList();
@@ -750,7 +773,13 @@ public class CloudSolrClient extends SolrClient {
       cheader.add(UpdateRequest.REPFACT, rf);
     if (minRf != null)
       cheader.add(UpdateRequest.MIN_REPFACT, minRf);
-    
+    if (null != toleratedErrors) {
+      cheader.add("errors", toleratedErrors);
+      if (maxToleratedErrors < toleratedErrors.size()) {
+        // nocommit: populate metadata based on the toleratedErrors
+        throw new SolrException(ErrorCode.BAD_REQUEST, "nocommit: need better msg");
+      }
+    }
     condensed.add("responseHeader", cheader);
     return condensed;
   }