Re: Per-document Payloads

2007-10-20 Thread Michael Busch
John Wang wrote:
> Hi Michael:
>  Thanks for the info.
> 
>  I haven't played with payloads. Can you give me an example or point me
> to how it is used to solve this problem?
> 

Hi John,

I (quickly) put together a class that is able to store UIDs as payloads.
I believe the type of your UIDs is Integer?

To add the UID to a document use this method:
  /**
   *  Adds a unique docId (UID) to a document as a payload.
   */
  public void addUID(Document doc, int uid);

You can either load the UID from the disk or use a cache:

 /** Returns the UID for the passed-in docId.
  *  If you use this method in a Hitcollector and your Query
  *  contains OR-terms, then try to set
  *  BooleanQuery.setAllowDocsOutOfOrder(false) to improve performance.
  */
  public int getUID(IndexReader reader, int docId) throws IOException;

  /** Fills the passed-in array with UID-values. If the given array is
   *  null or too small, then a new array is created with
   *  cache.length = reader.maxDoc()
   */
  public int[] getCachedIds(IndexReader reader, int[] cache, int offset)
   throws IOException;

I put a little test program in main and it seems to work fine. However,
it's not thoroughly tested yet...

You might want to try it without using the cache first. The performance
might be good enough for your needs. If not, try using the cache, it
should fill up much faster compared to the FieldCache.

Another comment: If you're using Lucene 2.2, you need to replace the
lines "tp.seek(UID_TERM);" (see comments in the code below). Or use the
latest trunk version, it has a fix for this bug.

Let me know please if this improves your performance! Have fun...
- Michael

And here is the code:

import java.io.IOException;

import org.apache.lucene.analysis.Token;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.Field.Index;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.Payload;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TermPositions;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.RAMDirectory;

public class PerDocPayloadReaderWriter {
  public static final Term UID_TERM = new Term("_ID", "_UID");
  private SinglePayloadTokenStream singlePayloadTokenStream = new
SinglePayloadTokenStream();
  private TermPositions tp;
  private byte[] payloadBuffer = new byte[4];

  public static void main(String args[]) throws IOException {
PerDocPayloadReaderWriter pdp = new PerDocPayloadReaderWriter();
Directory dir = new RAMDirectory();
IndexWriter writer = new IndexWriter(dir, new StandardAnalyzer());
for (int i = 0; i < 10; i++) {
  Document d = new Document();
  // create dummy doc
  d.add(new Field("test", "This is a test.", Store.NO,
Index.TOKENIZED));
  pdp.addUID(d, 100 + i);

  writer.addDocument(d);
}
writer.close();

IndexReader reader = IndexReader.open(dir);
int[] uids = pdp.getCachedIds(reader, null, 0);

System.out.println("Caching:");
System.out.println("ID --> UID");

for (int i = 0; i < uids.length; i++) {
  System.out.println(i + "  --> " + uids[i]);
}

System.out.println("\nDirect access:");
System.out.println("ID --> UID");

for (int i = 0; i < uids.length; i++) {
  System.out.println(i + "  --> " + pdp.getUID(reader, i));
}
reader.close();
  }


  /** Fills the passed-in array with UID-values. If the given array is
null or too small, then
   * a new array is created with cache.length = reader.maxDoc()
   */
  public int[] getCachedIds(IndexReader reader, int[] cache, int offset)
throws IOException {
int maxDoc = reader.maxDoc();
if (cache == null || cache.length - offset > maxDoc) {
  cache = new int[maxDoc];
  offset = 0;
}

if (tp == null) {
  tp = reader.termPositions(UID_TERM);
} else {
  // if using Lucene 2.2 replace the following line with
  // tp = reader.termPositions(UID_TERM);
  tp.seek(UID_TERM);
}

while (tp.next()) {
  assert tp.doc() < maxDoc;
  if (!reader.isDeleted(tp.doc())) {
tp.nextPosition();
tp.getPayload(payloadBuffer, 0);
cache[tp.doc() + offset] = bytesToInt(payloadBuffer);
  }
}

return cache;
  }

