You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mk...@apache.org on 2018/04/10 18:34:05 UTC

lucene-solr:branch_7x: SOLR-12155: awake threads awaiting UIF. despite of exception.

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x f8f1e23b8 -> 6ac9a19aa


SOLR-12155: awake threads awaiting UIF.<init> despite of exception.


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

Branch: refs/heads/branch_7x
Commit: 6ac9a19aa89d3ff59408902700f0737bca14d3aa
Parents: f8f1e23
Author: Mikhail Khludnev <mk...@apache.org>
Authored: Tue Apr 10 21:20:20 2018 +0300
Committer: Mikhail Khludnev <mk...@apache.org>
Committed: Tue Apr 10 21:33:44 2018 +0300

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../solr/search/facet/UnInvertedField.java      |  55 +++++---
 .../org/apache/solr/util/TestInjection.java     |  10 ++
 .../request/TestUnInvertedFieldException.java   | 127 +++++++++++++++++++
 4 files changed, 178 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6ac9a19a/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 8e7fd8b..0ee704e 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -103,6 +103,8 @@ Bug Fixes
 * SOLR-12207: Just rethrowing AssertionError caused by jdk bug in reflection with invocation details.
  (ab, Dawid Weiss, Mikhail Khludnev)
 
+* SOLR-12155: Exception from UnInvertedField constructor puts threads to infinite wait. (Mikhail Khludnev)
+ 
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6ac9a19a/solr/core/src/java/org/apache/solr/search/facet/UnInvertedField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/UnInvertedField.java b/solr/core/src/java/org/apache/solr/search/facet/UnInvertedField.java
index 9c395b7..69b341a 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/UnInvertedField.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/UnInvertedField.java
@@ -35,6 +35,7 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.index.SlowCompositeReaderWrapper;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.TrieField;
@@ -44,6 +45,7 @@ import org.apache.solr.search.DocSet;
 import org.apache.solr.search.SolrCache;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.uninverting.DocTermOrds;
