Sunday, January 8, 2012

Searching relational content with Lucene's BlockJoinQuery

Lucene's 3.4.0 release adds a new feature called index-time join (also sometimes called sub-documents, nested documents or parent/child documents), enabling efficient indexing and searching of certain types of relational content.

Most search engines can't directly index relational content, as documents in the index logically behave like a single flat database table. Yet, relational content is everywhere! A job listing site has each company joined to the specific listings for that company. Each resume might have separate list of skills, education and past work experience. A music search engine has an artist/band joined to albums and then joined to songs. A source code search engine would have projects joined to modules and then files.

Perhaps the PDF documents you need to search are immense, so you break them up and index each section as a separate Lucene document; in this case you'll have common fields (title, abstract, author, date published, etc.) for the overall document, joined to the sub-document (section) with its own fields (text, page number, etc.). XML documents typically contain nested tags, representing joined sub-documents; emails have attachments; office documents can embed other documents. Nearly all search domains have some form of relational content, often requiring more than one join.

If such content is so common then how do search applications handle it today?

One obvious "solution" is to simply use a relational database instead of a search engine! If relevance scores are less important and you need to do substantial joining, grouping, sorting, etc., then using a database could be best overall. Most databases include some form a text search, some even using Lucene.

If you still want to use a search engine, then one common approach is to denormalize the content up front, at index-time, by joining all tables and indexing the resulting rows, duplicating content in the process. For example, you'd index each song as a Lucene document, copying over all fields from the song's joined album and artist/band. This works correctly, but can be horribly wasteful as you are indexing identical fields, possibly including large text fields, over and over.

Another approach is to do the join yourself, outside of Lucene, by indexing songs, albums and artist/band as separate Lucene documents, perhaps even in separate indices. At search-time, you first run a query against one collection, for example the songs. Then you iterate through all hits, gathering up (joining) the full set of corresponding albums and then run a second query against the albums, with a large OR'd list of the albums from the first query, repeating this process if you need to join to artist/band as well. This approach will also work, but doesn't scale well as you may have to create possibly immense follow-on queries.

Yet another approach is to use a software package that has already implemented one of these approaches for you! elasticsearch, Apache Solr, Apache Jackrabbit, Hibernate Search and many others all handle relational content in some way.

With BlockJoinQuery you can now directly search relational content yourself!

Let's work through a simple example: imagine you sell shirts online. Each shirt has certain common fields such as name, description, fabric, price, etc. For each shirt you have a number of separate stock keeping units or SKUs, which have their own fields like size, color, inventory count, etc. The SKUs are what you actually sell, and what you must stock, because when someone buys a shirt they buy a specific SKU (size and color).

Maybe you are lucky enough to sell the incredible Mountain Three-wolf Moon Short Sleeve Tee, with these SKUs (size, color):
  • small, blue
  • small, black
  • medium, black
  • large, gray
Perhaps a user first searches for "wolf shirt", gets a bunch of hits, and then drills down on a particular size and color, resulting in this query:
   name:wolf AND size=small AND color=blue
which should match this shirt. name is a shirt field while the size and color are SKU fields.

But if the user drills down instead on a small gray shirt:
   name:wolf AND size=small AND color=gray
then this shirt should not match because the small size only comes in blue and black.

How can you run these queries using BlockJoinQuery? Start by indexing each shirt (parent) and all of its SKUs (children) as separate documents, using the new IndexWriter.addDocuments API to add one shirt and all of its SKUs as a single document block. This method atomically adds a block of documents into a single segment as adjacent document IDs, which BlockJoinQuery relies on. You should also add a marker field to each shirt document (e.g. type = shirt), as BlockJoinQuery requires a Filter identifying the parent documents.

To run a BlockJoinQuery at search-time, you'll first need to create the parent filter, matching only shirts. Note that the filter must use FixedBitSet under the hood, like CachingWrapperFilter:
  Filter shirts = new CachingWrapperFilter(
                    new QueryWrapperFilter(
                      new TermQuery(
                        new Term("type", "shirt"))));
Create this filter once, up front and re-use it any time you need to perform this join.

Then, for each query that requires a join, because it involves both SKU and shirt fields, start with the child query matching only SKU fields:
  BooleanQuery skuQuery = new BooleanQuery();
  skuQuery.add(new TermQuery(new Term("size", "small")), Occur.MUST);
  skuQuery.add(new TermQuery(new Term("color", "blue")), Occur.MUST);