  /** Returns the UID for the passed-in docId.
   *  If you use this method in a Hitcollector and your Query contains
OR-terms,
   *  then try to set BooleanQuery.setAllowDocsOutOfOrder(false) to
improve performance.
   */
  public int getUID(IndexReader reader, int docId) throws IOException {
if (tp == null) {
  tp = reader.termPositions(UID_TERM);
} else if (tp.doc()> docId) {
  // if using Lucene 2.2 replace the following line with
  // tp = reader.ter

[jira] Commented: (LUCENE-743) IndexReader.reopen()

2007-10-20 Thread Michael McCandless (JIRA)

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

Michael McCandless commented on LUCENE-743:
---

It's not nearly this complex (we don't need two ref counts). If we
follow the simple rule that "every time reader X wants to use reader
Y, it increfs it" and "whenver reader X is done using reader Y, it
decrefs it",  all should work correctly.

Also we should think of "close()" as the way that the external user
does the decref of their reader.  We just special-case this call, by
setting isOpen=false, to make sure we don't double decref on a double
close call.

Let's walk through your example...

I'm assuming in your example you meant for reader2 and reader3 to also
be SegmentReaders?  Ie, the changes that are happening to the
single-segment index1 are just changes to norms and/or deletes.  If
not, the example is less interesting because reader1 will be closed
sooner :)

Also in your example let's insert missing "reader1.close()" as the
very first close?  (Else it will never be closed because it's RC never
hits 0).

When reader1 is created it has RC 1.

When multiReader1 is created, reader1 now has RC 2.

When multiReader2 is created, reader1 now has RC 3.

When reader2 is created (by reader1.reopen()), it incref's reader1
because it's sharing the sub-readers in reader1.  So reader1 now has
RC 4.

When reader3 was created (by reader2.reopen()), it incref's reader2
because it's sharing the sub-readers reader2 contains.  So reader1 is
still at RC 4 and reader2 is now at RC 2.

Now, we close.

After reader1.close() is called, reader1 sets isOpen=false (to prevent
double close by the user) and RC drops to 3.

With multiReader1.close(), multiReader1 is not at RC 0, and so it
decrefs all readers it was using, and so reader1 RC is now 2.

With multiReader2.close(), likewise it is now at RC 0 and so it
decrefs all readers it was using, and so reader1 RC is now 1.

With reader2.close(), it decrefs its own RC, however that brings its
RC to 1 (reader3 is still referring to it) and so it does not decref
the reader1 that it's referring to.

Finally, with reader3.close(), it is now at RC 0 and so it decrefs the
reader2 it refers to.  This brings reader2's RC to 0, and so reader2
decrefs the reader1 that it's referring to.  Which brings reader1's RC
to 0, and so reader1 finally closes all its internal sub-readers.


> IndexReader.reopen()
> 
>
> Key: LUCENE-743
> URL: https://issues.apache.org/jira/browse/LUCENE-743
> Project: Lucene - Java
>  Issue Type: Improvement
>  Components: Index
>Reporter: Otis Gospodnetic
>Assignee: Michael Busch
>Priority: Minor
> Fix For: 2.3
>
> Attachments: IndexReaderUtils.java, lucene-743-take2.patch, 
> lucene-743.patch, lucene-743.patch, lucene-743.patch, MyMultiReader.java, 
> MySegmentReader.java, varient-no-isCloneSupported.BROKEN.patch
>
>
> This is Robert Engels' implementation of IndexReader.reopen() functionality, 
> as a set of 3 new classes (this was easier for him to implement, but should 
> probably be folded into the core, if this looks good).

-- 
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: [EMAIL PROTECTED]
For additional commands, e-mail: [EMAIL PROTECTED]



[jira] Commented: (LUCENE-743) IndexReader.reopen()

2007-10-20 Thread Michael Busch (JIRA)

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

Michael Busch commented on LUCENE-743:
--

{quote}
I'm assuming in your example you meant for reader2 and reader3 to also
be SegmentReaders?
{quote}
Yes that's what I meant. Sorry, I didn't make that clear.

{quote}
Also in your example let's insert missing "reader1.close()" as the
very first close? (Else it will never be closed because it's RC never
hits 0).
{quote}
Doesn't what you describe change the semantics of MultiReader.close()?

If you do:
{code:java}
IndexReader reader1 = IndexReader.open(index1);  
IndexReader multiReader1 = new MultiReader(new IndexReader[] {reader1, 
IndexReader.open(index2)});
multiReader1.close();
{code}

then today multiReader1.close() also closes reader1. That's why I consciously 
omitted reader1.close().

Consequently, if you do
{code:java}
IndexReader reader1 = IndexReader.open(index1);  
IndexReader multiReader1 = new MultiReader(new IndexReader[] {reader1, 
IndexReader.open(index2)});
IndexReader multiReader2 = new MultiReader(new IndexReader[] {reader1, 
IndexReader.open(index3)});
multiReader1.close();
{code}
then multiReader2 is not usable anymore, because multiReader1.close() closes 
reader1. But that can be explicitly avoided by the user because it's known that 
multiReader1 and multiReader2 share the same reader.

Now, with the reopen() code:
{code:java}
IndexReader reader1 = IndexReader.open(index1);  // optimized index, reader1 is 
a SegmentReader
IndexReader multiReader1 = new MultiReader(new IndexReader[] {reader1, 
IndexReader.open(index2)});
... // modify index2
IndexReader multiReader2 = multiReader1.reopen();  
// only index2 changed, so multiReader2 uses reader1 and has to increment the 
refcount of reader1
{code}
The user gets a new reader instance from multiReader.reopen(), but can't tell 
which of the subreaders has been reopened and which are shared. That's why 
multiReader1.close() should not close reader1 in this case and we need 
refcounting in order to make this work.

So do you suggest that a MultiReader should increment the refcounts when it is 
opened as well (in the constructor)? I believe we can implement it like this, 
but as I said it changes the semantics of MultiReader.close() (and 
ParallelReader.close() is, I believe, the same). A user would then have to 
close subreaders manually.



> IndexReader.reopen()
> 
>
> Key: LUCENE-743
> URL: https://issues.apache.org/jira/browse/LUCENE-743
> Project: Lucene - Java
>  Issue Type: Improvement
>  Components: Index
>Reporter: Otis Gospodnetic
>Assignee: Michael Busch
>Priority: Minor
> Fix For: 2.3
>
> Attachments: IndexReaderUtils.java, lucene-743-take2.patch, 
> lucene-743.patch, lucene-743.patch, lucene-743.patch, MyMultiReader.java, 
> MySegmentReader.java, varient-no-isCloneSupported.BROKEN.patch
>
>
> This is Robert Engels' implementation of IndexReader.reopen() functionality, 
> as a set of 3 new classes (this was easier for him to implement, but should 
> probably be folded into the core, if this looks good).

-- 
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: [EMAIL PROTECTED]
For additional commands, e-mail: [EMAIL PROTECTED]



[jira] Issue Comment Edited: (LUCENE-743) IndexReader.reopen()

2007-10-20 Thread Michael Busch (JIRA)

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

michaelbusch edited comment on LUCENE-743 at 10/20/07 3:11 AM:


Hi Mike,

I'm not sure if I fully understand your comment. Consider the following (quite 
constructed) example:

