You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by md...@apache.org on 2021/11/09 03:09:25 UTC

[lucene-solr] branch branch_8_11 updated: SOLR-15762 Error on Join Query with sync cache (#2606)

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

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


The following commit(s) were added to refs/heads/branch_8_11 by this push:
     new f16cc69  SOLR-15762 Error on Join Query with sync cache (#2606)
f16cc69 is described below

commit f16cc6914bca2a82c2f7967dd41dc6072ab883b2
Author: Mike Drob <md...@apache.org>
AuthorDate: Mon Nov 8 21:06:17 2021 -0600

    SOLR-15762 Error on Join Query with sync cache (#2606)
    
    * SOLR-15762 Error on Join Query with sync cache
    
    When attempting to run a join query using a synchronous filtercache,
    throw an error early with a link to docs rather than a mysterious
    IllegalStateException.
    
    Co-authored-by: Thomas Wöckinger <tw...@silbergrau.com>
---
 .../java/org/apache/solr/search/CaffeineCache.java |  5 +++
 .../org/apache/solr/search/JoinQParserPlugin.java  |  5 +++
 .../src/java/org/apache/solr/search/SolrCache.java | 11 +++++
 .../search/join/InvalidConfigJoinQueryTest.java    | 47 ++++++++++++++++++++++
 4 files changed, 68 insertions(+)

diff --git a/solr/core/src/java/org/apache/solr/search/CaffeineCache.java b/solr/core/src/java/org/apache/solr/search/CaffeineCache.java
index 6f098a3..f182486 100644
--- a/solr/core/src/java/org/apache/solr/search/CaffeineCache.java
+++ b/solr/core/src/java/org/apache/solr/search/CaffeineCache.java
@@ -409,6 +409,11 @@ public class CaffeineCache<K, V> extends SolrCacheBase implements SolrCache<K, V
         limit, initialSize, isAutowarmingOn() ? (", " + getAutowarmDescription()) : "");
   }
 
+  @Override
+  public boolean isRecursionSupported() {
+    return async;
+  }
+
   //////////////////////// SolrInfoBean methods //////////////////////
 
   @Override
diff --git a/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
index 7cb3596..3384514 100644
--- a/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
@@ -588,6 +588,11 @@ class JoinQuery extends Query {
             // use the filterCache to get a DocSet
             if (toTermsEnum.docFreq() >= minDocFreqTo || resultBits == null) {
               // use filter cache
+              SolrCache<?, ?> filterCache = toSearcher.getFilterCache();
+              if (filterCache != null && !filterCache.isRecursionSupported()) {
+                throw new SolrException(SolrException.ErrorCode.INVALID_STATE,
+                    "Using join queries with synchronous filterCache is not supported! Details can be found in Solr Reference Guide under 'query-settings-in-solrconfig'.");
+              }
               DocSet toTermSet = toSearcher.getDocSet(toDeState);
               resultListDocs += toTermSet.size();
               if (resultBits != null) {
diff --git a/solr/core/src/java/org/apache/solr/search/SolrCache.java b/solr/core/src/java/org/apache/solr/search/SolrCache.java
index f7c7623..012c6a0 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrCache.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrCache.java
@@ -22,6 +22,7 @@ import org.apache.solr.util.IOFunction;
 
 import java.io.IOException;
 import java.util.Map;
+import java.util.function.Function;
 
 
 /**
@@ -174,4 +175,14 @@ public interface SolrCache<K,V> extends SolrInfoBean, SolrMetricProducer {
    * only on implementations that support it, it's a no-op otherwise.
    */
   void setMaxRamMB(int maxRamMB);
+
+  /**
+   * Check if this SolrCache supports recursive calls to {@link #computeIfAbsent(Object, IOFunction)}.
+   * Caches backed by {@link java.util.concurrent.ConcurrentHashMap#computeIfAbsent(Object, Function)} explicitly do
+   * not support that, but other caches might.
+   * @return whether this cache allows recursive computations
+   */
+  default boolean isRecursionSupported() {
+    return false;
+  }
 }
diff --git a/solr/core/src/test/org/apache/solr/search/join/InvalidConfigJoinQueryTest.java b/solr/core/src/test/org/apache/solr/search/join/InvalidConfigJoinQueryTest.java
new file mode 100644
index 0000000..d8021f7
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/join/InvalidConfigJoinQueryTest.java
@@ -0,0 +1,47 @@
+/*
+ * 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.search.join;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class InvalidConfigJoinQueryTest extends SolrTestCaseJ4 {
+
+  @BeforeClass
+  public static void before() throws Exception {
+    System.setProperty("solr.filterCache.async", "false");
+    initCore("solrconfig.xml", "schema.xml");
+  }
+
+  @Test
+  public void testInvalidFilterConfig() throws Exception {
+    UpdateRequest req = new UpdateRequest();
+    req.add(new SolrInputDocument("id", "0", "type_s", "org", "locid_s", "1"));
+    req.add(new SolrInputDocument("id", "1", "type_s", "loc", "orgid_s", "0"));
+
+    SolrClient client = new EmbeddedSolrServer(h.getCore());
+    req.commit(client, null);
+
+    assertThrows(SolrException.class, () -> assertJQ(req("q", "{!join from=id to=locid_s v=$q1}", "q1", "type_s:loc", "fl", "id", "sort", "id asc")));
+  }
+}