You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2021/02/24 22:03:37 UTC

[lucene-solr] branch reference_impl_dev updated: @1383 Pinning down where we can forward an update to ourself.

This is an automated email from the ASF dual-hosted git repository.

markrmiller pushed a commit to branch reference_impl_dev
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/reference_impl_dev by this push:
     new 333f8ae  @1383 Pinning down where we can forward an update to ourself.
333f8ae is described below

commit 333f8aea9e1cc901945cc0e07082838b4b5094c6
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Wed Feb 24 16:01:59 2021 -0600

    @1383 Pinning down where we can forward an update to ourself.
    
    Took 39 minutes
    
    Took 6 minutes
---
 .../src/java/org/apache/lucene/util/IOUtils.java   |  1 +
 .../org/apache/solr/update/SolrCmdDistributor.java | 30 +++++++++++++---------
 .../src/java/org/apache/solr/update/UpdateLog.java |  1 +
 .../processor/DistributedUpdateProcessor.java      |  8 ++++++
 .../processor/DistributedZkUpdateProcessor.java    | 26 ++++++++++++++-----
 .../apache/solr/cloud/DocValuesNotIndexedTest.java | 22 ++++++++--------
 .../org/apache/solr/search/join/BJQParserTest.java | 15 +++++++++--
 .../solr/common/cloud/ConnectionManager.java       |  7 +++++
 .../org/apache/solr/common/cloud/SolrZkClient.java |  7 ++++-
 .../solrj/impl/BaseSolrClientWireMockTest.java     |  1 +
 10 files changed, 87 insertions(+), 31 deletions(-)

diff --git a/lucene/core/src/java/org/apache/lucene/util/IOUtils.java b/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
index 80182bf..6420723 100644
--- a/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
+++ b/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
@@ -195,6 +195,7 @@ public final class IOUtils {
    */
   public static void deleteFilesIgnoringExceptions(Directory dir, Collection<String> files) {
     for(String name : files) {
+      if (name == null) continue;
       try {
         dir.deleteFile(name);
       } catch (Throwable ignored) {
diff --git a/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java b/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
index 7189c36..806f140 100644
--- a/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
+++ b/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
@@ -95,7 +95,7 @@ public class SolrCmdDistributor implements Closeable {
       solrClient.waitForOutstandingRequests();
     } else {
       //cancels.forEach(cancellable -> cancellable.cancel());
-      Error error = new Error();
+      Error error = new Error("ac");
       error.t = new AlreadyClosedException();
       AlreadyClosedUpdateCmd cmd = new AlreadyClosedUpdateCmd(null);
       allErrors.put(cmd, error);
@@ -176,7 +176,7 @@ public class SolrCmdDistributor implements Closeable {
       } else {
         uReq.deleteByQuery(cmd.query);
       }
-      submit(new Req(cmd, node, uReq, sync, rollupTracker, leaderTracker));
+      submit(new Req(cmd, node, uReq, sync, rollupTracker, leaderTracker), "del");
     }
   }
   
@@ -202,7 +202,7 @@ public class SolrCmdDistributor implements Closeable {
       if (cmd.isInPlaceUpdate()) {
         params.set(DistributedUpdateProcessor.DISTRIB_INPLACE_PREVVERSION, String.valueOf(cmd.prevVersion));
       }
-      submit(new Req(cmd, node, uReq, synchronous, rollupTracker, leaderTracker));
+      submit(new Req(cmd, node, uReq, synchronous, rollupTracker, leaderTracker), "add");
     }
     
   }
@@ -221,7 +221,7 @@ public class SolrCmdDistributor implements Closeable {
       uReq.setParams(params);
 
       addCommit(uReq, cmd);
-      submit(new Req(cmd, node, uReq, false));
+      submit(new Req(cmd, node, uReq, false), "commit");
     }
   }
 
@@ -235,7 +235,7 @@ public class SolrCmdDistributor implements Closeable {
         : AbstractUpdateRequest.ACTION.COMMIT, false, cmd.waitSearcher, cmd.maxOptimizeSegments, cmd.softCommit, cmd.expungeDeletes, cmd.openSearcher);
   }
 
