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 2020/07/10 22:16:44 UTC

[lucene-solr] 02/02: #58 Fix ZkSolrResourceLoader.

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

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

commit bf37ff47cc9bda35a2863539bfca4e8e2df7a33d
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Fri Jul 10 17:14:13 2020 -0500

    #58 Fix ZkSolrResourceLoader.
---
 .../apache/solr/cloud/ZkSolrResourceLoader.java    | 87 +++++++---------------
 .../java/org/apache/solr/core/CoreContainer.java   |  2 +
 .../solr/schema/ManagedIndexSchemaFactory.java     | 31 +++++---
 3 files changed, 52 insertions(+), 68 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java b/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
index a3dbc31..f22c4de 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
@@ -17,28 +17,31 @@
 package org.apache.solr.cloud;
 
 import java.io.ByteArrayInputStream;
-import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
 import java.nio.file.Path;
 
-import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.lucene.analysis.util.ResourceLoader;
 import org.apache.solr.common.cloud.ZkConfigManager;
-import org.apache.solr.common.cloud.ZooKeeperException;
+import org.apache.solr.common.util.XMLErrorLogger;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.core.SolrResourceNotFoundException;
 import org.apache.solr.schema.ZkIndexSchemaReader;
+import org.apache.solr.util.SystemIdResolver;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.ParserConfigurationException;
+
 /**
  * ResourceLoader that works with ZooKeeper.
  *
  */