{code:java}
IndexReader reader1 = IndexReader.open(index1);  // optimized index, reader1 is 
a SegmentReader
IndexReader multiReader1 = new MultiReader(new IndexReader[] {reader1, 
IndexReader.open(index2)});
... // modify index2
IndexReader multiReader2 = multiReader1.reopen();  
// only index2 changed, so multiReader2 uses reader1 and has to increment the 
refcount of reader1
... // modify index1
IndexReader reader2 = reader1.reopen();
// reader2 is a new instance of SegmentReader that shares resources with reader1
... // modify index1
IndexReader reader3 = reader2.reopen();
// reader3 is a new instance of SegmentReader that shares resources with 
reader1 and reader2
{code}

Now the user closes the readers in this order:
# multiReader1.close();
# multiReader2.close();
# reader2.close();
# reader3.close();

reader1 should be marked as closed after 2., right? Because 
multiReader1.close() and multiReader2.close() have to decrement reader1's 
refcount. But the underlying files have to remain open until after 4., because 
reader2 and reader3 use reader1's resources.

So don't we need 2 refcount values in reader1? One that tells us when the 
reader itself can be marked as closed, and one that tells when the resources 
can be closed? Then multiReader1 and multiReader2 would decrement the first 
refCount, whereas reader2 and reader3 both have to "know" reader1, so that they 
can decrement the second refcount.

I hope I'm just completely confused now and someone tells me that the whole 
thing is much simpler :-)



  was (Author: michaelbusch):
Hi Mike,

I'm not sure if I fully understand your comment. Consider the following (quite 
constructed) example:

{code:java}
IndexReader reader1 = IndexReader.open(index1);  // optimized index, reader1 is 
a SegmentReader
IndexReader multiReader1 = new MultiReader(new IndexReader[] {reader1, 
IndexReader.open(index2)});
... // modify index2
IndexReader multiReader2 = multiReader1.reopen();  
// only index2 changed, so multiReader2 uses reader1 and has to increment the 
refcount of reader1
... // modify index1
IndexReader reader2 = reader1.reopen();
// reader2 is a new instance that shares resources with reader1
... // modify index1
IndexReader reader3 = reader2.reopen();
// reader3 is a new instance that shares resources with reader1 and reader2
{code}

Now the user closes the readers in this order:
# multiReader1.close();
# multiReader2.close();
# reader2.close();
# reader3.close();

reader1 should be marked as closed after 2., right? Because 
multiReader1.close() and multiReader2.close() have to decrement reader1's 
refcount. But the underlying files have to remain open until after 4., because 
reader2 and reader3 use reader1's resources.

So don't we need 2 refcount values in reader1? One that tells us when the 
reader itself can be marked as closed, and one that tells when the resources 
can be closed? Then multiReader1 and multiReader2 would decrement the first 
refCount, whereas reader2 and reader3 both have to "know" reader1, so that they 
can decrement the second refcount.

I hope I'm just completely confused now and someone tells me that the whole 
thing is much simpler :-)


  
> IndexReader.reopen()
> 
>
> Key: LUCENE-743
> URL: https://issues.apache.org/jira/browse/LUCENE-743
> Project: Lucene - Java
>  Issue Type: Improvement
>  Components: Index
>Reporter: Otis Gospodnetic
>Assignee: Michael Busch
>Priority: Minor
> Fix For: 2.3
>
> Attachments: IndexReaderUtils.java, lucene-743-take2.patch, 
> lucene-743.patch, lucene-743.patch, lucene-743.patch, MyMultiReader.java, 
> MySegmentReader.java, varient-no-isCloneSupported.BROKEN.patch
>
>
> This is Robert Engels' implementation of IndexReader.reopen() functionality, 
> as a set of 3 new classes (this was easier for him to implement, but should 
> probably be folded into the core, if this looks good).

-- 
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: [EMAIL PROTECTED]
For additional commands, e-mail: [EMAIL PROTECTED]



[jira] Commented: (LUCENE-743) IndexReader.reopen()

2007-10-20 Thread Michael McCandless (JIRA)

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

Michael McCandless commented on LUCENE-743:
---

{quote}
If you do:
{code:java}
IndexReader reader1 = IndexReader.open(index1);  
IndexReader multiReader1 = new MultiReader(new IndexReader[] {reader1, 
IndexReader.open(index2)});
multiReader1.close();
{code}

