UnInvertingReader makes indexed fields look like docvalues fields. The caching itself is still done in FieldCache/FieldCacheImpl but you could perhaps wrap what is cached there to either screen out stuff or construct a new entry based on the user.
-Yonik On Thu, Aug 27, 2015 at 12:55 PM, Jamie Johnson <jej2...@gmail.com> wrote: > I think a custom UnInvertingReader would work as I could skip the process > of putting things in the cache. Right now in Solr 4.x though I am caching > based but including the users authorities in the key of the cache so we're > not rebuilding the UnivertedField on every request. Where in 5.x is the > object actually cached? Will this be possible in 5.x? > > On Thu, Aug 27, 2015 at 12:32 PM, Yonik Seeley <ysee...@gmail.com> wrote: > >> The FieldCache has become implementation rather than interface, so I >> don't think you're going to see plugins at that level (it's all >> package protected now). >> >> One could either subclass or re-implement UnInvertingReader though. >> >> -Yonik >> >> >> On Thu, Aug 27, 2015 at 12:09 PM, Jamie Johnson <jej2...@gmail.com> wrote: >> > Also in this vein I think that Lucene should support factories for the >> > cache creation as described @ >> > https://issues.apache.org/jira/browse/LUCENE-2394. I'm not endorsing >> the >> > patch that is provided (I haven't even looked at it) just the concept in >> > general. >> > >> > On Thu, Aug 27, 2015 at 12:01 PM, Jamie Johnson <jej2...@gmail.com> >> wrote: >> > >> >> That makes sense, then I could extend the SolrIndexSearcher by creating >> a >> >> different factory class that did whatever magic I needed. If you >> create a >> >> Jira ticket for this please link it here so I can track it! Again >> thanks >> >> >> >> On Thu, Aug 27, 2015 at 11:59 AM, Tomás Fernández Löbbe < >> >> tomasflo...@gmail.com> wrote: >> >> >> >>> I don't think there is a way to do this now. Maybe we should separate >> the >> >>> logic of creating the SolrIndexSearcher to a factory. Moving this logic >> >>> away from SolrCore is already a win, plus it will make it easier to >> unit >> >>> test and extend for advanced use cases. >> >>> >> >>> Tomás >> >>> >> >>> On Wed, Aug 26, 2015 at 8:10 PM, Jamie Johnson <jej2...@gmail.com> >> wrote: >> >>> >> >>> > Sorry to poke this again but I'm not following the last comment of >> how I >> >>> > could go about extending the solr index searcher and have the >> extension >> >>> > used. Is there an example of this? Again thanks >> >>> > >> >>> > Jamie >> >>> > On Aug 25, 2015 7:18 AM, "Jamie Johnson" <jej2...@gmail.com> wrote: >> >>> > >> >>> > > I had seen this as well, if I over wrote this by extending >> >>> > > SolrIndexSearcher how do I have my extension used? I didn't see a >> way >> >>> > that >> >>> > > could be plugged in. >> >>> > > On Aug 25, 2015 7:15 AM, "Mikhail Khludnev" < >> >>> mkhlud...@griddynamics.com> >> >>> > > wrote: >> >>> > > >> >>> > >> On Tue, Aug 25, 2015 at 2:03 PM, Jamie Johnson <jej2...@gmail.com >> > >> >>> > wrote: >> >>> > >> >> >>> > >> > Thanks Mikhail. If I'm reading the SimpleFacets class >> correctly, >> >>> out >> >>> > >> > delegates to DocValuesFacets when facet method is FC, what used >> to >> >>> be >> >>> > >> > FieldCache I believe. DocValuesFacets either uses DocValues or >> >>> builds >> >>> > >> then >> >>> > >> > using the UninvertingReader. >> >>> > >> > >> >>> > >> >> >>> > >> Ah.. got it. Thanks for reminding this details.It seems like even >> >>> > >> docValues=true doesn't help with your custom implementation. >> >>> > >> >> >>> > >> >> >>> > >> > >> >>> > >> > I am not seeing a clean extension point to add a custom >> >>> > >> UninvertingReader >> >>> > >> > to Solr, would the only way be to copy the FacetComponent and >> >>> > >> SimpleFacets >> >>> > >> > and modify as needed? >> >>> > >> > >> >>> > >> Sadly, yes. There is no proper extension point. Also, consider >> >>> > overriding >> >>> > >> SolrIndexSearcher.wrapReader(SolrCore, DirectoryReader) where the >> >>> > >> particular UninvertingReader is created, there you can pass the >> own >> >>> one, >> >>> > >> which refers to custom FieldCache. >> >>> > >> >> >>> > >> >> >>> > >> > On Aug 25, 2015 12:42 AM, "Mikhail Khludnev" < >> >>> > >> mkhlud...@griddynamics.com> >> >>> > >> > wrote: >> >>> > >> > >> >>> > >> > > Hello Jamie, >> >>> > >> > > I don't understand how it could choose DocValuesFacets (it >> >>> occurs on >> >>> > >> > > docValues=true) field, but then switches to >> >>> > >> UninvertingReader/FieldCache >> >>> > >> > > which means docValues=false. If you can provide more details >> it >> >>> > would >> >>> > >> be >> >>> > >> > > great. >> >>> > >> > > Beside of that, I suppose you can only implement and inject >> your >> >>> own >> >>> > >> > > UninvertingReader, I don't think there is an extension point >> for >> >>> > this. >> >>> > >> > It's >> >>> > >> > > too specific requirement. >> >>> > >> > > >> >>> > >> > > On Tue, Aug 25, 2015 at 3:50 AM, Jamie Johnson < >> >>> jej2...@gmail.com> >> >>> > >> > wrote: >> >>> > >> > > >> >>> > >> > > > as mentioned in a previous email I have a need to provide >> >>> security >> >>> > >> > > controls >> >>> > >> > > > at the term level. I know that Lucene/Solr doesn't support >> >>> this >> >>> > so >> >>> > >> I >> >>> > >> > had >> >>> > >> > > > baked something onto a 4.x baseline that was sufficient for >> my >> >>> use >> >>> > >> > cases. >> >>> > >> > > > I am now looking to move that implementation to 5.x and am >> >>> running >> >>> > >> into >> >>> > >> > > an >> >>> > >> > > > issue around faceting. Previously we were able to provide a >> >>> > custom >> >>> > >> > cache >> >>> > >> > > > implementation that would create separate cache entries >> given a >> >>> > >> > > particular >> >>> > >> > > > set of security controls, but in Solr 5 some faceting is >> >>> delegated >> >>> > >> to >> >>> > >> > > > DocValuesFacets which delegates to UninvertingReader in my >> case >> >>> > (we >> >>> > >> are >> >>> > >> > > not >> >>> > >> > > > storing DocValues). The issue I am running into is that >> before >> >>> > 5.x >> >>> > >> I >> >>> > >> > had >> >>> > >> > > > the ability to influence the FieldCache that was used at the >> >>> Solr >> >>> > >> level >> >>> > >> > > to >> >>> > >> > > > also include a security token into the key so each cache >> entry >> >>> was >> >>> > >> > scoped >> >>> > >> > > > to a particular level. With the current implementation the >> >>> > >> FieldCache >> >>> > >> > > > seems to be an internal detail that I can't influence in >> >>> anyway. >> >>> > Is >> >>> > >> > this >> >>> > >> > > > correct? I had noticed this Jira ticket >> >>> > >> > > > https://issues.apache.org/jira/browse/LUCENE-5427, is there >> >>> any >> >>> > >> > movement >> >>> > >> > > > on >> >>> > >> > > > this? Is there another way to influence the information >> that >> >>> is >> >>> > put >> >>> > >> > into >> >>> > >> > > > these caches? As always thanks in advance for any >> suggestions. >> >>> > >> > > > >> >>> > >> > > > -Jamie >> >>> > >> > > > >> >>> > >> > > >> >>> > >> > > >> >>> > >> > > >> >>> > >> > > -- >> >>> > >> > > Sincerely yours >> >>> > >> > > Mikhail Khludnev >> >>> > >> > > Principal Engineer, >> >>> > >> > > Grid Dynamics >> >>> > >> > > >> >>> > >> > > <http://www.griddynamics.com> >> >>> > >> > > <mkhlud...@griddynamics.com> >> >>> > >> > > >> >>> > >> > >> >>> > >> >> >>> > >> >> >>> > >> >> >>> > >> -- >> >>> > >> Sincerely yours >> >>> > >> Mikhail Khludnev >> >>> > >> Principal Engineer, >> >>> > >> Grid Dynamics >> >>> > >> >> >>> > >> <http://www.griddynamics.com> >> >>> > >> <mkhlud...@griddynamics.com> >> >>> > >> >> >>> > > >> >>> > >> >>> >> >> >> >> >>