You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2022/04/18 08:19:19 UTC

[GitHub] [lucene] boicehuang opened a new pull request, #816: LUCENE-10519: ThreadLocal.remove under G1GC takes 100% CPU

boicehuang opened a new pull request, #816:
URL: https://github.com/apache/lucene/pull/816

   See also: https://issues.apache.org/jira/browse/LUCENE-10519
   
   Solution
   ---
   We don't need to store entry twice in the hardRefs And ThreadLocals. Remove ThreadLocal from CloseableThreadLocal so that we would not be affected by the serious flaw of Java's built-in ThreadLocal.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] uschindler commented on pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by GitBox <gi...@apache.org>.
uschindler commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1110778339

   Something like:
   ```java
   private final Lock readLock, writeLock;
   
   CloseableThreadLocal#ctor() {
     super();
     var  rwLock = new ReadWriteLock();
     this readLock = rwLock.readLock();
     this.writeLock = rwLock.writeLock();
     //... more init here
   }
   
   // later in get() when reading instead of synchronized::
   readLock.lock();
   try {
      // read value for current thread
   } finally {
      readLock.unlock();
   }
   
   // later when cleaning up and modifiying map instead of synchronized:
   writeLock.lock();
   try {
      // modify the map and clean up / purge threads
   } finally {
      writeLock.unlock();
   }
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] boicehuang commented on a diff in pull request #816: LUCENE-10519: ThreadLocal.remove under G1GC takes 100% CPU

Posted by GitBox <gi...@apache.org>.
boicehuang commented on code in PR #816:
URL: https://github.com/apache/lucene/pull/816#discussion_r855931374


##########
lucene/core/src/java/org/apache/lucene/util/CloseableThreadLocal.java:
##########
@@ -123,12 +122,34 @@ public void close() {
     // Clear the hard refs; then, the only remaining refs to
     // all values we were storing are weak (unless somewhere
     // else is still using them) and so GC may reclaim them:
-    hardRefs = null;
-    // Take care of the current thread right now; others will be
-    // taken care of via the WeakReferences.
-    if (t != null) {
-      t.remove();
+    synchronized (lock) {
+      if (perThreadValues == null) {
+        return;
+      }
+
+      perThreadValues.clear();
+      perThreadValues = null;
+    }
+  }
+
+
+  /**
+   * Visible to test.
+   *
+   * @return per-thread values map.
+   */
+  Map<String,T> getValuesAfterPurge() {
+    Map<String, T> values = new HashMap<>();

Review Comment:
   Fixed, run the CI again, thanks. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jaisonbi commented on pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by GitBox <gi...@apache.org>.
jaisonbi commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1110743533

   > Stupid question: Why do you not open a bug report on OpenJDK? If this ThreadLocal implementation is working better than the original one AND it behaves exactly identical, why not ask OpenJDK people to replace theirs?
   
   Thanks for the comment. 
   
   Different TheadLocal instances are mixed stored in one ThreadLocalMap, it's the current JDK implementation.  
   The problem is that ThreadLocal instances cannot be removed immediately in CloseableThreadLocal, so making the problem worse. I added one comment in LUCENE-10519, just copying it here:
   
   ```
   Each read thread should has own StoredFieldsReader, I think that's why CloseableThreadLocal/ThreadLocal is used.
   
   When one CloseableThreadLocal instance is created, it can be used by multiple threads. 
   
   Suppose threads {Thread-1, Thread-2, Thread-3, Thread-4} are using same CloseableThreadLocal instance, each thread set own value.
   
   So each thread will store the ThreadLocal instance(t) into it's ThreadLocalMap:
   
       Thread-1: {ThreadLocalMap: [t]}
       Thread-2: {ThreadLocalMap: [t]}
       Thread-3: {ThreadLocalMap: [t]}
       Thread-4: {ThreadLocalMap: [t]}
   
   Suppose CloseableThreadLocal instance got closed by Thread-1. Only Thread-1 removed the ThreadLocal instance from ThreadLocalMap.  
       
       Thread-1: {ThreadLocalMap: []}
       Thread-2: {ThreadLocalMap: [t]}
       Thread-3: {ThreadLocalMap: [t]}
       Thread-4: {ThreadLocalMap: [t]}
       
   The other 3 threads only rely on GC reclaim them. Under G1 GC, the problem gets worse, since each mixed GC only collect partial old regions.
   So ThreadLocalMap entries table may get very huge. That's why  "ReentrantReadWriteLock$ReadLock.unlock" took long time(It need to take long time to expunge stale entries from a huge table)
   
   Regarding on the patch, we keep the similar behavior to ThreadLocal, but avoid use ThreadLocal anymore.
   
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] A-U commented on pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by GitBox <gi...@apache.org>.
A-U commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1303067203

   I tried to replace the `ThreadLocal` in `SolrQueryTimeoutImpl` with this new `CloseableThreadLocal`, some of the test cases were timeout when reading the DocValues. It still happened when switching the lock to `ReentrantReadWriteLock`. So it might be better to add a caveat for the usage of the `CloseableThreadLocal` for some scenarios with heavy reads.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] boicehuang commented on a diff in pull request #816: LUCENE-10519: ThreadLocal.remove under G1GC takes 100% CPU

