You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by "Ryan McKinley (JIRA)" <ji...@apache.org> on 2010/09/17 05:25:32 UTC

[jira] Created: (LUCENE-2649) FieldCache should include a BitSet for matching docs

FieldCache should include a BitSet for matching docs
----------------------------------------------------

                 Key: LUCENE-2649
                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
             Project: Lucene - Java
          Issue Type: Improvement
            Reporter: Ryan McKinley
             Fix For: 4.0


The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.

This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910654#action_12910654 ] 

Uwe Schindler commented on LUCENE-2649:
---------------------------------------

I am against the configuration option to enable the additional BitSet. The problem is that you cannot control it for each usage for the FieldCache, as it is a static flag. We agreed in the past that we will remove all static defaults from Lucene (e.g. BQ.maxClauseCount) together with sytem properties. This flag can cause strange problems with 3rd party code (like when you lower the BQ maxClauseCount and suddenly your queries fail).

The overhead by the OpenBitSet is very marginal (for integers only 1/32, as Yonik said). If you have memory problems with the FieldCache, these 1/32 would not hurt you, as you should think about your whole configuration then (liek moving from ints to shorts or something like that).

So: Please don't add any static defaults or sysprops! Please, please, please!

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914179#action_12914179 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

Uwe, I think we need to keep the native arrays.

bq. Java 6 will hopefully optimize away the additional method call

It only does if you have *one* implementation at the point where it is used.  We just got done specializing the ord sorting code with native arrays because of this - the speed hit was really non-trivial, and it happened with the latest versions of all of the JVMs I tested (oracle 1.6, oracle 1.7, ibm 1.6).  I see no relief for this issue on the horizon.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913646#action_12913646 ] 

Michael McCandless commented on LUCENE-2649:
--------------------------------------------


bq. I think this is a better option then adding a parameter to Parser since we can have an easy upgrade path. Parser is an interface, so we can not just add to it without breaking compatibility. To change things in 4.x, 3.x should have an upgrade path.

Hmm... I'd rather make an exception to 3.x, ie, allow the addition of
this method to the interface, than confuse the 4.x API, going forward,
with 2 classes?

Creating a custom FieldCache parser is an extremely advanced use
case... very few users do this, and those that do will grok this
method?

bq. However, I don't cache the Bits separately since this is an edge case that should be avoided, but at least does not fail if you are not consistent.

This makes me nervous since it can now lead to further cases of field
cache insanity, ie, you loaded it once w/o the valid bits, and again
w/ the valid bits, and now your values array is taking up 2X the RAM.

It's already bad enough that FC allows one kind of insanity :)

bq. This does cache a MatchAllBits even when 'cacheValidBits' is false, since that is small (a small class with one int)

Hmm... but if I pass false here, it shouldn't spend any time
allocating the bit set, building it, checking the bit set for "all
bits set", etc.?

{quote}
bq.     *  We don't have to @Deprecate for 4.0 - just remove it, and note this in MIGRATE.txt. (Though for 3.x we need the deprecation, so maybe do 3.x patch first, then remove deprecations for 4.0?).

My plan was to apply with deprecations to 4.x, then merge with 3.x.  Then replace the calls in 4.x, then remove the old functions.  Does this sound reasonable?
{quote}

OK that sounds like a good plan!

bq. Right, the ValidBits are only checked for docs that exists (and the FC values are only set for docs that exists -- this has not changed), and may contain false positives for deleted docs.  I think this is OK since most use cases (i can think of) deal with deletions anyway.   Any ideas how/if we should change this?

I think this is the right approach -- expecting FC's valid bits to
take deletions into account is too much.  We have IR.getDeletedDocs
for this.

But, eg this means classes like FCRF will still have to consult
deleted docs.

Really, "in general" we need a better way for the query execution path
to enforce deleted docs.  Eg if the FCRF will be AND'd w/ a query
that's already excluding del docs then it need not be careful about
deletions...

bq.  (I did not realize that the FC is reused after deletions -- so clever)

Ha!  There was a time when it didn't ;)


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914044#action_12914044 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

Now we're talking!

Q: why aren't the CachePopulator methods just directly on EntryConfig - was it easier to share implementations that way or something?

Also:
- It doesn't seem like we need two methods fillValidBits , fillByteValues - shouldn't it just be one method that looks at the config and fills in the appropriate entries based on cacheValidBits() and cacheValues()?
- We should allow an implementation to create subclasses of ByteValues, etc...  what about this method:
   public abstract CachedArray  fillEntry( CachedArray vals, IndexReader reader, String field, EntryConfig creator )
That way, an existing entry can be filled in (i.e. vals != null) or a new entry can be created.
Oh, wait, I see further down a "ByteValues createValue()" - if that's meant to be a method on CachePopulator, I guess it's all good - my main concern was being able to create subclasses of ByteValues and frields.

Anyway, all that's off the top of my head - I'm sure you've thought about it more at this point.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913703#action_12913703 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. I all cases this is easily possible by having a separate cache with separate population method for the bits.

This seems more complex, and less extensible.  What's the issue with just putting the bits reference down on CachedArray?

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12911157#action_12911157 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

Here is the code for ByteValues that:
# optionally stores the BitSet via static config
# does not cache a real BitSet unless only some docs match
# calculates numDocs/numTerms

