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:45 UTC

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

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 {
 
   }