You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by tf...@apache.org on 2017/05/08 22:07:22 UTC

[08/50] [abbrv] lucene-solr:jira/solr-10233: SOLR-9867: fixing TestSolrCLIRunExample.testTechproductsExample

SOLR-9867: fixing TestSolrCLIRunExample.testTechproductsExample

- SolrDispatchFilter.doFilter rejects invocation until init() is completed.
- introducing isLoaded=false, isLoading=true core status
- blocking shutdown until core loading stops
- looping run example tool while core is loading 1 min max.


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

Branch: refs/heads/jira/solr-10233
Commit: 32b7791fa8019aff2fca65c06deda48a6360da41
Parents: e912b7c
Author: Mikhail Khludnev <mk...@apache.org>
Authored: Sat Apr 29 00:25:28 2017 +0300
Committer: Mikhail Khludnev <mk...@apache.org>
Committed: Thu May 4 09:15:12 2017 +0300

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 +
 .../java/org/apache/solr/core/SolrCores.java    |  1 +
 .../solr/handler/admin/CoreAdminOperation.java  | 92 +++++++++++---------
 .../apache/solr/servlet/SolrDispatchFilter.java | 45 ++++++----
 .../src/java/org/apache/solr/util/SolrCLI.java  | 39 ++++++---
 .../apache/solr/util/TestSolrCLIRunExample.java | 47 ++++++----
 6 files changed, 136 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32b7791f/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 7eaafae..ab023cc 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -332,6 +332,9 @@ Other Changes
 
 * SOLR-10519: SolrCLI.atPath cannot handle children that begin with a slash. (Erick Erickson)
 