+import org.apache.solr.util.TestInjection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -106,7 +108,7 @@ public class UnInvertedField extends DocTermOrds {
   private SolrIndexSearcher.DocsEnumState deState;
   private final SolrIndexSearcher searcher;
 
-  private static UnInvertedField uifPlaceholder = new UnInvertedField();
+  private static final UnInvertedField uifPlaceholder = new UnInvertedField();
 
   private UnInvertedField() { // Dummy for synchronization.
     super("fake", 0, 0); // cheapest initialization I can find.
@@ -186,6 +188,8 @@ public class UnInvertedField extends DocTermOrds {
         searcher.maxDoc()/20 + 2,
         DEFAULT_INDEX_INTERVAL_BITS);
 
+    assert TestInjection.injectUIFOutOfMemoryError();
+
     final String prefix = TrieField.getMainValuePrefix(searcher.getSchema().getFieldType(field));
     this.searcher = searcher;
     try {
@@ -556,16 +560,17 @@ public class UnInvertedField extends DocTermOrds {
   //////////////////////////// caching /////////////////////////////
   //////////////////////////////////////////////////////////////////
 
+  @SuppressWarnings("unchecked")
   public static UnInvertedField getUnInvertedField(String field, SolrIndexSearcher searcher) throws IOException {
-    SolrCache<String,UnInvertedField> cache = searcher.getFieldValueCache();
+    SolrCache cache = searcher.getFieldValueCache();
     if (cache == null) {
       return new UnInvertedField(field, searcher);
     }
-    UnInvertedField uif = null;
+
     Boolean doWait = false;
     synchronized (cache) {
-      uif = cache.get(field);
-      if (uif == null) {
+      final Object val = cache.get(field);
+      if (val == null || (val instanceof Throwable)) {
         /**
          * We use this place holder object to pull the UninvertedField construction out of the sync
          * so that if many fields are accessed in a short time, the UninvertedField can be
@@ -573,8 +578,8 @@ public class UnInvertedField extends DocTermOrds {
          */
         cache.put(field, uifPlaceholder);
       } else {
-        if (uif != uifPlaceholder) {
-          return uif;
+        if (val != uifPlaceholder) {
+          return (UnInvertedField) val;
         }
         doWait = true; // Someone else has put the place holder in, wait for that to complete.
       }
@@ -582,35 +587,53 @@ public class UnInvertedField extends DocTermOrds {
     while (doWait) {
       try {
         synchronized (cache) {
-          uif = cache.get(field); // Should at least return the placeholder, NPE if not is OK.
-          if (uif != uifPlaceholder) { // OK, another thread put this in the cache we should be good.
-            return uif;
+          final Object val = cache.get(field);
+          if (val != uifPlaceholder) { // OK, another thread put this in the cache we should be good.
+            if (val instanceof Throwable) {
+              rethrowAsSolrException(field, (Throwable) val);
+            } else {
+              return (UnInvertedField) val;
+            }
           }
           cache.wait();
         }
       } catch (InterruptedException e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Thread interrupted in getUninvertedField.");
+        rethrowAsSolrException(field, e);
       }
     }
 
-    uif = new UnInvertedField(field, searcher);
+    UnInvertedField uif = null;
+    try {
+      uif = new UnInvertedField(field, searcher);
+    }catch(Throwable e) {
+      synchronized (cache) {
+        cache.put(field, e); // signaling the failure
+        cache.notifyAll();
+      }
+      rethrowAsSolrException(field, e);
+    }
     synchronized (cache) {
       cache.put(field, uif); // Note, this cleverly replaces the placeholder.
       cache.notifyAll();
     }
-
     return uif;
   }
 
+  protected static void rethrowAsSolrException(String field, Throwable e) {
+    throw new SolrException(ErrorCode.SERVER_ERROR, 
+            "Exception occurs during uninverting "+field, e);
+  }
+
   // Returns null if not already populated
+  @SuppressWarnings({"rawtypes", "unchecked"})
   public static UnInvertedField checkUnInvertedField(String field, SolrIndexSearcher searcher) throws IOException {
-    SolrCache<String, UnInvertedField> cache = searcher.getFieldValueCache();
+    SolrCache cache = searcher.getFieldValueCache();
     if (cache == null) {
       return null;
     }
-    UnInvertedField uif = cache.get(field);  // cache is already synchronized, so no extra sync needed
+    Object uif = cache.get(field);  // cache is already synchronized, so no extra sync needed
     // placeholder is an implementation detail, keep it hidden and return null if that is what we got
-    return uif==uifPlaceholder ? null : uif;
+    return uif==uifPlaceholder || !(uif instanceof UnInvertedField)? null : (UnInvertedField) uif;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6ac9a19a/solr/core/src/java/org/apache/solr/util/TestInjection.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/TestInjection.java b/solr/core/src/java/org/apache/solr/util/TestInjection.java
index 821b37e..1685946 100644
--- a/solr/core/src/java/org/apache/solr/util/TestInjection.java
+++ b/solr/core/src/java/org/apache/solr/util/TestInjection.java
@@ -146,6 +146,8 @@ public class TestInjection {
 
   public static Integer delayBeforeSlaveCommitRefresh=null;
 
+  public static boolean uifOutOfMemoryError = false;
+
   public static void reset() {
     nonGracefullClose = null;
     failReplicaRequests = null;
@@ -161,6 +163,7 @@ public class TestInjection {
     failIndexFingerprintRequests = null;
     wrongIndexFingerprint = null;
     delayBeforeSlaveCommitRefresh = null;
+    uifOutOfMemoryError = false;
 
     for (Timer timer : timers) {
       timer.cancel();
@@ -470,4 +473,11 @@ public class TestInjection {
     return true;
   }
 
+  public static boolean injectUIFOutOfMemoryError() {
+    if (uifOutOfMemoryError ) {
+      throw new OutOfMemoryError("Test Injection");
+    }
+    return true;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6ac9a19a/solr/core/src/test/org/apache/solr/request/TestUnInvertedFieldException.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/request/TestUnInvertedFieldException.java b/solr/core/src/test/org/apache/solr/request/TestUnInvertedFieldException.java
new file mode 100644
index 0000000..56addf6
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/request/TestUnInvertedFieldException.java
@@ -0,0 +1,127 @@
+/*
+ * 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.request;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.util.NamedThreadFactory;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.util.ExecutorUtil.MDCAwareThreadPoolExecutor;
+import org.apache.solr.search.facet.UnInvertedField;
+import org.apache.solr.util.TestInjection;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestUnInvertedFieldException extends SolrTestCaseJ4 {
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig.xml","schema11.xml");
+  }
+
+  private int numTerms;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    numTerms = TestUtil.nextInt(random(), 10, 50);
+    createIndex(numTerms);
+  }
+
+  @After
+  @Override
+  public void tearDown() throws Exception {
+    super.tearDown();
+  }
+
+  String t(int tnum) {
+    return String.format(Locale.ROOT, "%08d", tnum);
+  }
+  
+  void createIndex(int nTerms) {
+    assertU(delQ("*:*"));
+    for (int i=0; i<nTerms; i++) {
+      assertU(adoc("id", Integer.toString(i), proto.field(), t(i) ));
+    }
+    assertU(commit()); 
+  }
+
+  Term proto = new Term("field_s","");
+
+  @Test
+  public void testConcurrentInit() throws Exception {
+    final SolrQueryRequest req = req("*:*");
+
+    List<Callable<UnInvertedField>> initCallables = new ArrayList<>();
+    for (int i=0;i< TestUtil.nextInt(random(), 10, 30);i++) {
+      initCallables.add(()-> UnInvertedField.getUnInvertedField(proto.field(), req.getSearcher()));
+    }
+
+    final ThreadPoolExecutor pool  = new MDCAwareThreadPoolExecutor(3, 
+        TestUtil.nextInt(random(), 3, 6), 10, TimeUnit.MILLISECONDS,
+        new LinkedBlockingQueue<Runnable>(), new NamedThreadFactory(getClass().getSimpleName()));
+
+    try {
+      TestInjection.uifOutOfMemoryError = true;
+      if (assertsAreEnabled) { // if they aren't, we check that injection is disabled in live
+        List<Future<UnInvertedField>> futures = initCallables.stream().map((c) -> pool.submit(c))
+            .collect(Collectors.toList());
+        for (Future<UnInvertedField> uifuture : futures) {
+          try {
+            final UnInvertedField uif = uifuture.get();
+          } catch (ExecutionException injection) {
+            SolrException solrException = (SolrException) injection.getCause();
+            assertEquals(ErrorCode.SERVER_ERROR.code, solrException.code());
+            assertSame(solrException.getCause().getClass(), OutOfMemoryError.class);
+          }
+          assertNull(UnInvertedField.checkUnInvertedField(proto.field(), req.getSearcher()));
+        }
+        TestInjection.uifOutOfMemoryError = false;
+      }
+      UnInvertedField prev = null;
+      List<Future<UnInvertedField>> futures = initCallables.stream().map((c) -> pool.submit(c))
+          .collect(Collectors.toList());
+      for (Future<UnInvertedField> uifuture : futures) {
+        final UnInvertedField uif = uifuture.get();
+        assertNotNull(uif);
+        assertSame(uif, UnInvertedField.checkUnInvertedField(proto.field(), req.getSearcher()));
+        if (prev != null) {
+          assertSame(prev, uif);
+        }
+        assertEquals(numTerms, uif.numTerms());
+        prev = uif;
+      }
+    } finally {
+      pool.shutdownNow();
+      req.close();
+    }
+  }
+}
+