-public class ZkSolrResourceLoader extends SolrResourceLoader {
+public class ZkSolrResourceLoader extends SolrResourceLoader implements ResourceLoader {
 
   private final String configSetZkPath;
   private ZkController zkController;
@@ -61,11 +64,7 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
   }
 
   /**
-   * Opens any resource by its name. By default, this will look in multiple
-   * locations to load the resource: $configDir/$resource from ZooKeeper.
-   * It will look for it in any jar
-   * accessible through the class loader if it cannot be found in ZooKeeper. 
-   * Override this method to customize loading resources.
+   * Opens any resource from zk by its name.
    * 
    * @return the stream for the named resource
    */
@@ -73,54 +72,31 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
   public InputStream openResource(String resource) throws IOException {
     InputStream is;
     String file = (".".equals(resource)) ? configSetZkPath : configSetZkPath + "/" + resource;
-    int maxTries = 10;
-    Exception exception = null;
-    while (maxTries -- > 0) {
-      try {
-        if (zkController.pathExists(file)) {
-          Stat stat = new Stat();
-          byte[] bytes = zkController.getZkClient().getData(file, null, stat, true);
-          return new ZkByteArrayInputStream(bytes, stat);
-        } else {
-          //Path does not exists. We only retry for session expired exceptions.
-          break;
-        }
-      } catch (KeeperException.SessionExpiredException e) {
-        exception = e;
-        if (!zkController.getCoreContainer().isShutDown()) {
-          // Retry in case of session expiry
-          try {
-            Thread.sleep(1000);
-            log.debug("Sleeping for 1s before retrying fetching resource={}", resource);
-          } catch (InterruptedException ie) {
-            Thread.currentThread().interrupt();
-            throw new IOException("Could not load resource=" + resource, ie);
-          }
-        }
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        throw new IOException("Error opening " + file, e);
-      } catch (Exception e) {
-        throw new IOException("Error opening " + file, e);
+
+    try {
+
+      Stat stat = new Stat();
+      byte[] bytes = zkController.getZkClient().getData(file, null, stat, true);
+      if (bytes == null) {
+
+        throw new SolrResourceNotFoundException("Can't find resource '" + resource
+                + "' in classpath or '" + configSetZkPath + "', cwd="
+                + System.getProperty("user.dir"));
       }
-    }
+      return new ZkByteArrayInputStream(bytes, stat);
 
-    if (exception != null) {
-      throw new IOException("We re-tried 10 times but was still unable to fetch resource=" + resource + " from ZK", exception);
-    }
 
-    try {
-      // delegate to the class loader (looking into $INSTANCE_DIR/lib jars)
-      is = resourceClassLoader.getResourceAsStream(resource.replace(File.separatorChar, '/'));
-    } catch (Exception e) {
-      throw new IOException("Error opening " + resource, e);
-    }
-    if (is == null) {
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new IOException("Error opening " + file, e);
+    } catch (KeeperException.NoNodeException e) {
       throw new SolrResourceNotFoundException("Can't find resource '" + resource
-          + "' in classpath or '" + configSetZkPath + "', cwd="
-          + System.getProperty("user.dir"));
+              + "' in classpath or '" + configSetZkPath + "', cwd="
+              + System.getProperty("user.dir"));
+    } catch (Exception e) {
+      throw new IOException("Error opening " + file, e);
     }
-    return is;
+
   }
 
   public static class ZkByteArrayInputStream extends ByteArrayInputStream{
@@ -137,13 +113,6 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
     }
   }
 
-  @Override
-  public String getConfigDir() {
-    throw new ZooKeeperException(
-        ErrorCode.SERVER_ERROR,
-        "ZkSolrResourceLoader does not support getConfigDir() - likely, what you are trying to do is not supported in ZooKeeper mode");
-  }
-
   public String getConfigSetZkPath() {
     return configSetZkPath;
   }
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 6781044..e6262da 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -1335,6 +1335,7 @@ public class CoreContainer implements Closeable {
 
       return core;
     } catch (Exception e) {
+      log.error("Unable to create SolrCore", e);
       coreInitFailures.put(dcore.getName(), new CoreLoadFailure(dcore, e));
       if (e instanceof ZkController.NotInClusterStateException && !newCollection) {
         // this mostly happen when the core is deleted when this node is down
@@ -1347,6 +1348,7 @@ public class CoreContainer implements Closeable {
         IOUtils.closeQuietly(core);
       throw solrException;
     } catch (Throwable t) {
+      log.error("Unable to create SolrCore", t);
       SolrException e = new SolrException(ErrorCode.SERVER_ERROR, "JVM Error creating core [" + dcore.getName() + "]: " + t.getMessage(), t);
       coreInitFailures.put(dcore.getName(), new CoreLoadFailure(dcore, e));
       solrCores.removeCoreDescriptor(dcore);
diff --git a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java
index 300bd97..9a45a64 100644
--- a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java
+++ b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java
@@ -22,14 +22,17 @@ import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
 
 import org.apache.commons.io.IOUtils;
+import org.apache.lucene.analysis.util.ResourceLoader;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
+import org.apache.solr.common.AlreadyClosedException;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkCmdExecutor;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
@@ -50,16 +53,19 @@ public class ManagedIndexSchemaFactory extends IndexSchemaFactory implements Sol
 
   public static final String DEFAULT_MANAGED_SCHEMA_RESOURCE_NAME = "managed-schema";
   public static final String MANAGED_SCHEMA_RESOURCE_NAME = "managedSchemaResourceName";
+  private volatile CoreContainer coreContainer;
 
-  private boolean isMutable = true;
+  private volatile boolean isMutable = true;
   private String managedSchemaResourceName = DEFAULT_MANAGED_SCHEMA_RESOURCE_NAME;
+  private volatile String coreName;
+
   public String getManagedSchemaResourceName() { return managedSchemaResourceName; }
   private SolrConfig config;
-  private SolrResourceLoader loader;
-  public SolrResourceLoader getResourceLoader() { return loader; }
+  private ResourceLoader loader;
+  public ResourceLoader getResourceLoader() { return loader; }
   private String resourceName;
   private ManagedIndexSchema schema;
-  private SolrCore core;
+// / private SolrCore core;
   private ZkIndexSchemaReader zkIndexSchemaReader;
 
 
@@ -219,7 +225,7 @@ public class ManagedIndexSchemaFactory extends IndexSchemaFactory implements Sol
   private void warnIfNonManagedSchemaExists() {
     if ( ! resourceName.equals(managedSchemaResourceName)) {
       boolean exists = false;
-      SolrResourceLoader loader = config.getResourceLoader();
+      ResourceLoader loader = config.getResourceLoader();
       if (loader instanceof ZkSolrResourceLoader) {
         ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
         String nonManagedSchemaPath = zkLoader.getConfigSetZkPath() + "/" + resourceName;
@@ -260,7 +266,7 @@ public class ManagedIndexSchemaFactory extends IndexSchemaFactory implements Sol
    * and no exception will be thrown.
    */
   private void upgradeToManagedSchema() {
-    SolrResourceLoader loader = config.getResourceLoader();
+    ResourceLoader loader = config.getResourceLoader();
     if (loader instanceof ZkSolrResourceLoader) {
       zkUgradeToManagedSchema();
     } else {
@@ -308,7 +314,7 @@ public class ManagedIndexSchemaFactory extends IndexSchemaFactory implements Sol
    *@return the File for the named resource, or null if it can't be found
    */
   private File locateConfigFile(String resource) {
-    String location = config.getResourceLoader().resourceLocation(resource);
+    String location = ((SolrResourceLoader)config.getResourceLoader()).resourceLocation(resource);
     if (location.equals(resource) || location.startsWith("classpath:"))
       return null;
     return new File(location);
@@ -399,7 +405,8 @@ public class ManagedIndexSchemaFactory extends IndexSchemaFactory implements Sol
 
   @Override
   public void inform(SolrCore core) {
-    this.core = core;
+    this.coreContainer = core.getCoreContainer();
+    this.coreName = core.getName();
     if (loader instanceof ZkSolrResourceLoader) {
       this.zkIndexSchemaReader = new ZkIndexSchemaReader(this, core);
       ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
@@ -427,7 +434,13 @@ public class ManagedIndexSchemaFactory extends IndexSchemaFactory implements Sol
 
   public void setSchema(ManagedIndexSchema schema) {
     this.schema = schema;
-    core.setLatestSchema(schema);
+    try (SolrCore core = coreContainer.getCore(coreName)) {
+      if (core == null) {
+        throw new AlreadyClosedException();
+      }
+      core.setLatestSchema(schema);
+    }
+
   }
   
   public boolean isMutable() {