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 2018/05/28 05:07:31 UTC

lucene-solr:branch_7x: SOLR-12294: update processors loaded from runtime jars fail to load if they are specified in an update processor chain

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x 22e333f1d -> 40eb83980


SOLR-12294: update processors loaded from runtime jars fail to load if they are specified in an update processor chain


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/40eb8398
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/40eb8398
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/40eb8398

Branch: refs/heads/branch_7x
Commit: 40eb83980085179a2034bb51949d4ac5efdadffe
Parents: 22e333f
Author: Noble Paul <no...@apache.org>
Authored: Sat May 26 00:25:39 2018 +1000
Committer: Noble Paul <no...@apache.org>
Committed: Mon May 28 15:07:20 2018 +1000

----------------------------------------------------------------------
 .../java/org/apache/solr/core/PluginBag.java    |  12 +-
 .../processor/UpdateRequestProcessorChain.java  |  39 +++++++
 .../src/test-files/runtimecode/TestURP.java     |  30 +++++
 .../org/apache/solr/core/TestDynamicURP.java    | 110 +++++++++++++++++++
 .../solr/update/processor/RuntimeUrp.java       |   3 +
 .../solr/client/solrj/request/V2Request.java    |   5 +-
 6 files changed, 197 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/40eb8398/solr/core/src/java/org/apache/solr/core/PluginBag.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/PluginBag.java b/solr/core/src/java/org/apache/solr/core/PluginBag.java
index b0c68f2..abd1a44 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginBag.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginBag.java
@@ -44,6 +44,8 @@ import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.component.SearchComponent;
 import org.apache.solr.request.SolrRequestHandler;
+import org.apache.solr.update.processor.UpdateRequestProcessorChain;
+import org.apache.solr.update.processor.UpdateRequestProcessorFactory;
 import org.apache.solr.util.CryptoKeys;
 import org.apache.solr.util.SimplePostTool;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
