You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by as...@apache.org on 2022/04/27 12:57:26 UTC

[solr] branch branch_9x updated: SOLR-15830: Concurrent core reloads mess up commits when using Schema API (#462)

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

asalamon74 pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/solr.git


The following commit(s) were added to refs/heads/branch_9x by this push:
     new 3e4e230bfcb SOLR-15830: Concurrent core reloads mess up commits when using Schema API (#462)
3e4e230bfcb is described below

commit 3e4e230bfcb1d8e9889a0e705f6c397fbad42d86
Author: Andras Salamon <as...@apache.org>
AuthorDate: Tue Apr 26 12:19:25 2022 +0200

    SOLR-15830: Concurrent core reloads mess up commits when using Schema API (#462)
    
    Co-authored-by: Bence Szabo <sz...@gmail.com>
    (cherry picked from commit bdbb56cc0a38e964ba5c9c965c04c14e19cd7e67)
---
 solr/CHANGES.txt                                   |   2 +
 .../java/org/apache/solr/schema/SchemaManager.java |  29 ++++--
 .../cloud-managed-autocommit/conf/managed-schema   |  29 ++++++
 .../cloud-managed-autocommit/conf/solrconfig.xml   |  60 +++++++++++
 .../schema/TestManagedSchemaWithMultipleAdd.java   | 111 +++++++++++++++++++++
 5 files changed, 220 insertions(+), 11 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 86f37bb2486..77aa799153b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -47,6 +47,8 @@ Bug Fixes
 
 * SOLR-16168: Spellcheck NPE if invalid dictionary name is provided (Kevin Risden)
 
+* SOLR-15830: Concurrent core reloads mess up commits when using Schema API (Bence Szabo via Andras Salamon)
+
 Other Changes
 ---------------------
 * SOLR-15897: Remove <jmx/> from all unit test solrconfig.xml files. (Eric Pugh)
diff --git a/solr/core/src/java/org/apache/solr/schema/SchemaManager.java b/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
index c68d2b53082..de64499349b 100644
--- a/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
+++ b/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
@@ -45,6 +45,7 @@ import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.handler.SolrConfigHandler;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.rest.BaseSolrResource;
 import org.apache.solr.util.TimeOut;
@@ -128,17 +129,23 @@ public class SchemaManager {
           }
 
           try {
-            latestVersion =
-                ZkController.persistConfigResourceToZooKeeper(
-                    zkLoader,
-                    managedIndexSchema.getSchemaZkVersion(),
-                    managedIndexSchema.getResourceName(),
-                    sw.toString().getBytes(StandardCharsets.UTF_8),
-                    true);
-            req.getCore()
-                .getCoreContainer()
-                .reload(req.getCore().getName(), req.getCore().uniqueId);
-            break;
+            SolrConfigHandler configHandler =
+                ((SolrConfigHandler) req.getCore().getRequestHandler("/config"));
+            if (configHandler.getReloadLock().tryLock()) {
+              latestVersion =
+                  ZkController.persistConfigResourceToZooKeeper(
+                      zkLoader,
+                      managedIndexSchema.getSchemaZkVersion(),
+                      managedIndexSchema.getResourceName(),
+                      sw.toString().getBytes(StandardCharsets.UTF_8),
+                      true);
+              req.getCore()
+                  .getCoreContainer()
+                  .reload(req.getCore().getName(), req.getCore().uniqueId);
+              break;
+            } else {
+              log.info("Another reload is in progress. Not doing anything.");
+            }
           } catch (ZkController.ResourceModifiedInZkException e) {
             log.info("Schema was modified by another node. Retrying..");
           }
diff --git a/solr/core/src/test-files/solr/configsets/cloud-managed-autocommit/conf/managed-schema b/solr/core/src/test-files/solr/configsets/cloud-managed-autocommit/conf/managed-schema
new file mode 100644
index 00000000000..4124feab0c3
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/cloud-managed-autocommit/conf/managed-schema
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+<schema name="minimal" version="1.1">
+  <fieldType name="string" class="solr.StrField"/>
+  <fieldType name="int" class="${solr.tests.IntegerFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
+  <fieldType name="long" class="${solr.tests.LongFieldType}" docValues="${solr.tests.numeric.dv}" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
+  <dynamicField name="*" type="string" indexed="true" stored="true"/>
+  <!-- for versioning -->
+  <field name="_version_" type="long" indexed="true" stored="true"/>
+  <field name="_root_" type="string" indexed="true" stored="true" multiValued="false" required="false"/>
+  <field name="id" type="string" indexed="true" stored="true"/>
+  <dynamicField name="*_s"  type="string"  indexed="true"  stored="true" />
+  <uniqueKey>id</uniqueKey>
+</schema>
diff --git a/solr/core/src/test-files/solr/configsets/cloud-managed-autocommit/conf/solrconfig.xml b/solr/core/src/test-files/solr/configsets/cloud-managed-autocommit/conf/solrconfig.xml
new file mode 100644
index 00000000000..9ec69436b3c
--- /dev/null
+++ b/solr/core/src/test-files/solr/configsets/cloud-managed-autocommit/conf/solrconfig.xml
@@ -0,0 +1,60 @@
+<?xml version="1.0" ?>
+
+<!--
+ 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.
+-->
+
+<!-- Minimal solrconfig.xml with /select, /admin and /update only -->
+
+<config>
+
+  <dataDir>${solr.data.dir:}</dataDir>
+
+  <directoryFactory name="DirectoryFactory"
+                    class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
+
+  <schemaFactory class="ManagedIndexSchemaFactory">
+    <bool name="mutable">${managed.schema.mutable}</bool>
+    <str name="managedSchemaResourceName">managed-schema</str>
+  </schemaFactory>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+
+  <updateHandler class="solr.DirectUpdateHandler2">
+    <commitWithin>
+      <softCommit>${solr.commitwithin.softcommit:true}</softCommit>
+    </commitWithin>
+    <updateLog class="${solr.ulog:solr.UpdateLog}"></updateLog>
+
+    <autoCommit>
+      <maxTime>${solr.autoCommit.maxTime:-1}</maxTime>
+      <openSearcher>false</openSearcher>
+    </autoCommit>
+
+    <autoSoftCommit>
+      <maxTime>${solr.autoSoftCommit.maxTime:-1}</maxTime>
+    </autoSoftCommit>
+  </updateHandler>
+
+  <requestHandler name="/select" class="solr.SearchHandler">
+    <lst name="defaults">
+      <str name="echoParams">explicit</str>
+      <str name="indent">true</str>
+      <str name="df">text</str>
+    </lst>
+
+  </requestHandler>
+</config>
\ No newline at end of file
diff --git a/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaWithMultipleAdd.java b/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaWithMultipleAdd.java
new file mode 100644
index 00000000000..0ba0ae282a5
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/schema/TestManagedSchemaWithMultipleAdd.java
@@ -0,0 +1,111 @@
+/*
+ * 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.schema;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.request.schema.SchemaRequest;
+import org.apache.solr.client.solrj.response.schema.SchemaResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrInputDocument;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestManagedSchemaWithMultipleAdd extends SolrCloudTestCase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private static final int AUTOSOFTCOMMIT_MAXTIME_MS = 3000;
+
+  @BeforeClass
+  public static void createClusterAndInitSysProperties() throws Exception {
+    System.setProperty("managed.schema.mutable", "true");
+    System.setProperty("solr.autoSoftCommit.maxTime", Integer.toString(AUTOSOFTCOMMIT_MAXTIME_MS));
+    configureCluster(1)
+        .addConfig(
+            "conf1",
+            TEST_PATH().resolve("configsets").resolve("cloud-managed-autocommit").resolve("conf"))
+        .configure();
+  }
+
+  @AfterClass
+  public static void afterRestartWhileUpdatingTest() {
+    System.clearProperty("managed.schema.mutable");
+    System.clearProperty("solr.autoSoftCommit.maxTime");
+  }
+
+  @Test
+  public void test() throws Exception {
+    String collection = "testschemaapi";
+    CollectionAdminRequest.createCollection(collection, "conf1", 1, 1)
+        .process(cluster.getSolrClient());
+    testAddFieldAndMultipleDocument(collection);
+  }
+
+  private void testAddFieldAndMultipleDocument(String collection)
+      throws IOException, SolrServerException, InterruptedException {
+
+    CloudSolrClient cloudClient = cluster.getSolrClient();
+
+    String fieldName = "myNewField1";
+    addStringField(fieldName, collection, cloudClient);
+
+    UpdateRequest ureq = new UpdateRequest();
+    int numDocs = 1000;
+    for (int i = 0; i < numDocs; i++) {
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", i);
+      doc.addField(fieldName, "value" + i);
+      ureq = ureq.add(doc);
+    }
+    cloudClient.request(ureq, collection);
+
+    // The issue we test in this class does not appear when using explicit commits.
+    // Because of this we are waiting for autoSoftCommit to finish if there is one.
+    Thread.sleep(AUTOSOFTCOMMIT_MAXTIME_MS + 500);
+
+    assertEquals(
+        numDocs, cloudClient.query(collection, new SolrQuery("*:*")).getResults().getNumFound());
+  }
+
+  private void addStringField(String fieldName, String collection, CloudSolrClient cloudClient)
+      throws IOException, SolrServerException {
+    Map<String, Object> fieldAttributes = new LinkedHashMap<>();
+    fieldAttributes.put("name", fieldName);
+    fieldAttributes.put("type", "string");
+    SchemaRequest.AddField addFieldUpdateSchemaRequest =
+        new SchemaRequest.AddField(fieldAttributes);
+    SchemaResponse.UpdateResponse addFieldResponse =
+        addFieldUpdateSchemaRequest.process(cloudClient, collection);
+    assertEquals(0, addFieldResponse.getStatus());
+    assertNull(addFieldResponse.getResponse().get("errors"));
+
+    if (log.isInfoEnabled()) {
+      log.info("added new field = {}", fieldName);
+    }
+  }
+}