You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2020/06/27 07:18:44 UTC

[lucene-solr] branch jira/solr14404_1 created (now 2d32d43)

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

noble pushed a change to branch jira/solr14404_1
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git.


      at 2d32d43  support for openResource in PackageResourceLoader

This branch includes the following new commits:

     new 2d32d43  support for openResource in PackageResourceLoader

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[lucene-solr] 01/01: support for openResource in PackageResourceLoader

Posted by no...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 2d32d4303abefd81d7ff11051721c723979fc7b7
Author: noble <no...@apache.org>
AuthorDate: Sat Jun 27 17:18:02 2020 +1000

    support for openResource in PackageResourceLoader
---
 .../apache/solr/api/CustomContainerPlugins.java    | 18 +++++++-
 .../java/org/apache/solr/pkg/PackageLoader.java    | 54 ++++++++++++++++++++++
 .../apache/solr/handler/TestContainerPlugin.java   | 45 +++++++++++++++++-
 3 files changed, 113 insertions(+), 4 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java b/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
index 9246dac..c26322f 100644
--- a/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
+++ b/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
@@ -30,8 +30,10 @@ import java.util.Map;
 import java.util.Objects;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.lucene.analysis.util.ResourceLoaderAware;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.request.beans.PluginMeta;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.annotation.JsonProperty;
 import org.apache.solr.common.cloud.ClusterPropertiesListener;
 import org.apache.solr.common.util.Pair;
