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 2013/10/10 06:10:34 UTC

svn commit: r1530856 - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/client/solrj/embedded/ core/src/java/org/apache/solr/core/ core/src/test-files/solr/ core/src/test/org/apache/solr/cloud/ test-framework/src/java/org/apache/solr/cloud/

Author: markrmiller
Date: Thu Oct 10 04:10:33 2013
New Revision: 1530856

URL: http://svn.apache.org/r1530856
Log:
SOLR-5317: SolrCore persistence bugs if defining SolrCores in solr.xml.

Added:
    lucene/dev/trunk/solr/core/src/test-files/solr/solr-no-core-old-style.xml   (with props)
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrXMLCoresLocator.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
    lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
    lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1530856&r1=1530855&r2=1530856&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Thu Oct 10 04:10:33 2013
@@ -121,6 +121,9 @@ Bug Fixes
 * SOLR-5306: Extra collection creation parameters like collection.configName are
   not being respected. (Mark Miller, Liang Tianyu, Nathan Neulinger)
 
+* SOLR-5317: SolrCore persistence bugs if defining SolrCores in solr.xml.
+  (Mark Miller, Yago Riveiro)
+
 Security
 ----------------------
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java?rev=1530856&r1=1530855&r2=1530856&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/client/solrj/embedded/JettySolrRunner.java Thu Oct 10 04:10:33 2013
@@ -398,37 +398,40 @@ public class JettySolrRunner {
       startedBefore = true;
     }
     
