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

[lucene-solr] branch master 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 master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


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

commit 01da67c728df1e46ed82fee1a178546c0665b421
Author: noble <no...@apache.org>
AuthorDate: Tue Sep 29 15:25:22 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 | 195 +++++++++++++--------
 5 files changed, 201 insertions(+), 74 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 0f5fbc879..bea5d0c 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,9 +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.util.LuceneTestCase;
+import org.apache.lucene.analysis.core.WhitespaceTokenizerFactory;
+import org.apache.lucene.analysis.pattern.PatternReplaceCharFilterFactory;
 import org.apache.lucene.util.ResourceLoader;
 import org.apache.lucene.util.ResourceLoaderAware;
 import org.apache.solr.client.solrj.SolrClient;
@@ -64,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;
@@ -77,11 +82,8 @@ import static org.apache.solr.core.TestSolrConfigHandler.getFileContent;
 import static org.apache.solr.filestore.TestDistribPackageStore.checkAllNodesForFile;
 import static org.apache.solr.filestore.TestDistribPackageStore.readFile;
 import static org.apache.solr.filestore.TestDistribPackageStore.uploadKey;
-import static org.hamcrest.CoreMatchers.containsString;
 
 @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
-@LuceneTestCase.AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-14901")
 public class TestPackages extends SolrCloudTestCase {
 
   @Before
@@ -187,23 +189,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" );
 
@@ -253,19 +255,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" );
 
@@ -284,15 +286,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" );
 
@@ -323,15 +325,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" );
 
@@ -339,15 +341,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" );
 
@@ -368,15 +370,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" );
 
@@ -398,15 +400,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" );
 
@@ -422,10 +424,10 @@ public class TestPackages extends SolrCloudTestCase {
       plugins.put("create-queryparser", p);
 
       v2r = new V2Request.Builder( "/c/"+COLLECTION_NAME+ "/config")
-              .withMethod(SolrRequest.METHOD.POST)
-              .withPayload(plugins)
-              .forceV2(true)
-              .build();
+          .withMethod(SolrRequest.METHOD.POST)
+          .withPayload(plugins)
+          .forceV2(true)
+          .build();
       cluster.getSolrClient().request(v2r);
       assertTrue(C.informCalled);
       assertTrue(C2.informCalled);
@@ -445,11 +447,8 @@ public class TestPackages extends SolrCloudTestCase {
       cluster.shutdown();
     }
   }
-
-  @SuppressWarnings({"unchecked"})
-  private void executeReq(String uri, JettySolrRunner jetty,
-                          @SuppressWarnings({"rawtypes"})Utils.InputStreamConsumer parser,
-                          @SuppressWarnings({"rawtypes"})Map expected) throws Exception {
+  @SuppressWarnings({"unchecked","rawtypes"})
+  private void executeReq(String uri, JettySolrRunner jetty, Utils.InputStreamConsumer parser, Map expected) throws Exception {
     try(HttpSolrClient client = (HttpSolrClient) jetty.newClient()){
       TestDistribPackageStore.assertResponseValues(10,
           () -> {
@@ -463,8 +462,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,
@@ -644,7 +643,6 @@ public class TestPackages extends SolrCloudTestCase {
     }
   }
 
-  @SuppressWarnings("rawtypes")
   public void testSchemaPlugins() throws Exception {
     String COLLECTION_NAME = "testSchemaLoadingColl";
     System.setProperty("managed.schema.mutable", "true");
@@ -659,7 +657,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,
@@ -670,20 +668,20 @@ public class TestPackages extends SolrCloudTestCase {
       add.pkg = "schemapkg";
       add.files = Arrays.asList(FILE1,FILE2);
       V2Request req = new V2Request.Builder("/cluster/package")
-              .forceV2(true)
-              .withMethod(SolrRequest.METHOD.POST)
-              .withPayload(Collections.singletonMap("add", add))
-              .build();
+          .forceV2(true)
+          .withMethod(SolrRequest.METHOD.POST)
+          .withPayload(Collections.singletonMap("add", add))
+          .build();
       req.process(cluster.getSolrClient());
 
       TestDistribPackageStore.assertResponseValues(10,
-              () -> new V2Request.Builder("/cluster/package").
-                      withMethod(SolrRequest.METHOD.GET)
-                      .build().process(cluster.getSolrClient()),
-              Utils.makeMap(
-                      ":result:packages:schemapkg[0]:version", "1.0",
-                      ":result:packages:schemapkg[0]:files[0]", FILE1
-              ));
+          () -> new V2Request.Builder("/cluster/package").
+              withMethod(SolrRequest.METHOD.GET)
+              .build().process(cluster.getSolrClient()),
+          Utils.makeMap(
+              ":result:packages:schemapkg[0]:version", "1.0",
+              ":result:packages:schemapkg[0]:files[0]", FILE1
+          ));
 
       CollectionAdminRequest
               .createCollection(COLLECTION_NAME, "conf1", 2, 2)
@@ -698,27 +696,27 @@ 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)
-              .withPayload(Collections.singletonMap("add", add))
-              .build();
+          .forceV2(true)
+          .withMethod(SolrRequest.METHOD.POST)
+          .withPayload(Collections.singletonMap("add", add))
+          .build();
       req.process(cluster.getSolrClient());
 
       TestDistribPackageStore.assertResponseValues(10,
-              () -> new V2Request.Builder("/cluster/package").
-                      withMethod(SolrRequest.METHOD.GET)
-                      .build().process(cluster.getSolrClient()),
-              Utils.makeMap(
-                      ":result:packages:schemapkg[0]:version", "2.0",
-                      ":result:packages:schemapkg[0]:files[0]", FILE1
-              ));
+          () -> new V2Request.Builder("/cluster/package").
+              withMethod(SolrRequest.METHOD.GET)
+              .build().process(cluster.getSolrClient()),
+          Utils.makeMap(
+              ":result:packages:schemapkg[0]:version", "2.0",
+              ":result:packages:schemapkg[0]:files[0]", FILE1
+          ));
 
       verifySchemaComponent(cluster.getSolrClient(), COLLECTION_NAME, "/schema/fieldtypes/myNewTextFieldWithAnalyzerClass",
-              Utils.makeMap(":fieldType:analyzer:charFilters[0]:_packageinfo_:version" ,"2.0",
-                      ":fieldType:analyzer:tokenizer:_packageinfo_:version","2.0",
-                      ":fieldType:_packageinfo_:version","2.0"));
+          Utils.makeMap(":fieldType:analyzer:charFilters[0]:_packageinfo_:version" ,"2.0",
+              ":fieldType:analyzer:tokenizer:_packageinfo_:version","2.0",
+              ":fieldType:_packageinfo_:version","2.0"));
 
     } finally {
       cluster.shutdown();
@@ -729,14 +727,14 @@ public class TestPackages extends SolrCloudTestCase {
   private void verifySchemaComponent(SolrClient client, String COLLECTION_NAME, String path,
                                      Map expected) throws Exception {
     SolrParams params = new MapSolrParams((Map) Utils.makeMap("collection", COLLECTION_NAME,
-            WT, JAVABIN,
-            "meta", "true"));
+        WT, JAVABIN,
+        "meta", "true"));
 
     GenericSolrRequest req = new GenericSolrRequest(SolrRequest.METHOD.GET,path
-            , params);
+        , params);
     TestDistribPackageStore.assertResponseValues(10,
-            client,
-            req, expected);
+        client,
+        req, expected);
   }
 
   public static void postFileAndWait(MiniSolrCloudCluster cluster, String fname, String path, String sig) throws Exception {
@@ -748,7 +746,7 @@ public class TestPackages extends SolrCloudTestCase {
         path, sig);// has file, but no signature
 
     TestDistribPackageStore.checkAllNodesForFile(cluster, path, Utils.makeMap(
-            ":files:" + path + ":sha512",
+        ":files:" + path + ":sha512",
         sha512
     ), false);
   }
@@ -759,7 +757,56 @@ public class TestPackages extends SolrCloudTestCase {
       fail("should have failed with message : " + expectErrorMsg);
     } catch (BaseHttpSolrClient.RemoteExecutionException e) {
       String msg = e.getMetaData()._getStr(errPath, "");
-      assertThat(msg, containsString(expectErrorMsg));
+      assertTrue("should have failed with message: " + expectErrorMsg + "actual message : " + msg,
+          msg.contains(expectErrorMsg)
+      );
     }
   }
+
+  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();
+  }
+
 }