+* SOLR-9867: Adding isLoading=true as core status. Fixing start after stop scenario in bin/solr
+  (Mikhail Khludnev)
+
 ==================  6.5.1 ==================
 
 Bug Fixes

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32b7791f/solr/core/src/java/org/apache/solr/core/SolrCores.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCores.java b/solr/core/src/java/org/apache/solr/core/SolrCores.java
index ef6fca5..87ffac3 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCores.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCores.java
@@ -95,6 +95,7 @@ class SolrCores implements Observer {
   // We are shutting down. You can't hold the lock on the various lists of cores while they shut down, so we need to
   // make a temporary copy of the names and shut them down outside the lock.
   protected void close() {
+    waitForLoadingCoresToFinish(30*1000);
     Collection<SolrCore> coreList = new ArrayList<>();
 
     

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32b7791f/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
index a5dad1e..fbf24a1 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
@@ -303,50 +303,56 @@ enum CoreAdminOperation implements CoreAdminOp {
   static NamedList<Object> getCoreStatus(CoreContainer cores, String cname, boolean isIndexInfoNeeded) throws IOException {
     NamedList<Object> info = new SimpleOrderedMap<>();
 
-    if (!cores.isLoaded(cname)) { // Lazily-loaded core, fill in what we can.
-      // It would be a real mistake to load the cores just to get the status
-      CoreDescriptor desc = cores.getUnloadedCoreDescriptor(cname);
-      if (desc != null) {
-        info.add(NAME, desc.getName());
-        info.add("instanceDir", desc.getInstanceDir());
-        // None of the following are guaranteed to be present in a not-yet-loaded core.
-        String tmp = desc.getDataDir();
-        if (StringUtils.isNotBlank(tmp)) info.add("dataDir", tmp);
-        tmp = desc.getConfigName();
-        if (StringUtils.isNotBlank(tmp)) info.add("config", tmp);
-        tmp = desc.getSchemaName();
-        if (StringUtils.isNotBlank(tmp)) info.add("schema", tmp);
-        info.add("isLoaded", "false");
-      }
+    if (cores.isCoreLoading(cname)) {
+      info.add(NAME, cname);
+      info.add("isLoaded", "false");
+      info.add("isLoading", "true");
     } else {
-      try (SolrCore core = cores.getCore(cname)) {
-        if (core != null) {
-          info.add(NAME, core.getName());
-          info.add("instanceDir", core.getResourceLoader().getInstancePath().toString());
-          info.add("dataDir", normalizePath(core.getDataDir()));
-          info.add("config", core.getConfigResource());
-          info.add("schema", core.getSchemaResource());
-          info.add("startTime", core.getStartTimeStamp());
-          info.add("uptime", core.getUptimeMs());
-          if (cores.isZooKeeperAware()) {
-            info.add("lastPublished", core.getCoreDescriptor().getCloudDescriptor().getLastPublished().toString().toLowerCase(Locale.ROOT));
-            info.add("configVersion", core.getSolrConfig().getZnodeVersion());
-            SimpleOrderedMap cloudInfo = new SimpleOrderedMap<>();
-            cloudInfo.add(COLLECTION, core.getCoreDescriptor().getCloudDescriptor().getCollectionName());
-            cloudInfo.add(SHARD, core.getCoreDescriptor().getCloudDescriptor().getShardId());
-            cloudInfo.add(REPLICA, core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName());
-            info.add("cloud", cloudInfo);
-          }
-          if (isIndexInfoNeeded) {
-            RefCounted<SolrIndexSearcher> searcher = core.getSearcher();
-            try {
-              SimpleOrderedMap<Object> indexInfo = LukeRequestHandler.getIndexInfo(searcher.get().getIndexReader());
-              long size = getIndexSize(core);
-              indexInfo.add("sizeInBytes", size);
-              indexInfo.add("size", NumberUtils.readableSize(size));
-              info.add("index", indexInfo);
-            } finally {
-              searcher.decref();
+      if (!cores.isLoaded(cname)) { // Lazily-loaded core, fill in what we can.
+        // It would be a real mistake to load the cores just to get the status
+        CoreDescriptor desc = cores.getUnloadedCoreDescriptor(cname);
+        if (desc != null) {
+          info.add(NAME, desc.getName());
+          info.add("instanceDir", desc.getInstanceDir());
+          // None of the following are guaranteed to be present in a not-yet-loaded core.
+          String tmp = desc.getDataDir();
+          if (StringUtils.isNotBlank(tmp)) info.add("dataDir", tmp);
+          tmp = desc.getConfigName();
+          if (StringUtils.isNotBlank(tmp)) info.add("config", tmp);
+          tmp = desc.getSchemaName();
+          if (StringUtils.isNotBlank(tmp)) info.add("schema", tmp);
+          info.add("isLoaded", "false");
+        }
+      } else {
+        try (SolrCore core = cores.getCore(cname)) {
+          if (core != null) {
+            info.add(NAME, core.getName());
+            info.add("instanceDir", core.getResourceLoader().getInstancePath().toString());
+            info.add("dataDir", normalizePath(core.getDataDir()));
+            info.add("config", core.getConfigResource());
+            info.add("schema", core.getSchemaResource());
+            info.add("startTime", core.getStartTimeStamp());
+            info.add("uptime", core.getUptimeMs());
+            if (cores.isZooKeeperAware()) {
+              info.add("lastPublished", core.getCoreDescriptor().getCloudDescriptor().getLastPublished().toString().toLowerCase(Locale.ROOT));
+              info.add("configVersion", core.getSolrConfig().getZnodeVersion());
+              SimpleOrderedMap cloudInfo = new SimpleOrderedMap<>();
+              cloudInfo.add(COLLECTION, core.getCoreDescriptor().getCloudDescriptor().getCollectionName());
+              cloudInfo.add(SHARD, core.getCoreDescriptor().getCloudDescriptor().getShardId());
+              cloudInfo.add(REPLICA, core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName());
+              info.add("cloud", cloudInfo);
+            }
+            if (isIndexInfoNeeded) {
+              RefCounted<SolrIndexSearcher> searcher = core.getSearcher();
+              try {
+                SimpleOrderedMap<Object> indexInfo = LukeRequestHandler.getIndexInfo(searcher.get().getIndexReader());
+                long size = getIndexSize(core);
+                indexInfo.add("sizeInBytes", size);
+                indexInfo.add("size", NumberUtils.readableSize(size));
+                info.add("index", indexInfo);
+              } finally {
+                searcher.decref();
+              }
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32b7791f/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index 39ccadc..02b0460 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -16,18 +16,6 @@
  */
 package org.apache.solr.servlet;
 
-import javax.servlet.FilterChain;
-import javax.servlet.FilterConfig;
-import javax.servlet.ServletException;
-import javax.servlet.ServletInputStream;
-import javax.servlet.ServletOutputStream;
-import javax.servlet.ServletRequest;
-import javax.servlet.ServletResponse;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletRequestWrapper;
-import javax.servlet.http.HttpServletResponse;
-import javax.servlet.http.HttpServletResponseWrapper;
-
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -46,10 +34,18 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import com.codahale.metrics.jvm.ClassLoadingGaugeSet;
-import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
-import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
-import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.ServletInputStream;
+import javax.servlet.ServletOutputStream;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.HttpServletResponseWrapper;
+
 import org.apache.commons.io.FileCleaningTracker;
 import org.apache.commons.io.input.CloseShieldInputStream;
 import org.apache.commons.io.output.CloseShieldOutputStream;
@@ -78,6 +74,11 @@ import org.apache.solr.util.SolrFileCleaningTracker;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.codahale.metrics.jvm.ClassLoadingGaugeSet;
+import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
+import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
+import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
+
 /**
  * This filter looks at the incoming URL maps them to handlers defined in solrconfig.xml
  *
@@ -136,6 +137,8 @@ public class SolrDispatchFilter extends BaseSolrFilter {
   public void init(FilterConfig config) throws ServletException
   {
     log.trace("SolrDispatchFilter.init(): {}", this.getClass().getClassLoader());
+    CoreContainer coresInit = null;
+    try{
 
     SolrRequestParsers.fileCleaningTracker = new SolrFileCleaningTracker();
 
@@ -166,9 +169,9 @@ public class SolrDispatchFilter extends BaseSolrFilter {
       String solrHome = (String) config.getServletContext().getAttribute(SOLRHOME_ATTRIBUTE);
       ExecutorUtil.addThreadLocalProvider(SolrRequestInfo.getInheritableThreadLocalProvider());
 
-      this.cores = createCoreContainer(solrHome == null ? SolrResourceLoader.locateSolrHome() : Paths.get(solrHome),
+      coresInit = createCoreContainer(solrHome == null ? SolrResourceLoader.locateSolrHome() : Paths.get(solrHome),
                                        extraProperties);
-      this.httpClient = cores.getUpdateShardHandler().getHttpClient();
+      this.httpClient = coresInit.getUpdateShardHandler().getHttpClient();
       setupJvmMetrics();
       log.debug("user.dir=" + System.getProperty("user.dir"));
     }
@@ -181,7 +184,10 @@ public class SolrDispatchFilter extends BaseSolrFilter {
       }
     }
 
-    log.trace("SolrDispatchFilter.init() done");
+    }finally{
+      log.trace("SolrDispatchFilter.init() done");
+      this.cores = coresInit; // crucially final assignment 
+    }
   }
 
   private void setupJvmMetrics()  {
@@ -308,6 +314,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
   
   public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain, boolean retry) throws IOException, ServletException {
     if (!(request instanceof HttpServletRequest)) return;
+    
     try {
 
       if (cores == null || cores.isShutDown()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32b7791f/solr/core/src/java/org/apache/solr/util/SolrCLI.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/SolrCLI.java b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
index a6a40a0..4ab1751 100644
--- a/solr/core/src/java/org/apache/solr/util/SolrCLI.java
+++ b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
@@ -16,6 +16,12 @@
  */
 package org.apache.solr.util;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.solr.common.SolrException.ErrorCode.FORBIDDEN;
+import static org.apache.solr.common.SolrException.ErrorCode.UNAUTHORIZED;
+import static org.apache.solr.common.params.CommonParams.DISTRIB;
+import static org.apache.solr.common.params.CommonParams.NAME;
+
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -111,16 +117,12 @@ import org.noggit.ObjectBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.solr.common.SolrException.ErrorCode.FORBIDDEN;
-import static org.apache.solr.common.SolrException.ErrorCode.UNAUTHORIZED;
-import static org.apache.solr.common.params.CommonParams.DISTRIB;
-import static org.apache.solr.common.params.CommonParams.NAME;
-
 /**
  * Command-line utility for working with Solr.
  */
 public class SolrCLI {
+  private static final long MAX_WAIT_FOR_CORE_LOAD_NANOS = TimeUnit.NANOSECONDS.convert(1, TimeUnit.MINUTES);
+
   /**
    * Defines the interface to a Solr tool that can be run from this command-line app.
    */
@@ -1399,18 +1401,29 @@ public class SolrCLI {
   public static boolean safeCheckCoreExists(String coreStatusUrl, String coreName) {
     boolean exists = false;
     try {
-      Map<String,Object> existsCheckResult = getJson(coreStatusUrl);
-      Map<String,Object> status = (Map<String, Object>)existsCheckResult.get("status");
-      Map<String,Object> coreStatus = (Map<String, Object>)status.get(coreName);
-      Map<String,Object> failureStatus = (Map<String, Object>)existsCheckResult.get("initFailures");
-      String errorMsg = (String) failureStatus.get(coreName);
-      exists = coreStatus != null && coreStatus.containsKey(NAME) || errorMsg != null;
+      boolean wait = false;
+      final long startWaitAt = System.nanoTime();
+      do{
+        if (wait) {
+          final int clamPeriodForStatusPollMs = 1000;
+          Thread.sleep(clamPeriodForStatusPollMs);
+        }
+        Map<String,Object> existsCheckResult = getJson(coreStatusUrl);
+        Map<String,Object> status = (Map<String, Object>)existsCheckResult.get("status");
+        Map<String,Object> coreStatus = (Map<String, Object>)status.get(coreName);
+        Map<String,Object> failureStatus = (Map<String, Object>)existsCheckResult.get("initFailures");
+        String errorMsg = (String) failureStatus.get(coreName);
+        final boolean hasName = coreStatus != null && coreStatus.containsKey(NAME);
+        exists = hasName || errorMsg != null;
+        wait = hasName && errorMsg==null && "true".equals(coreStatus.get("isLoading"));
+      }while (wait &&
+          System.nanoTime() - startWaitAt < MAX_WAIT_FOR_CORE_LOAD_NANOS);
     } catch (Exception exc) {
       // just ignore it since we're only interested in a positive result here
     }
     return exists;
   }
-
+  
   /**
    * Supports create_collection command in the bin/solr script.
    */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32b7791f/solr/core/src/test/org/apache/solr/util/TestSolrCLIRunExample.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/util/TestSolrCLIRunExample.java b/solr/core/src/test/org/apache/solr/util/TestSolrCLIRunExample.java
index 02d91b0..6bce8226 100644
--- a/solr/core/src/test/org/apache/solr/util/TestSolrCLIRunExample.java
+++ b/solr/core/src/test/org/apache/solr/util/TestSolrCLIRunExample.java
@@ -31,6 +31,7 @@ import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 
@@ -47,8 +48,8 @@ import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.cloud.MiniSolrCloudCluster;
 import org.apache.solr.common.SolrInputDocument;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -66,8 +67,15 @@ public class TestSolrCLIRunExample extends SolrTestCaseJ4 {
   public static void beforeClass() throws IOException {
     assumeFalse("FIXME: This test does not work with whitespace in CWD (https://issues.apache.org/jira/browse/SOLR-8877)",
         Paths.get(".").toAbsolutePath().toString().contains(" "));
+    // to be true
+    System.setProperty("solr.directoryFactory", "solr.NRTCachingDirectoryFactory");
   }
 
+  @AfterClass
+  public static void cleanupDirectoryFactory() throws IOException {
+    System.clearProperty("solr.directoryFactory");
+  }
+  
   /**
    * Overrides the call to exec bin/solr to start Solr nodes to start them using the Solr test-framework
    * instead of the script, since the script depends on a full build.
@@ -289,7 +297,6 @@ public class TestSolrCLIRunExample extends SolrTestCaseJ4 {
     }
   }
 
-  @Ignore
   @Test
   public void testTechproductsExample() throws Exception {
     testExample("techproducts");
@@ -309,6 +316,7 @@ public class TestSolrCLIRunExample extends SolrTestCaseJ4 {
     File solrExampleDir = tmpDir.toFile();
     File solrServerDir = solrHomeDir.getParentFile();
 
+    for (int i =0; i<2; i++){
     // need a port to start the example server on
     int bindPort = -1;
     try (ServerSocket socket = new ServerSocket(0)) {
@@ -333,7 +341,8 @@ public class TestSolrCLIRunExample extends SolrTestCaseJ4 {
 
     SolrCLI.RunExampleTool tool = new SolrCLI.RunExampleTool(executor, System.in, stdoutSim);
     try {
-      tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), toolArgs));
+      final int status = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), toolArgs));
+      assertEquals("it should be ok "+tool+" "+Arrays.toString(toolArgs),0, status);
     } catch (Exception e) {
       log.error("RunExampleTool failed due to: " + e +
           "; stdout from tool prior to failure: " + baos.toString(StandardCharsets.UTF_8.name()));
@@ -352,25 +361,31 @@ public class TestSolrCLIRunExample extends SolrTestCaseJ4 {
 
     if ("techproducts".equals(exampleName)) {
       HttpSolrClient solrClient = getHttpSolrClient("http://localhost:" + bindPort + "/solr/" + exampleName);
-      SolrQuery query = new SolrQuery("*:*");
-      QueryResponse qr = solrClient.query(query);
-      long numFound = qr.getResults().getNumFound();
-      if (numFound == 0) {
-        // brief wait in case of timing issue in getting the new docs committed
-        log.warn("Going to wait for 1 second before re-trying query for techproduct example docs ...");
-        try {
-          Thread.sleep(1000);
-        } catch (InterruptedException ignore) {
-          Thread.interrupted();
+      try{
+        SolrQuery query = new SolrQuery("*:*");
+        QueryResponse qr = solrClient.query(query);
+        long numFound = qr.getResults().getNumFound();
+        if (numFound == 0) {
+          // brief wait in case of timing issue in getting the new docs committed
+          log.warn("Going to wait for 1 second before re-trying query for techproduct example docs ...");
+          try {
+            Thread.sleep(1000);
+          } catch (InterruptedException ignore) {
+            Thread.interrupted();
+          }
+          numFound = solrClient.query(query).getResults().getNumFound();
         }
-        numFound = solrClient.query(query).getResults().getNumFound();
+        assertTrue("expected 32 docs in the " + exampleName + " example but found " + numFound + ", output: " + toolOutput,
+            numFound == 32);
+      }finally{
+        
+        solrClient.close();
       }
-      assertTrue("expected 32 docs in the " + exampleName + " example but found " + numFound + ", output: " + toolOutput,
-          numFound == 32);
     }
 
     // stop the test instance
     executor.execute(org.apache.commons.exec.CommandLine.parse("bin/solr stop -p " + bindPort));
+    }
   }
 
   /**