-    if( dataDir != null) {
+    if (dataDir != null) {
       System.setProperty("solr.data.dir", dataDir);
     }
-    if( solrUlogDir != null) {
+    if (solrUlogDir != null) {
       System.setProperty("solr.ulog.dir", solrUlogDir);
     }
-    if(shards != null) {
+    if (shards != null) {
       System.setProperty("shard", shards);
     }
     if (coreNodeName != null) {
       System.setProperty("coreNodeName", coreNodeName);
     }
-    
-    if (!server.isRunning()) {
-      server.start();
-    }
-    synchronized (JettySolrRunner.this) {
-      int cnt = 0;
-      while (!waitOnSolr) {
-        this.wait(100);
-        if (cnt++ == 5) {
-          throw new RuntimeException("Jetty/Solr unresponsive");
+    try {
+      
+      if (!server.isRunning()) {
+        server.start();
+      }
+      synchronized (JettySolrRunner.this) {
+        int cnt = 0;
+        while (!waitOnSolr) {
+          this.wait(100);
+          if (cnt++ == 5) {
+            throw new RuntimeException("Jetty/Solr unresponsive");
+          }
         }
       }
+    } finally {
+      
+      System.clearProperty("shard");
+      System.clearProperty("solr.data.dir");
+      System.clearProperty("coreNodeName");
+      System.clearProperty("solr.ulog.dir");
     }
     
-    System.clearProperty("shard");
-    System.clearProperty("solr.data.dir");
-    System.clearProperty("coreNodeName");
-    System.clearProperty("solr.ulog.dir");
-
   }
 
   public void stop() throws Exception {

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java?rev=1530856&r1=1530855&r2=1530856&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java Thu Oct 10 04:10:33 2013
@@ -17,25 +17,7 @@
 
 package org.apache.solr.core;
 
-import com.google.common.collect.Maps;
-import org.apache.solr.cloud.ZkController;
-import org.apache.solr.cloud.ZkSolrResourceLoader;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.common.cloud.ZooKeeperException;
-import org.apache.solr.common.util.ExecutorUtil;
-import org.apache.solr.handler.admin.CollectionsHandler;
-import org.apache.solr.handler.admin.CoreAdminHandler;
-import org.apache.solr.handler.admin.InfoHandler;
-import org.apache.solr.handler.component.ShardHandlerFactory;
-import org.apache.solr.logging.LogWatcher;
-import org.apache.solr.schema.IndexSchema;
-import org.apache.solr.schema.IndexSchemaFactory;
-import org.apache.solr.util.DefaultSolrThreadFactory;
-import org.apache.solr.util.FileUtils;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import static com.google.common.base.Preconditions.checkNotNull;
 
 import java.io.File;
 import java.text.SimpleDateFormat;
@@ -57,11 +39,27 @@ import java.util.concurrent.ExecutorComp
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
 
-import static com.google.common.base.Preconditions.checkNotNull;
+import org.apache.solr.cloud.ZkController;
+import org.apache.solr.cloud.ZkSolrResourceLoader;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.ZooKeeperException;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.handler.admin.CollectionsHandler;
+import org.apache.solr.handler.admin.CoreAdminHandler;
+import org.apache.solr.handler.admin.InfoHandler;
+import org.apache.solr.handler.component.ShardHandlerFactory;
+import org.apache.solr.logging.LogWatcher;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.IndexSchemaFactory;
+import org.apache.solr.util.DefaultSolrThreadFactory;
+import org.apache.solr.util.FileUtils;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Maps;
 
 
 /**

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrXMLCoresLocator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrXMLCoresLocator.java?rev=1530856&r1=1530855&r2=1530856&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrXMLCoresLocator.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrXMLCoresLocator.java Thu Oct 10 04:10:33 2013
@@ -19,6 +19,7 @@ package org.apache.solr.core;
 
 import com.google.common.base.Charsets;
 import com.google.common.collect.ImmutableList;
+
 import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -28,6 +29,8 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -140,10 +143,13 @@ public class SolrXMLCoresLocator impleme
 
   @Override
   public final void persist(CoreContainer cc, CoreDescriptor... coreDescriptors) {
-    doPersist(buildSolrXML(cc.getCoreDescriptors()));
+    List<CoreDescriptor> cds = new ArrayList<CoreDescriptor>(cc.getCoreDescriptors().size() + coreDescriptors.length);
+    cds.addAll(cc.getCoreDescriptors());
+    cds.addAll(Arrays.asList(coreDescriptors));
+    doPersist(buildSolrXML(cds));
   }
 
-  protected void doPersist(String xml) {
+  protected synchronized void doPersist(String xml) {
     File file = new File(cfg.config.getResourceLoader().getInstanceDir(), ConfigSolr.SOLR_XML_FILE);
     try {
       Writer writer = new OutputStreamWriter(new FileOutputStream(file), Charsets.UTF_8);
@@ -163,12 +169,14 @@ public class SolrXMLCoresLocator impleme
 
   @Override
   public void delete(CoreContainer cc, CoreDescriptor... coreDescriptors) {
-    this.persist(cc, coreDescriptors);
+    // coreDescriptors is kind of a useless param - we persist the current state off cc
+    this.persist(cc);
   }
 
   @Override
   public void rename(CoreContainer cc, CoreDescriptor oldCD, CoreDescriptor newCD) {
-    this.persist(cc, oldCD, newCD);
+    // we don't need those params, we just write out the current cc state
+    this.persist(cc);
   }
 
   @Override

Added: lucene/dev/trunk/solr/core/src/test-files/solr/solr-no-core-old-style.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/solr-no-core-old-style.xml?rev=1530856&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/solr-no-core-old-style.xml (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/solr-no-core-old-style.xml Thu Oct 10 04:10:33 2013
@@ -0,0 +1,41 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+
+<!--
+ All (relative) paths are relative to the installation path
+  
+  persistent: Save changes made via the API to this file
+  sharedLib: path to a lib directory that will be shared across all cores
+-->
+<solr persistent="${solr.xml.persist:true}">
+
+  <!--
+  adminPath: RequestHandler path to manage cores.  
+    If 'null' (or absent), cores will not be manageable via request handler
+  -->
+  <cores adminPath="/admin/cores" defaultCoreName="collection1" host="127.0.0.1" hostPort="${hostPort:8983}" 
+         hostContext="${hostContext:solr}" zkClientTimeout="${solr.zkclienttimeout:30000}" numShards="${numShards:3}" shareSchema="${shareSchema:false}" 
+         genericCoreNodeNames="${genericCoreNodeNames:true}" leaderVoteWait="0"
+         distribUpdateConnTimeout="${distribUpdateConnTimeout:15000}" distribUpdateSoTimeout="${distribUpdateSoTimeout:120000}">
+    <shardHandlerFactory name="shardHandlerFactory" class="HttpShardHandlerFactory">
+      <int name="socketTimeout">${socketTimeout:120000}</int>
+      <int name="connTimeout">${connTimeout:15000}</int>
+    </shardHandlerFactory>
+  </cores>
+  
+</solr>

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java?rev=1530856&r1=1530855&r2=1530856&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java Thu Oct 10 04:10:33 2013
@@ -100,7 +100,10 @@ public class CollectionsAPIDistributedZk
   
   CompletionService<Request> completionService;
   Set<Future<Request>> pending;
+  
+  // we randomly use a second config set rather than just one
   private boolean secondConfigSet = random().nextBoolean();
+  private boolean oldStyleSolrXml = false;
   
   @BeforeClass
   public static void beforeThisClass2() throws Exception {
@@ -112,6 +115,14 @@ public class CollectionsAPIDistributedZk
   public void setUp() throws Exception {
     super.setUp();
     
+    useJettyDataDir = false;
+    
+    oldStyleSolrXml = random().nextBoolean();
+    if (oldStyleSolrXml) {
+      System.err.println("Using old style solr.xml");
+    } else {
+      System.err.println("Using new style solr.xml");
+    }
     if (secondConfigSet ) {
       String zkHost = zkServer.getZkHost();
       String zkAddress = zkServer.getZkAddress();
@@ -144,7 +155,8 @@ public class CollectionsAPIDistributedZk
   }
   
   protected String getSolrXml() {
-    return "solr-no-core.xml";
+    // test old style and new style solr.xml
+    return oldStyleSolrXml ? "solr-no-core-old-style.xml" : "solr-no-core.xml";
   }
 
   
@@ -572,6 +584,27 @@ public class CollectionsAPIDistributedZk
       // poll for a second - it can take a moment before we are ready to serve
       waitForNon403or404or503(collectionClient);
     }
+    
+    // sometimes we restart one of the jetty nodes
+    if (random().nextBoolean()) {
+      JettySolrRunner jetty = jettys.get(random().nextInt(jettys.size()));
+      ChaosMonkey.stop(jetty);
+      ChaosMonkey.start(jetty);
+      
+      for (Entry<String,List<Integer>> entry : collectionInfosEntrySet) {
+        String collection = entry.getKey();
+        List<Integer> list = entry.getValue();
+        checkForCollection(collection, list, null);
+        
+        String url = getUrlFromZk(collection);
+        
+        HttpSolrServer collectionClient = new HttpSolrServer(url);
+        
+        // poll for a second - it can take a moment before we are ready to serve
+        waitForNon403or404or503(collectionClient);
+      }
+    }
+
     ZkStateReader zkStateReader = getCommonCloudSolrServer().getZkStateReader();
     for (int j = 0; j < cnt; j++) {
       waitForRecoveriesToFinish("awholynewcollection_" + j, zkStateReader, false);
@@ -759,10 +792,12 @@ public class CollectionsAPIDistributedZk
         .getFilter()).getCores();
     Collection<SolrCore> theCores = cores.getCores();
     for (SolrCore core : theCores) {
-      // look for core props file
-      assertTrue("Could not find expected core.properties file",
-          new File((String) core.getStatistics().get("instanceDir"),
-              "core.properties").exists());
+      if (!oldStyleSolrXml) {
+        // look for core props file
+        assertTrue("Could not find expected core.properties file",
+            new File((String) core.getStatistics().get("instanceDir"),
+                "core.properties").exists());
+      }
       
       assertEquals(
           SolrResourceLoader.normalizeDir(jetty.getSolrHome() + File.separator

Modified: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java?rev=1530856&r1=1530855&r2=1530856&view=diff
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java (original)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java Thu Oct 10 04:10:33 2013
@@ -148,7 +148,7 @@ public abstract class AbstractDistribZkT
       for (Map.Entry<String,Slice> entry : slices.entrySet()) {
         Map<String,Replica> shards = entry.getValue().getReplicasMap();
         for (Map.Entry<String,Replica> shard : shards.entrySet()) {
-          if (verbose) System.out.println("rstate:"
+          if (verbose) System.out.println("replica:" + shard.getValue().getName() + " rstate:"
               + shard.getValue().getStr(ZkStateReader.STATE_PROP)
               + " live:"
               + clusterState.liveNodesContain(shard.getValue().getNodeName()));

Modified: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java?rev=1530856&r1=1530855&r2=1530856&view=diff
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java (original)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java Thu Oct 10 04:10:33 2013
@@ -121,6 +121,7 @@ public abstract class AbstractFullDistri
   protected Map<String,CloudJettyRunner> shardToLeaderJetty = new HashMap<String,CloudJettyRunner>();
   private boolean cloudInit;
   protected boolean checkCreatedVsState;
+  protected boolean useJettyDataDir = true;
   
   public static class CloudJettyRunner {
     public JettySolrRunner jetty;
@@ -346,8 +347,8 @@ public abstract class AbstractFullDistri
           getClass().getName() + "-jetty" + cnt + "-" + System.currentTimeMillis());
       jettyDir.mkdirs();
       setupJettySolrHome(jettyDir);
-      JettySolrRunner j = createJetty(jettyDir, getDataDir(testDir + "/jetty"
-          + cnt), null, "solrconfig.xml", null);
+      JettySolrRunner j = createJetty(jettyDir, useJettyDataDir ? getDataDir(testDir + "/jetty"
+          + cnt) : null, null, "solrconfig.xml", null);
       jettys.add(j);
       SolrServer client = createNewSolrServer(j.getLocalPort());
       clients.add(client);