[ 
https://issues.apache.org/jira/browse/LUCENE-9791?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17290140#comment-17290140
 ] 

Mike Drob commented on LUCENE-9791:
-----------------------------------

>From the comments on the PR:

{quote}As a result it is now possible to call BytesRefHash#find concurrently as 
long as there are
no concurrent modifications to BytesRefHash instance and it is correctly 
published.
{quote}

My comment:

{quote}This feels like a trap waiting for somebody to fall in. I think a better 
solution would be a pool of buffers and proper concurrency controls around 
them, but don't know what the performance would look like.

At a minimum, we need to update the method javadoc to explain what is going on 
and what proper usage looks like.
{quote}

Robert's response:

{quote}sorry I don't think it is a trap really. Everything else using this 
thing is using it in a per-thread way. A lot of lucene works this way, it keeps 
things fast.

I don't think we should slow down all the correct users and add complex 
concurrency, just because we have one module abusing it. This would be 
equivalent to making all java code use HashTable/StringBuffer when most code 
only needs HashMap/StringBuilder. We could easily slow down important things 
such as indexing. See my comments on the issue.
{quote}

I think we agree here. Specifically, I find Robert's earlier comment in this 
JIRA very good - that we should not be trying to make this partially thread 
safe. If we're going to be doing changes like this to make it thread safe, that 
is the case where I would push for a full solution, but acknowledge that the 
performance of that would likely be a high cost.

I think it is a problem if we start saying that you can use it concurrently 
_this way_ but not concurrently _that way_. Eventually somebody gets confused. 
Better to just draw a line in the sand and say this class is or is not thread 
safe and be done with it.

Maybe another idea is to have a FrozenBytesRefHash that is explicitly safe for 
concurrent lookup because it does not support modification?

> Monitor (aka Luwak) has concurrency issues related to BytesRefHash#find
> -----------------------------------------------------------------------
>
>                 Key: LUCENE-9791
>                 URL: https://issues.apache.org/jira/browse/LUCENE-9791
>             Project: Lucene - Core
>          Issue Type: Bug
>          Components: core/other
>    Affects Versions: master (9.0), 8.7, 8.8
>            Reporter: Paweł Bugalski
>            Priority: Major
>         Attachments: LUCENE-9791.patch
>
>          Time Spent: 40m
>  Remaining Estimate: 0h
>
> _org.apache.lucene.monitor.Monitor_ can sometimes *NOT* match a document that 
> should be matched by one of registered queries if match operations are run 
> concurrently from multiple threads. 
> This is because sometimes in a concurrent environment 
> _TermFilteredPresearcher_ might not select a query that could later on match 
> one of documents being matched.
> Internally _TermFilteredPresearcher_ is using a term acceptor: an instance of 
> _org.apache.lucene.monitor.QueryIndex.QueryTermFilter_. _QueryTermFilter_ is 
> correctly initialized under lock and its internal state (a map of 
> _org.apache.lucene.util.BytesRefHash_ instances) is correctly published. 
> Later one when those instances are used concurrently a problem with 
> _org.apache.lucene.util.BytesRefHash#find_ is triggered since it is not 
> thread safe.
> _org.apache.lucene.util.BytesRefHash#find_ internally is using a private 
> _org.apache.lucene.util.BytesRefHash#equals_ method, which is using an 
> instance field _scratch1_ as a temporary buffer to compare its _ByteRef_ 
> parameter with contents of _ByteBlockPool_. This is not thread safe and can 
> cause incorrect answers as well as _ArrayOutOfBoundException_. 
> __
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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

Reply via email to