{code:java}

    @Override
    protected ByteValues createValue(IndexReader reader, Entry entryKey) throws IOException {
      Entry entry = entryKey;
      String field = entry.field;
      ByteParser parser = (ByteParser) entry.custom;
      if (parser == null) {
        return wrapper.getByteValues(reader, field, FieldCache.DEFAULT_BYTE_PARSER);
      }
      int numDocs = 0;
      int numTerms = 0;
      int maxDoc = reader.maxDoc();
      final byte[] retArray = new byte[maxDoc];
      Bits valid = null;
      Terms terms = MultiFields.getTerms(reader, field);
      if (terms != null) {
        final TermsEnum termsEnum = terms.iterator();
        final Bits delDocs = MultiFields.getDeletedDocs(reader);
        final OpenBitSet validBits = new OpenBitSet( maxDoc );
        DocsEnum docs = null;
        try {
          while(true) {
            final BytesRef term = termsEnum.next();
            if (term == null) {
              break;
            }
            final byte termval = parser.parseByte(term);
            docs = termsEnum.docs(delDocs, docs);
            while (true) {
              final int docID = docs.nextDoc();
              if (docID == DocsEnum.NO_MORE_DOCS) {
                break;
              }
              retArray[docID] = termval;
              validBits.set( docID );
              numDocs++;
            }
            numTerms++;
          }
        } catch (StopFillCacheException stop) {}
        
        // If all non-deleted docs are valid we don't need the bitset in memory
        if( numDocs > 0 && CachedArray.CACHE_VALID_ARRAY_BITS ) {
          boolean matchesAllDocs = true;
          for( int i=0; i<maxDoc; i++ ) {
            if( !delDocs.get(i) && !validBits.get(i) ) {
              matchesAllDocs = false;
              break;
            }
          }
          if( matchesAllDocs ) {
            valid = new Bits.MatchAllBits( maxDoc );
          }
          else {
            valid = validBits;
          }
        }
      }
      if( numDocs < 1 ) {
        valid = new Bits.MatchNoBits( maxDoc );
      }
      return new ByteValues( retArray, valid, numDocs, numTerms );
    }
{code}

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Updated: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Robert Muir (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Robert Muir updated LUCENE-2649:
--------------------------------

    Component/s: Search

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Search
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914169#action_12914169 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

Hmmm, that would also seem to transform the FieldCache into a more generic index reader cache - not a bad idea!

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914973#action_12914973 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq.  It's just that perhaps we should "upgrade" the entry, if it was first created w/false and then the current call passes true?

We just need to watch the thread safety of stuff like this.
For Bits, it should be trivial... if you didn't ask for bits, you shouldn't be looking at it.
fasterButMoreRAM is different... upgrading an existing entry could be tricky.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Search
>            Reporter: Ryan McKinley
>            Assignee: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914082#action_12914082 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

bq. Q: why aren't the CachePopulator methods just directly on EntryConfig - was it easier to share implementations that way or something?

Two reasons (but I can be talked out of it)
1. this approach separates what you are asking for (bits/values/etc) from how they are actually generated (the "populator").  Something makes me uncomfortable about the caller asking for Values needing to also know how they are generated.  Seems easy to mess up.  With this approach the 'populator' is attached to the field cache, and defines how stuff is read, vs the 'EntryConfig' that defines what the user is asking for (particulary since they may change what they are asking for in subsequent calls)

2. The 'populator' is attached to the FieldCache so it has consistent behavior across subsequet calls to getXxxxValues().  Note that with this approach, if you ask the field cache for just the 'values' then later want the 'bits' it uses the same populator and adds the results to the existing CachedArray value.


bq. It doesn't seem like we need two methods fillValidBits , fillByteValues

The 'fillValidBits' just fills up the valid bits w/o actually parsing (or caching) the values.  This is useful when:
1. you only want the ValidBits, but not the values (Mike seems to want this)
2. you first ask for just values, then later want the bits.  

Thinking some more, I think the populator should look like this:
{code:java}
public abstract class CachePopulator 
  {
    public abstract ByteValues   createByteValues(   IndexReader reader, String field, EntryConfig config ) throws IOException;
    public abstract ShortValues  createShortValues(  IndexReader reader, String field, EntryConfig config ) throws IOException;
    public abstract IntValues    createIntValues(    IndexReader reader, String field, EntryConfig config ) throws IOException;
    public abstract FloatValues  createFloatValues(  IndexReader reader, String field, EntryConfig config ) throws IOException;
    public abstract DoubleValues createDoubleValues( IndexReader reader, String field, EntryConfig config ) throws IOException;
    
    public abstract void fillByteValues(   ByteValues   vals, IndexReader reader, String field, EntryConfig config ) throws IOException;
    public abstract void fillShortValues(  ShortValues  vals, IndexReader reader, String field, EntryConfig config ) throws IOException;
    public abstract void fillIntValues(    IntValues    vals, IndexReader reader, String field, EntryConfig config ) throws IOException;
    public abstract void fillFloatValues(  FloatValues  vals, IndexReader reader, String field, EntryConfig config ) throws IOException;
    public abstract void fillDoubleValues( DoubleValues vals, IndexReader reader, String field, EntryConfig config ) throws IOException;

    // This will only fill in the ValidBits w/o parsing any actual values
    public abstract void fillValidBits( CachedArray  vals, IndexReader reader, String field, EntryConfig config ) throws IOException;
  }
{code}

The default 'create' implementation could look something like this:

{code:java}

    @Override
    public ShortValues createShortValues( IndexReader reader, String field, EntryConfig config ) throws IOException 
    {
      if( config == null ) {
        config = new SimpleEntryConfig();
      }
      ShortValues vals = new ShortValues();
      if( config.cacheValues() ) {
        this.fillShortValues(vals, reader, field, config);
      }
      else if( config.cacheValidBits() ) {
        this.fillValidBits(vals, reader, field, config);
      }
      else {
        throw new RuntimeException( "the config must cache values and/or bits" );
      }
      return vals;
    }
{code}

And the Cache 'createValue' would looks somethign like this:
{code:java}

  static final class ByteCache extends Cache {
    ByteCache(FieldCache wrapper) {
      super(wrapper);
    }
    
    @Override
    protected final ByteValues createValue(IndexReader reader, Entry entry, CachePopulator populator) throws IOException {
      String field = entry.field;
      EntryConfig config = (EntryConfig)entry.custom;
      if (config == null) {
        return wrapper.getByteValues(reader, field, new SimpleEntryConfig() );
      }
      return populator.createByteValues(reader, field, config);
    }
  }
{code}

thoughts?  This would open up lots more of the field cache... so if we go this route, lets make sure it addresses the other issues people have with FieldCache.  IIUC, the other big request is to load the values from an external source -- that should be possible with this approach.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913353#action_12913353 ] 

Uwe Schindler commented on LUCENE-2649:
---------------------------------------

Also set the Lucene default to true, as I want to improve sorting and FCRF.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914178#action_12914178 ] 

Uwe Schindler commented on LUCENE-2649:
---------------------------------------

bq. Anyone know what the deal is with IndexReader:

It is to share the cache for clones of IndexReaders or when SegmentReaders are reopended with different deleted docs. In this case the underlying Reader is the same, so it should use its cache (e.g. when deleted docs are added, you dont need to invalidate the cache).

For more info, ask Mike McCandless!

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Mark Miller (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910660#action_12910660 ] 

Mark Miller commented on LUCENE-2649:
-------------------------------------

bq. I was suggesting handling it the same way as FieldCache.Parser - it's set on the SortField. But instead of just being able to control parsing of a term (which is too limited), it needs to be able to control everything. (This would solve Shai's needs too)

We started down this path with LUCENE-831 - you could pass some *UnInverter on the sort field if i remember right, so that pretty much everything could be overridden. It has come up a lot - we really need this level of customizability eventually.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913699#action_12913699 ] 

Uwe Schindler commented on LUCENE-2649:
---------------------------------------

I think it should be a separate entry in the cache. Only that its only regenerated, if it does not already exist for the field/IR combination. So there are these combinations:

- cache is purged (inititially)
- user calls getBytes(field) -> bytes[] is filled, also the Bits
- user calls additionally getInts(field) for some reason -> int[] is filled, but as Bits already exit they dont need to be filled
- user calls getBits(field) -> returns the pre-filled bits from one of the previous calls

Alternatively:

- Cache is purged (initially)
- User calls getBits(field), there are no bits available until now, bits are populated from TermEnum/TermDocs, but no byte[] or int[], as not requested

With a config option, one could switch automatic polulation of Bits off, so in the first combination, only the last call to getBits() would populate bit set.

I all cases this is easily possible by having a separate cache with separate population method for the bits. If some method like getBytes() also populates the Bits, it should simply add the created Bits manually to the cache.

Does this sound like a plan?

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Updated: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ryan McKinley updated LUCENE-2649:
----------------------------------

    Attachment: LUCENE-2649-FieldCacheWithBitSet.patch

Here is a new patch that incorporates most of the ideas we have discussed.  I tried addressing the larger issue of the FieldCache mess in LUCENE-2665, but that is too big to tackle in one go.  After getting something to *almost* work for LUCENE-2665, I then just took the EntryCreator stuff and am adding that to 'LUCENE-2649'

As such, some of the choices about how EntryCreator works are based on future plans, and my feel akward today.  Speciffically:

1. In the future, a Cache on the IndexReader should not necessarily be tied to a field name.  To do this, the field name parameter should be part of the EntryCreator.  In this first pass, we will need to pass the field name twice:
{code:java}
IntValues  vals = cache.getInts(reader, "fieldName", new IntValuesCreator( "fieldName", parser, flags ) )
{code}
I think the tradeoff is OK, and makes fewer changes in the future.

2. In the future, the EntryCreator.getCacheKey() should be the only key stored.  To fall within the existing structure, the entire EntryCreator is stored on the 'custom' field on the internal cache, but the equals and hashCode values are bubbled up.  This makes more sense for LUCENE-2665.  For now we need to be careful that the EntryCreator classes are reasonable things to store as Keys (it includes the Parser, etc)

-----------------------------

I added a bunch of tests to exercise how sequential calls with different options behave.

I think this patch is ready to commit to /trunk -- when it is in, I'll make a patch for LUCENE-2665.

Since /trunk is now a bit more experimental, and I feel pretty good about the feedback this has had I will probably jump the gun and commit soon

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Search
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910461#action_12910461 ] 

Uwe Schindler commented on LUCENE-2649:
---------------------------------------

That looks exactly like I proposed it!

The only thing: For DocTerms the approach is not needed? You can check for null, so the Bits interface is not needed. As the OpenBitSets are created with the exact size and don't need to grow, you can use fastSet to speed up creation by doing no bounds checks.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914095#action_12914095 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

{quote}
I think getCachePopulator() should be under EntryConfig - that way people can provide their own (and extend ByteValues to include more info)
{quote}

So you think it is better for *each* call to define how the cache works rather then having that as an attribute of the FieldCache (that could be extended).  The on thing that concerns me is that that forces all users of the FieldCache to be in sync.

In this proposal, you could set the CachePopulator on the FieldCache. 

{quote}
Otherwise, we'll forever be locked into a lowest common denominator of only adding info that everyone can agree on.
{quote}

This is why I just added the 'createXxxxValues' functions on CachePopulator -- a subclass could add other values.

------------

It looks like the basic difference between what we are thinking is that the Populator is attached to the FieldCache rather then each call to the FieldCache.  From my point of view, this would make it easier for system with a schema (like solr) have consistent results across all calls, rather then making each request to the FieldCache need to know about the schema -> parsers -> populator

but I can always be convinced ;)

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913347#action_12913347 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

Any thoughts on this?