-  private void submit(final Req req) {
+  private void submit(final Req req, String tag) {
 
     if (cancelExeption != null) {
       Throwable exp = cancelExeption;
@@ -261,7 +261,7 @@ public class SolrCmdDistributor implements Closeable {
         solrClient.request(req.uReq);
       } catch (Exception e) {
         log.error("Exception sending synchronous dist update", e);
-        Error error = new Error();
+        Error error = new Error(tag);
         error.t = e;
         error.req = req;
         if (e instanceof SolrException) {
@@ -298,7 +298,7 @@ public class SolrCmdDistributor implements Closeable {
             return;
           }
 
-          Error error = new Error();
+          Error error = new Error(tag);
           error.t = t;
           error.req = req;
           if (t instanceof SolrException) {
@@ -313,7 +313,7 @@ public class SolrCmdDistributor implements Closeable {
           if (retry) {
             log.info("Retrying distrib update on error: {}", t.getMessage());
             try {
-              submit(req);
+              submit(req, tag);
             } catch (AlreadyClosedException e) {
               
             }
@@ -325,7 +325,7 @@ public class SolrCmdDistributor implements Closeable {
       });
     } catch (Exception e) {
       log.error("Exception sending dist update", e);
-      Error error = new Error();
+      Error error = new Error(tag);
       error.t = e;
       error.req = req;
       if (e instanceof SolrException) {
@@ -333,7 +333,7 @@ public class SolrCmdDistributor implements Closeable {
       }
       if (checkRetry(error)) {
         log.info("Retrying distrib update on error: {}", e.getMessage());
-        submit(req);
+        submit(req, "root");
       } else {
         allErrors.put(req.cmd, error);
       }
@@ -422,16 +422,22 @@ public class SolrCmdDistributor implements Closeable {
   public static volatile Diagnostics.Callable testing_errorHook;  // called on error when forwarding request.  Currently data=[this, Request]
 
   public static class Error {
+    public final String tag;
     public volatile Throwable t;
     public volatile int statusCode = -1;
 
     public volatile Req req;
+
+    public Error(String tag) {
+      this.tag = tag;
+    }
     
     public String toString() {
       StringBuilder sb = new StringBuilder();
       sb.append("SolrCmdDistributor$Error: statusCode=").append(statusCode);
-      sb.append("; throwable=").append(String.valueOf(t));
-      sb.append("; req=").append(String.valueOf(req));
+      sb.append("; throwable=").append(t);
+      sb.append("; req=").append(req);
+      sb.append("; tag=").append(tag);
       return sb.toString();
     }
   }
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLog.java b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
index af8b8f7..47ae031 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
@@ -2207,6 +2207,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
     }
 
     private void waitForAllUpdatesGetExecuted(OrderedExecutor executor, LongAdder pendingTasks) {
+      if (executor == null) return;
       executor.shutdown();
       while (pendingTasks.sum() > 0) {
         executor.awaitTermination();
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
index 122331a..081f677 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
@@ -1304,8 +1304,16 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
           if (null != eMeta) {
             metadata.addAll(eMeta);
           }
+          String cause = metadata.get("cause");
+          if (cause != null) {
+            metadata.remove("cause");
+          } else {
+            cause = "";
+          }
+          metadata.add("cause", error.tag + " " + cause);
         }
       }
+
       if (0 < metadata.size()) {
         this.setMetadata(metadata);
       }
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
index b3b58d5..a0b50b2 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
@@ -246,13 +246,18 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
             });
           }
 
-          if (log.isDebugEnabled()) log.debug(
-              "processCommit - Do a local commit for leader");
-          try {
-            doLocalCommit(cmd);
-          } catch (IOException e) {
-            throw new SolrException(ErrorCode.SERVER_ERROR, e);
+          if (log.isDebugEnabled()) {
+            log.debug(
+                "processCommit - Do a local commit for leader");
           }
+
+          worker.collect("localCommit", () -> {
+            try {
+              doLocalCommit(cmd);
+            } catch (IOException e) {
+              throw new SolrException(ErrorCode.SERVER_ERROR, e);
+            }
+          });
         } else {
           // zk
 
@@ -347,6 +352,8 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
       } else {
         params.set(DISTRIB_UPDATE_PARAM, (isLeader || isSubShardLeader ? DistribPhase.FROMLEADER.toString() : DistribPhase.TOLEADER.toString()));
       }
+
+
       params.set(DISTRIB_FROM, Replica.getCoreUrl(zkController.getBaseUrl(), req.getCore().getName()));
 
       if (req.getParams().get(UpdateRequest.MIN_REPFACT) != null) {
@@ -354,6 +361,13 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
         params.set(UpdateRequest.MIN_REPFACT, req.getParams().get(UpdateRequest.MIN_REPFACT));
       }
 
+      for (SolrCmdDistributor.Node node : nodes) {
+        if (node.getCoreName().equals(desc.getName())) {
+          log.error("IllegalState, trying to send an update to ourself");
+          throw new IllegalStateException("IllegalState, trying to send an update to ourself");
+        }
+      }
+
       if (cmd.isInPlaceUpdate()) {
         params.set(DISTRIB_INPLACE_PREVVERSION, String.valueOf(cmd.prevVersion));
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java b/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java
index 4a6b8c6..ce8b400 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DocValuesNotIndexedTest.java
@@ -75,17 +75,9 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
   @BeforeClass
   public static void beforeDocValuesNotIndexedTest() throws Exception {
     System.setProperty(SolrTestCaseJ4.NUMERIC_DOCVALUES_SYSPROP, "true");
-    SolrTestCaseJ4.randomizeNumericTypesProperties();
-
-    System.setProperty("managed.schema.mutable", "true");
-    configureCluster(2)
-        .addConfig("conf1", SolrTestUtil.configset("cloud-managed"))
-        .configure();
+    System.setProperty(SolrTestCaseJ4.USE_NUMERIC_POINTS_SYSPROP, "true");
 
-    // Need enough shards that we have some shards that don't have any docs on them.
-    CollectionAdminRequest.createCollection(COLLECTION, "conf1", 4, 1)
-        .setMaxShardsPerNode(3)
-        .process(cluster.getSolrClient());
+    SolrTestCaseJ4.randomizeNumericTypesProperties();
 
     fieldsToTestSingle =
         Collections.unmodifiableList(Arrays.asList(
@@ -136,6 +128,16 @@ public class DocValuesNotIndexedTest extends SolrCloudTestCase {
             new FieldProps("sortableGSL", "sortabletext")
         ));
 
+    System.setProperty("managed.schema.mutable", "true");
+    configureCluster(2)
+        .addConfig("conf1", SolrTestUtil.configset("cloud-managed"))
+        .configure();
+
+    // Need enough shards that we have some shards that don't have any docs on them.
+    CollectionAdminRequest.createCollection(COLLECTION, "conf1", 4, 1)
+        .setMaxShardsPerNode(3)
+        .process(cluster.getSolrClient());
+
     List<Update> updateList = new ArrayList<>(fieldsToTestSingle.size() +
         fieldsToTestMulti.size() + fieldsToTestGroupSortFirst.size() + fieldsToTestGroupSortLast.size() +
         4);
diff --git a/solr/core/src/test/org/apache/solr/search/join/BJQParserTest.java b/solr/core/src/test/org/apache/solr/search/join/BJQParserTest.java
index 7de6b92..56f2726 100644
--- a/solr/core/src/test/org/apache/solr/search/join/BJQParserTest.java
+++ b/solr/core/src/test/org/apache/solr/search/join/BJQParserTest.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.ListIterator;
 import java.util.Locale;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import javax.xml.xpath.XPathConstants;
 
@@ -35,6 +36,8 @@ import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.util.TimeOut;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.search.QParser;
@@ -291,8 +294,16 @@ public class BJQParserTest extends SolrTestCaseJ4 {
   @Test
   public void testCacheHit() throws IOException {
     try (SolrCore core = h.getCore()) {
-      Gauge parentFilterCache = (Gauge) (core.getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.perSegFilter"));
-      Gauge filterCache = (Gauge) (core.getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache"));
+      Gauge parentFilterCache = null;
+      Gauge filterCache = null;
+      TimeOut timeout = new TimeOut(1, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+      while (!timeout.hasTimedOut()) {
+        parentFilterCache = (Gauge) (core.getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.perSegFilter"));
+        filterCache = (Gauge) (core.getCoreMetricManager().getRegistry().getMetrics().get("CACHE.searcher.filterCache"));
+        if (parentFilterCache != null && filterCache != null) {
+          break;
+        }
+      }
 
       Map<String,Object> parentsBefore = (Map<String,Object>) parentFilterCache.getValue();
 
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
index 4981080..3792697 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
@@ -136,6 +136,13 @@ public class ConnectionManager implements Watcher, Closeable {
     public abstract boolean isClosed();
   }
 
+  public ConnectionManager(String name, SolrZkClient client, String zkServerAddress, int zkSessionTimeout) {
+    this.name = name;
+    this.client = client;
+    this.zkServerAddress = zkServerAddress;
+    this.zkSessionTimeout = zkSessionTimeout;
+  }
+
   public ConnectionManager(String name, SolrZkClient client, String zkServerAddress, int zkSessionTimeout, OnReconnect onConnect, BeforeReconnect beforeReconnect) {
     this.name = name;
     this.client = client;
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
index 20a3c5e..f07e5cb 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
@@ -18,6 +18,7 @@ package org.apache.solr.common.cloud;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.common.AlreadyClosedException;
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.StringUtils;
@@ -128,7 +129,8 @@ public class SolrZkClient implements Closeable {
   public SolrZkClient() {
     assert (closeTracker = new CloseTracker()) != null;
     zkClientConnectTimeout = 0;
-    connManager = null;
+    connManager = new ConnectionManager("ZooKeeperConnection Watcher:"
+        + zkServerAddress, this, zkServerAddress, zkClientTimeout);
   }
 
   public SolrZkClient(String zkServerAddress, int zkClientTimeout) {
@@ -1154,6 +1156,9 @@ public class SolrZkClient implements Closeable {
 
   public SolrZooKeeper getSolrZooKeeper() {
     ZooKeeper keeper = connManager.getKeeper();
+    if (keeper == null) {
+      throw new AlreadyClosedException("No ZooKeeper object");
+    }
     return (SolrZooKeeper) keeper;
   }
 
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BaseSolrClientWireMockTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BaseSolrClientWireMockTest.java
index 67ccf7a..4be8562 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BaseSolrClientWireMockTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BaseSolrClientWireMockTest.java
@@ -166,6 +166,7 @@ public abstract class BaseSolrClientWireMockTest extends SolrTestCase {
       qtp.close();
       qtp = null;
     }
+
     if (mockSolr != null) {
       mockSolr.stop();
       mockSolr = null;