@@ -125,9 +127,13 @@ public class PluginBag<T> implements AutoCloseable {
   public PluginHolder<T> createPlugin(PluginInfo info) {
     if ("true".equals(String.valueOf(info.attributes.get("runtimeLib")))) {
       log.debug(" {} : '{}'  created with runtimeLib=true ", meta.getCleanTag(), info.name);
-      return new LazyPluginHolder<T>(meta, info, core, "true".equals(System.getProperty("enable.runtime.lib")) ?
+      LazyPluginHolder<T> holder = new LazyPluginHolder<>(meta, info, core, "true".equals(System.getProperty("enable.runtime.lib")) ?
           core.getMemClassLoader() :
           core.getResourceLoader(), true);
+
+      return meta.clazz == UpdateRequestProcessorFactory.class ?
+          (PluginHolder<T>) new UpdateRequestProcessorChain.LazyUpdateProcessorFactoryHolder(holder) :
+          holder;
     } else if ("lazy".equals(info.attributes.get("startup")) && meta.options.contains(SolrConfig.PluginOpts.LAZY)) {
       log.debug("{} : '{}' created with startup=lazy ", meta.getCleanTag(), info.name);
       return new LazyPluginHolder<T>(meta, info, core, core.getResourceLoader(), false);
@@ -361,6 +367,10 @@ public class PluginBag<T> implements AutoCloseable {
       if (pluginInfo != null) return pluginInfo.className;
       return null;
     }
+
+    public PluginInfo getPluginInfo() {
+      return pluginInfo;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/40eb8398/solr/core/src/java/org/apache/solr/update/processor/UpdateRequestProcessorChain.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/UpdateRequestProcessorChain.java b/solr/core/src/java/org/apache/solr/update/processor/UpdateRequestProcessorChain.java
index 15da495..646c92e 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/UpdateRequestProcessorChain.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/UpdateRequestProcessorChain.java
@@ -31,6 +31,7 @@ import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.PluginBag;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
@@ -321,6 +322,44 @@ public final class UpdateRequestProcessorChain implements PluginInfoInitialized
     }
   }
 
+  public static class LazyUpdateProcessorFactoryHolder extends PluginBag.PluginHolder<UpdateRequestProcessorFactory> {
+    private volatile UpdateRequestProcessorFactory lazyFactory;
+
+    public LazyUpdateProcessorFactoryHolder(final PluginBag.LazyPluginHolder holder) {
+      super(holder.getPluginInfo());
+      lazyFactory = new LazyUpdateRequestProcessorFactory(holder);
+    }
+
+    @Override
+    public UpdateRequestProcessorFactory get() {
+      // don't initialize the delegate now. wait for the actual instance creation
+      return lazyFactory;
+    }
+
+    public class LazyUpdateRequestProcessorFactory extends UpdateRequestProcessorFactory {
+      private final PluginBag.LazyPluginHolder holder;
+      UpdateRequestProcessorFactory delegate;
+
+      public LazyUpdateRequestProcessorFactory(PluginBag.LazyPluginHolder holder) {
+        this.holder = holder;
+      }
+
+      public UpdateRequestProcessorFactory getDelegate() {
+        return delegate;
+      }
+
+      @Override
+      public UpdateRequestProcessor getInstance(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next) {
+        if (delegate != null) return delegate.getInstance(req, rsp, next);
+
+        synchronized (this) {
+          if (delegate == null)
+            delegate = (UpdateRequestProcessorFactory) holder.get();
+        }
+        return delegate.getInstance(req, rsp, next);
+      }
+    }
+  }
   public static final Map<String, Class> implicits = new ImmutableMap.Builder()
       .put(TemplateUpdateProcessorFactory.NAME, TemplateUpdateProcessorFactory.class)
       .put(AtomicUpdateProcessorFactory.NAME, AtomicUpdateProcessorFactory.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/40eb8398/solr/core/src/test-files/runtimecode/TestURP.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/runtimecode/TestURP.java b/solr/core/src/test-files/runtimecode/TestURP.java
new file mode 100644
index 0000000..5959222
--- /dev/null
+++ b/solr/core/src/test-files/runtimecode/TestURP.java
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.processor.SimpleUpdateProcessorFactory;
+
+public class TestURP extends SimpleUpdateProcessorFactory {
+  @Override
+  protected void process(AddUpdateCommand cmd, SolrQueryRequest req, SolrQueryResponse rsp) {
+    cmd.solrDoc.addField("time_s", ""+System.nanoTime());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/40eb8398/solr/core/src/test/org/apache/solr/core/TestDynamicURP.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestDynamicURP.java b/solr/core/src/test/org/apache/solr/core/TestDynamicURP.java
new file mode 100644
index 0000000..03baadc
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/core/TestDynamicURP.java
@@ -0,0 +1,110 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.request.V2Request;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.handler.TestBlobHandler;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static java.util.Collections.singletonMap;
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
+import static org.apache.solr.core.TestDynamicLoading.getFileContent;
+
+public class TestDynamicURP extends SolrCloudTestCase {
+
+
+  private static final String COLLECTION = "testUrpColl";
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    System.setProperty("enable.runtime.lib", "true");
+    configureCluster(3)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+    SolrZkClient zkClient = cluster.getSolrClient().getZkStateReader().getZkClient();
+    String path = ZkStateReader.CONFIGS_ZKNODE + "/conf/solrconfig.xml";
+    byte[] data = zkClient.getData(path, null, null, true);
+
+    String solrconfigStr = new String(data, StandardCharsets.UTF_8);
+    zkClient.setData(path, solrconfigStr.replace("</config>",
+        "<updateRequestProcessorChain name=\"test_urp\" processor=\"testURP\" default=\"true\">\n" +
+        "    <processor class=\"solr.RunUpdateProcessorFactory\"/>\n" +
+        "  </updateRequestProcessorChain>\n" +
+        "\n" +
+        "  <updateProcessor class=\"org.apache.solr.core.TestURP\" name=\"testURP\" runtimeLib=\"true\"></updateProcessor>\n" +
+        "</config>").getBytes(StandardCharsets.UTF_8), true );
+
+
+    CollectionAdminRequest.createCollection(COLLECTION, "conf", 3, 1).process(cluster.getSolrClient());
+  }
+
+
+
+  @Test
+  public void testUrp() throws Exception {
+
+    ByteBuffer jar = getFileContent("runtimecode/runtimeurp.jar.bin");
+
+    String blobName = "urptest";
+    TestBlobHandler.postAndCheck(cluster.getSolrClient(), cluster.getRandomJetty(random()).getBaseUrl().toString(),
+        blobName, jar, 1);
+
+    new V2Request.Builder("/c/" + COLLECTION + "/config")
+        .withPayload(singletonMap("add-runtimelib", (MapWriter) ew1 -> ew1
+            .put("name", blobName)
+            .put("version", "1")))
+        .withMethod(POST)
+        .build()
+        .process(cluster.getSolrClient());
+    TestSolrConfigHandler.testForResponseElement(null,
+        cluster.getRandomJetty(random()).getBaseUrl().toString(),
+        "/"+COLLECTION+"/config/overlay",
+        cluster.getSolrClient(),
+        Arrays.asList("overlay", "runtimeLib", blobName, "version")
+        ,"1",10);
+
+    SolrInputDocument doc = new SolrInputDocument();
+    doc.addField("id", "123");
+    doc.addField("name_s", "Test URP");
+    new UpdateRequest()
+        .add(doc)
+        .commit(cluster.getSolrClient(), COLLECTION);
+    QueryResponse result = cluster.getSolrClient().query(COLLECTION, new SolrQuery("id:123"));
+    assertEquals(1, result.getResults().getNumFound());
+    Object time_s = result.getResults().get(0).getFirstValue("time_s");
+    assertNotNull(time_s);
+
+
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/40eb8398/solr/core/src/test/org/apache/solr/update/processor/RuntimeUrp.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/RuntimeUrp.java b/solr/core/src/test/org/apache/solr/update/processor/RuntimeUrp.java
index 7c71a15..889b0bf 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/RuntimeUrp.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/RuntimeUrp.java
@@ -30,6 +30,9 @@ public class RuntimeUrp extends SimpleUpdateProcessorFactory {
     UpdateRequestProcessorChain processorChain = req.getCore().getUpdateProcessorChain(req.getParams());
     List<String>  names = new ArrayList<>();
     for (UpdateRequestProcessorFactory p : processorChain.getProcessors()) {
+      if (p instanceof UpdateRequestProcessorChain.LazyUpdateProcessorFactoryHolder.LazyUpdateRequestProcessorFactory) {
+        p = ((UpdateRequestProcessorChain.LazyUpdateProcessorFactoryHolder.LazyUpdateRequestProcessorFactory) p).delegate;
+      }
       names.add(p.getClass().getSimpleName());
     }
     cmd.solrDoc.addField("processors_s", StrUtils.join(names,'>'));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/40eb8398/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java
index b65ce92..c854833 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java
@@ -19,6 +19,7 @@ package org.apache.solr.client.solrj.request;
 
 import java.io.IOException;
 import java.io.OutputStream;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -66,7 +67,9 @@ public class V2Request extends SolrRequest<V2Response> implements MapWriter {
     if (payload == null) return null;
     if (payload instanceof String) {
       return new RequestWriter.StringPayloadContentWriter((String) payload, JSON_MIME);
-
+    }
+    if (payload instanceof Map) {
+      payload = Utils.getDeepCopy((Map) payload, 5);
     }
     return new RequestWriter.ContentWriter() {
       @Override