Next, use BlockJoinQuery to translate hits from the SKU document space up to the shirt document space:
  BlockJoinQuery skuJoinQuery = new BlockJoinQuery(
    skuQuery, 
    shirts,
    ScoreMode.None);
The ScoreMode enum decides how scores for multiple SKU hits should be aggregated to the score for the corresponding shirt hit. In this query you don't need scores from the SKU matches, but if you did you can aggregate with Avg, Max or Total instead.

Finally you are now free to build up an arbitrary shirt query using skuJoinQuery as a clause:
  BooleanQuery query = new BooleanQuery();
  query.add(new TermQuery(new Term("name", "wolf")), Occur.MUST);
  query.add(skuJoinQuery, Occur.MUST);
You could also just run skuJoinQuery as-is if the query doesn't have any shirt fields.

Finally, just run this query like normal! The returned hits will be only shirt documents; if you'd also like to see which SKUs matched for each shirt, use BlockJoinCollector:
  BlockJoinCollector c = new BlockJoinCollector(
    Sort.RELEVANCE, // sort
    10,             // numHits
    true,           // trackScores
    false           // trackMaxScore
    );
  searcher.search(query, c);
The provided Sort must use only shirt fields (you cannot sort by any SKU fields). When each hit (a shirt) is competitive, this collector will also record all SKUs that matched for that shirt, which you can retrieve like this:
  TopGroups hits = c.getTopGroups(
    skuJoinQuery,
    skuSort,
    0,   // offset
    10,  // maxDocsPerGroup
    0,   // withinGroupOffset
    true // fillSortFields
  );
Set skuSort to the sort order for the SKUs within each shirt. The first offset hits are skipped (use this for paging through shirt hits). Under each shirt, at most maxDocsPerGroup SKUs will be returned. Use withinGroupOffset if you want to page within the SKUs. If fillSortFields is true then each SKU hit will have values for the fields from skuSort.

The hits returned by BlockJoinCollector.getTopGroups are SKU hits, grouped by shirt. You'd get the exact same results if you had denormalized up-front and then used grouping to group results by shirt.

You can also do more than one join in a single query; the joins can be nested (parent to child to grandchild) or parallel (parent to child1 and parent to child2).

However, there are some important limitations of index-time joins:
  • The join must be computed at index-time and "compiled" into the index, in that all joined child documents must be indexed along with the parent document, as a single document block.

  • Different document types (for example, shirts and SKUs) must share a single index, which is wasteful as it means non-sparse data structures like FieldCache entries consume more memory than they would if you had separate indices.

  • If you need to re-index a parent document or any of its child documents, or delete or add a child, then the entire block must be re-indexed. This is a big problem in some cases, for example if you index "user reviews" as child documents then whenever a user adds a review you'll have to re-index that shirt as well as all its SKUs and user reviews.

  • There is no QueryParser support, so you need to programmatically create the parent and child queries, separating according to parent and child fields.

  • The join can currently only go in one direction (mapping child docIDs to parent docIDs), but in some cases you need to map parent docIDs to child docIDs. For example, when searching songs, perhaps you want all matching songs sorted by their title. You can't easily do this today because the only way to get song hits is to group by album or band/artist.

  • The join is a one (parent) to many (children), inner join.
As usual, patches are welcome!

There is work underway to create a more flexible, but likely less performant, query-time join capability, which should address a number of the above limitations.

