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/24 20:09:29 UTC

[1/3] lucene-solr:jira/SOLR-445: SOLR-445: removing questionable isLeader check; beasting the tests w/o this code didn't demonstrate any problems

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/SOLR-445 da3ea40e8 -> b08c284b2


SOLR-445: removing questionable isLeader check; beasting the tests w/o this code didn't demonstrate any problems


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

Branch: refs/heads/jira/SOLR-445
Commit: 39884c0b0c02b4090640d6268a45a1cf5f54f3e0
Parents: da3ea40
Author: Chris Hostetter <ho...@apache.org>
Authored: Thu Mar 24 11:45:45 2016 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Thu Mar 24 11:45:45 2016 -0700

----------------------------------------------------------------------
 .../processor/TolerantUpdateProcessor.java      | 39 +++-----------------
 1 file changed, 5 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/39884c0b/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 316a8d0..96851b3 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
@@ -147,37 +147,22 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
   
   @Override
   public void processAdd(AddUpdateCommand cmd) throws IOException {
-    boolean isLeader = true; // set below during 'try'   // nocommit: is this var really needed (see below)
     BytesRef id = null;
     
     try {
       // force AddUpdateCommand to validate+cache the id before proceeding
       id = cmd.getIndexedId();
-      // if the id is missing from doc, act like we're the leader, let downstream throw error
-      isLeader = (null == id) || isLeader(cmd); // nocommit: is this needed? see below...
       
       super.processAdd(cmd);
 
     } catch (Throwable t) { 
       firstErrTracker.caught(t);
+      knownErrors.add(new ToleratedUpdateError
+                      (CmdType.ADD,
+                       getPrintableId(id),
+                       t.getMessage()));
       
-      if (isLeader || distribPhase.equals(DistribPhase.NONE)) {
-        // nocommit: should we skip if condition and always do this? see comment in else...
-        
-        knownErrors.add(new ToleratedUpdateError
-                        (CmdType.ADD,
-                         getPrintableId(id),
-                         t.getMessage()));
-        if (knownErrors.size() > maxErrors) {
-          firstErrTracker.throwFirst();
-        }
-      } else {
-        // nocommit: is this if/else even relevant or important? ...
-        //
-        // 1) the factory won't even instantiate "this" if we are a replica being forwarded from our leader
-        // 2) so aren't we by definition either the leader or DistribPhase.NONE ?
-        // 3) even if we aren't, is there any downside to simplifying the code and always waiting until maxErors?
-        
+      if (knownErrors.size() > maxErrors) {
         firstErrTracker.throwFirst();
       }
     }
@@ -335,20 +320,6 @@ public class TolerantUpdateProcessor extends UpdateRequestProcessor {
     return uniqueKeyField.getType().indexedToReadable(ref, new CharsRefBuilder()).toString();
   }
 
-  // nocommit: 1) is this method even needed? 2) is this method correct? 3) javadocs
-  private boolean isLeader(AddUpdateCommand cmd) {
-    if(!cmd.getReq().getCore().getCoreDescriptor().getCoreContainer().isZooKeeperAware())
-      return true;
-    String collection = cmd.getReq().getCore().getCoreDescriptor().getCollectionName();
-    DocCollection coll = zkController.getClusterState().getCollection(collection);
-
-    SolrParams params = req.getParams();
-    String route = req.getParams().get(ShardParams._ROUTE_);
-    Slice slice = coll.getRouter().getTargetSlice(cmd.getHashableId(), cmd.getSolrInputDocument(), route, params, coll);
-    return slice.getLeader().getName().equals(req.getCore().getCoreDescriptor().getCloudDescriptor().getCoreNodeName());
-
-  }
-
   /**
    * Simple helper class for "tracking" any exceptions encountered.
    * 


[2/3] lucene-solr:jira/SOLR-445: SOLR-445: remove test - we have more complete coverage in TestTolerantUpdateProcessorCloud which uses the more robust SolrCloudTestCase model

Posted by ho...@apache.org.
SOLR-445: remove test - we have more complete coverage in TestTolerantUpdateProcessorCloud which uses the more robust SolrCloudTestCase model


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

Branch: refs/heads/jira/SOLR-445
Commit: 1d8cdd27993a46ae17c4ac308504513a33f01a15
Parents: 39884c0
Author: Chris Hostetter <ho...@apache.org>
Authored: Thu Mar 24 11:52:55 2016 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Thu Mar 24 11:52:55 2016 -0700

----------------------------------------------------------------------
 .../DistribTolerantUpdateProcessorTest.java     | 157 -------------------
 1 file changed, 157 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1d8cdd27/solr/core/src/test/org/apache/solr/cloud/DistribTolerantUpdateProcessorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/DistribTolerantUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/cloud/DistribTolerantUpdateProcessorTest.java
deleted file mode 100644
index 5555e31..0000000
--- a/solr/core/src/test/org/apache/solr/cloud/DistribTolerantUpdateProcessorTest.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.cloud;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.List;
-import java.util.HashSet;
-import java.util.Set;
-
-
-import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.response.QueryResponse;
-import org.apache.solr.client.solrj.response.UpdateResponse;
-import org.apache.solr.common.SolrDocumentList;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.SimpleOrderedMap;
-import org.junit.Test;
-
-// nocommit: should we just remove this class? does it give us anything not covered by TestTolerantUpdateProcessorCloud?
-public class DistribTolerantUpdateProcessorTest extends AbstractFullDistribZkTestBase {
-  
-
-  @Override
-  protected String getCloudSolrConfig() {
-    return "solrconfig-distrib-update-processor-chains.xml";
-  }
-
-  @ShardsFixed(num = 2)
-  public void testValidAdds() throws Exception {
-    del("*:*");
-    handle.clear();
-    handle.put("maxScore", SKIPVAL);
-    handle.put("QTime", SKIPVAL);
-    handle.put("timestamp", SKIPVAL);
-    SolrInputDocument validDoc = sdoc("id", "1", "text", "the quick brown fox");
-    indexDoc("tolerant-chain-max-errors-10", validDoc);
-    validDoc = sdoc("id", "2", "text", "the quick brown fox");
-    indexDoc("tolerant-chain-max-errors-not-set", validDoc);
-
-    commit();
-    ModifiableSolrParams query = new ModifiableSolrParams();
-    query.add("q", "*:*");
-    QueryResponse queryResponse = queryServer(query);
-    SolrDocumentList solrDocuments = queryResponse.getResults();
-    assertEquals("Expected to match 2 docs", solrDocuments.size(), 2);
-
-    query = new ModifiableSolrParams();
-    query.add("q", "id:1");
-    queryResponse = queryServer(query);
-    solrDocuments = queryResponse.getResults();
-    assertEquals("Expected to match 1 doc", solrDocuments.size(), 1);
-
-    query = new ModifiableSolrParams();
-    query.add("q", "id:2");
-    queryResponse = queryServer(query);
-    solrDocuments = queryResponse.getResults();
-    assertEquals("Expected to match 1 doc", solrDocuments.size(), 1);
-  }
-
-  @Test
-  @ShardsFixed(num = 2)
-  public void testInvalidAdds() throws Exception {
-    //del("*:*");
-    handle.clear();
-    handle.put("maxScore", SKIPVAL);
-    handle.put("QTime", SKIPVAL);
-    handle.put("timestamp", SKIPVAL);
-    SolrInputDocument invalidDoc = sdoc("id", 1, "iind", "b");
-    try {
-      // This doc should fail without being tolerant
-      indexDoc("not-tolerant", invalidDoc);
-      fail("Expecting exception");
-    } catch (SolrException e) {
-      // we can't make any reliable assertions about the error message, because
-      // it varies based on how the request was routed -- see SOLR-8830
-      assertTrue("not the type of error we were expecting: " + e.toString(),
-                 400 <= e.code() && e.code() < 500);
-    }
-    assertAddsSucceedWithErrors("tolerant-chain-max-errors-10",
-                                new SolrInputDocument[]{ invalidDoc,
-                                                         sdoc("id", 4, "text", "the brown fox") },
-                                null, "1");
-    commit();
-
-    ModifiableSolrParams query = new ModifiableSolrParams();
-    query.add("q", "id:4");
-    QueryResponse queryResponse = queryServer(query);
-    SolrDocumentList solrDocuments = queryResponse.getResults();
-    assertEquals("Expected to match 1 doc", 1, solrDocuments.size());
-
-  }
-
-  private void assertAddsSucceedWithErrors(String chain, SolrInputDocument[] docs,
-                                            SolrParams requestParams,
-                                            String... idsShouldFail) throws Exception {
-    
-    ModifiableSolrParams newParams = new ModifiableSolrParams(requestParams);
-    newParams.set("update.chain", chain);
-    UpdateResponse response = indexDoc(newParams, docs);
-    @SuppressWarnings("unchecked")
-    List<SimpleOrderedMap<String>> errors = (List<SimpleOrderedMap<String>>)
-      response.getResponseHeader().get("errors");
-    assertNotNull("Null errors in response: " + response.toString(), errors);
-
-    assertEquals("number of errors in response: " + response.toString(), idsShouldFail.length, errors.size());
-    
-    Set<String> addErrorIdsExpected = new HashSet<String>(Arrays.asList(idsShouldFail));
-    
-    for (SimpleOrderedMap<String> err : errors) {
-      assertEquals("error type not handled yet by this method",
-                   "ADD", err.get("type"));
-      
-      String id = err.get("id");
-      assertNotNull("null err id", id);
-      assertTrue("unexpected id in errors list: " + response.toString(),
-                 addErrorIdsExpected.contains(id));
-    }
-    
-  }
-
-  @Override
-  protected void setDistributedParams(ModifiableSolrParams params) {
-    //don't add anything, let SolrCloud handle this
-  }
-
-  private UpdateResponse indexDoc(String updateChain, SolrInputDocument doc) throws IOException, SolrServerException {
-    ModifiableSolrParams params = new ModifiableSolrParams();
-    params.set("update.chain", updateChain);
-    return indexDoc(params, doc);
-  }
-
-  private UpdateResponse indexDoc(SolrParams params, SolrInputDocument... docs)
-      throws IOException, SolrServerException {
-    int which = random().nextInt(clients.size());
-    SolrClient client = clients.get(which);
-    return add(client, params, docs);
-  }
-}


[3/3] lucene-solr:jira/SOLR-445: SOLR-445: fix logger declaration to satisfy precommit

Posted by ho...@apache.org.
SOLR-445: fix logger declaration to satisfy precommit


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

Branch: refs/heads/jira/SOLR-445
Commit: b08c284b26b1779d03693a45e219db89839461d0
Parents: 1d8cdd2
Author: Chris Hostetter <ho...@apache.org>
Authored: Thu Mar 24 12:08:57 2016 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Thu Mar 24 12:08:57 2016 -0700

----------------------------------------------------------------------
 .../apache/solr/update/processor/TolerantUpdateProcessor.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b08c284b/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 96851b3..f9437f5 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
@@ -22,7 +22,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
 import java.util.Set;
-
+import java.lang.invoke.MethodHandles;
 
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRefBuilder;
@@ -76,7 +76,7 @@ import org.slf4j.LoggerFactory;
  * @see TolerantUpdateProcessorFactory
  */
 public class TolerantUpdateProcessor extends UpdateRequestProcessor {
-  private static final Logger log = LoggerFactory.getLogger(TolerantUpdateProcessor.class);
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   
   /**
    * String to be used as document key for errors when a real uniqueKey can't be determined