then today multiReader1.close() also closes reader1. That's why I consciously 
omitted reader1.close().
{quote}

Ahh, I missed that MultiReader is allowed to close all readers that
were passed into it, when it is closed.  OK, let's leave
reader1.close() out of the example.

It's somewhat "aggressive" of MultiReader/ParallelReader to do that?
If you go and use those same sub-readers in other MultiReaders then
they closing of the first MultiReader will then break the other ones?

I think we are forced to keep this semantics, for backwards
compatibility.  But I don't really think MultiReader/ParallelReader
should actually be this aggressive.  Maybe in the future we can add
ctors for MultiReader/ParallelReader that accept a "doClose" boolean
to turn this off.

Anyway, it's simple to preserve this semantics with reference
counting.  It just means that IndexReader / MultiReader do not incref
the readers they receive, and, when they are done with those readers,
they must call their close(), not decref.  Ie they "borrow the
reference" that was passed in, rather than incref'ing their own
reference, to the child readers.

With that change, plus the change below, your example works fine.

{quote}
Consequently, if you do
{code:java}
IndexReader reader1 = IndexReader.open(index1);  
IndexReader multiReader1 = new MultiReader(new IndexReader[] {reader1, 
IndexReader.open(index2)});
IndexReader multiReader2 = new MultiReader(new IndexReader[] {reader1, 
IndexReader.open(index3)});
multiReader1.close();
{code}
then multiReader2 is not usable anymore, because multiReader1.close() closes 
reader1. But that can be explicitly avoided by the user because it's known that 
multiReader1 and multiReader2 share the same reader.
{quote}

This is why I don't like the semantics we have today -- I don't think
it's right that the multiReader1.close() breaks multiReader2.

{quote}
Now, with the reopen() code:
{code:java}
IndexReader reader1 = IndexReader.open(index1);  // optimized index, reader1 is 
a SegmentReader
IndexReader multiReader1 = new MultiReader(new IndexReader[] {reader1, 
IndexReader.open(index2)});
... // modify index2
IndexReader multiReader2 = multiReader1.reopen();  
// only index2 changed, so multiReader2 uses reader1 and has to increment the 
refcount of reader1
{code}
The user gets a new reader instance from multiReader.reopen(), but can't tell 
which of the subreaders has been reopened and which are shared. That's why 
multiReader1.close() should not close reader1 in this case and we need 
refcounting in order to make this work.
{quote}

Both of these cases are easy to fix with reference counting: we just
have to change ensureOpen() to assert that RC > 0 instead of
closed==false.  Ie, a reader may still be used as long as its RC is
still non-zero.


> IndexReader.reopen()
> 
>
> Key: LUCENE-743
> URL: https://issues.apache.org/jira/browse/LUCENE-743
> Project: Lucene - Java
>  Issue Type: Improvement
>  Components: Index
>Reporter: Otis Gospodnetic
>Assignee: Michael Busch
>Priority: Minor
> Fix For: 2.3
>
> Attachments: IndexReaderUtils.java, lucene-743-take2.patch, 
> lucene-743.patch, lucene-743.patch, lucene-743.patch, MyMultiReader.java, 
> MySegmentReader.java, varient-no-isCloneSupported.BROKEN.patch
>
>
> This is Robert Engels' implementation of IndexReader.reopen() functionality, 
> as a set of 3 new classes (this was easier for him to implement, but should 
> probably be folded into the core, if this looks good).

-- 
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: [EMAIL PROTECTED]
For additional commands, e-mail: [EMAIL PROTECTED]



[jira] Commented: (LUCENE-743) IndexReader.reopen()

2007-10-20 Thread Michael McCandless (JIRA)

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

Michael McCandless commented on LUCENE-743:
---

{quote}
I think we are forced to keep this semantics, for backwards
compatibility.  But I don't really think MultiReader/ParallelReader
should actually be this aggressive.  Maybe in the future we can add
ctors for MultiReader/ParallelReader that accept a "doClose" boolean
to turn this off.
{quote}

Actually I retract this: it's no longer necessary as long as we change
ensureOpen to assert that RC > 0 instead of closed==false.

I think this is actually a nice unexpected side-effect of using
reference counting: it resolves this overly aggressive behavior of
MultiReader/ParallelReader.


> IndexReader.reopen()
> 
>
> Key: LUCENE-743
> URL: https://issues.apache.org/jira/browse/LUCENE-743
> Project: Lucene - Java
>  Issue Type: Improvement
>  Components: Index
>Reporter: Otis Gospodnetic
>Assignee: Michael Busch
>Priority: Minor
> Fix For: 2.3
>
> Attachments: IndexReaderUtils.java, lucene-743-take2.patch, 
> lucene-743.patch, lucene-743.patch, lucene-743.patch, MyMultiReader.java, 
> MySegmentReader.java, varient-no-isCloneSupported.BROKEN.patch
>
>
> This is Robert Engels' implementation of IndexReader.reopen() functionality, 
> as a set of 3 new classes (this was easier for him to implement, but should 
> probably be folded into the core, if this looks good).

-- 
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: [EMAIL PROTECTED]
For additional commands, e-mail: [EMAIL PROTECTED]



Re: Per-document Payloads (was: Re: lucene indexing and merge process)