I think the best move forward is to:
a. optimize as much as possible
b. drop the no-parser function option
c. optionally store the bitset via static config (ugly, but lesser of many ugly options)
d. set lucene default=false (actually I don't care)
e. set solr default=true

Unless there are objections, I will clean up the patch, fix javadoc, tests, etc

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913672#action_12913672 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

bq. Hmm... I'd rather make an exception to 3.x, ie, allow the addition of this method to the interface, than confuse the 4.x API, going forward, with 2 classes?

That is OK with me.  Would be cleaner and simpler.  (though semantically it does not make sense to me -- why ask the parser what to cache?)


bq. >>    This does cache a MatchAllBits even when 'cacheValidBits' is false, since that is small (a small class with one int)
bq. Hmm... but if I pass false here, it shouldn't spend any time allocating the bit set, building it, checking the bit set for "all bits set", etc.?

Well it does not try *hard*, only if numDocs==maxDocs, it does not look at anything.  If the cost of caching new MatchAllBits( maxDocs ) isn't worth occasional win by knowing all the values are valid, then I will remove it.


bq. So how about we add a getBits(field)

Interesting... i'll mess for a bit and let you know what I think :)

rather then throwing an exception, that might be a flag, since I could imagin many thigns would use the Bits if they exist and do something else if they dont






> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914960#action_12914960 ] 

Michael McCandless commented on LUCENE-2649:
--------------------------------------------

I think we have a synchronization issue on the call to validate?  Ie, more than 1 thread can enter validate, and eg compute the valid bits (if they weren't computed on the first create())?

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Search
>            Reporter: Ryan McKinley
>            Assignee: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914989#action_12914989 ] 

Michael McCandless commented on LUCENE-2649:
--------------------------------------------

{quote}
We just need to watch the thread safety of stuff like this.
For Bits, it should be trivial... if you didn't ask for bits, you shouldn't be looking at it.
fasterButMoreRAM is different... upgrading an existing entry could be tricky.
{quote}

And, also, we need to decide the policy.  Ie I think it's best if whoever gets there first, wins, for this case.  Ie we should not in fact upgrade to fasterButMoreRAM if the existing entry isn't...

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Search
>            Reporter: Ryan McKinley
>            Assignee: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914175#action_12914175 ] 

Uwe Schindler commented on LUCENE-2649:
---------------------------------------

+1

Maybe we should also look at CSF again and what Simon did (LUCENE-2186). In my opinion, the field cache's public API should not look different from CSF, so one can simply also sort against a CSF.

I know, some people here will hurt me if I suggest to remove tha native arrays and instead provide getter methods like in the ValueSource approach. The native arrays are unflexible and Java 6 will hopefully optimize away the additional method call (at least I have seen no speed penalty when trying with CSF's getter API). Cool things could be done like materializing the FieldCache to disk using mmap by e.g. FileChannel.map(...).order(ByteOrder.BIG_ENDIAN).asFloatBuffer() which is then accessible using get(docId). I tested this and works very fine for sorting in Lucene! Java uses internally source code specialization to return different classes depending on native byte order that access the underlying ByteBuffer directly (not manually combining 4 bytes into a float). So the get(docId) call is only bounds checks and *one* mmaped memory access.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12915072#action_12915072 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. if you pass a different parser for the same field, it will give you an error.

If that were true, it would have broken solr in multiple places.  It may not be ideal - but it's not an error.
So I'm a little confused - why does solr still work fine?

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Search
>            Reporter: Ryan McKinley
>            Assignee: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910598#action_12910598 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. Whatever you call it (using Factory is fine) but isn't that what you mentioned to be insufficient? I mean this is something you would pass to a FieldCache.getXXX, right? 

I was suggesting handling it the same way as FieldCache.Parser - it's set on the SortField.  But instead of just being able to control parsing of a term (which is too limited), it needs to be able to control everything. (This would solve Shai's needs too)

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Updated: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ryan McKinley updated LUCENE-2649:
----------------------------------

    Attachment: LUCENE-2649-FieldCacheWithBitSet.patch

A slightly simplified version

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12915039#action_12915039 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

Thanks Mike, I'll take a look and fix the small issues later tonight.

bq. I think we should hold off on backport to 3.x until we stabilize LUCENE-2665?

+1, I think it makes sense to backport to 3.x when there is a clear upgrade path.  



bq. I think we have a synchronization issue on the call to validate?

Looks like it.  There are two approaches we could take.  

1. synchronize from the cache:
{code:java}
      if( key.creator.shouldValidate() ) {
        synchronized( key ) {
          key.creator.validate( (T)value, reader);
        }
      }
{code}

2. make each creator responcible for validation.  For example, the DoubleValuesEntryCreator would look like:
{code:java}

  @Override
  public DoubleValues validate(DoubleValues entry, IndexReader reader) throws IOException {
    boolean ok = false;
    if( hasOption(OPTION_CACHE_VALUES) ) {
      ok = true;
      if( entry.values == null ) {
        synchronized( this ) {
          if( entry.values == null ) {
            fillDoubleValues(entry, reader, field);
          }
        }
      }
    }
    if( hasOption(OPTION_CACHE_BITS) ) {
      ok = true;
      if( entry.valid == null ) {
        synchronized( this ) {
          if( entry.valid == null ) {
            fillValidBits(entry, reader, field);
          }
        }
      }
    }
    if( !ok ) {
      throw new RuntimeException( "the config must cache values and/or bits" );
    }
    return entry;
  }
{code}
That is a bit more complicated, but avoids synchonization when things are valid.  Thoughts?


bq. I think it's best if whoever gets there first, wins, for this case

Yes, this is the current behavior -- the validate method does not do anything:
{code:java}
  public T validate(T entry, IndexReader reader) throws IOException {
    // TODO? nothing? perhaps subsequent call with FASTER_BUT_MORE_RAM?
    return entry;
  }
{code}


- - - - - - - -

The other key behavior that we should note is that with CachedArrayEntryCreators, if you pass a different parser for the same field, it will give you an error.  Previously this created two cache entries (and then added something to the insanity log).  If someone wants to do that, they could override the XxxEntryCreator.getCacheKey() to return key that includes the Parser.  By default this seems like an error to me.





> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Search
>            Reporter: Ryan McKinley
>            Assignee: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910663#action_12910663 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

I'm all for dropping the static flag and always calculating the valid bits -- it makes things accurate with minimal cost.  

I am sympathetic to folks who don't want this, and I'm not sure the cleanest way to support both options, or even if it is actually worthwhile.

Do people see this 'option' as a showstopper?  If so, is there an easy way to configure?  without statics, the flag would need to be fetched from each parser, and the parser does not know what FieldCache it is used from (using FieldCache.DEFAULT is just as bad as the static flag IIUC)



> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913701#action_12913701 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq.  If we have a separate getBits( field ) call, should the Bits be added to the XxxValues class? I suspect not.

Seems a lot easier just to put all the optional stuff on the *Values class - more performant too (and avoid more synchronizing lookups)


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Updated: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ryan McKinley updated LUCENE-2649:
----------------------------------

    Attachment: LUCENE-2649-FieldCacheWithBitSet.patch

This patch replaces the cached primitive[] with a CachedObject.  The object heiarch looks like this:

{code:java}

    public abstract static class CachedObject { 
    
  }

  public abstract static class CachedArray extends CachedObject {
    public final Bits valid;
    public CachedArray( Bits valid ) {
      this.valid = valid;
    }
  };

  public static final class ByteValues extends CachedArray {
    public final byte[] values;
    public ByteValues( byte[] values, Bits valid ) {
      super( valid );
      this.values = values;
    }
  };
  ...
{code}

Then this @deprecates the getBytes() classes and replaces them with getByteValues()

{code:java}

  public ByteValues getByteValues(IndexReader reader, String field)
  throws IOException;

  public ByteValues getByteValues(IndexReader reader, String field, ByteParser parser)
  throws IOException;
  
{code}

then repeat for all the other types!

All tests pass with this patch, but i have not added any tests for the BitSet (yet)

If people like the general look of this approach, I will clean it up and add some tests, javadoc cleanup etc


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910645#action_12910645 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. oh right - thats true. Is a global flag sufficient? 

Yeah, solr could just always default it to on.  We don't know what kind of ad-hoc queries people will throw at solr and the 3% size increase (general case 1/32) seems completely worth it.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Simon Willnauer (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910539#action_12910539 ] 

Simon Willnauer commented on LUCENE-2649:
-----------------------------------------

bq. Should we make it optional, whether the valid bitset should be computed? Many apps wouldn't need it, so it just ties up (admittedly smallish amounts of) RAM unnecessarily?
+1 we can save that overhead and high level apps can enable it by default if needed.



> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910628#action_12910628 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

Uwe: "For DocTerms the approach is not needed..."

Ya I realized this after looking at the patch I first submitted.  In the first patch, the cache holds a CachedObject rather then just an Object.  In the second, I changed back to just an Object so it does not need to wrap the DocTerms or DocTermsIndex

For the RangeFilter, with optional Bits calculation, that could would look somethign like:
{code:java}

        LongValues cached = FieldCache.DEFAULT.getLongValues(reader, field, (FieldCache.LongParser) parser);
        final long[] values = cached.values;
        if( cached.valid == null ) {
          // ignore deleted docs if range doesn't contain 0
          return new FieldCacheDocIdSet(reader, !(inclusiveLowerPoint <= 0L && inclusiveUpperPoint >= 0L)) {
            @Override
            boolean matchDoc(int doc) {
              return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
            }
          };
        }
        else {
          final Bits valid = cached.valid;
          return new FieldCacheDocIdSet(reader, true) {
            @Override
            boolean matchDoc(int doc) {
              return valid.get(doc) && values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
            }
          };
        }
{code}

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Issue Comment Edited: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913712#action_12913712 ] 

Yonik Seeley edited comment on LUCENE-2649 at 9/22/10 2:46 PM:
---------------------------------------------------------------

bq. But we could probably still achieve these two benefits while using a single class for looking up everything "cached" about a field? Ie, the CachedArray could return non-null bits but null values?

Exactly.
And with NRT and increasing number of segments, the number of synchronized lookups per segment could really start to matter.

      was (Author: yseeley@gmail.com):
    bq. But we could probably still achieve these two benefits while using a single class for looking up everything "cached" about a field? Ie, the CachedArray could return non-null bits but null values?

Exactly.
And with NRT and increasing number of segments, the number of synchronized lookups per second could really start to matter.
  
> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914194#action_12914194 ] 

Uwe Schindler commented on LUCENE-2649:
---------------------------------------

I just wanted to mention that, so the design of the new FC is more flexible in that case. I am just pissed of because of these arrays and no flexibility :-(

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913685#action_12913685 ] 

Michael McCandless commented on LUCENE-2649:
--------------------------------------------

bq. That is OK with me. Would be cleaner and simpler. (though semantically it does not make sense to me - why ask the parser what to cache?)

Yeah that is weird.  Maybe we rename Parser -> EntryCreator?


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913633#action_12913633 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

If folks think that being able to tell a real "0" from a missing value is not useful for Lucene, we
could extend Ryan's CacheConfig to include a factory method that creates / populates ByteValues, IntValues, etc.
Then all the bitset stuff could be kept in Solr only.  I'm sensitive about pushing stuff into Lucene that is *only* useful for Solr.



> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910803#action_12910803 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

Maybe, but I'm still not sure this cleans things up enough to be worth the trouble -- ideally the API should be easy to have consistent results.  I don't like that it would be too easy to mess things up if you the application does not use the same parser from various components (that may be in different libraries etc).  Conceptually it makes sense to have settings about what is or is not cached attached to the FieldCache itself, not to the things that ask the FieldCache for its values -- and letting whoever asks first set the behavior for the next guy who asks (regardless of what they ask for!).  

If we are going to make it essentially required to always pass in the right Parser/EntryCreator, we should at least remove all the ways of not passing one in -- since that call is saying "use what ever is there, and the next guy who asks should be ok with it too"

Does something like the EntryCreator idea fix -- or at least begin to fix -- the other FieldCache issues?  If not, is it really worth introducing just to avoid a static variable?

I think the best near term option is live with the static initializer, and fix it when the we rework the FieldCache to fix a host of other issues.  For solr the default will be set to always calculate, for lucene... we will let Mike and Uwe duke it out :)









> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12915084#action_12915084 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. Ok, that makes sense - what I mean is that you can not ask for the same type with different parsers.

Yep, seems fine since people can work around it if they need to.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Search
>            Reporter: Ryan McKinley
>            Assignee: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Updated: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ryan McKinley updated LUCENE-2649:
----------------------------------

    Attachment: LUCENE-2649-FieldCacheWithBitSet.patch

I added a static flag to CachedArray:
{code:java}
  public abstract static class CachedArray {
    public static boolean CACHE_VALID_ARRAY_BITS = false;
    
    public final Bits valid;
    public CachedArray( Bits valid ) {
      this.valid = valid;
    }
  };
{code}
and then set it to true in the SolrCore static initalizer.

If folks are ok with this approach, I'll clean up the javadocs etc

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914123#action_12914123 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. In solr, is there one FieldCache for all all cores, or does each core get its own FieldCache? 

There is a single FieldCache for all cores (same as in Lucene).

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910464#action_12910464 ] 

Uwe Schindler commented on LUCENE-2649:
---------------------------------------

When this is committed, we can improve also some Lucene parts: FieldCacheRangeFilter does not need to do extra deletion checks and instead use the Bits interface to find missing/non-valued documents. Lucene's sorting Collectors can be improved to have a consistent behaviour for missing values (like Solr's sortMissingFirst/Last).

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Issue Comment Edited: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910707#action_12910707 ] 

Uwe Schindler edited comment on LUCENE-2649 at 9/17/10 2:11 PM:
----------------------------------------------------------------

I am against that option! No static defaults! (and if it *must* be there - default it to true on Lucene, too).

bq. the number of values set 

This is OpenBitSet.cardinality() ? I dont think we should add this extra cost during creation, as it can be retrieved quite easy if really needed.

      was (Author: thetaphi):
    I am against that option! No static defaults!

bq. the number of values set 

This is OpenBitSet.cardinality() ? I dont think we should add this extra cost during creation, as it can be retrieved quite easy if really needed.
  
> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914177#action_12914177 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

bq. Hmmm, that would also seem to transform the FieldCache into a more generic index reader cache - not a bad idea! 

But is it a good one?!  This would let the FieldCache just focus on the synchronized cache mechenism, and the each EntryCreator would need to do its own Parsing etc


Anyone know what the deal is with IndexReader:
{code:java}
  /** Expert */
  public Object getCoreCacheKey() {
    return this;
  }
{code}



> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12915106#action_12915106 ] 

Uwe Schindler commented on LUCENE-2649:
---------------------------------------

bq. What you can not do is as for a IntValues using: FieldCache.DEFAULT_INT_PARSER, then switch to FieldCache.NUMERIC_UTILS_INT_PARSER for the same field.

Just to note, my previous FieldCache impl did this correctly. When you requested a FieldCache without parser it created the parser=null entry and additionally the real used parser as two separate entries. When you later use directly (instead of automatics) passing in the trie parser, it returns the additionally created entry in the cache. If you request using another parser, it creates a new entry. So i was (theoretically) possible to parse the same field as int or long and additionally as trie or plain number (but the latter throwed of couse some NFE), so not really useful.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Search
>            Reporter: Ryan McKinley
>            Assignee: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914115#action_12914115 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

preface, I don't really know how FieldCache is used, so my assumptions could be way off...

In solr, is there one FieldCache for all all cores, or does each core get its own FieldCache?  

I figured each core would create a single CachePopulator (with a reference to the schema) and attach it to the FieldCache.  If that is not possible, then ya, it will be better to put that in the request.

bq. Are EntryConfig objects stored as keys anywhere? We need to be very careful about memory leaks.

Yes, the EntryConfig is part of the 'Entry' and gets stored as a key.  


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910700#action_12910700 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

Are people generally ok with the idea of global on/off?  I think that is a reasonable approach... I agree that we should avoid static fields to control behavior.  But do we avoid it at the cost of not allowing the option, or waiting till we rework FieldCache?

If the consensus is that FieldCache needs to be reworked *before* somethign like this could be added, that's fine... i'll move on to other things.  Any relatively easy suggestions for how to enable the option without a global static?  (Note that FieldCache is already a global static -- at leaset FieldCache.DEFAULT is referenced a lot)

Perhaps this could/should live in /trunk until a cleaner solution is viable?





> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910717#action_12910717 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

If ya care - don't pass a null parser!  Otherwise you get the default.

bq. This is OpenBitSet.cardinality()

Which isn't free... and calculating it over and over again is silly if you care about those numbers.

bq. I dont think we should add this extra cost during creation,

I don't think it will add extra cost.  I could be wrong, but I don't think it will be measurable.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Robert Muir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913503#action_12913503 ] 

Robert Muir commented on LUCENE-2649:
-------------------------------------

bq. Also set the Lucene default to true, as I want to improve sorting and FCRF.

bq. I know it's only 3% (for ints... 12.5% for bytes), but, 3% here, 3% there and suddenly we're talking real money.

I'm having trouble understanding the use case for this bitset.

The jira issue says to add a bitset, but doesnt explain why.

The linked thread talks about this being useful for sorting missing values last, but I don't think this justifies
increasing the size of fieldcache by default.


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12911128#action_12911128 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

I thought of  an optimization that could reduce memory usage...  

If all non-deleted documents have a value, we don't need a real BitSet -- just a Bits implementation that always returns true.

That should save 3% (or 12.5%) here and there.

- - - - - -

On other thing to consider... do we want to remove the getXXXX functions that do not pass in a Parser?  passing in null, is equivalent?

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910599#action_12910599 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. open up FieldCache to allow the application to populate the entries from other sources

+1

bq. specifically payloads

If CSF did not exist, I'd be totally on board with this...  but it looks to be right around the corner now.  Are there any advantages to using payloads over CSF for fieldcache population?

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913485#action_12913485 ] 

Michael McCandless commented on LUCENE-2649:
--------------------------------------------

bq. Also set the Lucene default to true

Please don't!

bq. as I want to improve sorting and FCRF.

But: sorting, FCRF must continue to work if the app chooses not to
load valid bits, right?

Other feedback on current patch:

  * We don't have to @Deprecate for 4.0 -- just remove it, and note
    this in MIGRATE.txt.  (Though for 3.x we need the deprecation, so
    maybe do 3.x patch first, then remove deprecations for 4.0?).

  * FieldCache.EntryCreator looks orphan'd?

It looks like the valid bits will not reflect deletions (by design),
right?  Ie caller cannot rely on valid always incorporating deleted
docs.  (Eg the MatchAll opto disregards deletions, and, a reopened
segment can have new deletions yet shares the FC entry).

The static config still also bothers me... and, going that route means
we must agree on a default (which is looking hard!).

What if we:

  * Allow specifying "loadValidBits" on each load (eg via Parser or
    separate arg to FC.getXXValues), but,

  * We separately cache the valid bits (we'd still have the XXXValues
    returned, to include the valid bits & values).

This way if an app "messes up", they do not end up double-storing the
actual values, ie the worst that happens is they have to re-invert
just to generate the valid bits.  Even that should be fairly rare, ie,
if they use MissingStringLastComparator it'll init both values & valid
bits entries in the cache on the first go.


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910805#action_12910805 ] 

Uwe Schindler commented on LUCENE-2649:
---------------------------------------

Supporting different parsers is not an issue at all. You can call getBytes() with different parsers, you simply create two entries in the cache, as each parser produces a different cache instance. And getBytes() without parser is also fine, as then you get the default parser from the cache (which would not create a third instance!). - [Parser is part of the cache key]

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Marvin Humphrey (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910668#action_12910668 ] 

Marvin Humphrey commented on LUCENE-2649:
-----------------------------------------

> So: Please don't add any static defaults or sysprops! Please, please, please!

+1

No global variables which control behavior, please.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910635#action_12910635 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. public void setCacheValidBitsForFields( Set<String> names );

Solr doesn't even know all of the fields at the time it reads it's schema.  And even if it did... this would seem to break multi-core or anything trying to have more than one index where the fields are different.  Seems like this needs to be passed down via SortField, just like FieldCache.Parser.  A factory makes this a more generic method than adding additional params to SortField every time we think of something like this... then we can add stuff like getFieldCacheParser() and other stuff to the factory.


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Assigned: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ryan McKinley reassigned LUCENE-2649:
-------------------------------------

    Assignee: Ryan McKinley

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Search
>            Reporter: Ryan McKinley
>            Assignee: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910699#action_12910699 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. Let's try to find some baby-step (even if not clean - we know FieldCache, somehow, needs to be fixed more generally) for today?

The cheapest option might be:

{quote}
  public interface Parser extends Serializable {
    public boolean recordMissing();
  }
{quote}

A better option is to replace FieldCache.Parser in SortField to be FieldCache.EntryCreator.

Oh, and if we're recording all the set bits, it would be really nice to record both
- the number of values set
- the number of unique values encountered

Both should be zero or non-measurable cost (a counter++ that does not produce a data dependency can be executed in parallel on a free int execution unit)


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "J.J. Larrea (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913740#action_12913740 ] 

J.J. Larrea commented on LUCENE-2649:
-------------------------------------

I only just waded through this thread, so apologies in advance if this is redundant or off-topic...

It seems to me that there could and should be a standalone enhancement to FieldCache/FCImpl to support Boolean-valued fields. 

Since there is no native array-of-bits in Java, it could have the signature:

    BitSet getBits(IndexReader reader, String field, BooleanParser parser)  [implementation returning an OpenBitSet for efficiency]

A pre-supplied BooleanParser implementation StringMatchBooleanParser could map any of one of a set of uncased strings to true, and a default subclass eg. DefaultStringMatchBooleanParser could supply { "T", "TRUE", "1", "Y", "YES" } for the set of strings.  So the defaulted and typical case getBits( ir, "field" ) would do what one typically expects of boolean-valued fields.

With that in place, then couldn't one simply define a parser that indicates value present for a docID regardless of what the term value is:

    public static BooleanParser AlwaysReturnTrueBooleanParser = new BooleanParser() { public boolean parseByte(BytesRef term) { return true; } }

    BitSet getValueExists(IndexReader reader, String field) {
       return  getBits( ir, field, AlwaysReturnTrueBooleanParser );
    }
 
Then a client (e.g. FieldComparator implementation) interested in ValueExists values could ask for them, and they would be independently cached from whatever other field type cache(s) were requested on that field by the same or different clients.  The only cost would be iterating the Term/docID iterators a second time (as for additional cache variants on the same field) - minor.

Does this make sense?

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914109#action_12914109 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

It's all doable though I guess - even if EntryConfig objects are used as cache keys, we could store a weak reference to the solr core.
So I say, proceed with what you think will make it easy for Lucene users - and don't focus on what will be easy for Solr.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Issue Comment Edited: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910699#action_12910699 ] 

Yonik Seeley edited comment on LUCENE-2649 at 9/17/10 2:01 PM:
---------------------------------------------------------------

bq. Let's try to find some baby-step (even if not clean - we know FieldCache, somehow, needs to be fixed more generally) for today?

The cheapest option might be:

{code}
  public interface Parser extends Serializable {
    public boolean recordMissing();
  }
{code}

A better option is to replace FieldCache.Parser in SortField to be FieldCache.EntryCreator.

Oh, and if we're recording all the set bits, it would be really nice to record both
- the number of values set
- the number of unique values encountered

Both should be zero or non-measurable cost (a counter++ that does not produce a data dependency can be executed in parallel on a free int execution unit)


      was (Author: yseeley@gmail.com):
    bq. Let's try to find some baby-step (even if not clean - we know FieldCache, somehow, needs to be fixed more generally) for today?

The cheapest option might be:

{quote}
  public interface Parser extends Serializable {
    public boolean recordMissing();
  }
{quote}

A better option is to replace FieldCache.Parser in SortField to be FieldCache.EntryCreator.

Oh, and if we're recording all the set bits, it would be really nice to record both
- the number of values set
- the number of unique values encountered

Both should be zero or non-measurable cost (a counter++ that does not produce a data dependency can be executed in parallel on a free int execution unit)

  
> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914184#action_12914184 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

bq. remove tha native arrays and instead provide getter methods like in the ValueSource approach

What would this look like?  Are you suggesting rather then having:
{code:java}
class ByteValues {
  public byte[] values;
};
{code}

we have:
{code:java}
class ByteValues {
  public byte getValue( int doc )
};
{code}

or are you suggesting like DocValues that has, intVal, longVal, floatVal, even though only one *really* makes sense?

Is this something that wold need the proposed FieldCache API to change?  or could it be implemented via EntryCreator?

If we like the more general cache, that probably needs its own issue (wow scope creep!)









> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "J.J. Larrea (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913795#action_12913795 ] 

J.J. Larrea commented on LUCENE-2649:
-------------------------------------

{quote}
Actually, that's major - otherwise I would have already just done that and stored it in a separate cache for Solr's needs.
{quote}

Is the one-time-per-IndexReader-lifecycle cost of multiplying the cache load time by some factor < 2.0 (since the term values don't need to be decoded), really so terrible that one has to contemplate global state variables, or a constant increase in cache memory, or significant API changes, or the potential for double-allocation (with then an additional 1x cache load time), or increased code complexity, ...?  Even with all the lovely Solr support for parallel pre-warming?

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910554#action_12910554 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. Should we make it optional, whether the valid bitset should be computed?

The trick is how to implement that (unless you mean just set it to true/false for all fields at once).  Putting a flag on the FieldCache.getXXX methods is insufficient.
Only the application knows if some of it's future uses of that field will require the bitset for matching docs, but it's Lucene that's often making the calls to the field cache.

Perhaps FieldCache.Parser was originally just too narrow in scope - it should have been a factory method for handling all decisions about creating and populating a field cache entry?

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910711#action_12910711 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

I like the idea of optionally caching the numdocs and unique values -- that would make sorting by this field faster -- the ArrayValues class could be easily augmented with this.

The problem with augmenting the Parser class as you suggest is that we would have to rejiggy everything that touches parser.  We would need different default classes for things that want or don't want the missing records.  How do we handle this big:
{code:java}
if (parser == null) {
        try {
          return wrapper.getIntValues(reader, field, DEFAULT_INT_PARSER);
        } catch (NumberFormatException ne) {
          return wrapper.getIntValues(reader, field, NUMERIC_UTILS_INT_PARSER);      
        }
      }
{code}
yuck


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Issue Comment Edited: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Simon Willnauer (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910568#action_12910568 ] 

Simon Willnauer edited comment on LUCENE-2649 at 9/17/10 9:59 AM:
------------------------------------------------------------------

bq. Perhaps FieldCache.Parser was originally just too narrow in scope - it should have been a factory method for handling all decisions about creating and populating a field cache entry?
I guess we need to be able to manually configure FieldCache with some kind of FieldType. There have been several issues mentioning this and it keeps coming up again and again. I think it is just time to rethink Fieldable / Field and move towards some kind of flexible type definition for Fields in Lucene. A FieldType could then have a FieldCache Attribute which contains all necessary info including the parser and flags like the one we are talking about. Yet, before I get too excieted about FieldType, yeah something with a wider scope than FieldCache.Parser would work in this case. I don't know how far the FieldType is away but it can eventually replace whatever is going to be implemented here in regards to that flag. 

I think by default we should not enable the Bits feature but it must be explicitly set via whatever mechanism we gonna use.



      was (Author: simonw):
    bp. Perhaps FieldCache.Parser was originally just too narrow in scope - it should have been a factory method for handling all decisions about creating and populating a field cache entry?
I guess we need to be able to manually configure FieldCache with some kind of FieldType. There have been several issues mentioning this and it keeps coming up again and again. I think it is just time to rethink Fieldable / Field and move towards some kind of flexible type definition for Fields in Lucene. A FieldType could then have a FieldCache Attribute which contains all necessary info including the parser and flags like the one we are talking about. Yet, before I get too excieted about FieldType, yeah something with a wider scope than FieldCache.Parser would work in this case. I don't know how far the FieldType is away but it can eventually replace whatever is going to be implemented here in regards to that flag. 

I think by default we should not enable the Bits feature but it must be explicitly set via whatever mechanism we gonna use.


  
> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913700#action_12913700 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

bq. If we do this then we can leave cutover to XXXValues (still a good idea) as a separate issue?

I'm already in deep... I'd like to keep the XxxValues in this patch, and use a different one to bring everything up-to-date with the new API in 3.x/4.x



> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914089#action_12914089 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

Oh... I mis-matched the parens when I was looking at your proposal (hence the confusion).

I think getCachePopulator() should be under EntryConfig - that way people can provide their own (and extend ByteValues to include more info)
Otherwise, we'll forever be locked into a lowest common denominator of only adding info that everyone can agree on.


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914959#action_12914959 ] 

Michael McCandless commented on LUCENE-2649:
--------------------------------------------

bq. Apologies for 'CTR' rather then 'RTC' -- we can always revert if I jumped the gun!

Better to ask forgiveness than permission :)

In fact I'm +1 on switching Lucene's trunk to CTR model instead, now
that we have 3.x as the stable branch.  We have enough "policemen"
around here that I think this'd work well.

The changes look great Ryan -- nice work!

Some smallish feedback:

  * I see some windows eol's snuck in... can you change the
    svn:eol-style of all the new sources to "native"?

  * Some classes are missing copyright header (at least EntryKey,
    SimpleEntryKey)

  * Shouldn't we only incr .numDocs if the bit wasn't already set?
    (To be robust if docs have more than one value).  Ie we can use
    OpenBits.getAndSet.  Maybe then add and assert that numDoc <=
    maxDoc in the end...

  * Then, we can pass null for the delDocs to the enums, and then we
    don't need a 2nd pass to detect matchAllDocs (just test if
    .numDocs == maxDoc())?

I think we should hold off on backport to 3.x until we stabilize
LUCENE-2665?

It looks like you've also fixed LUCENE-2527 with this?  Ie the
fasterButMoreRAM=true|false now cache to the same key?  It's just that
perhaps we should "upgrade" the entry, if it was first created w/
false and then the current call passes true?


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Search
>            Reporter: Ryan McKinley
>            Assignee: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Issue Comment Edited: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914194#action_12914194 ] 

Uwe Schindler edited comment on LUCENE-2649 at 9/23/10 3:49 PM:
----------------------------------------------------------------

I just wanted to mention that, so the design of the new FC is more flexible in that case. I am just pissed of because of these arrays and no flexibility :-(

The FC impl should be in line with the CSF aproach from LUCENE-2186.

      was (Author: thetaphi):
    I just wanted to mention that, so the design of the new FC is more flexible in that case. I am just pissed of because of these arrays and no flexibility :-(
  
> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914136#action_12914136 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

Passing it in would also allow a way to get rid of the StopFillCacheException hack for NumericField in the future.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914551#action_12914551 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

bq. Can't we first just allow the retrieval of ByteValues, etc, that also have Bits on it?

I sure *hope* so... otherwise, I doubt anything will move forward.  

My hope with LUCENE-2665, is to fix the basic problem -- in a way that does not close the door to the improvements we want to make in the future.  I think/hope we are close.  

If I'm missing something let me know, so I can stop wasting my time.

(I don't even know what CSF is, and do not want to learn right now)


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Search
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914398#action_12914398 ] 

Michael McCandless commented on LUCENE-2649:
--------------------------------------------

I like where this all is going!!  We can finally fix FC!!

bq. In my opinion, the field cache's public API should not look different from CSF, so one can simply also sort against a CSF.

+1

>From the consumption standpoint, they (FC and CSF) really ought to be
one and the same.

What's "unique" about FieldCache is that it derives its values via
uninversion... this is nice because there's no index change, but it's
slow at reader open time.  It's also error-proned (you may hit
multiple values per doc, these values may have gone through analysis,
etc.)

CSF, instead, actually computes the values during indexing and stores
the raw, pre-computed array, in the index.

They are both just different sources for the same thing.

Also, an app should be free to plugin its own external source, and it
should present this same "values source" API.

bq. Uwe, I think we need to keep the native arrays.

I think the API should allow for optional retrieval of the backing
array (and we should [manually, for today] specialize the sort
comparators), but primary access should be a method call eg
ByteValues.getValue(int docID).


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Search
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913698#action_12913698 ] 

Michael McCandless commented on LUCENE-2649:
--------------------------------------------

bq. If we have a separate getBits( field ) call, should the Bits be added to the XxxValues class? I suspect not.

Yeah I think it need not be.

But, EntryCreator still should be able to state that it'd like the bits computed & cached as a side effect?  And, if the bits wasn't already computed, then they'd be computed on-demand?  (This enables caching only valid bits and not the values array).

If we do this then we can leave cutover to XXXValues (still a good idea) as a separate issue?

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913679#action_12913679 ] 

Michael McCandless commented on LUCENE-2649:
--------------------------------------------

bq. Regardless of if there is a separate getBits(field), I think we should add/use ByteValues, IntValues, etc. It's just so much more extensible going forward.

+1

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910728#action_12910728 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

bq. If ya care - don't pass a null parser! Otherwise you get the default. 

What if I care, but somethign else (that does not care) asks for the value first?  Seems odd to have so much depend on *who* asks for the value first

bq. A better option is to replace FieldCache.Parser in SortField to be FieldCache.EntryCreator.

How would that work?  What if a filter creates the cache before the SortField?  

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Issue Comment Edited: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913699#action_12913699 ] 

Uwe Schindler edited comment on LUCENE-2649 at 9/22/10 2:25 PM:
----------------------------------------------------------------

I think it should be a separate entry in the cache. Only that its only regenerated, if it does not already exist for the field/IR combination. So there are these combinations:

- cache is purged (inititially)
- user calls getBytes(field) -> bytes[] is filled, also the Bits
- user calls additionally getInts(field) for some reason -> int[] is filled, but as Bits already exit they dont need to be filled
- user calls getBits(field) -> returns the pre-filled bits from one of the previous calls

Alternatively:

- Cache is purged (initially)
- User calls getBits(field), there are no bits available until now, bits are populated from TermEnum/TermDocs, but no byte[] or int[], as not requested

With a config option, one could switch automatic polulation of Bits off, so in the first combination, only the last call to getBits() would populate bit set.

I all cases this is easily possible by having a separate cache with separate population method for the bits. If some method like getBytes() also populates the Bits, it should simply add the created Bits manually to the cache.

Instead of a global config option, we could simply add a Boolean to the getXxx methods, to tell the cache if it should also populate Bits (if not already done?). The default would maybe false for Lucene, but solr would always pass true.

Does this sound like a plan?

      was (Author: thetaphi):
    I think it should be a separate entry in the cache. Only that its only regenerated, if it does not already exist for the field/IR combination. So there are these combinations:

- cache is purged (inititially)
- user calls getBytes(field) -> bytes[] is filled, also the Bits
- user calls additionally getInts(field) for some reason -> int[] is filled, but as Bits already exit they dont need to be filled
- user calls getBits(field) -> returns the pre-filled bits from one of the previous calls

Alternatively:

- Cache is purged (initially)
- User calls getBits(field), there are no bits available until now, bits are populated from TermEnum/TermDocs, but no byte[] or int[], as not requested

With a config option, one could switch automatic polulation of Bits off, so in the first combination, only the last call to getBits() would populate bit set.

I all cases this is easily possible by having a separate cache with separate population method for the bits. If some method like getBytes() also populates the Bits, it should simply add the created Bits manually to the cache.

Does this sound like a plan?
  
> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913659#action_12913659 ] 

Uwe Schindler commented on LUCENE-2649:
---------------------------------------

I am also strongly +1 for the additional Bits interface (as Ryan did, it does not always need to be a real OpenBitSet, so when no deletions and all things set, we can use a dummy one).
I had often use cases where i needed the information, if this document really has a value set or not, and i don't use Solr so much.

{quote}
And being able to distinguish missing values, eg to sort them last, or
to do something else, is useful. Once we do this we should also
[eventually] move "sort missing last" capability into Lucene's
comparators.
{quote}

+1

{quote}
I think this is the right approach - expecting FC's valid bits to
take deletions into account is too much. We have IR.getDeletedDocs
for this.
{quote}

We don't need to AND them together, maybe simply wrap the OpenBitset by a custom Bits impl, that ands in the getter? But as deletions are separated in IndexReader and the cache can reuse the cache even when new deletions are added, i think keeping it separate is fine.

About the whole bit set: Do we really need to couple the Bits interface to the type? Because if you exchange the parser/native type (e.g. parse ints as byte), the valid docs are still the same, only the native type representation is different. So how about we add a getBits(field) method to FieldCache that returns the valid docs. If field was not yet retrieved as a native type it could throw IllegalStateEx, else it would return the Bits interface (globally, but per field, but not per parser/datatype) created during the last FC polulation run? We have then also the possibility to disable the default generation of Bits and do it lazily (which should run faster, as it does not need to parse the values, only enumerate terms and termdocs).

{quote}
Really, "in general" we need a better way for the query execution path
to enforce deleted docs. Eg if the FCRF will be AND'd w/ a query
that's already excluding del docs then it need not be careful about
deletions...
{quote}

Thats another thing, but maybe we remove deleted docs completely from query processing and simply apply it like a filter before the collector. Not sure about the implications and performance.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12915077#action_12915077 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. What is an example?

Facet component, query elevation component, stats component, and sorting.
For example, if you facet on the popularity field, it will currently use strings - but if you sort on the popularity field, it will ask the FieldCache for ints.

But all of these instances in Solr involve treating the same field as both a string and as a number - perhaps you actually need to ask for numbers, but with different parsers, to trigger the exception?



> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Search
>            Reporter: Ryan McKinley
>            Assignee: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914188#action_12914188 ] 

Michael McCandless commented on LUCENE-2649:
--------------------------------------------

bq. I see no relief for this issue on the horizon.

We need to specialize the code... either manually or automatically...

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910736#action_12910736 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. What if I care, but somethign else (that does not care) asks for the value first? Seems odd to have so much depend on who asks for the value first

As long as it *can* be passed everywhere that matters, then it's up to the application - which knows if it ever needs the missing values or not for that field.  For solr, we could make it configurable per-field... but I'd prob default it to ON to avoid unpredictable weirdness.

bq. What if a filter creates the cache before the SortField?

If we have a filter that uses the field cache, then it should also be able to specify the same stuff that SortField can.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913428#action_12913428 ] 

Uwe Schindler commented on LUCENE-2649:
---------------------------------------

Hi Ryan,

few comments:
- the tests if all/no bits are set and so the special Bits implementations are returned are fine, but the special case for all bits are valid may be a little bit ineffective and seldom
- please use the correct Java code style ("{" should be at the end of previous line and not in separate line for method declarations), the Eclipse code style is available in Wiki

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910707#action_12910707 ] 

Uwe Schindler commented on LUCENE-2649:
---------------------------------------

I am against that option! No static defaults!

bq. the number of values set 

This is OpenBitSet.cardinality() ? I dont think we should add this extra cost during creation, as it can be retrieved quite easy if really needed.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910746#action_12910746 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. How would that work?

We could start off simple - add only recordMissing functionallity and punt on the rest, while still leaving a place to add it.

{code}
class FieldCache {

  public static class EntryCreator {
    public boolean recordMissing() {
      return false;
    }
    
    public abstract Parser getParser();
  }
{code}

Not even sure if a whole hierarchy is needed at this point... in the future, we'd prob want

{code}
  public static EntryCreatorInt extends EntryCreator {
    public IntValues getIntValues(IndexReader reader, String field) {... code currently in FieldCacheImpl that fills the fieldCahe...}
    ...
  }
{code}


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12915080#action_12915080 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

Ok, that makes sense -- what I mean is that you can not ask for the same *type* with different parsers.  Bytes, vs Strings, vs Ints all have different keys and within the FieldCache, they are even stored in different Maps

What you can not do is as for a IntValues using: FieldCache.DEFAULT_INT_PARSER, then switch to FieldCache.NUMERIC_UTILS_INT_PARSER for the same field.

make sense?

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Search
>            Reporter: Ryan McKinley
>            Assignee: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913742#action_12913742 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. The only cost would be iterating the Term/docID iterators a second time

Actually, that's major - otherwise I would have already just done that and stored it in a separate cache for Solr's needs.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913675#action_12913675 ] 

Uwe Schindler commented on LUCENE-2649:
---------------------------------------

bq. Hmm... I'd rather make an exception to 3.x, ie, allow the addition of this method to the interface, than confuse the 4.x API, going forward, with 2 classes?

Same here, we already defined the FieldCache "interface" as subject to change. Mabye we should simply remove it in trunk and only have a class? This interface was never of any use, because you were not able to supply any other field cache implementation (the DEFAULT field is *final* because all fields in interfaces are defined as *final* by the Java Language Spec.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913677#action_12913677 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

Regardless of if there is a separate getBits(field), I think we should add/use ByteValues, IntValues, etc.  It's just so much more extensible going forward.


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913680#action_12913680 ] 

Michael McCandless commented on LUCENE-2649:
--------------------------------------------

bq.  So how about we add a getBits(field) method to FieldCache that returns the valid docs. 

This would be great!

Assuming we separately cache the valid docs, we could then allow caching *only* valid docs, for apps that want to know if a doc has a value but do not need the full array of values RAM resident.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910653#action_12910653 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

FYI, I like the idea of revisiting the FieldCache, but i don't see a straightforward path.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910446#action_12910446 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

See some discussion here:
http://search.lucidimagination.com/search/document/b6a531f7b73621f1/trie_fields_and_sortmissinglast

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Simon Willnauer (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910589#action_12910589 ] 

Simon Willnauer commented on LUCENE-2649:
-----------------------------------------

bq. Also, if I have to list out all the fields I'm going to use, that's another big step backwards.
I don't think that this is needed at all neither would it be a step backwards - not even near to that. But since we aren't on an issue about FieldType lets just drop it...

bq. A factory would be a pretty straightforward way to increase the power, by allowing users to populate the entry through any mechanism, and optionally do extra calculations when the entry is populated (think statistics, sum-of-squares, etc).
Whatever you call it (using Factory is fine) but isn't that what you mentioned to be insufficient? I mean this is something you would pass to a FieldCache.getXXX, right? 

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914185#action_12914185 ] 

Uwe Schindler commented on LUCENE-2649:
---------------------------------------

Yonik: I was expecting this answer...

The reason is that my current contact (it's also your's) has exactly that problem also with norms (but also FC), that they want to lazily load values for sorting/norms (see the very old issue LUCENE-505). At least we should have a TopFieldDocCollector that can alternatively to native arrays also use a ValueSource-like aproach with getter methods - so you could sort against a CSF. Even if it is 20% slower, in some cases thats the only way to get a suitable search experience. Not always speed is the most important thing, sometimes also space requirements or warmup times. I would have no problem with providing both and chosing the implementation that is most speed-effective. So if no native arrays are provided by the FieldCache use getter methods.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913734#action_12913734 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

bq. more performant too (and avoid more synchronizing lookups)

This was my motivation for putting it all together and using the options as part of the key.  But yes, inconsistent usage will eat up RAM.  That is better then my original fear that inconsistent usage would give you unpredictable results!

Also, with the current Cache implementaion, we would need to somehow be able to add two cache entries from within Cache.createEntry() -- either rewrite all that or hack in some way to pass the FieldCache to the createEntry method.

Keeping the values and bits in different cache entries is pretty ugly (especially for the normal case where we want them both all the time)
{code:java}
    return new ByteValues() {
      (byte[])caches.get(Byte.TYPE).get(reader, new Entry(field, parser??)),
      (Bits)caches.get(Byte.TYPE).get(reader, new Entry(field, parser??)),
    };
{code}


bq. But we could probably still achieve these two benefits while using a single class for looking up everything "cached" about a field? Ie, the CachedArray could return non-null bits but null values?

Brainstorming here...  if Parser -> EnteryCreator and the 'EntryCreator.hashCode()' is used as the map key (as it is now)

{code:java}
  public abstract class EntryCreator implements Serializable {
    public boolean cacheValidBits() {
      return false;
    }
    public boolean cacheValues() {
      return true;
    }
  }
{code}

The trick would be to use the same *key* regardless of what we ask for (values but no bits - bits but no values - bits and values, etc) and then fill up whatever is missing if it is not in the existing cache.

That might suggest that the 'class' could be the key, but setting the cacheValidBits/values would need to get implemented by inheratance, so that is out.

other ideas?  directions I am not thinking about?






> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914191#action_12914191 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. If we like the more general cache, that probably needs its own issue

Correct - with a more general fieldCache, one could implement alternatives that MMAP files, etc.  But those alternative implementations certainly should not be included in this issue.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910618#action_12910618 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

This is a band-aid, but we could consider adding something like:
{code:java}
  public void setCacheValidBitsForFields( Set<String> names );
{code}
on FieldCache, then checking if the field is in that set before making the BitSet

When solr reads the schema, it could look for any fields have sortMissingLast and then call:
{code:java}
  FieldCache.DEFAULT.setCacheValidBitsForFields()
{code}

The factory idea also sounds good, but i don't see how would work without big big changes

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913712#action_12913712 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. But we could probably still achieve these two benefits while using a single class for looking up everything "cached" about a field? Ie, the CachedArray could return non-null bits but null values?

Exactly.
And with NRT and increasing number of segments, the number of synchronized lookups per second could really start to matter.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Updated: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ryan McKinley updated LUCENE-2649:
----------------------------------

    Attachment: LUCENE-2649-FieldCacheWithBitSet.patch

Here is a new patch that removes the static config.  Rather then put a property on Parser class, I added a class:
{code:java}
  public abstract static class CacheConfig {
    public abstract boolean cacheValidBits();
  }
{code}
and this gets passed to the getXXXValues function:
{code:java}
ByteValues getByteValues(IndexReader reader, String field, ByteParser parser, CacheConfig config)
{code}

I think this is a better option then adding a parameter to Parser since we can have an easy upgrade path.  Parser is an interface, so we can not just add to it without breaking compatibility.  To change things in 4.x, 3.x should have an upgrade path.

I took Mike's suggestion and include the CacheConfig hashcode in the Cache key -- however, I don't cache the Bits separately since this is an edge case that *should* be avoided, but at least does not fail if you are not consistent.

This does cache a MatchAllBits even when 'cacheValidBits' is false, since that is small (a small class with one int)

-----------

bq.     *  We don't have to @Deprecate for 4.0 - just remove it, and note this in MIGRATE.txt. (Though for 3.x we need the deprecation, so maybe do 3.x patch first, then remove deprecations for 4.0?).

My plan was to apply with deprecations to 4.x, then merge with 3.x.  Then replace the calls in 4.x, then remove the old functions.  Does this sound reasonable?

I would like this to get in 3.x since we could then remove many solr types in 4.x and have a 3.x migration path.

bq.  * FieldCache.EntryCreator looks orphan'd?

dooh, thanks


bq. It looks like the valid bits will not reflect deletions (by design), right? Ie caller cannot rely on valid always incorporating deleted docs. (Eg the MatchAll opto disregards deletions, and, a reopened segment can have new deletions yet shares the FC entry).

Right, the ValidBits are only checked for docs that exists (and the FC values are only set for docs that exists -- this has not changed), and may contain false positives for deleted docs.  I think this is OK since most use cases (i can think of) deal with deletions anyway.   Any ideas how/if we should change this?  (I did not realize that the FC is reused after deletions -- so clever)

----------------

bq. I'm having trouble understanding the use case for this bitset.

My motivation is for supporting the supportMissingLast feature in solr sorting (that could now be pushed to lucene).  For example if I have a bunch of documents and only some have the field "bytes" -- sorting 'bytes desc' works great, but sorting 'bytes asc' puts all the documents that do not have the field 'bytes' first since the FieldCache thinks they are all zero.

If we get this working in solr, we can deprecate and delete all the "sortable" number fields and have that same functionality on Trie* fields.







> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910576#action_12910576 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. I guess we need to be able to manually configure FieldCache with some kind of FieldType.

I don't know how well that would work.  For one, there's only one FieldCache, so configuring it with anything seems problematic.
Also, if I have to list out all the fields I'm going to use, that's another big step backwards.

A factory would be a pretty straightforward way to increase the power, by allowing users to populate the entry through any mechanism, and optionally do extra calculations when the entry is populated (think statistics, sum-of-squares, etc).

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913857#action_12913857 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

bq Is the one-time-per-IndexReader-lifecycle cost of multiplying the cache load time by some factor < 2.0 ... really so terrible 

it can be... on a big index just iterating all the terms/docs take a long time.  Try the LukeRequestHandler on an index with a million+ docs!

-------------

Here is different variation, it changes *lots* but if we are talking about changing Parser from interface to class, then I guess the cat can be out of the bag.

What about something like: 
{code:java|title=FieldCache.java}
  ...

  
  public class EntryConfig implements Serializable 
  {
    public Parser getParser() {
      return null;
    }
    public boolean cacheValidBits() {
      return false;
    }
    public boolean cacheValues() {
      return true;
    }
    
    /**
     * The HashCode is used as part of the Cache Key (along with the field name).  
     * To allow multiple calls with different parameters, make sure the hashCode 
     * does not include the specific instance and parameters.
     */
    public int hashCode()
    {
      return EntryConfig.class.hashCode();
    }
  }
  
  
  public abstract class CachePopulator 
  {
    public abstract void fillValidBits(  CachedArray vals, IndexReader reader, String field, EntryConfig creator ) throws IOException;
    public abstract void fillByteValues( CachedArray vals, IndexReader reader, String field, EntryConfig creator ) throws IOException;
    ...
  }

  public abstract CachePopulator getCachePopulator();

...

  public ByteValues getByteValues(IndexReader reader, String field, EntryConfig creator )

...

{code}


The field cache implementation would make sure what you asked for is filled up before passing it back (though i think this has some concurrency issue)
{code:java}

  public ByteValues getByteValues(IndexReader reader, String field, EntryConfig config) throws IOException
  {
    ByteValues vals = (ByteValues) caches.get(Byte.TYPE).get(reader, new Entry(field, config));
    if( vals.values == null && config.cacheValues() ) {
      populator.fillByteValues(vals, reader, field, config);
    }
    if( vals.valid == null && config.cacheValidBits() ) {
      populator.fillValidBits(vals, reader, field, config);
    }
    return vals;
  }
{code}

The Cache would then delegate the creation to the populator:
{code:java}

    @Override
    protected final ByteValues createValue(IndexReader reader, Entry entry, CachePopulator populator) throws IOException {
      String field = entry.field;
      EntryConfig config = (EntryConfig)entry.custom;
      if (config == null) {
        return wrapper.getByteValues(reader, field, new EntryConfig() );
      }
      ByteValues vals = new ByteValues();
      if( config.cacheValues() ) {
        populator.fillByteValues(vals, reader, field, config);
      }
      else if( config.cacheValidBits() ) {
        populator.fillValidBits(vals, reader, field, config);
      }
      else {
        throw new RuntimeException( "the config must cache values and/or bits" );
      }
      return vals;
    }
{code}

The fillByteValues would be the same code as always, but I think the CachedArray should make sure the same parser is used everytime
{code:java}

    @Override
    public void fillByteValues( CachedArray vals, IndexReader reader, String field, EntryConfig config ) throws IOException
    {
      ByteParser parser = (ByteParser) config.getParser();
      if( parser == null ) {
        parser = FieldCache.DEFAULT_BYTE_PARSER;
      }
      // Make sure it is the same parser
      int parserHashCode = parser.hashCode();
      if( vals.parserHashCode != null && vals.parserHashCode != parserHashCode ) {
        throw new RuntimeException( "Subsequent calls with different parser!" );
      }
      vals.parserHashCode = parserHashCode;
     ...
{code}

This is different then the current code where asking for the cached values with two different parsers (that return different hashcodes) will make two entries in the cache.

This approach would let us:
* cache values and bits independently or together
* subsequent calls with different parameters should behave reasonably
* If CachePopulator is pluggable/extendable that may make some other issues easier
* lets us use CachePopulator outside of the cache context (perhaps useful)





> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913709#action_12913709 ] 

Uwe Schindler commented on LUCENE-2649:
---------------------------------------

It's not more complicated its easier. The Bits are a real separate thing, its just a cache of the information, if values are assigned or not. It does not depend on the data type like byte, int,... Its just separate. And as said before, if somebody requests the same field in different types, it would only have one bits. Also one could request the Bits alone, or could first request a field without bits (using a boolean) and later again with bits, in which case those are lazily loaded

Th implementation would be simple, similar to the way, how the caches are filled for NumericFields (it adds the values two times to the cache, with the null parser and the real used parser). I this case youl would simply request the bits cache on e.g. the int[] creation, if not availabe, also build the bits in parallel and add to bits cache.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910677#action_12910677 ] 

Michael McCandless commented on LUCENE-2649:
--------------------------------------------

I know it's only 3% (for ints... 12.5% for bytes), but, 3% here, 3% there and suddenly we're talking real money.

Lucene can only stay lean and mean if we don't allow these little 3% losses here and there!!

Let's try to find some baby-step (even if not clean -- we know FieldCache, somehow, needs to be fixed more generally) for today?


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910740#action_12910740 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. I agree that we should avoid static fields to control behavior. But do we avoid it at the cost of not allowing the option, or waiting till we rework FieldCache?

I agree with this sentiment - progress, not perfection.  Being able to turn it on or off for everything in the process is better than nothing at all.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914163#action_12914163 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

Ok, as I look more, I think it may be worth some even bigger changes!  

Is there any advantage to having a different map for each Type?  The double (and triple) cache can get a bit crazy and lead to so much duplication

What about moving to a FieldCache that is centered around the very basic API:

{code:java}
public <T> T get(IndexReader reader, String field, EntryCreator<T> creator)
{code}

Entry creator would be something like
{code:java}
public abstract static class EntryCreator<T> implements Serializable 
  {
    public abstract T create( IndexReader reader, String field );
    public abstract void validate( T entry, IndexReader reader, String field );
    
    /**
     * NOTE: the hashCode is used as part of the cache Key, so make sure it 
     * only changes if you want different entries for the same field
     */
    @Override
    public int hashCode()
    {
      return EntryCreator.class.hashCode();
    }
  }
{code}

We could add all the utility functions that cast stuff to ByteValues etc.  We would also make sure that the Map does not use the EntryCreator as a key, but uses it to generate a key.

A sample EntryCreator would look like this:
{code:java}

class BytesEntryCreator extends FieldCache.EntryCreator<ByteValues> {

  @Override
  public ByteValues create(IndexReader reader, String field) 
  {
    // all the normal walking stuff using whatever parameters we have specified
  }

  @Override
  public void validate(ByteValues entry, IndexReader reader, String field) 
  {
    // all the normal walking stuff using whatever parameters we have specified
  }  
}
{code}

Thoughts on this approach?  


Crazy how a seemingly simple issue just explodes :(

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913688#action_12913688 ] 

Uwe Schindler commented on LUCENE-2649:
---------------------------------------

We should also allow the parser to stop iterating term without the strange StopFillCacheException (needed for Numeric fields).

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Simon Willnauer (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910568#action_12910568 ] 

Simon Willnauer commented on LUCENE-2649:
-----------------------------------------

bp. Perhaps FieldCache.Parser was originally just too narrow in scope - it should have been a factory method for handling all decisions about creating and populating a field cache entry?
I guess we need to be able to manually configure FieldCache with some kind of FieldType. There have been several issues mentioning this and it keeps coming up again and again. I think it is just time to rethink Fieldable / Field and move towards some kind of flexible type definition for Fields in Lucene. A FieldType could then have a FieldCache Attribute which contains all necessary info including the parser and flags like the one we are talking about. Yet, before I get too excieted about FieldType, yeah something with a wider scope than FieldCache.Parser would work in this case. I don't know how far the FieldType is away but it can eventually replace whatever is going to be implemented here in regards to that flag. 

I think by default we should not enable the Bits feature but it must be explicitly set via whatever mechanism we gonna use.



> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Shai Erera (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910625#action_12910625 ] 

Shai Erera commented on LUCENE-2649:
------------------------------------

bq. Are there any advantages to using payloads over CSF for fieldcache population?

Well .. payloads already exist (in my code :)), while CSF is "just around the corner" for a long time. While the two ultimately achieve the same goal, CSF is more generic than just payloads, and if we'd want to take advantage of it w/ FieldCache, I assume we'll need to make more changes to FieldCache, because w/ CSF, people can store arbitrary byte[] and request to cache them. So sorting data is a subset of CSF indeed, but I think the road to CSF + CSF-FieldCache integration is long. But perhaps I'm not up-to-date and there is progress / someone actually working on CSF?

Anyway, opening up FC to read from payloads seems to me a much easier solution, because besides reading the stuff from the payload, the rest of the classes continue to work the same (TopFieldCollector, Comparators etc.).

Maybe a slight change to SortField will be required as well though, not sure yet.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913710#action_12913710 ] 

Michael McCandless commented on LUCENE-2649:
--------------------------------------------

bq. What's the issue with just putting the bits reference down on CachedArray?

One risk was added insanity cases, ie you looked up once w/ the bits and later w/o and it double-stores the values array.

Another gain of separating the bits retrieval is it becomes possible to get only the valid bits (ie, w/o a value array), for apps that just want to know if a given doc had a field.

But we could probably still achieve these two benefits while using a single class for looking up everything "cached" about a field?  Ie, the CachedArray could return non-null bits but null values?

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914100#action_12914100 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

bq. In this proposal, you could set the CachePopulator on the FieldCache. 

Hmmm, OK, as long as it's possible.

bq. From my point of view, this would make it easier for system with a schema (like solr) have consistent results across all calls, rather then making each request to the FieldCache need to know about the schema -> parsers -> populator

I think this may make it a lot harder from Solr's point of view.
- it's essentially a static... so it had better not ever be configurable from the schema or solrconfig, or it will break multi-core.
- if we ever *did* want to treat fields differently (load some values from a DB, etc), we'd want to look that up in the schema - but we don't have a reference to the scema in the populator, and we wouldn't want to store one there (again, we have multiple schemas).  So... we could essentially create custom EntryConfig object and then our custom CachePopulator could delegate to the entry config  (and we've essentially re-invented a way to be able to specify the populator on a per-field basis).

Are EntryConfig objects stored as keys anywhere?   We need to be very careful about memory leaks.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913647#action_12913647 ] 

Michael McCandless commented on LUCENE-2649:
--------------------------------------------

bq. My motivation is for supporting the supportMissingLast feature in solr sorting (that could now be pushed to lucene).

{quote}
If folks think that being able to tell a real "0" from a missing value is not useful for Lucene, we
could extend Ryan's CacheConfig to include a factory method that creates / populates ByteValues, IntValues, etc.
Then all the bitset stuff could be kept in Solr only.  I'm sensitive about pushing stuff into Lucene that is *only* useful for Solr.
{quote}

I'm very much +1 for making this (exposing thea valid bitset) possible
in Lucene.

Users have asked over time how they can tell if a given doc has a field value.

And being able to distinguish missing values, eg to sort them last, or
to do something else, is useful.  Once we do this we should also
[eventually] move "sort missing last" capability into Lucene's
comparators.


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910640#action_12910640 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

oh right -- thats true.  Is a global flag sufficient?  

In lucene it could default to false and in solr default to true.

I know we don't want to just keep adding more things to memory, but I'm not sure there is a huge win by selectively enabling and disabling some fields.

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914515#action_12914515 ] 

Yonik Seeley commented on LUCENE-2649:
--------------------------------------

It feels like incremental improvement keeps being held hostage...
Can't we first just allow the retrieval of ByteValues, etc, that also have Bits on it?
Changing everything to go through getValue() should be a separate issue (and CSF isn't even finalized/committed yet).

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Search
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Updated: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ryan McKinley updated LUCENE-2649:
----------------------------------

    Attachment: LUCENE-2649-FieldCacheWithBitSet.patch

Here is a (hopefully) final patch that adds a bunch of tests to exercise the the 'valid' bits (and check that MatchAll is used when appropriate)

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Shai Erera (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910591#action_12910591 ] 

Shai Erera commented on LUCENE-2649:
------------------------------------

One thing I've wanted to do for a long time, but didn't get to doing it, is open up FieldCache to allow the application to populate the entries from other sources - specifically pyloads. I wrote a sorting solution which relies solely on payloads, and wanted to contribute it to Lucene, but due to lack's of FieldCache hook points, I didn't find the time to do the necessary refactoring.

Sorting based on payloads-data has several advantages:
# It's much faster to read than iterating on the lexicon and parsing the term values into sortable values.
# If your application needs to cater sort over 10s of millions of documents, or if it needs to keep its RAM usage low, you can do the sort while reading the payload data as the search happens. It's faster than if it was in RAM, but the current FieldCache does not allow you to sort w/o RAM consumption.
# You don't inflate your lexicon w/ sort values, affecting other searches. In some situations, you can add a unique term per document for the sort values (such as when sorting by date and require up to a millisecond precision).

I'm bringing it up so that if you consider any refactoring to FieldCache, I'd appreciate if you can keep that in mind. If the right hooks will open up, I'll make time to contribute my sort-by-payload package. If you don't, then it'll need to wait until I can find the time to do the refactoring.



> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910513#action_12910513 ] 

Michael McCandless commented on LUCENE-2649:
--------------------------------------------

Looks great!

Should we make it optional, whether the valid bitset should be computed?  Many apps wouldn't need it, so it just ties up (admittedly smallish amounts of) RAM unnecessarily?

bq. Lucene's sorting Collectors can be improved to have a consistent behaviour for missing values (like Solr's sortMissingFirst/Last).

+1

Shouldn't we pull Solr's sortMissingFirst/Last down into Lucene?

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913695#action_12913695 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

bq. Regardless of if there is a separate getBits(field), I think we should add/use ByteValues, IntValues, etc. It's just so much more extensible going forward.

If we have a separate getBits( field ) call, should the Bits be added to the XxxValues class?  I suspect not.

 


> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Ryan McKinley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12915074#action_12915074 ] 

Ryan McKinley commented on LUCENE-2649:
---------------------------------------

bq. If that were true, it would have broken solr in multiple places

What is an example?  The only thing I have seen is stuff that tries one parser and if it throws an exception, tries another (that is supported)

As is, the cached value keeps the hash code for the parser created the values.  If we validate (true by default) then it checks that the passed in hash code matches the one that is stored.  Perhaps different Parsers are returning the same hashCode?    

Does it make sense to create a different key depending on what Parser we use?  Is it normal to interpret the same field multiple different ways?

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Search
>            Reporter: Ryan McKinley
>            Assignee: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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


[jira] Commented: (LUCENE-2649) FieldCache should include a BitSet for matching docs

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2649?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12914508#action_12914508 ] 

Uwe Schindler commented on LUCENE-2649:
---------------------------------------

{quote}
bq. Uwe, I think we need to keep the native arrays.

I think the API should allow for optional retrieval of the backing
array (and we should [manually, for today] specialize the sort
comparators), but primary access should be a method call eg
ByteValues.getValue(int docID).
{quote}

Exactly. Maybe do it like with NIO buffers: they have methods hasArray(), array() and arrayOffset(), the two last ones throw UnsupportedOp, if first is false. We already have quite a lot TopFieldDocCollectors impls as inner classes, a few more choosen by hasArray()... haha :-)

> FieldCache should include a BitSet for matching docs
> ----------------------------------------------------
>
>                 Key: LUCENE-2649
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2649
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Search
>            Reporter: Ryan McKinley
>             Fix For: 4.0
>
>         Attachments: LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch, LUCENE-2649-FieldCacheWithBitSet.patch
>
>
> The FieldCache returns an array representing the values for each doc.  However there is no way to know if the doc actually has a value.
> This should be changed to return an object representing the values *and* a BitSet for all valid docs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


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