Posted by GitBox <gi...@apache.org>.
boicehuang commented on code in PR #816:
URL: https://github.com/apache/lucene/pull/816#discussion_r855737016


##########
lucene/core/src/java/org/apache/lucene/util/CloseableThreadLocal.java:
##########
@@ -123,12 +122,34 @@ public void close() {
     // Clear the hard refs; then, the only remaining refs to
     // all values we were storing are weak (unless somewhere
     // else is still using them) and so GC may reclaim them:
-    hardRefs = null;
-    // Take care of the current thread right now; others will be
-    // taken care of via the WeakReferences.
-    if (t != null) {
-      t.remove();
+    synchronized (lock) {
+      if (perThreadValues == null) {
+        return;
+      }
+
+      perThreadValues.clear();
+      perThreadValues = null;
+    }
+  }
+
+
+  /**
+   * Visible to test.
+   *
+   * @return per-thread values map.
+   */
+  Map<String,T> getValuesAfterPurge() {
+    Map<String, T> values = new HashMap<>();

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] uschindler commented on pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by GitBox <gi...@apache.org>.
uschindler commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1111271174

   Here is the map in thread:
   https://github.com/AdoptOpenJDK/openjdk-jdk11/blob/master/src/java.base/share/classes/java/lang/Thread.java#L178-L180
   
   And this is how it is accessed/populated:
   https://github.com/AdoptOpenJDK/openjdk-jdk11/blob/master/src/java.base/share/classes/java/lang/ThreadLocal.java#L161-L173
   
   It's cool from the locking/concurrency perspective, but problematic with GC.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] uschindler commented on pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by GitBox <gi...@apache.org>.
uschindler commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1111146277

   > I want to see these servers make real effort to not use 10,000 threads.
   
   Why does Elasticsearch need so many threads? They have a selector based connection handling! And Solr is ongoing to clean thread pools up.
   
   IMHO, Elasticserach should use fixed size threadpools (there can be many thread in it), but what they should not do (to be a friend of G1GC): purge unused threads. So a thread pool that grows size is fine (often useful when you don't know how many threads you need at beginning), but never shrink pool size. All my jetty servers have fixed pools, BTW.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] rmuir commented on a diff in pull request #816: LUCENE-10519: ThreadLocal.remove under G1GC takes 100% CPU

Posted by GitBox <gi...@apache.org>.
rmuir commented on code in PR #816:
URL: https://github.com/apache/lucene/pull/816#discussion_r852294751


##########
lucene/core/src/java/org/apache/lucene/util/CloseableThreadLocal.java:
##########
@@ -123,12 +121,27 @@ public void close() {
     // Clear the hard refs; then, the only remaining refs to
     // all values we were storing are weak (unless somewhere
     // else is still using them) and so GC may reclaim them:
-    hardRefs = null;
-    // Take care of the current thread right now; others will be
-    // taken care of via the WeakReferences.
-    if (t != null) {
-      t.remove();
+    perThreadValues = null;

Review Comment:
   Can we be a bit more friendly, instead of just nulling the reference in close? We can empty the map.
   Something like (being friendly to closeable):
   ```
   var values = perThreadValues;
   if (values != null) {
     values.clear();
   }
   perThreadValues = null;
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jaisonbi commented on pull request #816: LUCENE-10519: ThreadLocal.remove under G1GC takes 100% CPU

Posted by GitBox <gi...@apache.org>.
jaisonbi commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1109236031

   This issue has no relationship with "creating and destroying too many threads", but it's caused by **too many "ThreadLocal" objects created from same threads**.
   
   Based on the implementation of ThreadLocal:
   1. All the "ThreadLocal" objects created from same thread will be stored in one ThreadLocalMap.
   2. Each thread has it's own ThreadLocalMap.
   
   The investigation on the composition of the "ThreadLocalMap": 
   
   "**management**" thread: Most entries(Each ThreadLocal object is wrapped into an Entry) are "CompressingStoredFieldsReader"
   
   ```
     final CloseableThreadLocal<StoredFieldsReader> fieldsReaderLocal = new CloseableThreadLocal<StoredFieldsReader>() {
       @Override
       protected StoredFieldsReader initialValue() {
         return fieldsReaderOrig.clone();
       }
     };
   ```
   
   "**write**" thread: Most entries are "PerThreadIDVersionAndSeqNoLookup"
   
   ```
     static final ConcurrentMap<IndexReader.CacheKey, CloseableThreadLocal<PerThreadIDVersionAndSeqNoLookup[]>> lookupStates =
        ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency();
   ```
   
   Either "CompressingStoredFieldsReader" or "PerThreadIDVersionAndSeqNoLookup" is created by "CloseableThreadLocal". 
   
   ReentrantReadWriteLock will create it's own ThreadLocal object. So the ReentrantReadWrite lock aquired from "write" thread, it will share the same ThreadLocalMap with "PerThreadIDVersionAndSeqNoLookup".  There're too many entries stored into ThreadLocalMap, this is why "ThreadLocal#remove" running with high CPU usage.
   
   So I think the heavy usage of "ThreadLocal" from Lucene's current machanism is one major reason of this issue.
     


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] uschindler commented on pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by GitBox <gi...@apache.org>.
uschindler commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1110661444

   Stupid question: Why do you not open a bug report on OpenJDK? If this ThreadLocal implementation is working better than the original one AND it behaves exactly identical, why not ask OpenJDK people to replace theirs?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jpountz commented on pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by GitBox <gi...@apache.org>.
jpountz commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1110954677

   Maybe another option would be to get rid of these threadlocals. We're relying less on them than we used too, e.g. doc values readers are no longer cached in threadlocals since we moved to iterators. Maybe it would be good enough to pull a new clone of the stored fields / term vectors readers for every document we need to retrieve? And regarding analyzers, we already have a reuse mechanism via `PerField`, which we only use for `StringTokenStream`, not for token streams produced by analyzers: maybe we could reuse state via `PerField` as well so that we would no longer need to cache state in threadlocals?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] LUCENE-10519: Improvement for CloseableThreadLocal [lucene]

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz closed pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal
URL: https://github.com/apache/lucene/pull/816


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] xiaoshi2013 commented on pull request #816: LUCENE-10519: ThreadLocal.remove under G1GC takes 100% CPU

Posted by GitBox <gi...@apache.org>.
xiaoshi2013 commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1107515602

   Great


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] rmuir commented on pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by GitBox <gi...@apache.org>.
rmuir commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1111241274

   > Elasticsearch uses one unbounded threadpool
   
   I do find it hilarious how casually you state this. Only in java, man :)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jpountz commented on pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by GitBox <gi...@apache.org>.
jpountz commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1111221948

   > IMHO, Elasticserach should use fixed size threadpools (there can be many threads in it, the problem is huge pool that spawns new threads all the time and discards them because they are over minimum size).
   
   Elasticsearch uses one unbounded threadpool, but this threadpool shouldn't normally access analyzers or index readers. Apparently, we had a bug and we had this threadpool sometimes access index reader for the purpose of retrieving their memory usage. What is unclear is whether this is the reason why some users have been seeing this threadlocal behavior, or if it's something that can generally happen due to the fact that Elasticsearch often handles lots (possibly in the order of 10k) of segments per node, which translates into as many threadlocals for stored fields and term vectors.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] uschindler commented on pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by GitBox <gi...@apache.org>.
uschindler commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1111262421

   In short: ThreadLocals in Analyzers is ok, because even with many threads (100.000 is no problem), because you have a map per thread pointing to few analyzer's threadlocals with a weak reference.
   
   But having a ThreadLocal in each SegmentReader is a bad idea, because you register link using the weak ref to the ThreadLocal in every thread, possibly 10.000 Segmentreaders in hundreds of threads over time.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] boicehuang commented on pull request #816: LUCENE-10519: ThreadLocal.remove under G1GC takes 100% CPU

Posted by GitBox <gi...@apache.org>.
boicehuang commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1109518481

   @mikemccand  can you also help review this improvement for `CloseableThreadLocal`?  thanks.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] rmuir commented on pull request #816: LUCENE-10519: ThreadLocal.remove under G1GC takes 100% CPU

Posted by GitBox <gi...@apache.org>.
rmuir commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1108554479

   Thanks for getting the tests passing.
   
   I'm honestly torn on this issue, here are my thoughts:
   1. If you have 100% cpu in `java.lang.ThreadLocal.remove()`, you are creating and destroying **far too many*** threads. It is not lucene's fault.  The application needs to be fixed to not churn out so many threads. This problem is unique to the java ecosystem, I don't see it with any other programming languages. Again, there are two problems: a) developers using far too many threads (e.g. solr defaulting to `10000`!!!), and b) developers using "resizable thread pools" (min != max), which doesn't help anything and only magnifies these kinds of issues. The resizable ones just cause constant "churn" and GC pressure, and I know tomcat, jetty, etc love to default to that, but its so inefficient. Java developers should use reasonable fixed-sized threadpools just like everyone else does. Lucene can't fix these apps, they need to be fixed themselves.
   2. I don't like that lucene now has basically a reimplementation of ThreadLocal in its codebase to cater to the problems of such bad applications. IMO, we are a search engine library, we should use `java.lang.ThreadLocal` and if apps use threads in an insane way, they should get the OOM that they deserve.
   3. CloseableThreadLocal is confusing because it is doing two different things as opposed to java Threadlocal. At least let's consider updating the javadocs. The major differences are a) explicit close() for all threads, basically this clears the map when the storage is not needed anymore, and b) faster purging of values for threads that are "dead but not yet GC'd" via periodic liveliness check.
   4. That being said, I don't have real opposition to this patch, but I want us to be careful about correctness. I am also concerned about not hurting the performance of well-behaved apps that don't do bad things. I'm not the best one to review the concurrency/performance implications, as I only have a small 2-core laptop and I can barely remember how Java language works. But let's not punish apps that use threads reasonably. I'm not concerned about performance of badly behaved apps, because they need to fix how they use threads, and we can't help them do that.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] boicehuang commented on a diff in pull request #816: LUCENE-10519: ThreadLocal.remove under G1GC takes 100% CPU

Posted by GitBox <gi...@apache.org>.
boicehuang commented on code in PR #816:
URL: https://github.com/apache/lucene/pull/816#discussion_r854857934


##########
lucene/core/src/java/org/apache/lucene/util/CloseableThreadLocal.java:
##########
@@ -123,12 +121,27 @@ public void close() {
     // Clear the hard refs; then, the only remaining refs to
     // all values we were storing are weak (unless somewhere
     // else is still using them) and so GC may reclaim them:
-    hardRefs = null;
-    // Take care of the current thread right now; others will be
-    // taken care of via the WeakReferences.
-    if (t != null) {
-      t.remove();
+    perThreadValues = null;

Review Comment:
   @rmuir  Fixed, thanks! Can you help to review again? Appreciated! 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] boicehuang commented on pull request #816: LUCENE-10519: ThreadLocal.remove under G1GC takes 100% CPU

Posted by GitBox <gi...@apache.org>.
boicehuang commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1107689248

   > `./gradlew check` still fails. looks like the new test catches interruptedexception and doesn't use the exception. in our codebase compiler/linter will fail on this. please annotate the variable with `@SuppressWarnings("unused")`
   
   @rmuir  fixed. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] uschindler commented on pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by GitBox <gi...@apache.org>.
uschindler commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1111142462

   I agree with both of you:
   - We should avoid ThreadLocal at places where it can be done in another way (like for StoredFieldsReaders, it can just create a new one, escape analysis will throw it away! Warning: testing please with many iterations and tiered compilation turned on)
   - At other places (analyzers): just use default ThreadLocal
   
   If we remove CloseableThreadLocal please also look at other "dead" classes in utils package. While looking for a WeakConcurrentHashMap I found oal.util.WeakIdentityHashMap, which seems no longer used (maybe in Solr, then move there). I implemented it long time ago for AttributeSource, but that's obsolete. Also VirtualMethod (the sophisticated Policeman overridden method checker is also dead, only one usage in TestFramework -> move there).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] rmuir commented on pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by GitBox <gi...@apache.org>.
rmuir commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1111240565

   I do agree with removing any threadlocals on SegmentCoreReaders, if this is somehow possible to do, without annoying people who bring back 10,000 search results :)
   
   But this is a very different thing than threadlocals on the analyzers, which are totally reasonable and don't create objects for every segment.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] hydrogen666 commented on pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by "hydrogen666 (via GitHub)" <gi...@apache.org>.
hydrogen666 commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1555456315

   Any progress of this PR?
   @uschindler 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] boicehuang commented on a diff in pull request #816: LUCENE-10519: ThreadLocal.remove under G1GC takes 100% CPU

Posted by GitBox <gi...@apache.org>.
boicehuang commented on code in PR #816:
URL: https://github.com/apache/lucene/pull/816#discussion_r855737016


##########
lucene/core/src/java/org/apache/lucene/util/CloseableThreadLocal.java:
##########
@@ -123,12 +122,34 @@ public void close() {
     // Clear the hard refs; then, the only remaining refs to
     // all values we were storing are weak (unless somewhere
     // else is still using them) and so GC may reclaim them:
-    hardRefs = null;
-    // Take care of the current thread right now; others will be
-    // taken care of via the WeakReferences.
-    if (t != null) {
-      t.remove();
+    synchronized (lock) {
+      if (perThreadValues == null) {
+        return;
+      }
+
+      perThreadValues.clear();
+      perThreadValues = null;
+    }
+  }
+
+
+  /**
+   * Visible to test.
+   *
+   * @return per-thread values map.
+   */
+  Map<String,T> getValuesAfterPurge() {
+    Map<String, T> values = new HashMap<>();

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] rmuir commented on pull request #816: LUCENE-10519: ThreadLocal.remove under G1GC takes 100% CPU

Posted by GitBox <gi...@apache.org>.
rmuir commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1106454791

   `./gradlew check` still fails. looks like the new test catches interruptedexception and doesn't use the exception. in our codebase compiler/linter will fail on this. please annotate the variable with `@SuppressWarnings("unused")`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] boicehuang commented on pull request #816: LUCENE-10519: ThreadLocal.remove under G1GC takes 100% CPU

Posted by GitBox <gi...@apache.org>.
boicehuang commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1108216412

   > `./gradlew check` still fails.
   
    I have successfully run `gradle :lucene:core:spotlessApply` and `gradle :lucene:check` locally.  Please run the CI again, thanks.
    BTW, sorry for bothering you, can you give me the right to start workflows? 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] rmuir commented on a diff in pull request #816: LUCENE-10519: ThreadLocal.remove under G1GC takes 100% CPU

Posted by GitBox <gi...@apache.org>.
rmuir commented on code in PR #816:
URL: https://github.com/apache/lucene/pull/816#discussion_r855131404


##########
lucene/core/src/java/org/apache/lucene/util/CloseableThreadLocal.java:
##########
@@ -123,12 +122,34 @@ public void close() {
     // Clear the hard refs; then, the only remaining refs to
     // all values we were storing are weak (unless somewhere
     // else is still using them) and so GC may reclaim them:
-    hardRefs = null;
-    // Take care of the current thread right now; others will be
-    // taken care of via the WeakReferences.
-    if (t != null) {
-      t.remove();
+    synchronized (lock) {
+      if (perThreadValues == null) {
+        return;
+      }
+
+      perThreadValues.clear();
+      perThreadValues = null;
+    }
+  }
+
+
+  /**
+   * Visible to test.
+   *
+   * @return per-thread values map.
+   */
+  Map<String,T> getValuesAfterPurge() {
+    Map<String, T> values = new HashMap<>();

Review Comment:
   the CI build failed, i think we need to `import java.util.HashMap`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] LUCENE-10519: Improvement for CloseableThreadLocal [lucene]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1880904741

   This PR has not had activity in the past 2 weeks, labeling it as stale. If the PR is waiting for review, notify the dev@lucene.apache.org list. Thank you for your contribution!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


Re: [PR] LUCENE-10519: Improvement for CloseableThreadLocal [lucene]

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1880940378

   I'm closing this PR. Since this PR got opened, we removed threadlocals from `SegmentReader` (#11998) so the number of threadlocal objects no longer scales with the number of open segments, which should help mitigate the issue that prompted this PR.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] uschindler commented on a diff in pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by GitBox <gi...@apache.org>.
uschindler commented on code in PR #816:
URL: https://github.com/apache/lucene/pull/816#discussion_r859491311


##########
lucene/core/src/test/org/apache/lucene/util/TestCloseableThreadLocal.java:
##########
@@ -48,4 +49,67 @@ protected Object initialValue() {
       return TEST_VALUE;
     }
   }
+
+  public void testSetGetValueWithMultiThreads() {
+    final int CONCURRENT_THREADS = 5;

Review Comment:
   please don't spell local variables in all-upper.



##########
lucene/core/src/test/org/apache/lucene/util/TestCloseableThreadLocal.java:
##########
@@ -48,4 +49,67 @@ protected Object initialValue() {
       return TEST_VALUE;
     }
   }
+
+  public void testSetGetValueWithMultiThreads() {
+    final int CONCURRENT_THREADS = 5;
+    final int LOOPS = 10000;

Review Comment:
   this should be multiplied by the test multiplier. We have a function ("atLeast") in the base test class.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] uschindler commented on pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by GitBox <gi...@apache.org>.
uschindler commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1110769232

   One suggestion: Use a `ReadWriteLock` (https://docs.oracle.com/en/java/javase/11/docs/api/java.base/java/util/concurrent/locks/ReadWriteLock.html) instead of synchronization around the map using an `Object lock`. This makes sure that multiple threads can read from the map in parallel.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] uschindler commented on pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by GitBox <gi...@apache.org>.
uschindler commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1110698870

   > That being said, I don't have real opposition to this patch, but I want us to be careful about correctness. I am also concerned about not hurting the performance of well-behaved apps that don't do bad things. I'm not the best one to review the concurrency/performance implications, as I only have a small 2-core laptop and I can barely remember how Java language works. But let's not punish apps that use threads reasonably. I'm not concerned about performance of badly behaved apps, because they need to fix how they use threads, and we can't help them do that.
   
   See my previous comment: I fully agree and I doubt that this will not affect performance for well-behaving apps. With that patch every `get()` goes through a lock, because the underlying map has no concurrent variant (missing `WeakConcurrentHashMap`) in Java.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] rmuir commented on pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by GitBox <gi...@apache.org>.
rmuir commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1110988180

   The analyzers need to not be slow at query-time too. A threadlocal is a reasonable datastructure to use, you see them in every programming language: https://en.wikipedia.org/wiki/Thread-local_storage
   
   I want to see these servers make real effort to not use 10,000 threads.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] uschindler commented on pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by GitBox <gi...@apache.org>.
uschindler commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1111248678

   > What is unclear is whether this is the reason why some users have been seeing this threadlocal behavior, or if it's something that can generally happen due to the fact that Elasticsearch often handles lots (possibly in the order of 10k) of segments per node, which translates into as many threadlocals for stored fields and term vectors.
   
   That's another issue: You should only create ThreadLocals at some "central" places in your code (e.g. a central one in IndexReader or similar) and use it as per-thread holder for everything you need to remember. You should not have thousands of objects with separate threadlocals. The problem with that is: every thread has a weak map keyed by ThreadLocal (`Map<ThreadLocal,Object> locals`. The setter in Threadlocal calls it like `Thread.currentThread().locals.set(this, value)`, same for get). So when the ThreadLocal objects come and go, the Thread's map collects instances of ThreadLocals not yet garbege collected. Actually this is why they are so fast and it is really clever, because this map is only accessed from inside the thread's context so no locking is needed. But the downside is that it is somehow "inverse". The reference goes from Thread to ThreadLocal, so it must be weak. And this requires cleanup.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] rmuir commented on pull request #816: LUCENE-10519: ThreadLocal.remove under G1GC takes 100% CPU

Posted by GitBox <gi...@apache.org>.
rmuir commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1107875110

   `./gradlew check` still fails.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jaisonbi commented on pull request #816: LUCENE-10519: Improvement for CloseableThreadLocal

Posted by GitBox <gi...@apache.org>.
jaisonbi commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1110760113

   > Actually the implementation does not look wrong. I mostly like that it removes the `ThreadLocal<WeakReference<T>>`.
   > 
   > The problem now is that every call goes through the WeakHashMap with a lock around. The original implementation was always first trying the native thread local (almost lock-free) and only if the value disappeared it was checking the hard references (with a lock). So in most cases getting the value was fast (because the value is still on the thread local).
   > 
   > It would be better to use a ConcurrentHashMap to store the values, but there exists now weak version of it, so the keys (threads) in the map do not disappear automatically once thread has died.
   
   We considered "CloseableThreadLocal#get" and "CloseableThreadLocal#set" as low-frequency calls, so choose WeakHashMap.
   Ya, ConcurrentHashMap would be a better approach,  we can trigger "purge" more frequently to remove the died threads.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] rmuir commented on pull request #816: LUCENE-10519: ThreadLocal.remove under G1GC takes 100% CPU

Posted by GitBox <gi...@apache.org>.
rmuir commented on PR #816:
URL: https://github.com/apache/lucene/pull/816#issuecomment-1109349516

   See this is my problem, applications using threadpools of absurd sizes (unbounded, 10000, etc), and then blaming lucene for their GC woes. at this point the DSM-IV is needed, because developers are in denial.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org