2007-10-20 Thread Nicolas Lalevée
Le samedi 20 octobre 2007, Michael Busch a écrit :
> John Wang wrote:
> >  I can tried to get some numbers for leading an int[] array vs
> > FieldCache.getInts().
>
> I've had a similar performance problem when I used the FieldCache. The
> loading performance is apparently so slow, because each value is stored
> as a term in the dictionary. For loading the cache it is necessary to
> iterate over all terms for the field in the dictionary. And for each
> term it's posting list is opened to check which documents have that value.
>
> If you store unique docIds, then there are no two documents that share
> the same value. That means, that each value gets its own entry in the
> dictionary and to load each value it is necessary to perform two random
> I/O seeks (one for term lookup + one to open the posting list).
>
> In my app it took for a big index several minutes to fill the cache like
> that.
>
> To speed things up I did essentially what Ning suggested. Now I store
> the values as payloads in the posting list of an artificial term. To
> fill my cache it's only necessary to perform a couple of I/O seeks for
> opening the posting list of the specific term, then it is just a
> sequential scan to load all values. With this approach the time for
> filling the cache went down from minutes to seconds!
>
> Now this approach is already much better than the current field cache
> implementation, but it still can be improved. In fact, we already have a
> mechanism for doing that: the norms. Norms are stored with a fixed size,
> which means both random access and sequential scan are optimal. Norms
> are also cached in memory, and filling that cache is much faster
> compared to the current FieldCache approach.
>
> I was therefore thinking about adding per-document payloads to Lucene
> (we can also call it document-metadata). The API could look like this:
>
> Document d = new Document();
> byte[] uidValue = ...
> d.addMetadata("uid", uidValue);
>
> And on the retrieval side all values could either be loaded into the
> field cache, or, if the index is too big, a new API can be used:
>
> IndexReader reader = IndexReader.open(...);
> DocumentMetadataIterator it = reader.metadataIterator("uid");
>
> where DocumentMetadataIterator is an interface similar to TermDocs:
>
> interface DocumentMetadataIterator {
>   void seek(String name);
>   boolean next();
>   boolean skipTo(int doc);
>
>   int doc();
>   byte[] getMetadata();
> }
>
> The next question would be how to store the per-doc payloads (PDP). If
> all values have the same length (as the unique docIds), then we should
> store them as efficiently as possible, like the norms. However, we still
> want to offer the flexibility of having variable-length values. For this
> case we could use a new data structure similar to our posting list.
>
> PDPList   --> FixedLengthPDPList |  SkipList>
> FixedLengthPDPList--> ^SegSize
> VariableLengthPDPList --> 
> Payload   --> Byte^PayloadLength
> PayloadLength --> VInt
> SkipList  --> see frq.file
>
> Because we don't have global field semantics Lucene should automatically
> pick the "right" data structure. This could work like this: When the
> DocumentsWriter writes a segment it checks whether all values of a PDP
> have the same length. If yes, it stores them as FixedLengthPDPList, if
> not, then as VariableLengthPDPList.
> When the SegmentMerger merges two or more segments it checks if all
> segments have a FixedLengthPDPList with the same length for a PDP. If
> not, it writes a VariableLengthPDPList to the new segment.
>
> I think this would be a nice new feature for Lucene. We could then have
> user-defined and Lucene-specific PDPs. For example, norms would be in
> the latter category (this way we would get rid of the special code for
> norms, as they could be handled as PDPs). It would also be easy to add
> new features in the future, like splitting the norms into two values: a
> norm and a boost value.
>
> OK lot's of thoughts, I'm sure I'll get lot's of comments too ... ;)

lot's thoughts, that makes me think of LUCENE-662 ;)

Nicolas

>
> - Michael
>
> > Thanks
> >
> > -John
> >
> > On 10/19/07, Michael McCandless <[EMAIL PROTECTED]> wrote:
> >> It seems like there are (at least) two angles here for getting better
> >> performance from FieldCache:
> >>
> >>   1) Be incremental: with reopen() we should only have to update a
> >>  subset of the array in the FieldCache, according to the changed
> >>  segments.  This is what Hoss is working on and Mark was referring
> >>  to and I think it's very important!
> >>
> >>   2) Parsing is slow (?): I'm guessing one of the reasons that John
> >>  added the _X.udt file was because it's much faster to load an
> >>  array of already-parsed ints than to ask FieldCache to populate
> >>  itself.
> >>
> >> Even if we do #1, I think #2 could be a big win (in addition)?  John
> >> do you have any numbers of how much faster it

Re: lucene indexing and merge process

2007-10-20 Thread Grant Ingersoll

John,

For case 1, can you describe your document structure?  Do you have a  
lot of other fields besides the UID field?  Most importantly, do you  
have some large fields?


Did you give the FieldSelector mechanism a try?

In fact, I think you may even be able to create a caching  
FieldSelector implementation.  We could a add a FieldSelectorResult,  
something like LOAD_AND_CACHE that then caches the info for that Doc,  
Field combination.  Would have to investigate further, but it seems  
like it might work.


Just thinking out loud...

-Grant

On Oct 18, 2007, at 10:38 AM, Erik Hatcher wrote:

Forwarding this to java-dev per request.  Seems like the best place  
to discuss this topic.


Erik


Begin forwarded message:


From: "John Wang" <[EMAIL PROTECTED]>
Date: October 17, 2007 5:43:29 PM EDT
To: [EMAIL PROTECTED]
Subject: lucene indexing and merge process

Hi Erik:

We are revamping our search system here at LinekdIn. And we  
are using Lucene.


One issue we ran across is that we store an UID in Lucene  
which we map to the DB storage. So given a docid, to lookup its  
UID, we have the following solutions:


1) Index it as a Stored field and get it from reader.document  
(very slow if recall is large)
2) Load/Warmup the FieldCache (for large corpus, loading up the  
indexreader can be slow)
3) construct it using the FieldCache and persist it on disk  
everytime the index changes. (not suitable for real time indexing,  
e.g. this process will degrade as # of documents get large)


None of the above solutions turn out to be adequate for our  
requirements.


 What we end up doing is to modify Lucene code by changing  
SegmentReader,DocumentWriter,and FieldWriter classes by taking  
advantage of the Lucene Segment/merge process. E.g:


 For each segment, we store a .udt file, which is an int[]  
array, (by changing the FieldWriter class)


 And SegmentReader will load the .udt file into an array.

 And merge happens seemlessly.

 Because the tight encapsulation around these classes, e.g.  
private and final methods, it is very difficult to extend Lucene  
while avoiding branch into our own version. Is there a way we can  
open up and make these classes extensible? We'd be happy to  
contribute what we have done.


 I guess to tackle the problem from a different angle: is  
there a way to incorporate FieldCache into the segments (it is  
strictly in memory now), and build disk versions while indexing.



 Hope I am making sense.

I did not send this out to the mailing list because I wasn't  
sure if this is a dev question or an user question, feel free to  
either forward it to the right mailing list or let me know and I  
can forward it.



Thanks

-John




-
To unsubscribe, e-mail: [EMAIL PROTECTED]
For additional commands, e-mail: [EMAIL PROTECTED]



--
Grant Ingersoll
http://www.grantingersoll.com/
http://lucene.grantingersoll.com
http://www.paperoftheweek.com/



-
To unsubscribe, e-mail: [EMAIL PROTECTED]
For additional commands, e-mail: [EMAIL PROTECTED]



Re: Per-document Payloads (was: Re: lucene indexing and merge process)

2007-10-20 Thread Grant Ingersoll


On Oct 19, 2007, at 6:53 PM, Michael Busch wrote:


John Wang wrote:


 I can tried to get some numbers for leading an int[] array vs
FieldCache.getInts().


I've had a similar performance problem when I used the FieldCache. The
loading performance is apparently so slow, because each value is  
stored

as a term in the dictionary. For loading the cache it is necessary to
iterate over all terms for the field in the dictionary. And for each
term it's posting list is opened to check which documents have that  
value.


If you store unique docIds, then there are no two documents that share
the same value. That means, that each value gets its own entry in the
dictionary and to load each value it is necessary to perform two  
random

I/O seeks (one for term lookup + one to open the posting list).

In my app it took for a big index several minutes to fill the cache  
like

that.

To speed things up I did essentially what Ning suggested. Now I store
the values as payloads in the posting list of an artificial term. To
fill my cache it's only necessary to perform a couple of I/O seeks for
opening the posting list of the specific term, then it is just a
sequential scan to load all values. With this approach the time for
filling the cache went down from minutes to seconds!

Now this approach is already much better than the current field cache
implementation, but it still can be improved. In fact, we already  
have a
mechanism for doing that: the norms. Norms are stored with a fixed  
size,

which means both random access and sequential scan are optimal. Norms
are also cached in memory, and filling that cache is much faster
compared to the current FieldCache approach.

I was therefore thinking about adding per-document payloads to Lucene
(we can also call it document-metadata). The API could look like this:

Document d = new Document();
byte[] uidValue = ...
d.addMetadata("uid", uidValue);

And on the retrieval side all values could either be loaded into the
field cache, or, if the index is too big, a new API can be used:

IndexReader reader = IndexReader.open(...);
DocumentMetadataIterator it = reader.metadataIterator("uid");

where DocumentMetadataIterator is an interface similar to TermDocs:

interface DocumentMetadataIterator {
  void seek(String name);
  boolean next();
  boolean skipTo(int doc);

  int doc();
  byte[] getMetadata();
}

The next question would be how to store the per-doc payloads (PDP). If
all values have the same length (as the unique docIds), then we should
store them as efficiently as possible, like the norms. However, we  
still
want to offer the flexibility of having variable-length values. For  
this

case we could use a new data structure similar to our posting list.

PDPList   --> FixedLengthPDPList | 
FixedLengthPDPList--> ^SegSize
VariableLengthPDPList --> 
Payload   --> Byte^PayloadLength
PayloadLength --> VInt
SkipList  --> see frq.file

Because we don't have global field semantics Lucene should  
automatically

pick the "right" data structure. This could work like this: When the
DocumentsWriter writes a segment it checks whether all values of a PDP
have the same length. If yes, it stores them as FixedLengthPDPList, if
not, then as VariableLengthPDPList.
When the SegmentMerger merges two or more segments it checks if all
segments have a FixedLengthPDPList with the same length for a PDP. If
not, it writes a VariableLengthPDPList to the new segment.

I think this would be a nice new feature for Lucene. We could then  
have

user-defined and Lucene-specific PDPs. For example, norms would be in
the latter category (this way we would get rid of the special code for
norms, as they could be handled as PDPs). It would also be easy to add
new features in the future, like splitting the norms into two  
values: a

norm and a boost value.


Some randomly pieced together thoughts (I may not even be fully awake  
yet :-)  so feel free to tell me I'm not understanding this correctly)


My first thought was how is this different from just having a binary  
field, but if I understand correctly it is to be stored in a separate  
file?


Now you are proposing a faster storage mechanism for them,  
essentially, since they are to be stored separately from the  
Documents themselves?   But the other key is they are all stored next  
to each other, right, so the scan is a lot faster?


I think one of the questions that will come up from users is when  
should I use addMetadata and when should I use addField?  Why make  
the distinction to the user?  Fields have always represented  
metadata, all your doing is optimizing the internal storage of them.   
So from an interface side of things, I would just make it a new Field  
type.  Essentially what we are doing is creating a two level document  
store, right?  First level contains all of the small metadata that is  
likely to be accessed on every hit, second level contains all of the  
non-essential fields,

Re: Per-document Payloads (was: Re: lucene indexing and merge process)

2007-10-20 Thread Yonik Seeley
On 10/20/07, Grant Ingersoll <[EMAIL PROTECTED]> wrote:
> I think one of the questions that will come up from users is when
> should I use addMetadata and when should I use addField?  Why make
> the distinction to the user?  Fields have always represented
> metadata, all your doing is optimizing the internal storage of them.
> So from an interface side of things, I would just make it a new Field
> type.

Same thing occured to me...
Fieldable.isStoredSeparately()?

I wouldn't mind this byte[] access to any type of field stored
separately (non binary fields too).  What about switching from char
counts to byte counts for indexed (String) fields that are stored
separately?

I guess fields that were stored separately would not be returned
unless asked for by name?

-Yonik

-
To unsubscribe, e-mail: [EMAIL PROTECTED]
For additional commands, e-mail: [EMAIL PROTECTED]



Re: Per-document Payloads (was: Re: lucene indexing and merge process)

2007-10-20 Thread Yonik Seeley
On 10/20/07, Yonik Seeley <[EMAIL PROTECTED]> wrote:
> What about switching from char
> counts to byte counts for indexed (String) fields that are stored
> separately?

In fact, what about switching to byte counts for all stored fields?
It should be much easier than the full-blown byte-counts for the term
index since it only involves stored fields.  It should make skipping
fields (lazy field loading) much faster too.

-Yonik

-
To unsubscribe, e-mail: [EMAIL PROTECTED]
For additional commands, e-mail: [EMAIL PROTECTED]



Re: Per-document Payloads (was: Re: lucene indexing and merge process)

2007-10-20 Thread Grant Ingersoll
https://issues.apache.org/jira/browse/LUCENE-510  is related, then, I  
presume


On Oct 20, 2007, at 11:09 AM, Yonik Seeley wrote:


On 10/20/07, Yonik Seeley <[EMAIL PROTECTED]> wrote:

What about switching from char
counts to byte counts for indexed (String) fields that are stored
separately?


In fact, what about switching to byte counts for all stored fields?
It should be much easier than the full-blown byte-counts for the term
index since it only involves stored fields.  It should make skipping
fields (lazy field loading) much faster too.

-Yonik

-
To unsubscribe, e-mail: [EMAIL PROTECTED]
For additional commands, e-mail: [EMAIL PROTECTED]





-
To unsubscribe, e-mail: [EMAIL PROTECTED]
For additional commands, e-mail: [EMAIL PROTECTED]



Re: Per-document Payloads (was: Re: lucene indexing and merge process)

2007-10-20 Thread Grant Ingersoll


On Oct 20, 2007, at 10:51 AM, Yonik Seeley wrote:


On 10/20/07, Grant Ingersoll <[EMAIL PROTECTED]> wrote:

I think one of the questions that will come up from users is when
should I use addMetadata and when should I use addField?  Why make
the distinction to the user?  Fields have always represented
metadata, all your doing is optimizing the internal storage of them.
So from an interface side of things, I would just make it a new Field
type.


Same thing occured to me...
Fieldable.isStoredSeparately()?

I wouldn't mind this byte[] access to any type of field stored
separately (non binary fields too).  What about switching from char
counts to byte counts for indexed (String) fields that are stored
separately?

I guess fields that were stored separately would not be returned
unless asked for by name?


Right, I would think the typical use case would be you want all the  
"small" fields to be returned w/ the document and the large fields to  
be lazily loaded.  I think it should be seamless to the user.   
Perhaps we could have a threshold value upon indexing, such that all  
fields below are determined to be small, and all above are large,  
then at retrieval time we just compare the byte count to the  
threshold and lazy load the large fields.


Just a thought.  There are probably several ways this could be handled.

-Grant

-
To unsubscribe, e-mail: [EMAIL PROTECTED]
For additional commands, e-mail: [EMAIL PROTECTED]



Re: Per-document Payloads

2007-10-20 Thread Michael Busch
Grant Ingersoll wrote:
> 
> Some randomly pieced together thoughts (I may not even be fully awake
> yet :-)  so feel free to tell me I'm not understanding this correctly)
> 
> My first thought was how is this different from just having a binary
> field, but if I understand correctly it is to be stored in a separate file?
> 
> Now you are proposing a faster storage mechanism for them, essentially,
> since they are to be stored separately from the Documents themselves?  
> But the other key is they are all stored next to each other, right, so
> the scan is a lot faster?
> 

Yes, scanning and skipping would be much faster, comparable to a posting
list. In fact, what I'm proposing is a new kind of posting list. Since
you mentioned the magic term "flexible indexing" already ;), let's take
a look at http://wiki.apache.org/lucene-java/FlexibleIndexing. Here 4
kinds of posting lists are proposed:

a. +

b. +

c. + >+

d. + >+

Today, we have c. and d. already. c. is the original Lucene format, and
d. can be achieved by storing the boost as a payload.

The new format I'm proposing actually covers a. and b. If you don't
store a payload it's basically a binary posting list without freq and
positions (a.). If you store the boost as a payload, then you have b.


> I think one of the questions that will come up from users is when should
> I use addMetadata and when should I use addField?  Why make the
> distinction to the user?  Fields have always represented metadata, all

I'd like to make a distinction because IMO these are two different use
cases. Not necessarily in terms of functionality, but in terms of
performance. You are right, you can store everything today as stored
fields, but if you want to use e. g. a stored value for scoring, then
performance is terrible. This is simply the nature of the store - it is
optimized for returning all stored fields for a document. Even a
FieldSelector doesn't help you too much, unless the docs contain very
big fields that you don't want to return. The reason is that two random
I/Os are necessary to find the stored fields of a document. Then only
sequential I/O has to be performed. And the overhead of loading e. g.
10KB instead of 2KB is not big, much less than two random I/Os, I believe.

Payloads are also much better in terms of cache utilization. Since they
are stored next to each other, and if accessed frequently (in every
search), then it's very likely that big portions of that posting list
will be in the cache.

So the answer to the question when to use a stored field and when to use
a payload should be: use payloads when you access the data during query
evaluation/scoring, use stored fields when you need the data to
construct a search result from a hit.

