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/05/23 21:35:53 UTC

svn commit: r1485835 - in /lucene/dev/trunk/solr/core/src/java/org/apache/solr: cloud/ZkController.java core/CoreContainer.java core/SolrCore.java

Author: markrmiller
Date: Thu May 23 19:35:52 2013
New Revision: 1485835

URL: http://svn.apache.org/r1485835
Log:
SOLR-4745: Do not pass a SolrCore in ZkController.preRegister()

Modified:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.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/SolrCore.java

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1485835&r1=1485834&r2=1485835&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java Thu May 23 19:35:52 2013
@@ -1277,12 +1277,9 @@ public final class ZkController {
     downloadFromZK(zkClient, ZkController.CONFIGS_ZKNODE + "/" + configName, dir);
   }
 
-  public void preRegister(SolrCore core) throws KeeperException, InterruptedException {
-    CoreDescriptor cd = core.getCoreDescriptor();
-    if (Slice.CONSTRUCTION.equals(cd.getCloudDescriptor().getShardState())) {
-      // set update log to buffer before publishing the core
-      core.getUpdateHandler().getUpdateLog().bufferUpdates();
-    }
+  public void preRegister(CoreDescriptor cd) throws KeeperException, InterruptedException {
+
+
     // before becoming available, make sure we are not live and active
     // this also gets us our assigned shard id if it was not specified
     publish(cd, ZkStateReader.DOWN, false);

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=1485835&r1=1485834&r2=1485835&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 May 23 19:35:52 2013
@@ -633,7 +633,7 @@ public class CoreContainer
     if (zkSys.getZkController() != null) {
       // this happens before we can receive requests
       try {
-        zkSys.getZkController().preRegister(core);
+        zkSys.getZkController().preRegister(core.getCoreDescriptor());
       } catch (KeeperException e) {
         log.error("", e);
         throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java?rev=1485835&r1=1485834&r2=1485835&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java Thu May 23 19:35:52 2013
@@ -62,6 +62,7 @@ import org.apache.lucene.store.IndexInpu
 import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.CommonParams.EchoParamStyle;
 import org.apache.solr.common.params.SolrParams;
@@ -845,6 +846,12 @@ public final class SolrCore implements S
     // from the core.
     resourceLoader.inform(infoRegistry);
     
+    CoreContainer cc = cd.getCoreContainer();
+    
+    if (cc != null && cc.isZooKeeperAware() && Slice.CONSTRUCTION.equals(cd.getCloudDescriptor().getShardState())) {
+      // set update log to buffer before publishing the core
+      getUpdateHandler().getUpdateLog().bufferUpdates();
+    }
     // For debugging   
 //    numOpens.incrementAndGet();
 //    openHandles.put(this, new RuntimeException("unclosed core - name:" + getName() + " refs: " + refCount.get()));