@@ -264,12 +266,24 @@ public class CustomContainerPlugins implements ClusterPropertiesListener {
       if (this.holders != null) return;
       Constructor constructor = klas.getConstructors()[0];
       if (constructor.getParameterTypes().length == 0) {
-        instance = constructor.newInstance();
+        instance = pkgVersion.getLoader().newInstance(klas.getName(), Object.class);
       } else if (constructor.getParameterTypes().length == 1 && constructor.getParameterTypes()[0] == CoreContainer.class) {
-        instance = constructor.newInstance(coreContainer);
+        instance = pkgVersion.getLoader().newInstance(
+            klas.getName(),
+            Object.class,
+            new String[0],
+            new Class[]{CoreContainer.class},
+            new Object[]{coreContainer});
       } else {
         throw new RuntimeException("Must have a no-arg constructor or CoreContainer constructor ");
       }
+      if (instance instanceof ResourceLoaderAware) {
+        try {
+          ((ResourceLoaderAware) instance).inform(pkgVersion.getLoader());
+        } catch (IOException e) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+        }
+      }
       this.holders = new ArrayList<>();
       for (Api api : AnnotatedApi.getApis(instance)) {
         holders.add(new ApiHolder((AnnotatedApi) api));
diff --git a/solr/core/src/java/org/apache/solr/pkg/PackageLoader.java b/solr/core/src/java/org/apache/solr/pkg/PackageLoader.java
index 1089288..63e801e 100644
--- a/solr/core/src/java/org/apache/solr/pkg/PackageLoader.java
+++ b/solr/core/src/java/org/apache/solr/pkg/PackageLoader.java
@@ -18,18 +18,27 @@
 package org.apache.solr.pkg;
 
 import java.io.Closeable;
+import java.io.FileInputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
+import java.nio.ByteBuffer;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.function.Supplier;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.util.SimplePostTool;
+import org.apache.zookeeper.server.ByteBufferInputStream;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -303,9 +312,12 @@ public class PackageLoader implements Closeable {
     }
   }
   static class PackageResourceLoader extends SolrResourceLoader {
+    List<Path> paths;
+
 
     PackageResourceLoader(String name, List<Path> classpath, Path instanceDir, ClassLoader parent) {
       super(name, classpath, instanceDir, parent);
+      this.paths = classpath;
     }
 
     @Override
@@ -329,6 +341,48 @@ public class PackageLoader implements Closeable {
     public  <T> void addToInfoBeans(T obj) {
       //do not do anything. It should be handled externally
     }
+
+    @Override
+    public InputStream openResource(String resource) throws IOException {
+      for (Path path : paths) {
+        try(FileInputStream in = new FileInputStream(path.toFile())) {
+          ZipInputStream zis = new ZipInputStream(in);
+          try {
+            ZipEntry entry;
+            while ((entry = zis.getNextEntry()) != null) {
+              if (resource == null || resource.equals(entry.getName())) {
+                SimplePostTool.BAOS out = new SimplePostTool.BAOS();
+                byte[] buffer = new byte[2048];
+                int size;
+                while ((size = zis.read(buffer, 0, buffer.length)) != -1) {
+                  out.write(buffer, 0, size);
+                }
+                out.close();
+                return new ByteBufferStream(out.getByteBuffer());
+              }
+            }
+          } finally {
+            zis.closeEntry();
+          }
+        }
+      }
+
+      return null;
+    }
+  }
+
+  private static class ByteBufferStream extends ByteBufferInputStream implements Supplier<ByteBuffer> {
+    private final ByteBuffer buf ;
+
+    public ByteBufferStream(ByteBuffer buf) {
+      super(buf);
+      this.buf = buf;
+    }
+
+    @Override
+    public ByteBuffer get() {
+      return buf;
+    }
   }
 
   private static String findBiggest(String lessThan, List<String> sortedList) {
diff --git a/solr/core/src/test/org/apache/solr/handler/TestContainerPlugin.java b/solr/core/src/test/org/apache/solr/handler/TestContainerPlugin.java
index 2d417e1..f89fd64 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestContainerPlugin.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestContainerPlugin.java
@@ -18,11 +18,16 @@
 package org.apache.solr.handler;
 
 import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
+import java.util.function.Supplier;
 
 import com.google.common.collect.ImmutableMap;
+import org.apache.lucene.analysis.util.ResourceLoader;
+import org.apache.lucene.analysis.util.ResourceLoaderAware;
 import org.apache.solr.api.Command;
 import org.apache.solr.api.EndPoint;
 import org.apache.solr.client.solrj.SolrClient;
@@ -37,6 +42,7 @@ import org.apache.solr.cloud.MiniSolrCloudCluster;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.NavigableObject;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.filestore.PackageStoreAPI;
 import org.apache.solr.filestore.TestDistribPackageStore;
 import org.apache.solr.filestore.TestDistribPackageStore.Fetcher;
@@ -188,7 +194,7 @@ public class TestContainerPlugin extends SolrCloudTestCase {
       plugin.name = "myplugin";
       plugin.klass = "mypkg:org.apache.solr.handler.MyPlugin";
       plugin.version = add.version;
-      V2Request req1 = new V2Request.Builder("/cluster/plugin")
+      final V2Request req1 = new V2Request.Builder("/cluster/plugin")
           .forceV2(true)
           .withMethod(POST)
           .withPayload(singletonMap("add", plugin))
@@ -235,15 +241,50 @@ public class TestContainerPlugin extends SolrCloudTestCase {
               "/plugin/myplugin/class", plugin.klass,
               "/plugin/myplugin/version", "2.0"
           ));
-      // invoke the plugin and test thye output
+      // invoke the plugin and test the output
       TestDistribPackageStore.assertResponseValues(10,
           invokePlugin,
           ImmutableMap.of("/myplugin.version", "2.0"));
+
+      plugin.name = "plugin2";
+      plugin.klass = "mypkg:"+ C5.class.getName();
+      plugin.version = "2.0";
+      req1.process(cluster.getSolrClient());
+      assertNotNull(C5.classData);
+      assertEquals( 1452, C5.classData.limit());
     } finally {
       cluster.shutdown();
     }
   }
 
+  public static class C5 implements ResourceLoaderAware {
+    static ByteBuffer classData;
+    private  SolrResourceLoader resourceLoader;
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void inform(ResourceLoader loader) throws IOException {
+      this.resourceLoader = (SolrResourceLoader) loader;
+      try {
+        InputStream is = resourceLoader.openResource("org/apache/solr/handler/MyPlugin.class");
+        if (is instanceof Supplier) {
+          classData = ((Supplier<ByteBuffer>) is).get();
+        }
+      } catch (IOException e) {
+        //do not do anything
+      }
+    }
+
+    @EndPoint(method = GET,
+        path = "/$plugin-name/m2",
+        permission = PermissionNameProvider.Name.COLL_READ_PERM)
+    public void m2() {
+
+
+    }
+
+  }
+
   public static class C1 {
 
   }