50 comments:

  1. Love the Three-Wolf Moon reference.

    ReplyDelete
  2. Nice post. Could you elaborate on how the shirts are actually indexed? It is hard to understand the query options without knowing the structure of the documents.

    ReplyDelete
  3. Hi Anonymous,

    Each shirt is indexed as a document (with only shirt fields on it), and each SKU for that shirt is also indexed as its own document (with only SKU fields on it). Then you use BlockJoinQuery to "translate" (join) hits from the SKU-document space (a child query) up to the shirt document space.

    ReplyDelete
  4. Hi, your post is helpful.
    But I can't quite understand.
    Could you teach me how to build the nested index?
    I reference your code but it seems like doesn't work.

    ReplyDelete
  5. Hi, can you send an email to the Lucene user's list? (java-user@lucene.apache.org).

    ReplyDelete
  6. Any sense of a real limit on the number of child docs supported? For example, would putting 1,000s of child docs for a parent doc result in a major performance hit (index-time or query-time)? I wasn't sure if the use of the BitSet put a limit on this.

    Thanks! Great post.

    ReplyDelete
  7. Hi Chris,

    The only real limit for a high number of child docs per parent is RAM.

    At indexing time you need to send all those docs (plus the parent) as a single block (though, in 4.0, it's an Iterable, so you could actually stream them).

    At query time, if you're searching in the parent space, during collection, each parent will build an array of the child doc IDs (and scores, optionally), so that'll be 8 byte per child doc for each collected parent. If you're searching in the child space I think there's no added cost for having many children per parent.

    The BitSet should be fine: it's pre-allocated to the max number of docs, and its a single static (i.e. computed once and cached) BitSet for each segment.

    ReplyDelete
  8. Mike,

    What does the schema.xml look like for your example?

    Also, I am using SolrJ to add my documents. Does that support adding blocks like you are referencing? Can it support the Query example using the Filter?

    Sorry if these questions are dumb, but I am new to this space.

    Also, are you available for some hourly consulting so I could give you the details of my use case and you could help us with the best solution?

    Thank you.

    -Dave

    ReplyDelete
  9. Hi Dave,

    There is no schema.xml here (this is all Lucene-only code)...

    Unfortunately Solr can't yet add nor query doc blocks; there is an issue open to do this: https://issues.apache.org/jira/browse/SOLR-3076 but I'm not sure when it'll be done.

    You might want to check out ElasticSearch -- its Nested Type is using BlockJoinQuery under the hood: http://www.elasticsearch.org/guide/reference/mapping/nested-type.html

    ReplyDelete
  10. Hi

    I am new to lucene and have few questions realted to my project.

    I find two approaches to index using lucene for a database.

    1. Denormalize the database and store the entire data with duplicates.
    2. Maintain indexes for different tables and then join the results by searching them individually.

    Is the block join query similar to approach 2???

    In the example stated above shirt document space and sku document space have been referred does that mean they both have different indexes??

    can you provide some detailed explanation to parent and child docment concept??

    I have an oracle db and wondering if storing an index inside may be the most efficient way to go vs storing the index on a file system. has anyone done this before? is it possible with lucene version 3.x?

    ReplyDelete
  11. Hi Sundeep,

    You should email the Lucene user's list w/ questions (java-user@lucene.apache.org).

    Quick answers: BlockJoinQuery uses a single index, with some documents being parents and the rest being children, and, yes, it represents the join from the DB but the join must be "compiled in" at index time. There is now also a more flexible JoinQuery that doesn't require such up-front joining: the join is done instead at query time.

    There is a project to run Lucene inside Oracle (search for "Lucene Domain Index"), but I don't know much about it ...

    ReplyDelete
  12. Hi Mike,

    i have the smiler problem in writing a search for relational data set. so would u be able to give me a sample code of the working scenario. so i will be able to get a exact idea hows the things to be implement(as i am new to lucene this will help).

    ReplyDelete
  13. Hi Anonymous, can you email the Lucene user's list instead?

    ReplyDelete
  14. Hey Mike, i've been looking at your examples. Especially the multiple child one. My question is can I search for job java and qualification maths and just retrieve all found parent documents?
    Which collector do I use for this?

    Thanks

    ReplyDelete
  15. Also is it possible to parse this query directly?

    "+country:Belgium +ToParentBlockJoinQuery (+skill:java +year:[2006 TO 2012]) +ToParentBlockJoinQuery (+qualification:maths +year:[1980 TO 2012])"

    Or does it always need to be build with the BooleanQuery's etc?

    ReplyDelete
  16. Johan,

    Yes, you can retrieve just parent docs: just use any ordinary Collector (or IndexSearcher.search) with your ToParentBlockJoinQuery: the returned docIDs will be only parents.

    You only need to use the BlockJoinCollector if you want to also see all matched children per parent.

    There is no QueryParser support for BJQ yet ... though this issue is working towards adding something: https://issues.apache.org/jira/browse/SOLR-3076

    Until then you'll have to build it up programmatically. Patches welcome!

    ReplyDelete
  17. Hi Mike,

    What would be the best way to delete a a parent doc with its child documents?
    Do I just search the parent document and delete that one, or is there another step I should take.

    ReplyDelete
  18. Hi Johan,

    Deleting just the parent will work (if you are using ToParentBJQ), but to be more efficient you should also delete the child docs for that parent as well, else querying will spend time matching those child docs and joining up to the parent doc only to then notice that parent doc was deleted.

    ReplyDelete
    Replies
    1. Hi Mike,

      Can this be accomplished in one query? Lets say I know the "Id" field of the parent. Do I just do a booleanquery on docType and Id, then delete the found record(s)?

      Delete
    2. Hi Johan, yes deleting by query matching that parent doc should work fine. But be sure to test this! And please report back if it fails :)

      Delete
    3. Deleting the parent document works, but the child docs are still present. And if I search on a field of the parent it won't find anything, but if I search on a field of the child it will find something. Even fields that were stored in the parent document.

      I'm deleting like this:

      Query lQuery = NumericRangeQuery.newIntRange("Id", new java.lang.Integer(searchPerson.Id),
      new java.lang.Integer(searchPerson.Id), true, true);
      _writer.deleteDocuments(lQuery);

      Delete
    4. Hmmm that sounds like a bug, that searching on a field of the child and then joining up to the deleted parent, will return that parent document. Can you boil that down to a small test case and open an issue?

      That query should be fine for deletion, and it sounds like it's clearly succeeding in deleting the parent document...

      Delete
    5. Any work arounds for this? For me it's impossible to index my whole table again. The rest seems to work perfect (searching, indexing, joining, ...).

      Delete
    6. If you also delete the child docs, does it work?

      Delete
    7. Haven't tried yet, maybe it's best if I create a shared property like "personId: the_id" which I place on all my child documents and when I delete search on that field with the person id. I'll try it tomorow and share the result.

      Delete
    8. Ok deleting the child documents works. I added an extra fields (the same one on all my childs) which is named something like PersonId, with the same value in every child. On my delete request I do the following:
      Query lQuery = NumericRangeQuery.newIntRange("PersonId", new java.lang.Integer(searchPerson.Id),
      new java.lang.Integer(searchPerson.Id), true, true);
      _writer.deleteDocuments(lQuery);

      And that cleans it up nicely.

      Delete
    9. Super, I'm glad that worked.

      It's actually possible to make a single Term from your searchPerson.Id integer and then delete by Term instead; it should be faster (to apply the deletes) and use less RAM ... I'm not sure of the details, can you email the user's list (java-user@lucene.apache.org) if you want to explore this?

      Delete
  19. Key for me is that reindexing a child does not lead to a required reindexing of the parent. It would be great if there was a way to keep the elements of the block atomic in that sense.

    ReplyDelete
  20. Hi Marc,

    Unfortunately updating a child document without changing its docID is not easy! Updateable fields might get us that ( https://issues.apache.org/jira/browse/LUCENE-4258 ) ... not sure.

    In the mean-time you can use query time join instead?

    ReplyDelete
  21. Mike,
    I have one obsession about isolated single child update in a block. Let's imagine that we have many small segments in index ( as we recently moved to Lucene 4.0 with fancy concurrent flush).
    - When we add 10 docs blocks, after we inserted 9 children, let's just spin current docnum counter to 10 docs (assuming DocInvertorPerThread has such counter).
    - What we have afterwards: 0-8 docnums are children docs, then a gap - there will no docs with 9...19 and then parent goes with docnum 20.
    - Everything should works as-is on such index with gaps.
    - When we need to update child 5 we can prepare segment with single document, which will substitute doc#5, and then
    - we can merge those two segments in overlapping manner: the new doc will have one of free docNums from the gap.

    how do you think it's feasible?

    ReplyDelete
  22. Hi Mikhail,

    "Reserving" docID gaps could in theory work, but in order to efficiently overlay the updated segment (with doc#5 in your example) I think you'd need something like stacked segments (LUCENE-4258)?

    Even with LUCENE-4258, which will entail a perf hit at search time, it's not clear how often that's a good tradeoff (vs the cost of re-indexing all children + parents for that one doc block)...

    ReplyDelete
  23. Wow, great post!. Do you know if it is possible to get the BlockJoinCollector.getTopGroups semantics using ElasticSearch? We have some really large documents that have several children. If we could return only those children that matched out query, that would save on post processing and io for ES.

    Thanks for the great article!

    ReplyDelete
  24. Hi bobbytech,

    I think ElasticSearch's nested documents already provide this capability? Don't you only get back the children that matched?

    ReplyDelete
  25. Hey Mike,

    Actually, no ES currently doesn't support this. There is an open ticket for it here:

    https://github.com/elasticsearch/elasticsearch/issues/3022

    I'm not sure if there has been any advancements in the new Lucene 4.x line that allows for nested sorting as described there. Do you know if that is currently possible?

    Thanks again!

    ReplyDelete
    Replies
    1. Hi bobbytech,

      That's too bad ElasticSearch doesn't let you get at the specific child hits: this is an important capability of block join. E.g. I use this at http://jirasearch.mikemccandless.com (each comment on an issue is a child doc).

      The API certainly allows for this so I'm not sure why ElasticSearch doesn't return it ...

      There was https://issues.apache.org/jira/browse/LUCENE-4774 which Martijn fixed for Lucene 4.3, which seems relevant? It lets you sort the parents by the largest or smallest value for a given child field for all children under that parent. I would guess that Martijn did this in order to expose it in ElasticSearch ...

      Delete
  26. nice post man
    will save me a few days of code

    ReplyDelete
  27. Hi Mike,

    I'm using ToChildBlockJoinQuery in lucene 4.2.0 and encounter this problem:
    When I have a parent document with no children I get an ArrayIndexOutOfBoundException during search. The cause is in ToChildBlockJoinQuery.java:241

    Debugging this shows that when I have a parent document without children the assert in line 239 doesn't hold and it will keep incrementing the child until it gets the exception
    assert childDoc < parentDoc: "childDoc=" + childDoc + " parentDoc=" + parentDoc;

    To fix this I tried commenting line 223 to 225:

    if (acceptDocs != null && !acceptDocs.get(childDoc)) {
    continue nextChildDoc;
    }

    So far it seems to work for me.
    Is this a bug in lucene? Will my fix break something else?

    ReplyDelete
    Replies
    1. Hi Sally,

      Hmm, maybe first verify this is still a problem on the latest (4.7.0) release? And if so, open a Jira issue (https://issues.apache.org/jira/browse/LUCENE ) with the details? Just commenting out those lines is not right, because they we're not checking whether the document was deleted.

      Delete
    2. Hi Mike,

      Thanks for the reply. I can reproduce this issue with 4.7.0. I'm going to file the bug.

      Delete
  28. Hi Mike,

    I'm trying to do search parent using two child documents. You mentioned in blog that, we can do parallel joins also. I tried to do same using two ToParentBlockJoinQuery for two different child's and ANDed them using boolean query but that wont return any result. Currently I am using lucene 4.10.2 .

    ReplyDelete
  29. Hi Michael,
    I want to search text(columns) from each table which are there in different databases within the same server. For example i want to fetch brand, price, color(Which are there in different databases) for a particular mobile number. We have to implement Lucene search engine in our second hand mobile sale online project. So please help me out with code. Thank you so much.

    ReplyDelete
  30. Hi Michael,

    Does this support custom scoring based on parent and child fields? For example I want to rank the parent documents based on some mathematical function that would use a combination of children and parent fields. Is that supported/possible?

    ReplyDelete
  31. KSV,

    I believe you can only choose between basic aggregations of the children scores up to the parent's score (min, max, avg, total). Better to ask on Lucene's users list for more details: java-user@lucene.apache.org.

    ReplyDelete
  32. Hi Michael,
    i have usecase. there is a two .txt files containing millions of records. there is primarykey & foreign key relationship in data. i want to do inner join & result one single json file. please provide good solution asap. waiting for reply

    ReplyDelete
    Replies
    1. Looks like you asked this on the Lucene user's list and got some replies already!

      Delete
  33. This comment has been removed by the author.

    ReplyDelete
  34. Hi Mike,
    I was looking at the code for ToParentBlockJoinCollector. I am trying to use this feature for a related use case and saw that it uses a FixedBitSet as a parent filter. Why use a bit set at all? Isn't it the same time to navigate a bitset as it is to navigate a posting list? Wouldn't a scorer also achieve the same purpose?

    ReplyDelete