You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2020/07/13 15:32:32 UTC

[GitHub] [lucene-solr] noblepaul opened a new pull request #1669: SOLR-14151 Make schema components load from packages

noblepaul opened a new pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] noblepaul commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
noblepaul commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r454702783



##########
File path: solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
##########
@@ -198,6 +206,11 @@ synchronized void reloadLuceneSPI() {
     TokenizerFactory.reloadTokenizers(this.classLoader);
   }
 
+  public SolrCore getCore(){

Review comment:
       This is used in #1666 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] noblepaul commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
noblepaul commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r454700983



##########
File path: solr/core/src/java/org/apache/solr/handler/StreamHandler.java
##########
@@ -158,8 +158,8 @@ public Class getClazz() {
     }
 
     @Override
-    protected void initNewInstance(PackageLoader.Package.Version newest) {
-      clazz = newest.getLoader().findClass(pluginInfo.className, Expressible.class);
+    protected Object initNewInstance(PackageLoader.Package.Version newest) {
+      return clazz = newest.getLoader().findClass(pluginInfo.className, Expressible.class);

Review comment:
       Actually no. it can be anything what the listener wants it to be. In this case it's a class




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] noblepaul commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
noblepaul commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r454850576



##########
File path: solr/core/src/java/org/apache/solr/pkg/MultiPackageListener.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.
+ */
+
+package org.apache.solr.pkg;
+
+import org.apache.lucene.analysis.util.ResourceLoaderAware;
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrClassLoader;
+import org.apache.solr.core.SolrResourceLoader;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+/**
+ * A {@link SolrClassLoader} that is designed to listen to a set of packages.
+ * This class would register a listener each package that is loaded through this
+ * if any of those packages are updated , the onReload runnable is executed
+ * */
+public class MultiPackageListener implements SolrClassLoader , PackageListeners.Listener {

Review comment:
       name suggestions are welcome




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r455386922



##########
File path: solr/core/src/java/org/apache/solr/schema/IndexSchema.java
##########
@@ -188,6 +190,7 @@ public IndexSchema(String name, InputSource is, Version luceneVersion, SolrResou
   protected IndexSchema(Version luceneVersion, SolrResourceLoader loader, Properties substitutableProperties) {
     this.luceneVersion = Objects.requireNonNull(luceneVersion);
     this.loader = loader;
+    this.solrClassLoader = loader.getCore() == null? loader: loader.getCore().getSchemaPluginsLoader();

Review comment:
       Okay; lets get this done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r455510065



##########
File path: solr/core/src/java/org/apache/solr/core/SolrCore.java
##########
@@ -191,6 +191,11 @@
 
   private String name;
   private String logid; // used to show what name is set
+  /**
+   * A unique id to differentiate multiple instances of the same core
+   * If we reload a core, the name remains same , but the id will be new
+   */
+  public final UUID uniqueId = UUID.randomUUID();

Review comment:
       Okay... but the AtomicLong incrementAndGet technique is _very_ common for for an internal-id use case (I could list others in Lucene/Solr).  I have not seen a UUID used yet for this.  Up to you.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] noblepaul commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
noblepaul commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r454855401



##########
File path: solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
##########
@@ -752,6 +765,9 @@ public void close() throws IOException {
   }
 
 
+  public CoreContainer getCoreContainer(){

Review comment:
       I may remove it altogether now




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] noblepaul commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
noblepaul commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r454702702



##########
File path: solr/core/src/java/org/apache/solr/core/SolrCore.java
##########
@@ -191,6 +191,11 @@
 
   private String name;
   private String logid; // used to show what name is set
+  /**
+   * A unique id to differentiate multiple instances of the same core
+   * If we reload a core, the name remains same , but the id will be new
+   */
+  public final UUID uniqueId = UUID.randomUUID();

Review comment:
       `AtomicLong` is mutable. We need an immutable object




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r454756640



##########
File path: solr/core/src/java/org/apache/solr/core/SolrCore.java
##########
@@ -191,6 +191,11 @@
 
   private String name;
   private String logid; // used to show what name is set
+  /**
+   * A unique id to differentiate multiple instances of the same core
+   * If we reload a core, the name remains same , but the id will be new
+   */
+  public final UUID uniqueId = UUID.randomUUID();

Review comment:
       The primitive long instance field would be final (immutable).  A static final AtomicLong (which is mutable) would only be used to generate a new core UID.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r455511455



##########
File path: solr/core/src/java/org/apache/solr/core/ConfigSet.java
##########
@@ -79,4 +75,14 @@ public NamedList getProperties() {
   public boolean isTrusted() {
     return trusted;
   }
+
+  /**Provide a Schema object on demand

Review comment:
       https://google.github.io/styleguide/javaguide.html#s7-javadoc
   
   Glad to see this supplier though.

##########
File path: solr/core/src/java/org/apache/solr/pkg/PackageListeningClassLoader.java
##########
@@ -35,7 +35,7 @@
  * This class would register a listener each package that is loaded through this
  * if any of those packages are updated , the onReload runnable is executed
  * */
-public class MultiPackageListener implements SolrClassLoader , PackageListeners.Listener {
+public class PackageListeningClassLoader implements SolrClassLoader , PackageListeners.Listener {

Review comment:
       Good name.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r454765026



##########
File path: solr/core/src/java/org/apache/solr/core/CoreContainer.java
##########
@@ -1588,20 +1590,32 @@ private CoreDescriptor reloadCoreDescriptor(CoreDescriptor oldDesc) {
     return ret;
   }
 
+  /**
+   * reloads a core
+   * refer {@link CoreContainer#reload(String, UUID)} for details
+   */
+  public void reload(String name) {
+    reload(name, null);
+  }
   /**
    * Recreates a SolrCore.
    * While the new core is loading, requests will continue to be dispatched to
    * and processed by the old core
    *
    * @param name the name of the SolrCore to reload
+   * @param coreId The unique Id of the core

Review comment:
       plus mention it's optional, and then if it's specified and the coreId doesn't match, then the reload is a no-op.  Maybe rename coreId to ifCoreIdMatches to reflect not only what it is but how it's used.

##########
File path: solr/core/src/java/org/apache/solr/core/ConfigSet.java
##########
@@ -30,15 +32,18 @@
 
   private final SolrConfig solrconfig;
 
-  private final IndexSchema indexSchema;
+  /**Provide a Schema object on demand
+   * The first Boolean is to signify a a forcefetch
+   */
+  private final Function<Boolean, IndexSchema> indexSchema;

Review comment:
       I think using the generic interface Function here is pushing the bounds of readability/intuitiveness.  Lets just provide an interface/class instead.  Perhaps get() and getForce() with no boolean arg needed.

##########
File path: solr/core/src/java/org/apache/solr/pkg/MultiPackageListener.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.
+ */
+
+package org.apache.solr.pkg;
+
+import org.apache.lucene.analysis.util.ResourceLoaderAware;
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrClassLoader;
+import org.apache.solr.core.SolrResourceLoader;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+/**
+ * A {@link SolrClassLoader} that is designed to listen to a set of packages.
+ * This class would register a listener each package that is loaded through this
+ * if any of those packages are updated , the onReload runnable is executed
+ * */
+public class MultiPackageListener implements SolrClassLoader , PackageListeners.Listener {

Review comment:
       I find this confusing... it could just be the name.  Something that implements SolrClassLoader should probably have a name that makes that aspect more pronounced and less so the listener aspect.

##########
File path: solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
##########
@@ -202,6 +202,38 @@ private void handleGET(SolrQueryRequest req, SolrQueryResponse rsp) {
     }
   }
 
+  /**If a plugin is loaded from a package, the version of the package being used should be added

Review comment:
       nit: another example of oddly formatted javadoc.

##########
File path: solr/core/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java
##########
@@ -86,7 +86,7 @@ public AbstractPluginLoader(String type, Class<T> pluginClassType)
    * @param node - the XML node defining this plugin
    */
   @SuppressWarnings("unchecked")
-  protected T create( SolrResourceLoader loader, String name, String className, Node node ) throws Exception
+  protected T create(SolrClassLoader loader, String name, String className, Node node ) throws Exception

Review comment:
       BTW I do like these sorts of changes where you've found SRL used where the caller really just needs a class loader.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r454450712



##########
File path: solr/core/src/java/org/apache/solr/schema/IndexSchema.java
##########
@@ -188,6 +190,7 @@ public IndexSchema(String name, InputSource is, Version luceneVersion, SolrResou
   protected IndexSchema(Version luceneVersion, SolrResourceLoader loader, Properties substitutableProperties) {
     this.luceneVersion = Objects.requireNonNull(luceneVersion);
     this.loader = loader;
+    this.solrClassLoader = loader.getCore() == null? loader: loader.getCore().getSchemaPluginsLoader();

Review comment:
       Continuing my idea... Imagine a "ZkConfigSetResourceProvider" and "FileSystemConfigSetResourceProvider" pair of classes implementing an interface ConfigSetResourceProvider that is only for serving a resource (InputStream), not a class.  Just method openResource ideally.  Then imagine a SRL that demands a ConfigSetResourceProvider, and a ClassLoader (or SolrClassLoader as you prefer).  See where this is going?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] noblepaul commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
noblepaul commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r454704926



##########
File path: solr/core/src/java/org/apache/solr/schema/IndexSchema.java
##########
@@ -188,6 +190,7 @@ public IndexSchema(String name, InputSource is, Version luceneVersion, SolrResou
   protected IndexSchema(Version luceneVersion, SolrResourceLoader loader, Properties substitutableProperties) {
     this.luceneVersion = Objects.requireNonNull(luceneVersion);
     this.loader = loader;
+    this.solrClassLoader = loader.getCore() == null? loader: loader.getCore().getSchemaPluginsLoader();

Review comment:
       >  What if SchemaPluginsLoader was an SRL itself, and delegated the resource-loading methods to the "real" SRL?
   
   Well, technically it's possible. The current SRL is a mess. At some point in the future we may end up making it clean and usable. Today it's not. We should clearly differentiate between places where we need to load resources and places where we need to load classes. A Minimal interface should be enough for loading classes. SRL is a heavy concrete class. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] noblepaul merged pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
noblepaul merged pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r454763137



##########
File path: solr/core/src/java/org/apache/solr/schema/IndexSchema.java
##########
@@ -188,6 +190,7 @@ public IndexSchema(String name, InputSource is, Version luceneVersion, SolrResou
   protected IndexSchema(Version luceneVersion, SolrResourceLoader loader, Properties substitutableProperties) {
     this.luceneVersion = Objects.requireNonNull(luceneVersion);
     this.loader = loader;
+    this.solrClassLoader = loader.getCore() == null? loader: loader.getCore().getSchemaPluginsLoader();

Review comment:
       In the design I propose here, all the debt I mentioned in SRL except instancePath/getConfigDir could move to the new SolrClassLoader because they are class-loading related and not resource-loading related.  InstancePath/getConfigDir is debt I could erase quickly.  Then I think we're in good shape to move forward with a debt-free SRL that is a simple combination of it's two components.  If that sounds nice to you, I could work on a POC immediately.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r454404343



##########
File path: solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
##########
@@ -198,6 +206,11 @@ synchronized void reloadLuceneSPI() {
     TokenizerFactory.reloadTokenizers(this.classLoader);
   }
 
+  public SolrCore getCore(){

Review comment:
       This is not used?

##########
File path: solr/core/src/java/org/apache/solr/pkg/PackageListeners.java
##########
@@ -96,15 +97,42 @@ private synchronized void invokeListeners(PackageLoader.Package pkg) {
 
 
   public interface Listener {
-    /**Name of the package or null to loisten to all package changes
+    /**Name of the package or null to listen to all package changes
      */
     String packageName();
 
     PluginInfo pluginInfo();
 
-    void changed(PackageLoader.Package pkg);
+    void changed(PackageLoader.Package pkg, Ctx ctx);
 
     PackageLoader.Package.Version getPackageVersion();
+    class Ctx {
+      private Map<String, Runnable > runLater;
+
+      /** If there are multiple packages to be updated and there are multiple listeners,
+       * This is executed after all of the {@link Listener#changed(PackageLoader.Package, Ctx)}
+       * calls are invoked. The name is a unique identifier that can be used by consumers to avoid duplicate
+       * If no deduplication is required, use null
+       * runnable objects
+       */
+      public void runLater(String name,  Runnable runnable  ) {
+        if(runLater == null) runLater = new LinkedHashMap<>();
+        if(name == null) {
+          name = runnable.getClass().getSimpleName() + "@" + runnable.hashCode();
+        }
+        runLater.put(name, runnable);
+      }
+      private void runLaterTasks(){
+        if(runLater == null) return;
+        new Thread(() -> runLater.forEach((s, runnable) -> {

Review comment:
       Could use find a Solr based ExecutorService for this instead?  It sets up MDC and we ensure it gets shut down.

##########
File path: solr/core/src/java/org/apache/solr/handler/SchemaHandler.java
##########
@@ -202,6 +202,38 @@ private void handleGET(SolrQueryRequest req, SolrQueryResponse rsp) {
     }
   }
 
+  /**If a plugin is loaded from a package, the version of the package being used should be added
+   * to the response
+   *
+   */
+  @SuppressWarnings("rawtypes")
+  private  void insertPackageInfo(Object o, SolrQueryRequest req) {
+    if(!req.getParams().getBool("meta",false)) return;

Review comment:
       nitpick: auto-format this code to be consistent with spaces

##########
File path: solr/core/src/java/org/apache/solr/handler/StreamHandler.java
##########
@@ -158,8 +158,8 @@ public Class getClazz() {
     }
 
     @Override
-    protected void initNewInstance(PackageLoader.Package.Version newest) {
-      clazz = newest.getLoader().findClass(pluginInfo.className, Expressible.class);
+    protected Object initNewInstance(PackageLoader.Package.Version newest) {
+      return clazz = newest.getLoader().findClass(pluginInfo.className, Expressible.class);

Review comment:
       This code here returns a class and not an instance.  This seems wrong?

##########
File path: solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
##########
@@ -752,6 +765,9 @@ public void close() throws IOException {
   }
 
 
+  public CoreContainer getCoreContainer(){

Review comment:
       this is not used?

##########
File path: solr/core/src/java/org/apache/solr/core/SolrClassLoader.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+package org.apache.solr.core;
+
+
+/**A generic interface to load plugin classes

Review comment:
       nit: see my longer comment on formatting javadoc.  This case right here really gets to me.

##########
File path: solr/core/src/java/org/apache/solr/pkg/PackageListeners.java
##########
@@ -96,15 +97,42 @@ private synchronized void invokeListeners(PackageLoader.Package pkg) {
 
 
   public interface Listener {
-    /**Name of the package or null to loisten to all package changes
+    /**Name of the package or null to listen to all package changes
      */
     String packageName();
 
     PluginInfo pluginInfo();
 
-    void changed(PackageLoader.Package pkg);
+    void changed(PackageLoader.Package pkg, Ctx ctx);
 
     PackageLoader.Package.Version getPackageVersion();
+    class Ctx {
+      private Map<String, Runnable > runLater;
+
+      /** If there are multiple packages to be updated and there are multiple listeners,
+       * This is executed after all of the {@link Listener#changed(PackageLoader.Package, Ctx)}
+       * calls are invoked. The name is a unique identifier that can be used by consumers to avoid duplicate
+       * If no deduplication is required, use null
+       * runnable objects
+       */
+      public void runLater(String name,  Runnable runnable  ) {
+        if(runLater == null) runLater = new LinkedHashMap<>();

Review comment:
       nit: please auto-format for space consistency

##########
File path: solr/core/src/java/org/apache/solr/pkg/PackageListeners.java
##########
@@ -96,15 +97,42 @@ private synchronized void invokeListeners(PackageLoader.Package pkg) {
 
 
   public interface Listener {
-    /**Name of the package or null to loisten to all package changes
+    /**Name of the package or null to listen to all package changes

Review comment:
       nit:  
   Single-line javadocs can entirely be in one line:
   
       /** Name of the package or null to listen to all package changes */
   
   Multi-line are formatted like this:
   
       /**
        * Summary sentence.
        * More info.
        */
   
   I see this formatting inconsistency in lots of your javadocs.  I know it's not a big deal yet it's still gives code not adhering to this a sloppy feel.  I find https://google.github.io/styleguide/javaguide.html#s7-javadoc useful to refer to, and it's perhaps the most popular Java style guide.
   
   You might try Opt-Cmd-L if you use IntelliJ on a Mac to reformat selected text.
   

##########
File path: solr/core/src/java/org/apache/solr/pkg/PackageListeners.java
##########
@@ -96,15 +97,42 @@ private synchronized void invokeListeners(PackageLoader.Package pkg) {
 
 
   public interface Listener {
-    /**Name of the package or null to loisten to all package changes
+    /**Name of the package or null to listen to all package changes
      */
     String packageName();
 
     PluginInfo pluginInfo();
 
-    void changed(PackageLoader.Package pkg);
+    void changed(PackageLoader.Package pkg, Ctx ctx);
 
     PackageLoader.Package.Version getPackageVersion();
+    class Ctx {
+      private Map<String, Runnable > runLater;
+
+      /** If there are multiple packages to be updated and there are multiple listeners,
+       * This is executed after all of the {@link Listener#changed(PackageLoader.Package, Ctx)}
+       * calls are invoked. The name is a unique identifier that can be used by consumers to avoid duplicate
+       * If no deduplication is required, use null
+       * runnable objects

Review comment:
       I think you mean, use null _for the name_ and not for the Runnable object.  The runnable is required.

##########
File path: solr/core/src/java/org/apache/solr/pkg/SchemaPluginsLoader.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.
+ */
+
+package org.apache.solr.pkg;
+
+import org.apache.lucene.analysis.util.ResourceLoaderAware;
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrClassLoader;
+import org.apache.solr.core.SolrResourceLoader;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+/**
+ * A {@link SolrClassLoader} that is specifically designed to load schema plugins from packages.
+ * This class would register a listener for any package that is used in a schema and reload the schema
+ * if any of those packages are updated
+ * */
+public class SchemaPluginsLoader implements SolrClassLoader {
+    final CoreContainer coreContainer;
+    final SolrResourceLoader loader;
+    final Function<String, String> pkgVersionSupplier;
+    private Map<String ,PackageAPI.PkgVersion> packageVersions =  new HashMap<>(1);

Review comment:
       nit: spacing issue.
   Also, please use 'final' for all fields that are final and not just some.  Likewise for 'private' for fields than can be private.

##########
File path: solr/core/src/java/org/apache/solr/schema/IndexSchema.java
##########
@@ -188,6 +190,7 @@ public IndexSchema(String name, InputSource is, Version luceneVersion, SolrResou
   protected IndexSchema(Version luceneVersion, SolrResourceLoader loader, Properties substitutableProperties) {
     this.luceneVersion = Objects.requireNonNull(luceneVersion);
     this.loader = loader;
+    this.solrClassLoader = loader.getCore() == null? loader: loader.getCore().getSchemaPluginsLoader();

Review comment:
       This seems problematic.  The code involved in index schema loading has access to two fields that both implement SolrClassLoader:  `loader` and `solrClassLoader` which you just added.  And then you changed many lines to use SolrClassLoader which just as well could have been as it was before -- `loader` (SRL).  I can see that you're doing this so that a new SchemaPluginsLoader thing could be used.  What if SchemaPluginsLoader was an SRL itself, and delegated the resource-loading methods to the "real" SRL?
   
   Put differently, we could create a synthetic SRL whose methods delegate to an appropriate plugin enabled ClassLoader and a real SRL for the configSet to find resources.  WDYT?

##########
File path: solr/core/src/java/org/apache/solr/core/SolrCore.java
##########
@@ -191,6 +191,11 @@
 
   private String name;
   private String logid; // used to show what name is set
+  /**
+   * A unique id to differentiate multiple instances of the same core
+   * If we reload a core, the name remains same , but the id will be new
+   */
+  public final UUID uniqueId = UUID.randomUUID();

Review comment:
       I don't think we need UUID because AFAICT the usage is entirely within the node.  Instead, just use a static AtomicLong counter like, for example, SearchHandler.ridCounter and then declare this instance field here as a simple primitive long.

##########
File path: solr/core/src/java/org/apache/solr/pkg/PackageAPI.java
##########
@@ -52,6 +42,11 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;

Review comment:
       nit: you re-ordered the imports in a way not consistent with most classes.  java package should come first.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] noblepaul commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
noblepaul commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r454854570



##########
File path: solr/core/src/java/org/apache/solr/schema/IndexSchema.java
##########
@@ -188,6 +190,7 @@ public IndexSchema(String name, InputSource is, Version luceneVersion, SolrResou
   protected IndexSchema(Version luceneVersion, SolrResourceLoader loader, Properties substitutableProperties) {
     this.luceneVersion = Objects.requireNonNull(luceneVersion);
     this.loader = loader;
+    this.solrClassLoader = loader.getCore() == null? loader: loader.getCore().getSchemaPluginsLoader();

Review comment:
       At this point I want this to get out of my way. I would fix this one instead of spending more time on SRL refactoring.
   
   I believe this is in a reasonably good shape. We can do more refactoring later.
   
   Let's do things that add value to users
   
   TBH, I don't care about any particular class here. If possible, I would just nuke SRL altogether. It's just a kitchen sink and reeks of bad designs all over
   
   I wish to use SRL minimally. We should have a less heavy interface in most places. I would say most places are happy to just have a `SolrClassLoader` 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] noblepaul commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
noblepaul commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r455476821



##########
File path: solr/core/src/java/org/apache/solr/core/SolrCore.java
##########
@@ -191,6 +191,11 @@
 
   private String name;
   private String logid; // used to show what name is set
+  /**
+   * A unique id to differentiate multiple instances of the same core
+   * If we reload a core, the name remains same , but the id will be new
+   */
+  public final UUID uniqueId = UUID.randomUUID();

Review comment:
       UUID is cheap and there is no advantage in using something else. Cores are not created so frequently that we should care too much about it




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] noblepaul commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
noblepaul commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r454700567



##########
File path: solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
##########
@@ -752,6 +765,9 @@ public void close() throws IOException {
   }
 
 
+  public CoreContainer getCoreContainer(){

Review comment:
       This is necessary for #1666 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r454456488



##########
File path: solr/core/src/java/org/apache/solr/schema/IndexSchema.java
##########
@@ -188,6 +190,7 @@ public IndexSchema(String name, InputSource is, Version luceneVersion, SolrResou
   protected IndexSchema(Version luceneVersion, SolrResourceLoader loader, Properties substitutableProperties) {
     this.luceneVersion = Objects.requireNonNull(luceneVersion);
     this.loader = loader;
+    this.solrClassLoader = loader.getCore() == null? loader: loader.getCore().getSchemaPluginsLoader();

Review comment:
       With that idea, there would only need to be one SRL class (no subclasses), and it'd be easy to create new instances based on those two primary components.  
   
   I'm sure there is some tech debt entanglements in SRL relating to tracking instancePath (there's a TODO I added in there, initLibs() to remove that one) and harder are waitingForCore, infoMBeans, waitingForResources, and of course managedResourceRegistry.  If those get moved off somehow, then I hope the picture I propose becomes more clear.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] noblepaul commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
noblepaul commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r454854570



##########
File path: solr/core/src/java/org/apache/solr/schema/IndexSchema.java
##########
@@ -188,6 +190,7 @@ public IndexSchema(String name, InputSource is, Version luceneVersion, SolrResou
   protected IndexSchema(Version luceneVersion, SolrResourceLoader loader, Properties substitutableProperties) {
     this.luceneVersion = Objects.requireNonNull(luceneVersion);
     this.loader = loader;
+    this.solrClassLoader = loader.getCore() == null? loader: loader.getCore().getSchemaPluginsLoader();

Review comment:
       At this point I want this to get out of my way. I would fix this one instead of spending more time on SRL refactoring.
   
   I believe this is in a reasonably good shape. We can do more refactoring later.
   
   Let's do things that add value to users
   
   TBH, I don't care about any particular class here. If possible, I would just nuke SRL altogether. It's just a kitchen sink and reeks of bad designs all over
   
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] noblepaul commented on a change in pull request #1669: SOLR-14151 Make schema components load from packages

Posted by GitBox <gi...@apache.org>.
noblepaul commented on a change in pull request #1669:
URL: https://github.com/apache/lucene-solr/pull/1669#discussion_r454701342



##########
File path: solr/core/src/java/org/apache/solr/pkg/PackageListeners.java
##########
@@ -96,15 +97,42 @@ private synchronized void invokeListeners(PackageLoader.Package pkg) {
 
 
   public interface Listener {
-    /**Name of the package or null to loisten to all package changes
+    /**Name of the package or null to listen to all package changes
      */
     String packageName();
 
     PluginInfo pluginInfo();
 
-    void changed(PackageLoader.Package pkg);
+    void changed(PackageLoader.Package pkg, Ctx ctx);
 
     PackageLoader.Package.Version getPackageVersion();
+    class Ctx {
+      private Map<String, Runnable > runLater;
+
+      /** If there are multiple packages to be updated and there are multiple listeners,
+       * This is executed after all of the {@link Listener#changed(PackageLoader.Package, Ctx)}
+       * calls are invoked. The name is a unique identifier that can be used by consumers to avoid duplicate
+       * If no deduplication is required, use null
+       * runnable objects
+       */
+      public void runLater(String name,  Runnable runnable  ) {
+        if(runLater == null) runLater = new LinkedHashMap<>();
+        if(name == null) {
+          name = runnable.getClass().getSimpleName() + "@" + runnable.hashCode();
+        }
+        runLater.put(name, runnable);
+      }
+      private void runLaterTasks(){
+        if(runLater == null) return;
+        new Thread(() -> runLater.forEach((s, runnable) -> {

Review comment:
       True. This needs to be done more gracefully.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org