> fields, right?  Perhaps in this way, if users were willing to commit to
> fixed length fields for the first level, we could also make field
> updating of these types of fields possible w/o having to reindex?
> 

Yes I was thinking the same. Just like norms.



-
To unsubscribe, e-mail: [EMAIL PROTECTED]
For additional commands, e-mail: [EMAIL PROTECTED]



Re: Per-document Payloads

2007-10-20 Thread Marvin Humphrey


On Oct 20, 2007, at 12:49 PM, Michael Busch wrote:


In fact, what I'm proposing is a new kind of posting list.


http://www.rectangular.com/pipermail/kinosearch/2007-July/001096.html

Marvin Humphrey
Rectangular Research
http://www.rectangular.com/



-
To unsubscribe, e-mail: [EMAIL PROTECTED]
For additional commands, e-mail: [EMAIL PROTECTED]



Re: Per-document Payloads

2007-10-20 Thread Marvin Humphrey


On Oct 19, 2007, at 3:53 PM, Michael Busch wrote:


The next question would be how to store the per-doc payloads (PDP). If
all values have the same length (as the unique docIds), then we should
store them as efficiently as possible, like the norms. However, we  
still
want to offer the flexibility of having variable-length values. For  
this

case we could use a new data structure similar to our posting list.

PDPList   --> FixedLengthPDPList | 
FixedLengthPDPList--> ^SegSize
VariableLengthPDPList --> 
Payload   --> Byte^PayloadLength
PayloadLength --> VInt
SkipList  --> see frq.file


There's another approach, which has the following advantages:

  * Simpler.
  * Pluggable.
  * More future proof.
  * More closely models IR Theory.
  * Easier for other implementations to deal with.
  * Breaks the tight binding between Lucene and its file format.

Start with a Posting base class.

  public class Posting {
private int docNum;
private int lastDocNum = 0;

public int getDocNum { return docNum; }

public void read(IndexInput inStream) {
  docNum += inStream.readVInt();
}

public void write(IndexOutput outStream) {
  outStream.writeVInt(docNum - lastDocNum);
}
  }

Then, PostingList (subclassed by SegPostingList and MultiPostingList,  
naturally).


  public abstract class PostingList {
 public abstract Posting getPosting();
 public abstract boolean next() throws IOException;
 public boolean skipTo(int target) throws IOException;
  }

Each field gets its own "postings" file within the segment, named  
_SEGNUM_FIELDNUM.p, where SEGNUM and FIELDNUM are encoded using base  
36.  Each of these files is a solid stack of serialized Postings.


Posting subclasses like ScorePosting, PayloadPosting, etc, implement  
their own read() and write() methods.  Thus, Posting subclasses  
wholly define their own file format -- instead of the current,  
brittle design, where read/write code is dispersed over multiple  
classes.  If some Posting types become obsolete, they can be  
deprecated, but PostingList and its subclasses won't require the  
addition of crufty special case code to stay back-compatible.


There's more (I've written a working implementation), but that's the  
gist.


Marvin Humphrey
Rectangular Research
http://www.rectangular.com/



-
To unsubscribe, e-mail: [EMAIL PROTECTED]
For additional commands, e-mail: [EMAIL PROTECTED]