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/09/29 05:51:46 UTC

[lucene-solr] branch branch_8x updated: SOLR-14901: TestPackages uses binary precompiled classes to refer to analysis factory FQCNs

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new f521906  SOLR-14901: TestPackages uses binary precompiled classes to refer to analysis factory FQCNs
f521906 is described below

commit f5219061252912c000f5d119d49bb315e5e6f1ae
Author: noble <no...@apache.org>
AuthorDate: Tue Sep 29 15:27:50 2020 +1000

    SOLR-14901: TestPackages uses binary precompiled classes to refer to analysis factory FQCNs
---
 .../MyPatternReplaceCharFilterFactory.java         |  29 ++++++
 .../src/test-files/runtimecode/MyTextField.java    |  23 +++++
 .../runtimecode/MyWhitespaceTokenizerFactory.java  |  28 ++++++
 .../test-files/runtimecode/schema-plugins.jar.bin  | Bin 6814 -> 1461 bytes
 .../src/test/org/apache/solr/pkg/TestPackages.java | 111 +++++++++++++++------
 5 files changed, 161 insertions(+), 30 deletions(-)

diff --git a/solr/core/src/test-files/runtimecode/MyPatternReplaceCharFilterFactory.java b/solr/core/src/test-files/runtimecode/MyPatternReplaceCharFilterFactory.java
new file mode 100644
index 0000000..72c0ee0
--- /dev/null
+++ b/solr/core/src/test-files/runtimecode/MyPatternReplaceCharFilterFactory.java
@@ -0,0 +1,29 @@
+/*
+ * 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 my.pkg;
+
+import java.util.Map;
+
+import org.apache.solr.pkg.TestPackages;
+
+public class MyPatternReplaceCharFilterFactory extends TestPackages.BasePatternReplaceCharFilterFactory {
+
+  public MyPatternReplaceCharFilterFactory(Map<String, String> args) {
+    super(args);
+  }
+}
diff --git a/solr/core/src/test-files/runtimecode/MyTextField.java b/solr/core/src/test-files/runtimecode/MyTextField.java
new file mode 100644
index 0000000..a88ad5e
--- /dev/null
+++ b/solr/core/src/test-files/runtimecode/MyTextField.java
@@ -0,0 +1,23 @@
+/*
+ * 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 my.pkg;
+
+import org.apache.solr.schema.TextField;
+
+public class MyTextField extends TextField {
+}
diff --git a/solr/core/src/test-files/runtimecode/MyWhitespaceTokenizerFactory.java b/solr/core/src/test-files/runtimecode/MyWhitespaceTokenizerFactory.java
new file mode 100644
index 0000000..30f363d
--- /dev/null
+++ b/solr/core/src/test-files/runtimecode/MyWhitespaceTokenizerFactory.java
@@ -0,0 +1,28 @@
+/*
+ * 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 my.pkg;
+
+import java.util.Map;
+
+import org.apache.solr.pkg.TestPackages;
+
+public class MyWhitespaceTokenizerFactory extends TestPackages.BaseWhitespaceTokenizerFactory {
+  public MyWhitespaceTokenizerFactory(Map<String, String> args) {
+    super(args);
+  }
+}
diff --git a/solr/core/src/test-files/runtimecode/schema-plugins.jar.bin b/solr/core/src/test-files/runtimecode/schema-plugins.jar.bin
index 4effbba..8115392 100644
Binary files a/solr/core/src/test-files/runtimecode/schema-plugins.jar.bin and b/solr/core/src/test-files/runtimecode/schema-plugins.jar.bin differ
diff --git a/solr/core/src/test/org/apache/solr/pkg/TestPackages.java b/solr/core/src/test/org/apache/solr/pkg/TestPackages.java
index 6ec5044..87134a3 100644
--- a/solr/core/src/test/org/apache/solr/pkg/TestPackages.java
+++ b/solr/core/src/test/org/apache/solr/pkg/TestPackages.java
@@ -17,6 +17,7 @@
 
 package org.apache.solr.pkg;
 
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
@@ -24,8 +25,12 @@ import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.concurrent.Callable;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipOutputStream;
 
 import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.lucene.analysis.core.WhitespaceTokenizerFactory;
+import org.apache.lucene.analysis.pattern.PatternReplaceCharFilterFactory;
 import org.apache.lucene.analysis.util.ResourceLoader;
 import org.apache.lucene.analysis.util.ResourceLoaderAware;
 import org.apache.solr.client.solrj.SolrClient;
@@ -63,6 +68,7 @@ import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.QParser;
 import org.apache.solr.search.QParserPlugin;
 import org.apache.solr.util.LogLevel;
+import org.apache.solr.util.SimplePostTool;
 import org.apache.solr.util.plugin.SolrCoreAware;
 import org.apache.zookeeper.data.Stat;
 import org.junit.After;
@@ -78,7 +84,6 @@ import static org.apache.solr.filestore.TestDistribPackageStore.uploadKey;
 import static org.apache.solr.filestore.TestDistribPackageStore.checkAllNodesForFile;
 
 @LogLevel("org.apache.solr.pkg.PackageLoader=DEBUG;org.apache.solr.pkg.PackageAPI=DEBUG")
-//@org.apache.lucene.util.LuceneTestCase.AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-13822") // leaks files
 public class TestPackages extends SolrCloudTestCase {
 
   @Before
@@ -185,23 +190,23 @@ public class TestPackages extends SolrCloudTestCase {
           .build();
       cluster.getSolrClient().request(v2r);
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "queryResponseWriter", "json1",
           "mypkg", "1.0" );
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "searchComponent", "get",
           "mypkg", "1.0" );
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "requestHandler", "/runtime",
           "mypkg", "1.0" );
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "updateProcessor", "myurp",
           "mypkg", "1.0" );
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "expressible", "mincopy",
           "mypkg", "1.0" );
 
@@ -251,19 +256,19 @@ public class TestPackages extends SolrCloudTestCase {
       add.files = Arrays.asList(new String[]{FILE2,URP2, EXPR1});
       req.process(cluster.getSolrClient());
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "queryResponseWriter", "json1",
           "mypkg", "1.1" );
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "searchComponent", "get",
           "mypkg", "1.1" );
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "requestHandler", "/runtime",
           "mypkg", "1.1" );
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "updateProcessor", "myurp",
           "mypkg", "1.1" );
 
@@ -282,15 +287,15 @@ public class TestPackages extends SolrCloudTestCase {
       req.process(cluster.getSolrClient());
 
       //now let's verify that the classes are updated
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "queryResponseWriter", "json1",
           "mypkg", "2.1" );
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "searchComponent", "get",
           "mypkg", "2.1" );
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "requestHandler", "/runtime",
           "mypkg", "2.1" );
 
@@ -321,15 +326,15 @@ public class TestPackages extends SolrCloudTestCase {
           .build();
       delete.process(cluster.getSolrClient());
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "queryResponseWriter", "json1",
           "mypkg", "2.1" );
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "searchComponent", "get",
           "mypkg", "2.1" );
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "requestHandler", "/runtime",
           "mypkg", "2.1" );
 
@@ -337,15 +342,15 @@ public class TestPackages extends SolrCloudTestCase {
       delVersion.version = "2.1";
       delete.process(cluster.getSolrClient());
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "queryResponseWriter", "json1",
           "mypkg", "1.1" );
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "searchComponent", "get",
           "mypkg", "1.1" );
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "requestHandler", "/runtime",
           "mypkg", "1.1" );
 
@@ -366,15 +371,15 @@ public class TestPackages extends SolrCloudTestCase {
       //the collections mypkg is set to use version 1.1
       //so no upgrade
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "queryResponseWriter", "json1",
           "mypkg", "1.1" );
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "searchComponent", "get",
           "mypkg", "1.1" );
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "requestHandler", "/runtime",
           "mypkg", "1.1" );
 
@@ -396,15 +401,15 @@ public class TestPackages extends SolrCloudTestCase {
           .process(cluster.getSolrClient());
 
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "queryResponseWriter", "json1",
           "mypkg", "2.1" );
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "searchComponent", "get",
           "mypkg", "2.1" );
 
-      verifyCmponent(cluster.getSolrClient(),
+      verifyComponent(cluster.getSolrClient(),
           COLLECTION_NAME, "requestHandler", "/runtime",
           "mypkg", "2.1" );
 
@@ -458,8 +463,8 @@ public class TestPackages extends SolrCloudTestCase {
     }
   }
 
-  private void verifyCmponent(SolrClient client, String COLLECTION_NAME,
-                              String componentType, String componentName, String pkg, String version) throws Exception {
+  private void verifyComponent(SolrClient client, String COLLECTION_NAME,
+                               String componentType, String componentName, String pkg, String version) throws Exception {
     @SuppressWarnings({"unchecked"})
     SolrParams params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME,
         WT, JAVABIN,
@@ -639,7 +644,6 @@ public class TestPackages extends SolrCloudTestCase {
     }
   }
 
-  @SuppressWarnings("rawtypes")
   public void testSchemaPlugins() throws Exception {
     String COLLECTION_NAME = "testSchemaLoadingColl";
     System.setProperty("managed.schema.mutable", "true");
@@ -654,7 +658,7 @@ public class TestPackages extends SolrCloudTestCase {
       byte[] derFile = readFile("cryptokeys/pub_key512.der");
       uploadKey(derFile, PackageStoreAPI.KEYS_DIR+"/pub_key512.der", cluster);
       postFileAndWait(cluster, "runtimecode/schema-plugins.jar.bin", FILE1,
-          "iSRhrogDyt9P1htmSf/krh1kx9oty3TYyWm4GKHQGlb8a+X4tKCe9kKk+3tGs+bU9zq5JBZ5txNXsn96aZem5A==");
+          "U+AdO/jgY3DtMpeFRGoTQk72iA5g/qjPvdQYPGBaXB5+ggcTZk4FoIWiueB0bwGJ8Mg3V/elxOqEbD2JR8R0tA==");
 
       String FILE2 = "/schemapkg/payload-component.jar";
       postFileAndWait(cluster, "runtimecode/payload-component.jar.bin", FILE2,
@@ -693,7 +697,7 @@ public class TestPackages extends SolrCloudTestCase {
       add = new Package.AddVersion();
       add.version = "2.0";
       add.pkg = "schemapkg";
-      add.files = Arrays.asList(new String[]{FILE1});
+      add.files = Arrays.asList(FILE1);
       req = new V2Request.Builder("/cluster/package")
           .forceV2(true)
           .withMethod(SolrRequest.METHOD.POST)
@@ -759,4 +763,51 @@ public class TestPackages extends SolrCloudTestCase {
       );
     }
   }
+
+  public static class BasePatternReplaceCharFilterFactory extends PatternReplaceCharFilterFactory {
+    public BasePatternReplaceCharFilterFactory(Map<String, String> args) {
+      super(args);
+    }
+  }
+
+  public static class BaseWhitespaceTokenizerFactory extends WhitespaceTokenizerFactory {
+
+    public BaseWhitespaceTokenizerFactory(Map<String, String> args) {
+      super(args);
+    }
+  }
+
+  /*
+  //copy the jav files to a package and then run the main method
+  public static void main(String[] args) throws Exception {
+    persistZip("/tmp/x.jar", MyPatternReplaceCharFilterFactory.class, MyTextField.class, MyWhitespaceTokenizerFactory.class);
+  }*/
+
+
+  public static ByteBuffer persistZip(String loc,
+                                      @SuppressWarnings({"rawtypes"}) Class... classes) throws IOException {
+    ByteBuffer jar = generateZip(classes);
+    try (FileOutputStream fos = new FileOutputStream(loc)) {
+      fos.write(jar.array(), 0, jar.limit());
+      fos.flush();
+    }
+    return jar;
+  }
+
+  public static ByteBuffer generateZip(@SuppressWarnings({"rawtypes"}) Class... classes) throws IOException {
+    SimplePostTool.BAOS bos = new SimplePostTool.BAOS();
+    try (ZipOutputStream zipOut = new ZipOutputStream(bos)) {
+      zipOut.setLevel(ZipOutputStream.DEFLATED);
+      for (@SuppressWarnings({"rawtypes"}) Class c : classes) {
+        String path = c.getName().replace('.', '/').concat(".class");
+        ZipEntry entry = new ZipEntry(path);
+        ByteBuffer b = SimplePostTool.inputStreamToByteArray(c.getClassLoader().getResourceAsStream(path));
+        zipOut.putNextEntry(entry);
+        zipOut.write(b.array(), 0, b.limit());
+        zipOut.closeEntry();
+      }
+    }
+    return bos.getByteBuffer();
+  }
+
 }