[jira] [Assigned] (LUCENE-5488) FilteredQuery.explain does not honor FilterStrategy
[ https://issues.apache.org/jira/browse/LUCENE-5488?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Busch reassigned LUCENE-5488: - Assignee: Michael Busch FilteredQuery.explain does not honor FilterStrategy --- Key: LUCENE-5488 URL: https://issues.apache.org/jira/browse/LUCENE-5488 Project: Lucene - Core Issue Type: Bug Components: core/search Affects Versions: 4.6.1 Reporter: John Wang Assignee: Michael Busch Attachments: LUCENE-5488.patch, LUCENE-5488.patch Some Filter implementations produce DocIdSets without the iterator() implementation, such as o.a.l.facet.range.Range.getFilter(). It is done with the intention to be used in conjunction with FilteredQuery with FilterStrategy set to be QUERY_FIRST_FILTER_STRATEGY for performance reasons. However, this behavior is not honored by FilteredQuery.explain where docidset.iterator is called regardless and causing such valid usages of above filter types to fail. The fix is to check bits() first and and fall back to iterator if bits is null. In which case, the input Filter is indeed bad. See attached unit test, which fails without this patch. -- This message was sent by Atlassian JIRA (v6.2#6252) - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
Make IndexingChain and friends protected?
Hi All, At Twitter we're using customized IndexingChains and also extend a lot of abstract classes like e.g. TermsHashConsumer. Most of these classes are currently package-private, because they were always considered expert APIs. I was wondering if we could switch from package-private to protected in combination with @lucene.internal? That way extensions and callers of these APIs would not have to be placed in the o.a.l.index package anymore. I'd be happy to work on a patch unless there are concerns about this change. Michael - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
Re: Make IndexingChain and friends protected?
Sounds good. I'll work on a patch. On 3/7/14 1:05 PM, Michael McCandless wrote: +1 Mike McCandless http://blog.mikemccandless.com On Fri, Mar 7, 2014 at 2:39 PM, Robert Muir rcm...@gmail.com wrote: if the change can mostly just expose the indexing chain and related abstract classe so that its properly pluggable, yet passes our documentation-lint task without unravelling the whole thing and making some of the crazier impl stuff public, I think it could be a change for the better overall. On Fri, Mar 7, 2014 at 2:15 PM, Michael Busch busch...@gmail.com wrote: Hi All, At Twitter we're using customized IndexingChains and also extend a lot of abstract classes like e.g. TermsHashConsumer. Most of these classes are currently package-private, because they were always considered expert APIs. I was wondering if we could switch from package-private to protected in combination with @lucene.internal? That way extensions and callers of these APIs would not have to be placed in the o.a.l.index package anymore. I'd be happy to work on a patch unless there are concerns about this change. Michael - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] [Commented] (LUCENE-5488) FilteredQuery.explain does not honor FilterStrategy
[ https://issues.apache.org/jira/browse/LUCENE-5488?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13921429#comment-13921429 ] Michael Busch commented on LUCENE-5488: --- Patch looks good, and all tests pass. Two minor things: - Could you remove the unnecessary changes to the import statements? - Could you create the patch file with svn diff? FilteredQuery.explain does not honor FilterStrategy --- Key: LUCENE-5488 URL: https://issues.apache.org/jira/browse/LUCENE-5488 Project: Lucene - Core Issue Type: Bug Components: core/search Affects Versions: 4.6.1 Reporter: John Wang Attachments: LUCENE-5488.patch Some Filter implementations produce DocIdSets without the iterator() implementation, such as o.a.l.facet.range.Range.getFilter(). It is done with the intention to be used in conjunction with FilteredQuery with FilterStrategy set to be QUERY_FIRST_FILTER_STRATEGY for performance reasons. However, this behavior is not honored by FilteredQuery.explain where docidset.iterator is called regardless and causing such valid usages of above filter types to fail. The fix is to check bits() first and and fall back to iterator if bits is null. In which case, the input Filter is indeed bad. See attached unit test, which fails without this patch. -- This message was sent by Atlassian JIRA (v6.2#6252) - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] [Resolved] (LUCENE-5401) Field.StringTokenStream#end() does not call super.end()
[ https://issues.apache.org/jira/browse/LUCENE-5401?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Busch resolved LUCENE-5401. --- Resolution: Fixed Field.StringTokenStream#end() does not call super.end() --- Key: LUCENE-5401 URL: https://issues.apache.org/jira/browse/LUCENE-5401 Project: Lucene - Core Issue Type: Bug Components: core/other Affects Versions: 4.6 Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: 4.6.1 Attachments: lucene-5401.patch Field.StringTokenStream#end() currently does not call super.end(). This prevents resetting the PositionIncrementAttribute to 0 in end(), which can lead to wrong positions in the index under certain conditions. I added a test to TestDocument which indexes two Fields with the same name, String values, indexed=true, tokenized=false and IndexOptions.DOCS_AND_FREQS_AND_POSITIONS. Without the fix the test fails. The first token gets the correct position 0, but the second token gets position 2 instead of 1. The reason is that in DocInverterPerField line 176 (which is just after the call to end()) we increment the position a second time, because end() didn't reset the increment to 0. All tests pass with the fix. -- This message was sent by Atlassian JIRA (v6.1.5#6160) - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] [Commented] (LUCENE-5401) Field.StringTokenStream#end() does not call super.end()
[ https://issues.apache.org/jira/browse/LUCENE-5401?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13873746#comment-13873746 ] Michael Busch commented on LUCENE-5401: --- Thanks, guys! I backported to 4.6.1 and just committed. (feels good after a looong time :) ) Field.StringTokenStream#end() does not call super.end() --- Key: LUCENE-5401 URL: https://issues.apache.org/jira/browse/LUCENE-5401 Project: Lucene - Core Issue Type: Bug Components: core/other Affects Versions: 4.6 Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: 4.6.1 Attachments: lucene-5401.patch Field.StringTokenStream#end() currently does not call super.end(). This prevents resetting the PositionIncrementAttribute to 0 in end(), which can lead to wrong positions in the index under certain conditions. I added a test to TestDocument which indexes two Fields with the same name, String values, indexed=true, tokenized=false and IndexOptions.DOCS_AND_FREQS_AND_POSITIONS. Without the fix the test fails. The first token gets the correct position 0, but the second token gets position 2 instead of 1. The reason is that in DocInverterPerField line 176 (which is just after the call to end()) we increment the position a second time, because end() didn't reset the increment to 0. All tests pass with the fix. -- This message was sent by Atlassian JIRA (v6.1.5#6160) - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
Re: Lucene / Solr 4.6.1
Yes, I committed LUCENE-5401. Thanks for waiting! On 1/16/14 11:05 AM, Simon Willnauer wrote: seems like we are good to go simon On Thu, Jan 16, 2014 at 1:59 PM, Simon Willnauer simon.willna...@gmail.com wrote: mark, we may wait for https://issues.apache.org/jira/browse/LUCENE-5401 to be committed and merged? simon On Thu, Jan 16, 2014 at 7:34 AM, Mark Miller markrmil...@gmail.com wrote: Whoops - just built this rc with ant 1.9.2 and smoke tester still wants just 1.8. I'll start another build tonight and send the vote thread in the morning. - Mark On Wed, Jan 15, 2014 at 3:14 PM, Simon Willnauer simon.willna...@gmail.com wrote: +1 On Wed, Jan 15, 2014 at 8:02 PM, Mark Miller markrmil...@gmail.com wrote: Unless there is an objection, I’m going to try and make a first RC tonight. - Mark - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org -- - Mark - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] [Created] (LUCENE-5401) Field.StringTokenStream#end() does not call super.end()
Michael Busch created LUCENE-5401: - Summary: Field.StringTokenStream#end() does not call super.end() Key: LUCENE-5401 URL: https://issues.apache.org/jira/browse/LUCENE-5401 Project: Lucene - Core Issue Type: Bug Components: core/other Affects Versions: 4.6 Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: 4.6.1 Field.StringTokenStream#end() currently does not call super.end(). This prevents resetting the PositionIncrementAttribute to 0 in end(), which can lead to wrong positions in the index under certain conditions. I added a test to TestDocument which indexes two Fields with the same name, String values, indexed=true, tokenized=false and IndexOptions.DOCS_AND_FREQS_AND_POSITIONS. Without the fix the test fails. The first token gets the correct position 0, but the second token gets position 2 instead of 1. The reason is that in DocInverterPerField line 176 (which is just after the call to end()) we increment the position a second time, because end() didn't reset the increment to 0. All tests pass with the fix. -- This message was sent by Atlassian JIRA (v6.1.5#6160) - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] [Updated] (LUCENE-5401) Field.StringTokenStream#end() does not call super.end()
[ https://issues.apache.org/jira/browse/LUCENE-5401?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Busch updated LUCENE-5401: -- Attachment: lucene-5401.patch Field.StringTokenStream#end() does not call super.end() --- Key: LUCENE-5401 URL: https://issues.apache.org/jira/browse/LUCENE-5401 Project: Lucene - Core Issue Type: Bug Components: core/other Affects Versions: 4.6 Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: 4.6.1 Attachments: lucene-5401.patch Field.StringTokenStream#end() currently does not call super.end(). This prevents resetting the PositionIncrementAttribute to 0 in end(), which can lead to wrong positions in the index under certain conditions. I added a test to TestDocument which indexes two Fields with the same name, String values, indexed=true, tokenized=false and IndexOptions.DOCS_AND_FREQS_AND_POSITIONS. Without the fix the test fails. The first token gets the correct position 0, but the second token gets position 2 instead of 1. The reason is that in DocInverterPerField line 176 (which is just after the call to end()) we increment the position a second time, because end() didn't reset the increment to 0. All tests pass with the fix. -- This message was sent by Atlassian JIRA (v6.1.5#6160) - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
Re: VOTE: release 4.0 (RC2)
+1 smoketest succeeded on macos 10.7.4. Michael On 10/6/12 1:10 AM, Robert Muir wrote: artifacts here: http://s.apache.org/lusolr40rc2 Thanks for the good inspection of rc#1 and finding bugs, which found test bugs and other bugs! I am happy this was all discovered and sorted out before release. vote stays open until wednesday, the weekend is just extra time for evaluating the RC. - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] [Commented] (LUCENE-3328) Specialize BooleanQuery if all clauses are TermQueries
[ https://issues.apache.org/jira/browse/LUCENE-3328?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13069061#comment-13069061 ] Michael Busch commented on LUCENE-3328: --- {quote} The ConjunctionTermScorer sorts the DocsEnums by their frequency in the ctor. The leader will always be the lowest frequent term in the set. is this what you mean here? {quote} Cool, yeah that's roughly what I meant. In general, it's best to always pick the lowest-df enum as leader: 1) after initialization 2) after a hit was found 3) whenever a doc matched m out of n enums, 1 m n I think what you described covers situation 1), does it also cover 2) and 3)? Specialize BooleanQuery if all clauses are TermQueries -- Key: LUCENE-3328 URL: https://issues.apache.org/jira/browse/LUCENE-3328 Project: Lucene - Java Issue Type: Improvement Components: core/search Affects Versions: 3.4, 4.0 Reporter: Simon Willnauer Fix For: 4.0 Attachments: LUCENE-3328.patch, LUCENE-3328.patch, LUCENE-3328.patch During work on LUCENE-3319 I ran into issues with BooleanQuery compared to PhraseQuery in the exact case. If I disable scoring on PhraseQuery and bypass the position matching, essentially doing a conjunction match, ExactPhraseScorer beats plain boolean scorer by 40% which is a sizeable gain. I converted a ConjunctionScorer to use DocsEnum directly but still didn't get all the 40% from PhraseQuery. Yet, it turned out with further optimizations this gets very close to PhraseQuery. The biggest gain here came from converting the hand crafted loop in ConjunctionScorer#doNext to a for loop which seems to be less confusing to hotspot. In this particular case I think code specialization makes lots of sense since BQ with TQ is by far one of the most common queries. I will upload a patch shortly -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] [Commented] (LUCENE-3328) Specialize BooleanQuery if all clauses are TermQueries
[ https://issues.apache.org/jira/browse/LUCENE-3328?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13068807#comment-13068807 ] Michael Busch commented on LUCENE-3328: --- Nice improvements! I'm wondering if you considered having ConjunctionTermScorer use the terms' IDF values to decide which iterator to advance when all are on the same docID? It should always be best to pick the rarest term. We've talked about doing that in the past, but it's hard to support this for any type of subclause, because you'd have to add the ability to estimate the IDFs of possible subclauses. But with this change it seems very feasible to try for BQs that only have TQ clauses. Specialize BooleanQuery if all clauses are TermQueries -- Key: LUCENE-3328 URL: https://issues.apache.org/jira/browse/LUCENE-3328 Project: Lucene - Java Issue Type: Improvement Components: core/search Affects Versions: 3.4, 4.0 Reporter: Simon Willnauer Fix For: 4.0 Attachments: LUCENE-3328.patch, LUCENE-3328.patch, LUCENE-3328.patch During work on LUCENE-3319 I ran into issues with BooleanQuery compared to PhraseQuery in the exact case. If I disable scoring on PhraseQuery and bypass the position matching, essentially doing a conjunction match, ExactPhraseScorer beats plain boolean scorer by 40% which is a sizeable gain. I converted a ConjunctionScorer to use DocsEnum directly but still didn't get all the 40% from PhraseQuery. Yet, it turned out with further optimizations this gets very close to PhraseQuery. The biggest gain here came from converting the hand crafted loop in ConjunctionScorer#doNext to a for loop which seems to be less confusing to hotspot. In this particular case I think code specialization makes lots of sense since BQ with TQ is by far one of the most common queries. I will upload a patch shortly -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] [Commented] (LUCENE-3023) Land DWPT on trunk
[ https://issues.apache.org/jira/browse/LUCENE-3023?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13027428#comment-13027428 ] Michael Busch commented on LUCENE-3023: --- Just a few minor comments, otherwise +1 to commit! (I'm super excited this is finally happening after the branch was created a year ago or so!) * In DocumentsWriterPerThreadPool: remove: {code} + //public abstract void clearThreadBindings(ThreadState perThread); + + //public abstract void clearAllThreadBindings(); + {code} * In ThreadAffinityDocumentsWriterThreadPool#getAndLock() we had talked about switching from a per-threadstate queue (safeway model) to a single queue (whole foods). I'm wondering if we should do that before we commit or change that later as a separate patch? * remove some commented out code in TestFlushByRamOrCountsPolicy#testHealthyness Land DWPT on trunk -- Key: LUCENE-3023 URL: https://issues.apache.org/jira/browse/LUCENE-3023 Project: Lucene - Java Issue Type: Task Affects Versions: CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Simon Willnauer Fix For: 4.0 Attachments: LUCENE-3023-svn-diff.patch, LUCENE-3023-ws-changes.patch, LUCENE-3023.patch, LUCENE-3023.patch, LUCENE-3023.patch, LUCENE-3023.patch, LUCENE-3023_CHANGES.patch, LUCENE-3023_CHANGES.patch, LUCENE-3023_iw_iwc_jdoc.patch, LUCENE-3023_simonw_review.patch, LUCENE-3023_svndiff.patch, LUCENE-3023_svndiff.patch, diffMccand.py, diffSources.patch, diffSources.patch, realtime-TestAddIndexes-3.txt, realtime-TestAddIndexes-5.txt, realtime-TestIndexWriterExceptions-assert-6.txt, realtime-TestIndexWriterExceptions-npe-1.txt, realtime-TestIndexWriterExceptions-npe-2.txt, realtime-TestIndexWriterExceptions-npe-4.txt, realtime-TestOmitTf-corrupt-0.txt With LUCENE-2956 we have resolved the last remaining issue for LUCENE-2324 so we can proceed landing the DWPT development on trunk soon. I think one of the bigger issues here is to make sure that all JavaDocs for IW etc. are still correct though. I will start going through that first. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] [Commented] (LUCENE-3023) Land DWPT on trunk
[ https://issues.apache.org/jira/browse/LUCENE-3023?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13027058#comment-13027058 ] Michael Busch commented on LUCENE-3023: --- Just wanted to say: you guys totally rock! Great teamwork here with all the work involved of getting the branch merged back. I'm sorry I couldn't help much in the last few weeks. Land DWPT on trunk -- Key: LUCENE-3023 URL: https://issues.apache.org/jira/browse/LUCENE-3023 Project: Lucene - Java Issue Type: Task Affects Versions: CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Simon Willnauer Fix For: 4.0 Attachments: LUCENE-3023-svn-diff.patch, LUCENE-3023-ws-changes.patch, LUCENE-3023.patch, LUCENE-3023.patch, LUCENE-3023.patch, LUCENE-3023.patch, LUCENE-3023_CHANGES.patch, LUCENE-3023_CHANGES.patch, LUCENE-3023_iw_iwc_jdoc.patch, LUCENE-3023_simonw_review.patch, LUCENE-3023_svndiff.patch, LUCENE-3023_svndiff.patch, diffMccand.py, diffSources.patch, diffSources.patch, realtime-TestAddIndexes-3.txt, realtime-TestAddIndexes-5.txt, realtime-TestIndexWriterExceptions-assert-6.txt, realtime-TestIndexWriterExceptions-npe-1.txt, realtime-TestIndexWriterExceptions-npe-2.txt, realtime-TestIndexWriterExceptions-npe-4.txt, realtime-TestOmitTf-corrupt-0.txt With LUCENE-2956 we have resolved the last remaining issue for LUCENE-2324 so we can proceed landing the DWPT development on trunk soon. I think one of the bigger issues here is to make sure that all JavaDocs for IW etc. are still correct though. I will start going through that first. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] [Commented] (LUCENE-2956) Support updateDocument() with DWPTs
[ https://issues.apache.org/jira/browse/LUCENE-2956?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13019019#comment-13019019 ] Michael Busch commented on LUCENE-2956: --- Cool patch! :) Though it worries me a little how complex the whole delete/update logic is becoming (not only the part this patch adds). Originally we decided to not go with sequenceIDs partly because we thought the implementation might be too complex, but I think it'd be simpler than the current approach that uses bits. The sequenceIDs approach we had in the beginning was also completely lockless in a very very simple way. Anyway, I have yet to take a closer look here. Just something that might be worth discussing. Support updateDocument() with DWPTs --- Key: LUCENE-2956 URL: https://issues.apache.org/jira/browse/LUCENE-2956 Project: Lucene - Java Issue Type: Bug Components: Index Affects Versions: Realtime Branch Reporter: Michael Busch Assignee: Simon Willnauer Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2956.patch With separate DocumentsWriterPerThreads (DWPT) it can currently happen that the delete part of an updateDocument() is flushed and committed separately from the corresponding new document. We need to make sure that updateDocument() is always an atomic operation from a IW.commit() and IW.getReader() perspective. See LUCENE-2324 for more details. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] [Commented] (LUCENE-2573) Tiered flushing of DWPTs by RAM with low/high water marks
[ https://issues.apache.org/jira/browse/LUCENE-2573?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13014724#comment-13014724 ] Michael Busch commented on LUCENE-2573: --- Awesome speedup! Finally all this work shows great results!! What's surprising is that the merge time is lower with DWPT. How can that be, considering we're doing more merges? Tiered flushing of DWPTs by RAM with low/high water marks - Key: LUCENE-2573 URL: https://issues.apache.org/jira/browse/LUCENE-2573 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Simon Willnauer Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch Now that we have DocumentsWriterPerThreads we need to track total consumed RAM across all DWPTs. A flushing strategy idea that was discussed in LUCENE-2324 was to use a tiered approach: - Flush the first DWPT at a low water mark (e.g. at 90% of allowed RAM) - Flush all DWPTs at a high water mark (e.g. at 110%) - Use linear steps in between high and low watermark: E.g. when 5 DWPTs are used, flush at 90%, 95%, 100%, 105% and 110%. Should we allow the user to configure the low and high water mark values explicitly using total values (e.g. low water mark at 120MB, high water mark at 140MB)? Or shall we keep for simplicity the single setRAMBufferSizeMB() config method and use something like 90% and 110% for the water marks? -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] [Commented] (LUCENE-2573) Tiered flushing of DWPTs by RAM with low/high water marks
[ https://issues.apache.org/jira/browse/LUCENE-2573?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13014046#comment-13014046 ] Michael Busch commented on LUCENE-2573: --- Thanks, Simon, for running the benchmarks! Good results overall, even though it's puzzling why flushing would be CPU intensive. We should probably do some profiling to figure out where the time is spent. I can probably do that Sunday, but feel free to beat me :) Tiered flushing of DWPTs by RAM with low/high water marks - Key: LUCENE-2573 URL: https://issues.apache.org/jira/browse/LUCENE-2573 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Simon Willnauer Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch Now that we have DocumentsWriterPerThreads we need to track total consumed RAM across all DWPTs. A flushing strategy idea that was discussed in LUCENE-2324 was to use a tiered approach: - Flush the first DWPT at a low water mark (e.g. at 90% of allowed RAM) - Flush all DWPTs at a high water mark (e.g. at 110%) - Use linear steps in between high and low watermark: E.g. when 5 DWPTs are used, flush at 90%, 95%, 100%, 105% and 110%. Should we allow the user to configure the low and high water mark values explicitly using total values (e.g. low water mark at 120MB, high water mark at 140MB)? Or shall we keep for simplicity the single setRAMBufferSizeMB() config method and use something like 90% and 110% for the water marks? -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] [Commented] (LUCENE-2573) Tiered flushing of DWPTs by RAM with low/high water marks
[ https://issues.apache.org/jira/browse/LUCENE-2573?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13013559#comment-13013559 ] Michael Busch commented on LUCENE-2573: --- Thanks Simon! I'll work on LUCENE-2956 next. Tiered flushing of DWPTs by RAM with low/high water marks - Key: LUCENE-2573 URL: https://issues.apache.org/jira/browse/LUCENE-2573 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Simon Willnauer Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch Now that we have DocumentsWriterPerThreads we need to track total consumed RAM across all DWPTs. A flushing strategy idea that was discussed in LUCENE-2324 was to use a tiered approach: - Flush the first DWPT at a low water mark (e.g. at 90% of allowed RAM) - Flush all DWPTs at a high water mark (e.g. at 110%) - Use linear steps in between high and low watermark: E.g. when 5 DWPTs are used, flush at 90%, 95%, 100%, 105% and 110%. Should we allow the user to configure the low and high water mark values explicitly using total values (e.g. low water mark at 120MB, high water mark at 140MB)? Or shall we keep for simplicity the single setRAMBufferSizeMB() config method and use something like 90% and 110% for the water marks? -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Created: (LUCENE-2956) Support updateDocument() with DWPTs
Support updateDocument() with DWPTs --- Key: LUCENE-2956 URL: https://issues.apache.org/jira/browse/LUCENE-2956 Project: Lucene - Java Issue Type: Bug Reporter: Michael Busch Assignee: Michael Busch Priority: Minor With separate DocumentsWriterPerThreads (DWPT) it can currently happen that the delete part of an updateDocument() is flushed and committed separately from the corresponding new document. We need to make sure that updateDocument() is always a atomic operation from a IW.commit() and IW.getReader() perspective. See LUCENE-2324 for more details. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Updated: (LUCENE-2956) Support updateDocument() with DWPTs
[ https://issues.apache.org/jira/browse/LUCENE-2956?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Busch updated LUCENE-2956: -- Component/s: Index Description: With separate DocumentsWriterPerThreads (DWPT) it can currently happen that the delete part of an updateDocument() is flushed and committed separately from the corresponding new document. We need to make sure that updateDocument() is always an atomic operation from a IW.commit() and IW.getReader() perspective. See LUCENE-2324 for more details. was: With separate DocumentsWriterPerThreads (DWPT) it can currently happen that the delete part of an updateDocument() is flushed and committed separately from the corresponding new document. We need to make sure that updateDocument() is always a atomic operation from a IW.commit() and IW.getReader() perspective. See LUCENE-2324 for more details. Affects Version/s: Realtime Branch Fix Version/s: Realtime Branch Support updateDocument() with DWPTs --- Key: LUCENE-2956 URL: https://issues.apache.org/jira/browse/LUCENE-2956 Project: Lucene - Java Issue Type: Bug Components: Index Affects Versions: Realtime Branch Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch With separate DocumentsWriterPerThreads (DWPT) it can currently happen that the delete part of an updateDocument() is flushed and committed separately from the corresponding new document. We need to make sure that updateDocument() is always an atomic operation from a IW.commit() and IW.getReader() perspective. See LUCENE-2324 for more details. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2956) Support updateDocument() with DWPTs
[ https://issues.apache.org/jira/browse/LUCENE-2956?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13004818#comment-13004818 ] Michael Busch commented on LUCENE-2956: --- An idea from Mike how to fix this problem: {quote} To avoid the full-stop, I think during the flush we can have two global delete pools. We carefully sweep all DWPTs and flush each, in succession. Any DWPT not yet flushed is free to continue indexing as normal, putting deletes into the first global pool, flushing as normal. But, a DWPT that has been flushed by the sweeper must instead put deletes for an updateDocument carefully into the 2nd pool, and not buffer the delete into DWPTs not yet flushed. {quote} Support updateDocument() with DWPTs --- Key: LUCENE-2956 URL: https://issues.apache.org/jira/browse/LUCENE-2956 Project: Lucene - Java Issue Type: Bug Components: Index Affects Versions: Realtime Branch Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch With separate DocumentsWriterPerThreads (DWPT) it can currently happen that the delete part of an updateDocument() is flushed and committed separately from the corresponding new document. We need to make sure that updateDocument() is always an atomic operation from a IW.commit() and IW.getReader() perspective. See LUCENE-2324 for more details. -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2573) Tiered flushing of DWPTs by RAM with low/high water marks
[ https://issues.apache.org/jira/browse/LUCENE-2573?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13004291#comment-13004291 ] Michael Busch commented on LUCENE-2573: --- bq. we need to fix LUCENE-2881 first too. Yeah, I haven't merged with trunk since we rolled back 2881, so we should fix it first, catch up with trunk, and then make deletes work. I might have a bit time tonight to work on 2881. Tiered flushing of DWPTs by RAM with low/high water marks - Key: LUCENE-2573 URL: https://issues.apache.org/jira/browse/LUCENE-2573 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Simon Willnauer Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch Now that we have DocumentsWriterPerThreads we need to track total consumed RAM across all DWPTs. A flushing strategy idea that was discussed in LUCENE-2324 was to use a tiered approach: - Flush the first DWPT at a low water mark (e.g. at 90% of allowed RAM) - Flush all DWPTs at a high water mark (e.g. at 110%) - Use linear steps in between high and low watermark: E.g. when 5 DWPTs are used, flush at 90%, 95%, 100%, 105% and 110%. Should we allow the user to configure the low and high water mark values explicitly using total values (e.g. low water mark at 120MB, high water mark at 140MB)? Or shall we keep for simplicity the single setRAMBufferSizeMB() config method and use something like 90% and 110% for the water marks? -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2573) Tiered flushing of DWPTs by RAM with low/high water marks
[ https://issues.apache.org/jira/browse/LUCENE-2573?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13004099#comment-13004099 ] Michael Busch commented on LUCENE-2573: --- bq. Awesome speedup!! YAY! Glad the branch is actually faster :) Thanks for helping out with this patch, Simon. I'll try to look at the patch soon. My last week was super busy. Tiered flushing of DWPTs by RAM with low/high water marks - Key: LUCENE-2573 URL: https://issues.apache.org/jira/browse/LUCENE-2573 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Simon Willnauer Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch, LUCENE-2573.patch Now that we have DocumentsWriterPerThreads we need to track total consumed RAM across all DWPTs. A flushing strategy idea that was discussed in LUCENE-2324 was to use a tiered approach: - Flush the first DWPT at a low water mark (e.g. at 90% of allowed RAM) - Flush all DWPTs at a high water mark (e.g. at 110%) - Use linear steps in between high and low watermark: E.g. when 5 DWPTs are used, flush at 90%, 95%, 100%, 105% and 110%. Should we allow the user to configure the low and high water mark values explicitly using total values (e.g. low water mark at 120MB, high water mark at 140MB)? Or shall we keep for simplicity the single setRAMBufferSizeMB() config method and use something like 90% and 110% for the water marks? -- This message is automatically generated by JIRA. For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13001214#comment-13001214 ] Michael Busch commented on LUCENE-2881: --- bq. maybe, we should store the FieldInfos inside the segments file? Hmmm I had the same thought while adding the ref to FieldInfos to SegmentInfo. Actually this is probably the right thing to do. At the same time we could switch to a human-readable format :) bq. I fear we may not necessarily ever stabilize on a fixed global name/number bimap, because we re-compute this map on every IW init? We could also store the global map on disk? addIndexes() would have to ignore the global map from the external index(es). Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Michael Busch Fix For: Realtime Branch, CSF branch, 4.0 Attachments: LUCENE-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Reopened: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Busch reopened LUCENE-2881: --- Reopening to make the described improvement that ensures consistent field numbers. Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Michael Busch Fix For: Realtime Branch, CSF branch, 4.0 Attachments: lucene-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13000185#comment-13000185 ] Michael Busch commented on LUCENE-2881: --- bq. I don't see any problems in FieldInfo number gaps. this should work just fine and guarantee the bulk copy just for now at least. I was thinking that we probably write field numbers as VInts in a lot of places, and it would therefore be less efficient to have gaps... but this is probably negligible. Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Michael Busch Fix For: Realtime Branch, CSF branch, 4.0 Attachments: lucene-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13000438#comment-13000438 ] Michael Busch commented on LUCENE-2881: --- bq. Can't we sync globally on the assignment of field name - number (the global map lookup)? And FieldInfos per-DWPT would share the same global map. Wouldn't that keep us consistent in the DWPT case? Yes. The global map is already shared across DWPTs and the lookup is synchronized on the global map. I think if we change the logic to always pick the next available global number we would increase the likelihood that fields get bulk-merged. It can't be perfect though, because e.g. if you use addIndexes() to add an external segment that has a different field number assignment. That's why we definitely have to keep the code that can fallback to a different local number if it's not possible to use the global number in a segment. But I agree that we should optimize for the normal indexing case. And it seems like we all agree that field number gaps are fine. Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Michael Busch Fix For: Realtime Branch, CSF branch, 4.0 Attachments: lucene-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13000153#comment-13000153 ] Michael Busch commented on LUCENE-2881: --- I mentioned on dev that assigning the same field number across segments is best effort now and wanted to explain in greater detail here how it works: There is now a global fieldName - fieldNumber bi-map in FieldInfos, which contains all fieldName/number pairs seen in a IndexWriter session. It is passed into each new FieldInfos that is created in the same IndexWriter session. Also, when a new IndexWriter is opened, the FieldInfos of all segments are read and the global map created - this is tested in a new unit test this issue adds. A FieldInfos has in addition to the reference to the global map also a private map, which holds all FieldInfo objects that belong to the corresponding segment (remember there's now a 1-1 mapping SegmentInfo-FieldInfos). Now the fieldNumber assignment strategy works as follows: If a new FI is added to FieldInfos, the global map is checked for the number of that field. If the field name hasn't been seen before, the smallest number available in the *local* map is picked (to keep the numbers dense). Otherwise, if we have seen the field before, the global number is used. The problem now might be, that the global number might already be taken in the local FieldInfos. In this case the global and local numbers for the same fieldName would differ. This is not a problem in terms of correctness, but could prevent that field from being efficiently bulk-merged. With DocumentsWriterPerThreads (DWPTs) in mind I don't see how we could guarantee consistent field numbering across DWPTs, that's why I implemented it in this best effort way. Here's an example on how we can get into a situation where a field would get different numbers in different segments: segment_1 has fields A and B, therefore these mappings A - 1, B - 2. Now in segment_2 the first field we add is C, which hasn't been seen ever before, so we pick locally number 1 for it. Then we add the next document which has field A, but since number 1 is already taken, it would get a different number than in segment_1. This means A would not get bulk merged. Hmm, after writing this example down I'm realizing that it would be better to just always pick the next available global field number for a new field, then, at least until we get DWPTs, we should never get different numbers across segments, I think? The disadvantage would be that FieldInfos could have gaps in the numbers. I implemented the current approach because I wanted to avoid those gaps, but having them would probably not be a big deal? Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Michael Busch Fix For: Realtime Branch, CSF branch, 4.0 Attachments: lucene-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h
Re: [HUDSON] Lucene-Solr-tests-only-trunk - Build # 5336 - Failure
On 2/27/11 2:47 AM, Simon Willnauer wrote: On Sat, Feb 26, 2011 at 11:02 PM, Michael Buschbusch...@gmail.com wrote: Well, after LUCENE-2881 assigning the same fieldNumber to the same fieldName across segments is best effort - not guaranteed anymore. It looks like in most cases it works fine, just very rarely we get different field numbers. I don't see how we can improve the best effort, because I don't think we can assign field numbers on flush (like codecIDs)? Can you elaborate this? I am not sure if I understand what you mean here! Sorry should have explained this better. I actually posted a comment describing what I meant with best effort on LUCENE-2881, so that it's easier to find in the future. Michael - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12999848#comment-12999848 ] Michael Busch commented on LUCENE-2881: --- bq. One question: we seem to have lost DocFieldProcessorPerThread.trimFields? I actually renamed it to doAfterFlush(). It now resets the whole hashmap in DocFieldProcessorPerThread, because we don't want to carry over any field settings into the next segment anymore with per-segment FieldInfos. I think this should be fine? Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Michael Busch Fix For: Realtime Branch, CSF branch, 4.0 Attachments: lucene-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
Re: [HUDSON] Lucene-Solr-tests-only-trunk - Build # 5336 - Failure
Well, after LUCENE-2881 assigning the same fieldNumber to the same fieldName across segments is best effort - not guaranteed anymore. It looks like in most cases it works fine, just very rarely we get different field numbers. I don't see how we can improve the best effort, because I don't think we can assign field numbers on flush (like codecIDs)? So the assert might be too strict now. It'd be good to disable the assert and see if the test still fails. I can't do that right now, but later tonight... Michael On 2/26/11 9:09 AM, Michael McCandless wrote: Alas no I cannot repro so far :( I tried the seed that failed, and I'm running while(1) on beast but no failure so far gonna be a tricky one!! Mike On Sat, Feb 26, 2011 at 8:43 AM, Simon Willnauer simon.willna...@googlemail.com wrote: I am running this now for a while with no failure :( mike can your reproduce? On Sat, Feb 26, 2011 at 2:30 PM, Michael McCandless luc...@mikemccandless.com wrote: Uh-oh! I suspect this failure is from LUCENE-2881 -- the assert that tripped was verifying all merges were done w/ bulk copy. So, somehow, in this test we are failing to consistently assign the same field number to the same field name... Mike On Sat, Feb 26, 2011 at 4:46 AM, Apache Hudson Server hud...@hudson.apache.org wrote: Build: https://hudson.apache.org/hudson/job/Lucene-Solr-tests-only-trunk/5336/ 1 tests failed. REGRESSION: org.apache.lucene.index.TestNRTThreads.testNRTThreads Error Message: null Stack Trace: junit.framework.AssertionFailedError: at org.apache.lucene.util.LuceneTestCase$LuceneTestCaseRunner.runChild(LuceneTestCase.java:1213) at org.apache.lucene.util.LuceneTestCase$LuceneTestCaseRunner.runChild(LuceneTestCase.java:1145) at org.apache.lucene.index.TestNRTThreads.testNRTThreads(TestNRTThreads.java:386) Build Log (for compile errors): [...truncated 3089 lines...] - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org -- Mike http://blog.mikemccandless.com - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12999244#comment-12999244 ] Michael Busch commented on LUCENE-2324: --- {quote} Somehow, we have to let each DWPT have some privacy, but, the field name - number binding should be global. I think Simon is going to open a separate issue to make something possible along these lines... {quote} This is done now (LUCENE-2881) and merged into the RT branch. {quote} The current plan w/ deletes is that a delete gets buffered 1) into the global pool (stored in DW and pushed whenever any DWPT flushes), as well as 2) per DWPT. The per-DWPT pools apply only to the segment flushed from that DWPT, while the global pool applies during coalescing (ie to all prior segments). {quote} I implemented and committed this approach. It's looking pretty good - almost all tests pass. Only TestStressIndexing2 is sometimes failing - but only when updateDocument() is called, not when I modify the test to only use add, delete-by-term and delete-by-query. {quote} To avoid the full-stop, I think during the flush we can have two global delete pools. We carefully sweep all DWPTs and flush each, in succession. Any DWPT not yet flushed is free to continue indexing as normal, putting deletes into the first global pool, flushing as normal. But, a DWPT that has been flushed by the sweeper must instead put deletes for an updateDocument carefully into the 2nd pool, and not buffer the delete into DWPTs not yet flushed. {quote} I haven't done this yet - it might fix the failing test I described. Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324.patch, LUCENE-2324.patch, LUCENE-2324.patch, LUCENE-2324.patch, lucene-2324.patch, lucene-2324.patch, test.out, test.out, test.out, test.out See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12999247#comment-12999247 ] Michael Busch commented on LUCENE-2324: --- {quote} Can anyone gimme a quick statement about what is left here or what the status of this issue is? I am at the point where I need to do some rather big changes to DocValues which I would not need if we have DWPT so I might rather help here before wasting time. {quote} I think it's very close! The new deletes approach is implemented, and various bugs are fixed. Also the latest trunk is merged in (including LUCENE-2881). Outstanding issues are to fix the updateDocument() problems, and finish flush-by-RAM (LUCENE-2573). Other than TestStressIndexing2 and TestNRTThreads (updateDocument problem) and a few tests that rely on flush-by-RAM, all core and contrib tests are passing now. Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324.patch, LUCENE-2324.patch, LUCENE-2324.patch, LUCENE-2324.patch, lucene-2324.patch, lucene-2324.patch, test.out, test.out, test.out, test.out See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
Re: [HUDSON] Lucene-trunk - Build # 1475 - Failure
I was also able to reproduce this failure locally. So the problem only happens for segments that had document(s) with term vectors, but which all hit non-aborting exceptions. In that case the corresponding FieldInfo(s) will have term vectors enabled, but no TV files will have been written due to the non-aborting exception. I committed a fix which clears the TV bits in FieldInfos in case SegmentWriteState.hasVectors is false on segment flush. This fixes the problem. I will probably keep looking if there's a more elegant way to fix this, but for now the build should be fixed. Michael On 2/23/11 2:24 AM, Simon Willnauer wrote: Michael did you try with the reproduce commandline? ant test-core -Dtestcase=TestIndexWriterExceptions -Dtestmethod=testDocumentsWriterExceptions -Dtests.seed=5748357769164696038:5339220614554941881 that one fails all the time on my machine simon On Wed, Feb 23, 2011 at 8:38 AM, Michael Buschbusch...@gmail.com wrote: I just ran this test locally ~15 times and no failure. Weird... I'll keep looking On 2/22/11 11:29 PM, Simon Willnauer wrote: hmm maybe this was caused by LUCENE-2881 but I am not sure. I try to dig this afternoon... simon On Wed, Feb 23, 2011 at 4:11 AM, Apache Hudson Server hud...@hudson.apache.orgwrote: Build: https://hudson.apache.org/hudson/job/Lucene-trunk/1475/ 1 tests failed. REGRESSION: org.apache.lucene.index.TestIndexWriterExceptions.testDocumentsWriterExceptions Error Message: file _1.tvx does not exist Stack Trace: junit.framework.AssertionFailedError: file _1.tvx does not exist at org.apache.lucene.util.LuceneTestCase$LuceneTestCaseRunner.runChild(LuceneTestCase.java:1213) at org.apache.lucene.util.LuceneTestCase$LuceneTestCaseRunner.runChild(LuceneTestCase.java:1145) at org.apache.lucene.index.IndexWriter.filesExist(IndexWriter.java:3413) at org.apache.lucene.index.IndexWriter.startCommit(IndexWriter.java:3467) at org.apache.lucene.index.IndexWriter.prepareCommit(IndexWriter.java:2375) at org.apache.lucene.index.IndexWriter.commitInternal(IndexWriter.java:2446) at org.apache.lucene.index.IndexWriter.closeInternal(IndexWriter.java:1098) at org.apache.lucene.index.IndexWriter.close(IndexWriter.java:1041) at org.apache.lucene.index.IndexWriter.close(IndexWriter.java:1005) at org.apache.lucene.index.TestIndexWriterExceptions.__CLR2_6_3n67xuk1m6p(TestIndexWriterExceptions.java:565) at org.apache.lucene.index.TestIndexWriterExceptions.testDocumentsWriterExceptions(TestIndexWriterExceptions.java:518) Build Log (for compile errors): [...truncated 13598 lines...] - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
Re: [HUDSON] Lucene-trunk - Build # 1475 - Failure
I just ran this test locally ~15 times and no failure. Weird... I'll keep looking On 2/22/11 11:29 PM, Simon Willnauer wrote: hmm maybe this was caused by LUCENE-2881 but I am not sure. I try to dig this afternoon... simon On Wed, Feb 23, 2011 at 4:11 AM, Apache Hudson Server hud...@hudson.apache.org wrote: Build: https://hudson.apache.org/hudson/job/Lucene-trunk/1475/ 1 tests failed. REGRESSION: org.apache.lucene.index.TestIndexWriterExceptions.testDocumentsWriterExceptions Error Message: file _1.tvx does not exist Stack Trace: junit.framework.AssertionFailedError: file _1.tvx does not exist at org.apache.lucene.util.LuceneTestCase$LuceneTestCaseRunner.runChild(LuceneTestCase.java:1213) at org.apache.lucene.util.LuceneTestCase$LuceneTestCaseRunner.runChild(LuceneTestCase.java:1145) at org.apache.lucene.index.IndexWriter.filesExist(IndexWriter.java:3413) at org.apache.lucene.index.IndexWriter.startCommit(IndexWriter.java:3467) at org.apache.lucene.index.IndexWriter.prepareCommit(IndexWriter.java:2375) at org.apache.lucene.index.IndexWriter.commitInternal(IndexWriter.java:2446) at org.apache.lucene.index.IndexWriter.closeInternal(IndexWriter.java:1098) at org.apache.lucene.index.IndexWriter.close(IndexWriter.java:1041) at org.apache.lucene.index.IndexWriter.close(IndexWriter.java:1005) at org.apache.lucene.index.TestIndexWriterExceptions.__CLR2_6_3n67xuk1m6p(TestIndexWriterExceptions.java:565) at org.apache.lucene.index.TestIndexWriterExceptions.testDocumentsWriterExceptions(TestIndexWriterExceptions.java:518) Build Log (for compile errors): [...truncated 13598 lines...] - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Resolved: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Busch resolved LUCENE-2881. --- Resolution: Fixed Committed revision 1073110. Thanks for reviewing the patch, Simon! Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Michael Busch Fix For: Realtime Branch, CSF branch, 4.0 Attachments: lucene-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12997252#comment-12997252 ] Michael Busch commented on LUCENE-2881: --- bq. I think we should initialize the codecID with a different value and replace the this.codecId != 0 check with something like this.codecId != -1. Yeah, I had the same though. I changed it to use -1 and use an assert now instead of throwing the exception. (will post the new patch shortly) bq. What exactly was the problem with the previous patch beside the codecID clone issue? Not sure if that's what caused your codecID issues, but the previous patch had a problem with assigning field numbers. It could happen that a global number for a FieldInfo was acquired, but that number wasn't available anymore in the local FieldInfos. I think this would be quite rare, but now I'm preventing this from happening. Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Michael Busch Fix For: Realtime Branch, CSF branch, 4.0 Attachments: lucene-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Updated: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Busch updated LUCENE-2881: -- Attachment: lucene-2881.patch - Uses -1 now as initial value for codecID. - updated to current trunk Let me know if it works without problems now in the doc values branch, Simon! Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Michael Busch Fix For: Realtime Branch, CSF branch, 4.0 Attachments: lucene-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Updated: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Busch updated LUCENE-2881: -- Attachment: lucene-2881.patch I fixed a bug in FieldInfos that could lead to wrong field numbers, that might have been related to the wrong behavior you're seeing, Simon. About codecIds: I made the fix to FieldInfo.clone() to set the codecId on the clone. I also made FieldInfo.codecId private and added getter and setter. The setter checks whether the new value for codecId is different from the previous one, and throws in exception in that case (unless it was set to the default 0 before, which I think means Preflex codec). All tests pass. Please let me know if that fixes your problem. If not then you should at least see the new exception that I added, which might make debugging easier. Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Michael Busch Fix For: Realtime Branch, CSF branch, 4.0 Attachments: lucene-2881.patch, lucene-2881.patch, lucene-2881.patch, lucene-2881.patch Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12992524#comment-12992524 ] Michael Busch commented on LUCENE-2881: --- Awesome, thanks for letting me know! I hope I'll be able to say the same about the RT branch after I tried it there... :) Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Michael Busch Fix For: Realtime Branch, CSF branch, 4.0 Attachments: lucene-2881.patch, lucene-2881.patch, lucene-2881.patch Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Updated: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Busch updated LUCENE-2881: -- Attachment: lucene-2881.patch * Creates for every segment a new FieldInfos * Changes FieldInfos, so that the FieldInfo numbers within a single FieldInfos don't have to be contiguous - this allows using the same numbering as the previous segment(s), even if not all fields are present in the new segment * Adds a global fieldName - fieldNumber map; if possible when a new field is added to a FieldInfo it tries to use an already assigned number for that field All tests pass. Though I need to verify if the global map works correctly (it'd probably be good to add a test for that). Also it'd be nice to remove hasVectors and hasProx from SegmentInfo, but we could also do that in a separate issue. Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Michael Busch Fix For: Realtime Branch, CSF branch, 4.0 Attachments: lucene-2881.patch Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Updated: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Busch updated LUCENE-2881: -- Attachment: lucene-2881.patch New patch that removes the tracking of 'hasVectors' and 'hasProx' in SegmentInfo. Instead SegmentInfo now has a reference to its corresponding FieldInfos. For backwards-compatibility reasons we can't completely remove the hasVectors and hasProx bytes from the serialized SegmentInfo yet. Eg. if someone uses addIndexes(Directory...) to add external old pre-4.0 segments to a new index, we upgrade the SegmentInfo to the latest version. However, we don't modify the FieldInfos of that segment, instead we just copy it over to the new dir. So the hasVector and hasProx bits in the FieldInfos might not be accurate and we have to keep those bits in the SegmentInfo instead. Not an ideal solution, but we can remove it entirely in Lucene 5.0 :). The alternative would be to rewrite the FieldInfos instead of just copying the files, but then we have to rewrite the cfs files. All core contrib tests pass. Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Michael Busch Fix For: Realtime Branch, CSF branch, 4.0 Attachments: lucene-2881.patch, lucene-2881.patch Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12992175#comment-12992175 ] Michael Busch commented on LUCENE-2881: --- Thanks for reviewing! bq. I think you should commit that patch. I'll port to docvalues and run some tests that rely on this issue. I just want to add another tests for the global fieldname-number map, after that I think it'll be ready to commit. Will do that tonight :) Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Michael Busch Fix For: Realtime Branch, CSF branch, 4.0 Attachments: lucene-2881.patch, lucene-2881.patch Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12992340#comment-12992340 ] Michael Busch commented on LUCENE-2881: --- bq. Maybe we can simply implement IterableFieldInfo? good idea - done. bq. Maybe we can rename SI#clearFilesCache() Actually I renamed it intentionally, because all this method does is really clearing the files cache. SI has a separate reset() method for resetting its state entirely. Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Michael Busch Fix For: Realtime Branch, CSF branch, 4.0 Attachments: lucene-2881.patch, lucene-2881.patch Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Updated: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Busch updated LUCENE-2881: -- Attachment: lucene-2881.patch New patch that adds a new junit for testing that field numbering is consistent across segments. It tests two cases: 1) one IW is used to write two segments; 2) two IWs are used to write two segments. And it also tests that addIndexes(Directory...) doesn't mess up the field numbering of the external segment. All tests pass. I'll commit this in a day or two if nobody objects. Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Michael Busch Fix For: Realtime Branch, CSF branch, 4.0 Attachments: lucene-2881.patch, lucene-2881.patch, lucene-2881.patch Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12985852#action_12985852 ] Michael Busch commented on LUCENE-2881: --- It would probably make sense to have a new class (maybe an extension of SegmentInfo) for in-memory (not-yet-flushed) segments that references the corresponding FieldInfos and SegmentDeletes. That'd be better I think that adding another map SegmentInfo - FieldInfos and we could then also remove the SegmentInfo - SegmentDeletes map (in BufferedDeletes). Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Fix For: Realtime Branch, CSF branch, 4.0 Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12985853#action_12985853 ] Michael Busch commented on LUCENE-2881: --- bq. i think we should do that on trunk and then merge to RT - do you have time to work on this soon? Yeah I agree. Hmm maybe I can spend some hours tonight on this, otherwise I don't think I'll have much time before Thursday. Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Fix For: Realtime Branch, CSF branch, 4.0 Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Assigned: (LUCENE-2881) Track FieldInfo per segment instead of per-IW-session
[ https://issues.apache.org/jira/browse/LUCENE-2881?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Busch reassigned LUCENE-2881: - Assignee: Michael Busch Track FieldInfo per segment instead of per-IW-session - Key: LUCENE-2881 URL: https://issues.apache.org/jira/browse/LUCENE-2881 Project: Lucene - Java Issue Type: Improvement Affects Versions: Realtime Branch, CSF branch, 4.0 Reporter: Simon Willnauer Assignee: Michael Busch Fix For: Realtime Branch, CSF branch, 4.0 Currently FieldInfo is tracked per IW session to guarantee consistent global field-naming / ordering. IW carries FI instances over from previous segments which also carries over field properties like isIndexed etc. While having consistent field ordering per IW session appears to be important due to bulk merging stored fields etc. carrying over other properties might become problematic with Lucene's Codec support. Codecs that rely on consistent properties in FI will fail if FI properties are carried over. The DocValuesCodec (DocValuesBranch) for instance writes files per segment and field (using the field id within the file name). Yet, if a segment has no DocValues indexed in a particular segment but a previous segment in the same IW session had DocValues, FieldInfo#docValues will be true since those values are reused from previous segments. We already work around this limitation in SegmentInfo with properties like hasVectors or hasProx which is really something we should manage per Codec Segment. Ideally FieldInfo would be managed per Segment and Codec such that its properties are valid per segment. It also seems to be necessary to bind FieldInfoS to SegmentInfo logically since its really just per segment metadata. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
Re: Lucene Google Summer of Code 2011
Oh my god, Uwe, I was hoping you would never write a sophisticated™ backwards® compatibility layer again! Michael On 1/24/11 12:39 PM, Uwe Schindler wrote: +1 I also have an idea from the attributes and TokenStream policeman. So I could even help mentoring. Uwe Simon Willnauersimon.willna...@googlemail.com schrieb: hey folks, Google has announce GSoC 2011 lately and mentoring organizations can start submitting applications by the end of feb (http://www.google-melange.com/document/show/gsoc_program/google/gsoc2011/timeline). I wonder if we should participate this year again? I think we have plenty of work to do and its a great opportunity to get fresh blood into the project on both ends Solr Lucene. I already have a couple of tasks / projects in mind though... Thoughts? - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org -- Uwe Schindler H.-H.-Meier-Allee 63, 28213 Bremen http://www.thetaphi.de - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12983613#action_12983613 ] Michael Busch commented on LUCENE-2324: --- So I'm wondering about the following problem with deletes: Suppose we open a new IW on an existing index with 2 segments _1 and _2. IW is set to maxBufferedDocs=1000. The app starts indexing with two threads, so two DWPTs are created. DWPT1 starts working on _3, DWPT2 on _4. Both remember that they must apply their deletes only to segments _1 and _2. After adding 500 docs thread 2 stops indexing for an hour, but thread 1 keeps working. While thread 2 is sleeping several segment flushes (_3, _5, _6, etc) happen. Now thread 2 wakes up again and adds another 500 docs, and also some deletes, so DWPT2 has to flush finally. How can it figure out to which docs the deletes to apply to? _1 and _2 are probably gone a long time ago. If we apply the deletes to all of _3 this would be a mistake too. I'm starting to think there's no way around sequenceIds? Even without RT. Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324.patch, LUCENE-2324.patch, LUCENE-2324.patch, lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch, test.out, test.out, test.out, test.out See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12983246#action_12983246 ] Michael Busch commented on LUCENE-2324: --- {quote} I ran a quick perf test here: I built the 10M Wikipedia index, Standard codec, using 6 threads. Trunk took 541.6 sec; RT took 518.2 sec (only a bit faster), but the test wasn't really fair because it flushed @ docCount=12870. {quote} Thanks for running the tests! Hmm that's a bit disappointing - we were hoping for more speedup. Flushing by docCount is currently per DWPT, so every initial segment in your test had 12870 docs. I guess there's a lot of merging happening. Maybe you could rerun with higher docCount? bq. But I can't test flush by RAM - that's not working yet on RT right? True. I'm going to add that soonish. There's one thread-safety bug related to deletes that needs to be fixed too. {quote} Then I ran a single-threaded test. Trunk took 1097.1 sec and RT took 1040.5 sec - a bit faster! Presumably in the noise (we don't expect a speedup?), but excellent that it's not slower... {quote} Yeah I didn't expect much speedup - cool! :) Maybe because some code is gone, like the WaitQueue, not sure how much overhead that added in the single-threaded case. {quote} I think we lost infoStream output on the details of flushing? I can't see when which DWPTs are flushing... {quote} Oh yeah, good point, I'll add some infoStream messages to DWPT! Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324.patch, LUCENE-2324.patch, LUCENE-2324.patch, lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch, test.out, test.out, test.out, test.out See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
Re: Let's drop Maven Artifacts !
On 1/18/11 9:13 AM, Robert Muir wrote: I can't help but remind myself, this is the same argument Oracle offered up for the whole reason hudson debacle (http://hudson-labs.org/content/whos-driving-thing) Declaring that I have a secret pocket of users that want XYZ isn't open source consensus. Well everyone using ant+ivy or maven as their build system likely consumes artifacts from maven repos. I'm surprised you're so much against keeping to publish. I too really really want to keep ant as Lucene's build tool. Maven has made me suicidal in the past. But I don't want to stop publishing artifacts to commonly used repos. I guess we could try to figure out how many people download the artifacts from m2 repos. Maybe they have download statistics? But then what? What number would justify stopping to publish? Michael - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12983346#action_12983346 ] Michael Busch commented on LUCENE-2324: --- bq. Why does DW.anyDeletions need to be sync'd? Hmm good point. Actually only the call to DW.pendingDeletes.any() needs to be synced, but not the loop that calls the DWPTs. {quote} In ThreadAffinityDWTP... it may be better if we had a single queue, where threads wait in line, if no DWPT is available? And when a DWPT finishes it then notifies any waiting threads? (Ie, instead of queue-per-DWPT). {quote} Whole foods instead of safeway? :) Yeah that would be fairer. A large doc (= a full cart) wouldn't block unlucky other docs. I'll make that change, good idea! {quote} I see the fieldInfos.update(dwpt.getFieldInfos()) (in DW.updateDocument) - is there a risk that two threads bring a new field into existence at the same time, but w/ different config? Eg one doc omitsTFAP and the other doesn't? Or, on flush, does each DWPT use its private FieldInfos to correctly flush the segment? (Hmm: do we seed each DWPT w/ the original FieldInfos created by IW on init?). {quote} Every DWPT has its own private FieldInfos. When a segment is flushed the DWPT uses its private FI and then it updates the original DW.fieldInfos (from IW), which is a synchronized call. The only consumer of DW.getFieldInfos() is SegmentMerger in IW. Hmm, given that IW.flush() isn't synchronized anymore I assume this can lead into a problem? E.g. the SegmentMerger gets a FieldInfos that's newer than the list of segments it's trying to flush? bq. How are we handling the case of open IW, do delete-by-term but no added docs? DW has a SegmentDeletes (pendingDeletes) which gets pushed to the last segment. We only add delTerms to DW.pendingDeletes if we couldn't push it to any DWPT. Btw. I think the whole pushDeletes business isn't working correctly yet, I'm looking into it. I need to understand the code that coalesces the deletes better. bq. In DW.deleteTerms... shouldn't we skip a DWPT if it has no buffered docs? Yeah, I did that already, but not committed yet. Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324.patch, LUCENE-2324.patch, LUCENE-2324.patch, lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch, test.out, test.out, test.out, test.out See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
Re: Let's drop Maven Artifacts !
On 1/18/11 10:44 AM, Mark Miller wrote: From my point of view, but perhaps I misremember: At some point, Grant or someone put in some Maven poms. I did. :) It was a ton of work and especially getting the maven-ant-tasks to work was a nightmare! I don't think anyone else really paid attention. All those patches were attached to a jira issue, and the issue was open for a while, with people asking for published maven artifacts. Later, as we did releases, and saw and dealt with these poms, most of us commented against Maven support. So can you explain what the problem with the maven support is? Isn't it enough to just call the ant target and copying the generated files somewhere? When I did releases I never thought it made the release any harder. Just two additional easy steps. It just feels to me like it slipped in - and really its the type of thing that should have been more discussed and thought out, and perhaps voted upon. Maven snuck into Lucene IMO. To my knowledge, the majority of core developers do not want maven in the build and/or frown on dealing with Maven. We could always have a little vote to gauge numbers - I just have not wanted to rush to another vote thread myself ;) Users are important too - but they don't get official votes - it's up to each of us to consider the User feelings/vote in our opinions/votes as we see fit IMO. - Mark - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
Re: Let's drop Maven Artifacts !
It's sad how aggressive these discussions get. There's really no reason. On 1/18/11 1:10 PM, Robert Muir wrote: On Tue, Jan 18, 2011 at 4:06 PM, Grant Ingersollgsing...@apache.org wrote: In other words, I don't see consensus for dropping it. When you have it, get back to me. Thats not how things are added to the release process. So currently, maven is not included in the release process. I don't care if your poll on the users list has 100% of users checking maven, you biased your poll already by mentioning that its because we are considering dropping maven support at the start of the email, so its total garbage. There's a lot of totally insane things I could poll the user list and get lots of responses for, that I think the devs would disagree with. - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
Re: Let's drop Maven Artifacts !
On 1/17/11 8:06 AM, Steven A Rowe wrote: On 1/17/2011 at 1:53 AM, Michael Busch wrote: I don't think any user needs the ability to run an ant target on Lucene's sources to produce maven artifacts I want to be able to make modifications to the Lucene source, install Maven snapshot artifacts in my local repository, then depend on those snapshots from other projects. I doubt I'm alone. This is something I would feel comfortable not supporting in Lucene out-of-the-box, because if someone needs to use modified sources it's not unreasonable to expect that they can also create their own pom files for the modified jars. I do think though that we should keep publishing official artifacts to a central repo. Michael - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
Re: Let's drop Maven Artifacts !
On 1/17/11 12:27 PM, Steven A Rowe wrote: This makes zero sense to me - no one will ever make their own POMs I did :) (for a different project though). - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12982900#action_12982900 ] Michael Busch commented on LUCENE-2324: --- My last commit yesterday made almost all test cases pass. The ones that test flush-by-ram are still failing. Also TestStressIndexing2 still fails. The reason has to do with how deletes are pushed into bufferedDeletes. E.g. if I call addDocument() instead of updateDocument() in TestStressIndexing.IndexerThread then the test passes. I need to look more into that problem, but otherwise it's looking good and we're pretty close! Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324.patch, lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch, test.out, test.out, test.out, test.out See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
Re: Let's drop Maven Artifacts !
On 1/16/11 11:08 AM, Shai Erera wrote: I think the reasonable solution is to have a modules/maven package, with build.xml that generates whatever needs to be generated. Whoever cares about maven should run the proper Ant targets, just like whoever cares about Eclipse/IDEA can now run ant eclipse/idea. We'd have an ant maven. If that's what you intend doing in 2657 then fine. The person who cares about maven is the one who puts a few lines of xml into their ivy or maven config files, which downloads automatically the specified version from a central repository. It's a very convenient thing and stopping to publish artifacts will require everyone who has such a build system setup to change the way they get their Lucene jar files. There is an impressive amount of tools available in maven repos, it'd probably not be good if something as popular as Lucene was missing there. I don't think any user needs the ability to run an ant target on Lucene's sources to produce maven artifacts - what they want is published artifacts in a central repo. I personally don't need Lucene to be in such a repo, but I wanted to point out why I think it can be very useful. Michael - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12982200#action_12982200 ] Michael Busch commented on LUCENE-2324: --- I just committed fixes for some failing tests. Eg. the addIndexes() problem is now fixed. The problem was that I had accidentally removed the following line in DW.addIndexes(): {code} // Update SI appropriately info.setDocStore(info.getDocStoreOffset(), newDsName, info.getDocStoreIsCompoundFile()); {code} info.setDocStore() calls clearFiles(), which empties a SegmentInfo-local cache of all filenames that belong to the corresponding segment. Since addIndexes() changes the segment name, it is important to refill that cache with the new file names. This was a sneaky bug. We should probably call clearFiles() explicitly there in addIndexes(). For now I added a comment. Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch, test.out, test.out, test.out See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12981832#action_12981832 ] Michael Busch commented on LUCENE-2324: --- bq. as we're iterating on ThreadStates and on a non-concurrent hashmap calling put while not in a lock? The threadBindings hashmap is a ConcurrentHashMap and the getActivePerThreadsIterator() is threadsafe I believe. Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch, test.out, test.out, test.out See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12981192#action_12981192 ] Michael Busch commented on LUCENE-2324: --- I made some progress with the concurrency model, especially removing the need for various locks to make everything easier. - DocumentsWriterPerThreadPool.ThreadState now extends ReentrantLock, which means that standard methods like lock() and unlock() can be used to reserve a DWPT for a task. - The max. number of DWPTs allowed (config.maxThreadStates) is instantiated up-front. Creating a DWPT is cheap, so this is not a performance concern; this makes it easier to push config changes to the DWPTs without synchronizing on the pool and without having to worry about newly created DWPTs getting the same config settings. - DocumentsWriterPerThreadPool.getActivePerThreadsIterator() gives the caller a static snapshot of the active DWPTs at the time the iterator was acquired, e.g. for flushAllThreads() or DW.abort(). Here synchronizing on the pool isn't necessary either. - deletes are now pushed to DW.pendingDeletes() if no active DWPTs are present. TODOs: - fix remaining testcases that still fail - fix RAM tracking and flush-by-RAM - write new testcases to test thread pool, thread assignment, etc - review if all cases that were discussed in the recent comments here work as expected (likely not :) ) - performance testing and code cleanup Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch, test.out, test.out See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12981380#action_12981380 ] Michael Busch commented on LUCENE-2324: --- bq. Really? That makes synchronized seem simpler? Well look at ThreadAffinityDocumentsWriterThreadPool. There I'm able to use things like tryLock() and getQueueLength(). Also DocumentsWriterPerThreadPool has a getAndLock() method, that can be used by DW for addDocument(), whereas DW.flush(), which needs to iterate the DWPTs, can lock the individual DWPTs directly. I think it's simpler, but I'm open to other suggestions of course :) bq. What about the memory used, eg, the non-use of byte[] recycling? I guess it'll be cleared on flush. Yeah, sure. That is independent on whether they're all created upfront or not. But yeah, after flush or abort we need to clear the DWPT's state to make sure they're not consuming unused RAM (as you described in your earlier comment). Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch, test.out, test.out See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12981390#action_12981390 ] Michael Busch commented on LUCENE-2324: --- bq. How do I currently get the ..er.. current version? Just do 'svn up' on the RT branch. bq. Regardless of everything else, I'd ask you not to extend random things This was a conscious decision, not random. Extending ReentrantLock is not an uncommon pattern, e.g. ConcurrentHashMap.Segment does exactly that. ThreadState basically is nothing but a lock that has a reference to the corresponding DWPT it protects. I encourage you to look at the code. Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch, test.out, test.out See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12981548#action_12981548 ] Michael Busch commented on LUCENE-2324: --- bq. DWPT.perDocAllocator and freeLevel can be removed? done. bq. DWPT's RecyclingByteBlockAllocator - DirectAllocator? done. Also removed more recycling code. bq. I don't think we need FlushControl anymore as the RAM tracking should occur in DW and there's no need for IW to [globally] wait for flushes. I removed flushControl from DW. bq. I'm curious if the file not found errors are gone. I think there's something wrong with TermVectors - several related test cases fail. We need to investigate more. Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch, test.out, test.out See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2312) Search on IndexWriter's RAM Buffer
[ https://issues.apache.org/jira/browse/LUCENE-2312?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12979633#action_12979633 ] Michael Busch commented on LUCENE-2312: --- bq. I believe the goal for RT readers is still point in time reader semantics. True. At twitter our RT solution also guarantees point-in-time readers (with one exception; see below). We have to provide at least a fixed macDoc per-query to guarantee consistency across terms (posting lists). Eg. imagine your query is 'a AND NOT b'. Say a occurs in doc 100. Now you don't find a posting in b's posting list for doc 100. Did doc 100 not have term b, or is doc 100 still being processed and that particular posting hasn't been written yet? If the reader's maxDoc however is set to 99 (the last completely indexed document) you can't get into this situation. Before every query we reopen the readers, which effectively simply updates the maxDoc. The one exception to point-in-time-ness are the df values in the dictionary, which for obvious reasons is tricky. I think a straightforward way to solve this problem is to count the df by iterating the corresponding posting list when requested. We could add a special counting method that just uses the skip lists to perform this task. Here the term buffer becomes even more important, and also documenting that docFreq() can be expensive in RT mode, ie. not O(1) as in non-RT mode, but rather O(log indexSize) in case we can get multi-level skip lists working in RT. Search on IndexWriter's RAM Buffer -- Key: LUCENE-2312 URL: https://issues.apache.org/jira/browse/LUCENE-2312 Project: Lucene - Java Issue Type: New Feature Components: Search Affects Versions: Realtime Branch Reporter: Jason Rutherglen Assignee: Michael Busch Fix For: Realtime Branch Attachments: LUCENE-2312-FC.patch, LUCENE-2312.patch In order to offer user's near realtime search, without incurring an indexing performance penalty, we can implement search on IndexWriter's RAM buffer. This is the buffer that is filled in RAM as documents are indexed. Currently the RAM buffer is flushed to the underlying directory (usually disk) before being made searchable. Todays Lucene based NRT systems must incur the cost of merging segments, which can slow indexing. Michael Busch has good suggestions regarding how to handle deletes using max doc ids. https://issues.apache.org/jira/browse/LUCENE-2293?focusedCommentId=12841923page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#action_12841923 The area that isn't fully fleshed out is the terms dictionary, which needs to be sorted prior to queries executing. Currently IW implements a specialized hash table. Michael B has a suggestion here: https://issues.apache.org/jira/browse/LUCENE-2293?focusedCommentId=12841915page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#action_12841915 -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12979649#action_12979649 ] Michael Busch commented on LUCENE-2324: --- bq. Longer term c) would be great, or, if IW has an ES then it'd send multiple flush jobs to the ES. Lost in abbreviations :) - Can you remind me what 'ES' is? bq. But, you're right: maybe we should sometimes prune DWPTs. Or simply stop recycling any RAM, so that a just-flushed DWPT is an empty shell. I'm not sure I understand what the problem here with recycling RAM is. Could someone elaborate? Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch, test.out, test.out See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12979654#action_12979654 ] Michael Busch commented on LUCENE-2324: --- bq. I think aborting a flush should only lose the docs in that one DWPT (as it is today). Yeah I'm convinced now I don't want the nuke the world approach. Btw, Mike, you're very good with giving things intuitive names :) bq. I think on commit if we hit an aborting exception flushing a given DWPT, we throw it then there. Yes sounds good. {quote} bq. Any segs already flushed remain flushed (but not committed). Any segs not yet flushed remain not yet flushed... If the segment are flushed, then they will be deleted? Or they will be made available in a subsequent and completely successful commit? {quote} The aborting exception might be thrown due to a disk-full situation. This can be fixed and commit() called again, which then would flush the remaining DWPTs and commit all flushed segments. Otherwise, those flushed segments will be orphaned and deleted sometime later by a different IW because they don't belong to any SegmentInfos. Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch, test.out, test.out See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12979671#action_12979671 ] Michael Busch commented on LUCENE-2324: --- {quote} Mainly that we could have DWPT(s) lying around unused, consuming [recycled] RAM, eg, from a sudden drop in the number of incoming threads after a flush. This is a drop the code, and put it back in if that was a bad idea solution. {quote} Ah thanks, got it. bq. Or simply stop recycling any RAM, so that a just-flushed DWPT is an empty shell. +1 Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch, test.out, test.out See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12979247#action_12979247 ] Michael Busch commented on LUCENE-2324: --- bq. I think the risk is a new DWPT likely will have been created during flush, which'd make the returning DWPT inutile. The DWPT will not be removed from the pool, just marked as busy during flush, like as its state is busy (or currently called non-idle in the code) during addDocumentI(). So no new DWPT would be created during flush if the maxThreadState limit was already reached. Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch, test.out, test.out See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12979248#action_12979248 ] Michael Busch commented on LUCENE-2324: --- {quote} I think start simple - the addDocument always happens? Ie it's never coordinated w/ the ongoing flush. It picks a free DWPT like normal, and since flush is single threaded, there should always be a free DWPT? {quote} Yeah I agree. The change I'll make then is to not have the global lock and return a DWPT immediately to the pool and set it to 'idle' after its flush completed. {quote} I think we should continue what we do today? Ie, if it's an 'aborting' exception, then the entire segment held by that DWPT is discarded? And we then throw this exc back to caller (and don't try to flush any other segments)? {quote} What I meant was the following situation: Suppose we have two DWPTs and IW.commit() is called. The first DWPT finishes flushing successfully, is returned to the pool and idle again. The second DWPT flush fails with an aborting exception. Should the segment of the first DWPT make it into the index or not? I think segment 1 shouldn't be committed, ie. a global flush should be all or nothing. This means we would have to delay the commit of the segments until all DWPTs flushed successfully. Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch, test.out, test.out See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12978466#action_12978466 ] Michael Busch commented on LUCENE-2324: --- {quote} I believe we can drop the delete in that case. We only need to buffer into DWPTs that have at least 1 doc. {quote} Yeah sounds right. {quote} If a given DWPT is flushing then we pick another? Ie the binding logic would naturally avoid DWPTs that are not available - either because another thread has it, or it's flushing. But it would prefer to use the same DWPT it used last time, if possible (affinity). {quote} This is actually what should be happening currently if the (default) ThreadAffinityThreadPool is used. I've to check the code again and maybe write a test specifically for that. bq. Also: I thought we don't have sequence IDs anymore? (At least, for landing DWPT; after that (for true RT) we need something like sequence IDs?). True, sequenceIDs are gone since the last merge. And yes, I still think we'll need them for RT. Even for the non-RT case sequenceIDs would have nice benefits. If methods like addDocument(), deleteDocuments(), etc. return the sequenceID they'd define a strict ordering on those operations and make it transparent for the application, which would be beneficial for document tracking and log replay. But anyway, let's add seqIDs back after the DWPT changes are done and in trunk. {quote} bq. We shouldn't do global waiting anymore - this is what's great about DWPT. However we'll have global waiting for the flush all threads case. I think that can move down to DW though. Or should it simply be a sync in/on IW? {quote} True, the only global lock that locks all thread states happens when flushAllThreads is called. This is called when IW explicitly triggers a flush, e.g. on close/commit. However, maybe this is not the right approach? I guess we don't really need the global lock. A thread performing the global flush could still acquire each thread state before it starts flushing, but return a threadState to the pool once that particular threadState is done flushing? A related question is: Do we want to piggyback on multiple threads when a global flush happens? Eg. Thread 1 called commit, Thread 2 shortly afterwards addDocument(). When should addDocument() happen? a) After all DWPTs finished flushing? b) After at least one DWPT finished flushing and is available again? c) Or should Thread 2 be used to help flushing DWPTs in parallel with Thread 1? a) is currently implemented, but I think not really what we want. b) is probably best for RT, because it means the lowest indexing latency for the new document to be added. c) probably means the best overall throughput (depending even on hardware like disk speed, etc) For whatever option we pick, we'll have to carefully think about error handling. It's quite straightforward for a) (just commit all flushed segments to SegmentInfos when the global flush completed succesfully). But for b) and c) it's unclear what should happen if a DWPT flush fails after some completed already successfully before. Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, LUCENE-2324-SMALL.patch, lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch, test.out, test.out See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2292) ByteBuffer Directory - allowing to store the index outside the heap
[ https://issues.apache.org/jira/browse/LUCENE-2292?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12974788#action_12974788 ] Michael Busch commented on LUCENE-2292: --- bq. This class uses ByteBuffer, which has its overhead over simple byte[], In my experience ByteBuffer has basically no performance overhead over byte[] if you construct it by wrapping a byte[]. The JVM seems smart enough to figure out that there's a good old array behind the ByteBuffer. But if I allocated the BB in any other way it was 2-4x slower in my simple tests on a mac with a sun JVM. So it might be the right thing to put these changes into RAMDirectory and have it by default wrap a byte[] and add an (expert) API to allow allocating the BB in other ways. ByteBuffer Directory - allowing to store the index outside the heap --- Key: LUCENE-2292 URL: https://issues.apache.org/jira/browse/LUCENE-2292 Project: Lucene - Java Issue Type: New Feature Components: Store Reporter: Shay Banon Attachments: LUCENE-2292.patch, LUCENE-2292.patch, LUCENE-2292.patch A byte buffer based directory with the benefit of being able to create direct byte buffer thus storing the index outside the JVM heap. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
RT branch status
After merging trunk into the RT branch it's finally compiling again and up-to-date. Several tests are failing now after the merge (43 out of 1427 are failing), which is not too surprising, because so many things have changed (segment-deletes, flush control, termsHash refactoring, removal of doc stores, etc). Especially IndexWriter and DocumentsWriter are in a somewhat messy state, but I wanted to share my current state, so I committed the merge. I'll try this week to understand the new changes (especially deletes) and make them work with the DWPT. The following areas need work: * deletes * thread-safety * error handling and aborting * flush-by-ram (LUCENE-2573) Also, some tests deadlock. Not surprisingly either, cause flushcontrol etc. introduce new synchronized blocks. Before the merge all tests were passing, except the ones testing flush-by-ram functionality. I'll keep working on getting the branch back into that state again soon. Help is definitely welcome! I'd love to get this branch ready so that we can merge it into trunk as soon as possible. As Mike's experiments show having DWPTs will not only be beneficial for RT search, but also increase indexing performance in general. Michael PS: Thanks for the patience! - To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2814) stop writing shared doc stores across segments
[ https://issues.apache.org/jira/browse/LUCENE-2814?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12973135#action_12973135 ] Michael Busch commented on LUCENE-2814: --- {quote} OK I committed to trunk. I'll let this bake for a while on trunk before backporting to 3.x... Thanks Earwin! {quote} Man, you guys really ruined my Sunday with this commit :) I got so many merge conflicts, that I decided to merge first only up to rev 1050655 (the rev before this commit) and up to HEAD in a second merge. I'm down to 64 compile errors (from 800), hopefully I can finish the merge tomorrow. Just wanted you to know that I'm making progress here with the DWPTs. stop writing shared doc stores across segments -- Key: LUCENE-2814 URL: https://issues.apache.org/jira/browse/LUCENE-2814 Project: Lucene - Java Issue Type: Improvement Components: Index Affects Versions: 3.1, 4.0 Reporter: Michael McCandless Assignee: Michael McCandless Attachments: LUCENE-2814.patch, LUCENE-2814.patch, LUCENE-2814.patch, LUCENE-2814.patch, LUCENE-2814.patch Shared doc stores enables the files for stored fields and term vectors to be shared across multiple segments. We've had this optimization since 2.1 I think. It works best against a new index, where you open an IW, add lots of docs, and then close it. In that case all of the written segments will reference slices a single shared doc store segment. This was a good optimization because it means we never need to merge these files. But, when you open another IW on that index, it writes a new set of doc stores, and then whenever merges take place across doc stores, they must now be merged. However, since we switched to shared doc stores, there have been two optimizations for merging the stores. First, we now bulk-copy the bytes in these files if the field name/number assignment is congruent. Second, we now force congruent field name/number mapping in IndexWriter. This means this optimization is much less potent than it used to be. Furthermore, the optimization adds *a lot* of hair to IndexWriter/DocumentsWriter; this has been the source of sneaky bugs over time, and causes odd behavior like a merge possibly forcing a flush when it starts. Finally, with DWPT (LUCENE-2324), which gets us truly concurrent flushing, we can no longer share doc stores. So, I think we should turn off the write-side of shared doc stores to pave the path for DWPT to land on trunk and simplify IW/DW. We still must support reading them (until 5.0), but the read side is far less hairy. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2814) stop writing shared doc stores across segments
[ https://issues.apache.org/jira/browse/LUCENE-2814?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12972227#action_12972227 ] Michael Busch commented on LUCENE-2814: --- The shared doc stores are actually already completely removed in the realtime branch (part of LUCENE-2324). Does someone want to help with the merge, then we can land the realtime branch (which is pretty much only DWPT and removing doc stores) in trunk sometime soon? stop writing shared doc stores across segments -- Key: LUCENE-2814 URL: https://issues.apache.org/jira/browse/LUCENE-2814 Project: Lucene - Java Issue Type: Improvement Components: Index Affects Versions: 3.1, 4.0 Reporter: Michael McCandless Assignee: Michael McCandless Attachments: LUCENE-2814.patch, LUCENE-2814.patch Shared doc stores enables the files for stored fields and term vectors to be shared across multiple segments. We've had this optimization since 2.1 I think. It works best against a new index, where you open an IW, add lots of docs, and then close it. In that case all of the written segments will reference slices a single shared doc store segment. This was a good optimization because it means we never need to merge these files. But, when you open another IW on that index, it writes a new set of doc stores, and then whenever merges take place across doc stores, they must now be merged. However, since we switched to shared doc stores, there have been two optimizations for merging the stores. First, we now bulk-copy the bytes in these files if the field name/number assignment is congruent. Second, we now force congruent field name/number mapping in IndexWriter. This means this optimization is much less potent than it used to be. Furthermore, the optimization adds *a lot* of hair to IndexWriter/DocumentsWriter; this has been the source of sneaky bugs over time, and causes odd behavior like a merge possibly forcing a flush when it starts. Finally, with DWPT (LUCENE-2324), which gets us truly concurrent flushing, we can no longer share doc stores. So, I think we should turn off the write-side of shared doc stores to pave the path for DWPT to land on trunk and simplify IW/DW. We still must support reading them (until 5.0), but the read side is far less hairy. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2814) stop writing shared doc stores across segments
[ https://issues.apache.org/jira/browse/LUCENE-2814?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12972275#action_12972275 ] Michael Busch commented on LUCENE-2814: --- Well I need to merge with the recent changes in trunk (especially LUCENE-2680). The merge is pretty hard, but I'm planning to spend most of my weekend on it. If I can get most tests to pass again (most were passing before the merge), then I think the only outstanding thing is LUCENE-2573 before we could land it in trunk. stop writing shared doc stores across segments -- Key: LUCENE-2814 URL: https://issues.apache.org/jira/browse/LUCENE-2814 Project: Lucene - Java Issue Type: Improvement Components: Index Affects Versions: 3.1, 4.0 Reporter: Michael McCandless Assignee: Michael McCandless Attachments: LUCENE-2814.patch, LUCENE-2814.patch Shared doc stores enables the files for stored fields and term vectors to be shared across multiple segments. We've had this optimization since 2.1 I think. It works best against a new index, where you open an IW, add lots of docs, and then close it. In that case all of the written segments will reference slices a single shared doc store segment. This was a good optimization because it means we never need to merge these files. But, when you open another IW on that index, it writes a new set of doc stores, and then whenever merges take place across doc stores, they must now be merged. However, since we switched to shared doc stores, there have been two optimizations for merging the stores. First, we now bulk-copy the bytes in these files if the field name/number assignment is congruent. Second, we now force congruent field name/number mapping in IndexWriter. This means this optimization is much less potent than it used to be. Furthermore, the optimization adds *a lot* of hair to IndexWriter/DocumentsWriter; this has been the source of sneaky bugs over time, and causes odd behavior like a merge possibly forcing a flush when it starts. Finally, with DWPT (LUCENE-2324), which gets us truly concurrent flushing, we can no longer share doc stores. So, I think we should turn off the write-side of shared doc stores to pave the path for DWPT to land on trunk and simplify IW/DW. We still must support reading them (until 5.0), but the read side is far less hairy. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2814) stop writing shared doc stores across segments
[ https://issues.apache.org/jira/browse/LUCENE-2814?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12972288#action_12972288 ] Michael Busch commented on LUCENE-2814: --- bq. I think taking things one step at a time would be good here? Probably still a smaller change than flex indexing ;) But yeah in general I agree that we should do things more incrementally. I think that's a mistake I've made with the RT branch so far. In this particular case it's just a bit sad to redo all this work now, because I think I got the removal of doc stores right in RT and all related tests to pass. stop writing shared doc stores across segments -- Key: LUCENE-2814 URL: https://issues.apache.org/jira/browse/LUCENE-2814 Project: Lucene - Java Issue Type: Improvement Components: Index Affects Versions: 3.1, 4.0 Reporter: Michael McCandless Assignee: Michael McCandless Attachments: LUCENE-2814.patch, LUCENE-2814.patch Shared doc stores enables the files for stored fields and term vectors to be shared across multiple segments. We've had this optimization since 2.1 I think. It works best against a new index, where you open an IW, add lots of docs, and then close it. In that case all of the written segments will reference slices a single shared doc store segment. This was a good optimization because it means we never need to merge these files. But, when you open another IW on that index, it writes a new set of doc stores, and then whenever merges take place across doc stores, they must now be merged. However, since we switched to shared doc stores, there have been two optimizations for merging the stores. First, we now bulk-copy the bytes in these files if the field name/number assignment is congruent. Second, we now force congruent field name/number mapping in IndexWriter. This means this optimization is much less potent than it used to be. Furthermore, the optimization adds *a lot* of hair to IndexWriter/DocumentsWriter; this has been the source of sneaky bugs over time, and causes odd behavior like a merge possibly forcing a flush when it starts. Finally, with DWPT (LUCENE-2324), which gets us truly concurrent flushing, we can no longer share doc stores. So, I think we should turn off the write-side of shared doc stores to pave the path for DWPT to land on trunk and simplify IW/DW. We still must support reading them (until 5.0), but the read side is far less hairy. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2814) stop writing shared doc stores across segments
[ https://issues.apache.org/jira/browse/LUCENE-2814?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12972302#action_12972302 ] Michael Busch commented on LUCENE-2814: --- bq. So, what's the plan? I can't really work on this much before Saturday. But during the weekend I can work on the RT merge and maybe try to pull out the docstore removal changes and create a separate patch. Have to see how hard that is. If it's not too difficult I'll post a separate patch, otherwise I'll commit the merge to RT and maybe convince you guys to help a bit with getting the RT branch ready for landing in trunk? :) stop writing shared doc stores across segments -- Key: LUCENE-2814 URL: https://issues.apache.org/jira/browse/LUCENE-2814 Project: Lucene - Java Issue Type: Improvement Components: Index Affects Versions: 3.1, 4.0 Reporter: Michael McCandless Assignee: Michael McCandless Attachments: LUCENE-2814.patch, LUCENE-2814.patch Shared doc stores enables the files for stored fields and term vectors to be shared across multiple segments. We've had this optimization since 2.1 I think. It works best against a new index, where you open an IW, add lots of docs, and then close it. In that case all of the written segments will reference slices a single shared doc store segment. This was a good optimization because it means we never need to merge these files. But, when you open another IW on that index, it writes a new set of doc stores, and then whenever merges take place across doc stores, they must now be merged. However, since we switched to shared doc stores, there have been two optimizations for merging the stores. First, we now bulk-copy the bytes in these files if the field name/number assignment is congruent. Second, we now force congruent field name/number mapping in IndexWriter. This means this optimization is much less potent than it used to be. Furthermore, the optimization adds *a lot* of hair to IndexWriter/DocumentsWriter; this has been the source of sneaky bugs over time, and causes odd behavior like a merge possibly forcing a flush when it starts. Finally, with DWPT (LUCENE-2324), which gets us truly concurrent flushing, we can no longer share doc stores. So, I think we should turn off the write-side of shared doc stores to pave the path for DWPT to land on trunk and simplify IW/DW. We still must support reading them (until 5.0), but the read side is far less hairy. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12970287#action_12970287 ] Michael Busch commented on LUCENE-2324: --- I started merging yesterday the latest trunk into realtime. The merge is rather hard, as you might imagine :) But I'm down from 600 compile errors to ~100. I can try to finish it this weekend. But I don't want to block you, if you want to go the patch route and have time now don't wait for me. Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12970289#action_12970289 ] Michael Busch commented on LUCENE-2324: --- bq. I started merging yesterday the latest trunk into realtime. As part of this I want to clean up the branch a bit and remove unnecessary changes (like refactorings) to make the merge back into trunk less difficult. When I'm done with the merge we should patch LUCENE-2680 into realtime. (or commit to trunk and merge trunk into realtime again) Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12969850#action_12969850 ] Michael Busch commented on LUCENE-2324: --- Ideally we should merge trunk into realtime after LUCENE-2680 is committed, get everything working there, and then merge realtime back into trunk? I agree that it totally makes sense to get DWPT into trunk as soon as possible (ie. not wait until all realtime stuff is done). Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12969865#action_12969865 ] Michael Busch commented on LUCENE-2324: --- Not sure if that's much easier though, because what you said is true: the realtime branch currently is basically the DWPT branch. Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2792) Add a simple FST impl to Lucene
[ https://issues.apache.org/jira/browse/LUCENE-2792?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12966691#action_12966691 ] Michael Busch commented on LUCENE-2792: --- Cool stuff, Mike! Could we use this for more efficient wildcard search? E.g. could we add posting lists for inner nodes to the index? Add a simple FST impl to Lucene --- Key: LUCENE-2792 URL: https://issues.apache.org/jira/browse/LUCENE-2792 Project: Lucene - Java Issue Type: New Feature Components: Index Reporter: Michael McCandless Assignee: Michael McCandless Fix For: 4.0 Attachments: FSTExample.png, LUCENE-2792.patch I implemented the algo described at http://citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.24.3698 for incrementally building a finite state transducer (FST) from sorted inputs. This is not a fully general FST impl -- it's only able to build up an FST incrementally from input/output pairs that are pre-sorted. Currently the inputs are BytesRefs, and the outputs are pluggable -- NoOutputs gets you a simple FSA, PositiveIntOutputs maps to a long, ByteSequenceOutput maps to a BytesRef. The implementation has a low memory overhead, so that it can handle a fairly large set of terms. For example, it can build the FSA for the 9.8M terms from a 10M document wikipedia index in ~8 seconds (on beast), using ~256 MB peak RAM, resulting in an FSA that's ~60 MB. It packs the FST as-it-builds into a compact byte[], and then exposes the API to read nodes/arcs directly from the byte[]. The FST can be quickly saved/loaded to/from a Directory since it's just a big byte[]. The format is similar to what Morfologik uses (http://sourceforge.net/projects/morfologik/). I think there are a number of possible places we can use this in Lucene. For example, I think many apps could hold the entire terms dict in RAM, either at the multi-reader level or maybe per-segment (mapping to file offset or to something else custom to the app), which may possibly be a good speedup for certain MTQs (though, because the format is packed into a byte[], there is a decode cost when visiting arcs). The builder can also prune as it goes, so you get a prefix trie pruned according to how many terms run through the nodes, which makes it faster and even less memory consuming. This may be useful as a replacement for our current binary search terms index since it can achieve higher term density for the same RAM consumption of our current index. As an initial usage to make sure this is exercised, I cutover the SimpleText codec, which currently fully loads all terms into a TreeMap (and has caused intermittent OOME in some tests), to use an FST instead. SimpleText uses a PairOutputs which is able to pair up any two other outputs, since it needs to map each input term to an int docFreq and long filePosition. All tests pass w/ SimpleText forced codec, and I think this is committable except I'd love to get some help w/ the generics (confession to the policeman: I had to add @SuppressWarnings({unchecked})) all over!! Ideally an FST is parameterized by its output type (Integer, BytesRef, etc.). I even added a new @nightly test that makes a largeish set of random terms and tests the resulting FST on different outputs :) I think it would also be easy to make a variant that uses char[] instead of byte[] as its inputs, so we could eg use this during analysis (Robert's idea). It's already be easy to have a CharSequence output type since the outputs are pluggable. Dawid Weiss (author of HPPC -- http://labs.carrotsearch.com/hppc.html -- and Morfologik -- http://sourceforge.net/projects/morfologik/) was very helpful iterating with me on this (thank you!). -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2662) BytesHash
[ https://issues.apache.org/jira/browse/LUCENE-2662?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12935735#action_12935735 ] Michael Busch commented on LUCENE-2662: --- bq. I think we should really close this since RT branch is not very active right now Sorry about that. I need to merge trunk into RT, then I'll get this change too. It's a big merge though with tons of conflicts... BytesHash - Key: LUCENE-2662 URL: https://issues.apache.org/jira/browse/LUCENE-2662 Project: Lucene - Java Issue Type: Improvement Components: Index Affects Versions: 4.0 Reporter: Jason Rutherglen Assignee: Simon Willnauer Priority: Minor Fix For: 4.0 Attachments: LUCENE-2662.patch, LUCENE-2662.patch, LUCENE-2662.patch, LUCENE-2662.patch, LUCENE-2662.patch, LUCENE-2662.patch This issue will have the BytesHash separated out from LUCENE-2186 -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2662) BytesHash
[ https://issues.apache.org/jira/browse/LUCENE-2662?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12935747#action_12935747 ] Michael Busch commented on LUCENE-2662: --- Yeah sitting in Stuttgart, going to hit the Weihnachtsmarkt soon - let's see how the merge goes after several glasses of Gluehwein :) BytesHash - Key: LUCENE-2662 URL: https://issues.apache.org/jira/browse/LUCENE-2662 Project: Lucene - Java Issue Type: Improvement Components: Index Affects Versions: 4.0 Reporter: Jason Rutherglen Assignee: Simon Willnauer Priority: Minor Fix For: 4.0 Attachments: LUCENE-2662.patch, LUCENE-2662.patch, LUCENE-2662.patch, LUCENE-2662.patch, LUCENE-2662.patch, LUCENE-2662.patch This issue will have the BytesHash separated out from LUCENE-2186 -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12910483#action_12910483 ] Michael Busch commented on LUCENE-2324: --- bq. Is this near-comittable? I think we need to: * merge trunk and make tests pass * finish flushing by RAM * make deletes work again Then it should be ready to commit. Sorry, was so busy the last weeks that I couldn't make much progress. Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2573) Tiered flushing of DWPTs by RAM with low/high water marks
[ https://issues.apache.org/jira/browse/LUCENE-2573?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12898675#action_12898675 ] Michael Busch commented on LUCENE-2573: --- Hi Jason, are you still working on the patch here? Tiered flushing of DWPTs by RAM with low/high water marks - Key: LUCENE-2573 URL: https://issues.apache.org/jira/browse/LUCENE-2573 Project: Lucene - Java Issue Type: Improvement Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Now that we have DocumentsWriterPerThreads we need to track total consumed RAM across all DWPTs. A flushing strategy idea that was discussed in LUCENE-2324 was to use a tiered approach: - Flush the first DWPT at a low water mark (e.g. at 90% of allowed RAM) - Flush all DWPTs at a high water mark (e.g. at 110%) - Use linear steps in between high and low watermark: E.g. when 5 DWPTs are used, flush at 90%, 95%, 100%, 105% and 110%. Should we allow the user to configure the low and high water mark values explicitly using total values (e.g. low water mark at 120MB, high water mark at 140MB)? Or shall we keep for simplicity the single setRAMBufferSizeMB() config method and use something like 90% and 110% for the water marks? -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2573) Tiered flushing of DWPTs by RAM with low/high water marks
[ https://issues.apache.org/jira/browse/LUCENE-2573?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12893683#action_12893683 ] Michael Busch commented on LUCENE-2573: --- Jason, are you still up for working on a patch for this one? We should probably get the realtime branch in a healthy state first and run some performance tests before we start working on all the fun stuff. Almost there! Tiered flushing of DWPTs by RAM with low/high water marks - Key: LUCENE-2573 URL: https://issues.apache.org/jira/browse/LUCENE-2573 Project: Lucene - Java Issue Type: Improvement Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Now that we have DocumentsWriterPerThreads we need to track total consumed RAM across all DWPTs. A flushing strategy idea that was discussed in LUCENE-2324 was to use a tiered approach: - Flush the first DWPT at a low water mark (e.g. at 90% of allowed RAM) - Flush all DWPTs at a high water mark (e.g. at 110%) - Use linear steps in between high and low watermark: E.g. when 5 DWPTs are used, flush at 90%, 95%, 100%, 105% and 110%. Should we allow the user to configure the low and high water mark values explicitly using total values (e.g. low water mark at 120MB, high water mark at 140MB)? Or shall we keep for simplicity the single setRAMBufferSizeMB() config method and use something like 90% and 110% for the water marks? -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2573) Tiered flushing of DWPTs by RAM with low/high water marks
[ https://issues.apache.org/jira/browse/LUCENE-2573?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12893789#action_12893789 ] Michael Busch commented on LUCENE-2573: --- bq. Michael, DWPT.numBytesUsed isn't currently being updated? You can delete that one. I factored all the memory allocation/tracking into DocumentsWriterRAMAllocator. You might have to get some memory related stuff from trunk, e.g. the balanceRAM() code and adapt it. Tiered flushing of DWPTs by RAM with low/high water marks - Key: LUCENE-2573 URL: https://issues.apache.org/jira/browse/LUCENE-2573 Project: Lucene - Java Issue Type: Improvement Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Now that we have DocumentsWriterPerThreads we need to track total consumed RAM across all DWPTs. A flushing strategy idea that was discussed in LUCENE-2324 was to use a tiered approach: - Flush the first DWPT at a low water mark (e.g. at 90% of allowed RAM) - Flush all DWPTs at a high water mark (e.g. at 110%) - Use linear steps in between high and low watermark: E.g. when 5 DWPTs are used, flush at 90%, 95%, 100%, 105% and 110%. Should we allow the user to configure the low and high water mark values explicitly using total values (e.g. low water mark at 120MB, high water mark at 140MB)? Or shall we keep for simplicity the single setRAMBufferSizeMB() config method and use something like 90% and 110% for the water marks? -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2573) Tiered flushing of DWPTs by RAM with low/high water marks
[ https://issues.apache.org/jira/browse/LUCENE-2573?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12893928#action_12893928 ] Michael Busch commented on LUCENE-2573: --- I'm not 100% sure, I need to review the code to refresh my memory... Tiered flushing of DWPTs by RAM with low/high water marks - Key: LUCENE-2573 URL: https://issues.apache.org/jira/browse/LUCENE-2573 Project: Lucene - Java Issue Type: Improvement Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Now that we have DocumentsWriterPerThreads we need to track total consumed RAM across all DWPTs. A flushing strategy idea that was discussed in LUCENE-2324 was to use a tiered approach: - Flush the first DWPT at a low water mark (e.g. at 90% of allowed RAM) - Flush all DWPTs at a high water mark (e.g. at 110%) - Use linear steps in between high and low watermark: E.g. when 5 DWPTs are used, flush at 90%, 95%, 100%, 105% and 110%. Should we allow the user to configure the low and high water mark values explicitly using total values (e.g. low water mark at 120MB, high water mark at 140MB)? Or shall we keep for simplicity the single setRAMBufferSizeMB() config method and use something like 90% and 110% for the water marks? -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Resolved: (LUCENE-2561) Fix exception handling and thread safety in realtime branch
[ https://issues.apache.org/jira/browse/LUCENE-2561?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Busch resolved LUCENE-2561. --- Resolution: Fixed TestStressIndexing2 is not failing because of concurrency problems, so I'm closing this issue. All contrib tests pass now too! The reason why TestStressIndexing2 is failing is that deletes and sequenceIDs aren't fully implemented yet. The remapDeletes step is still commented out, which results in a wrong behavior as soon as segment merges happen while deletes are buffered. (I'll use LUCENE-2558 to fix that) Fix exception handling and thread safety in realtime branch --- Key: LUCENE-2561 URL: https://issues.apache.org/jira/browse/LUCENE-2561 Project: Lucene - Java Issue Type: Bug Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: lucene-2561.patch Several tests are currently failing in the realtime branch - most of them due to thread safety problems (often exceptions in ConcurrentMergeScheduler) and in tests that test for aborting and non-aborting exceptions. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Created: (LUCENE-2573) Tiered flushing of DWPTs by RAM with low/high water marks
Tiered flushing of DWPTs by RAM with low/high water marks - Key: LUCENE-2573 URL: https://issues.apache.org/jira/browse/LUCENE-2573 Project: Lucene - Java Issue Type: Improvement Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Now that we have DocumentsWriterPerThreads we need to track total consumed RAM across all DWPTs. A flushing strategy idea that was discussed in LUCENE-2324 was to use a tiered approach: - Flush the first DWPT at a low water mark (e.g. at 90% of allowed RAM) - Flush all DWPTs at a high water mark (e.g. at 110%) - Use linear steps in between high and low watermark: E.g. when 5 DWPTs are used, flush at 90%, 95%, 100%, 105% and 110%. Should we allow the user to configure the low and high water mark values explicitly using total values (e.g. low water mark at 120MB, high water mark at 140MB)? Or shall we keep for simplicity the single setRAMBufferSizeMB() config method and use something like 90% and 110% for the water marks? -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2573) Tiered flushing of DWPTs by RAM with low/high water marks
[ https://issues.apache.org/jira/browse/LUCENE-2573?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12893256#action_12893256 ] Michael Busch commented on LUCENE-2573: --- Yeah I like that better too. Will implement that approach. Tiered flushing of DWPTs by RAM with low/high water marks - Key: LUCENE-2573 URL: https://issues.apache.org/jira/browse/LUCENE-2573 Project: Lucene - Java Issue Type: Improvement Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Now that we have DocumentsWriterPerThreads we need to track total consumed RAM across all DWPTs. A flushing strategy idea that was discussed in LUCENE-2324 was to use a tiered approach: - Flush the first DWPT at a low water mark (e.g. at 90% of allowed RAM) - Flush all DWPTs at a high water mark (e.g. at 110%) - Use linear steps in between high and low watermark: E.g. when 5 DWPTs are used, flush at 90%, 95%, 100%, 105% and 110%. Should we allow the user to configure the low and high water mark values explicitly using total values (e.g. low water mark at 120MB, high water mark at 140MB)? Or shall we keep for simplicity the single setRAMBufferSizeMB() config method and use something like 90% and 110% for the water marks? -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-1799) Unicode compression
[ https://issues.apache.org/jira/browse/LUCENE-1799?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12893292#action_12893292 ] Michael Busch commented on LUCENE-1799: --- Yonik can you give more details about how you ran your tests? Was it an isolated string encoding test or does BOCU slow down overall indexing speed by 29%-80% (which would be hard to believe). Unicode compression --- Key: LUCENE-1799 URL: https://issues.apache.org/jira/browse/LUCENE-1799 Project: Lucene - Java Issue Type: New Feature Components: Store Affects Versions: 2.4.1 Reporter: DM Smith Priority: Minor Attachments: LUCENE-1779.patch, LUCENE-1799.patch, LUCENE-1799.patch, LUCENE-1799.patch, LUCENE-1799.patch, LUCENE-1799.patch, LUCENE-1799.patch, LUCENE-1799.patch, LUCENE-1799.patch, LUCENE-1799.patch, LUCENE-1799.patch, LUCENE-1799.patch, LUCENE-1799.patch, LUCENE-1799.patch, LUCENE-1799.patch, LUCENE-1799_big.patch In lucene-1793, there is the off-topic suggestion to provide compression of Unicode data. The motivation was a custom encoding in a Russian analyzer. The original supposition was that it provided a more compact index. This led to the comment that a different or compressed encoding would be a generally useful feature. BOCU-1 was suggested as a possibility. This is a patented algorithm by IBM with an implementation in ICU. If Lucene provide it's own implementation a freely avIlable, royalty-free license would need to be obtained. SCSU is another Unicode compression algorithm that could be used. An advantage of these methods is that they work on the whole of Unicode. If that is not needed an encoding such as iso8859-1 (or whatever covers the input) could be used. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Updated: (LUCENE-2561) Fix exception handling and thread safety in realtime branch
[ https://issues.apache.org/jira/browse/LUCENE-2561?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Busch updated LUCENE-2561: -- Attachment: lucene-2561.patch The patch fixes most of the threading and exception issues. Now 99% of the core tests pass! Some failures are expected, because some features aren't implemented yet (e.g. flush by RAM or maxBufferedDeletes). A test though that I still want to fix with this patch is TestStressIndexing2 - not sure yet what's going on. Other changes: - Factored ReaderPool out of IndexWriter into its own class - Added a FilterDirectory that forwards all method calls to a delegate - Use an extended FilterDirectory in DW to track all files the consumers and codecs open, so that they can be closed on abort - Fixed some more nocommits Using the FilterDirectory might not be the cleanest approach? Maybe a IndexOutputFactory or something would be cleaner? Or maybe on abort we should just delete all files that have the prefix of the segment(s) the DWPT(s) were working on? This should be possible now that the shared doc stores are gone and no files are shared anymore across segments. Fix exception handling and thread safety in realtime branch --- Key: LUCENE-2561 URL: https://issues.apache.org/jira/browse/LUCENE-2561 Project: Lucene - Java Issue Type: Bug Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: lucene-2561.patch Several tests are currently failing in the realtime branch - most of them due to thread safety problems (often exceptions in ConcurrentMergeScheduler) and in tests that test for aborting and non-aborting exceptions. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2561) Fix exception handling and thread safety in realtime branch
[ https://issues.apache.org/jira/browse/LUCENE-2561?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12892918#action_12892918 ] Michael Busch commented on LUCENE-2561: --- Committed the current patch to the realtime branch. (revision 979856) Leaving this issue open to fix TestStressIndexing2 and for more cleanup. Fix exception handling and thread safety in realtime branch --- Key: LUCENE-2561 URL: https://issues.apache.org/jira/browse/LUCENE-2561 Project: Lucene - Java Issue Type: Bug Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: lucene-2561.patch Several tests are currently failing in the realtime branch - most of them due to thread safety problems (often exceptions in ConcurrentMergeScheduler) and in tests that test for aborting and non-aborting exceptions. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Created: (LUCENE-2571) Indexing performance tests with realtime branch
Indexing performance tests with realtime branch --- Key: LUCENE-2571 URL: https://issues.apache.org/jira/browse/LUCENE-2571 Project: Lucene - Java Issue Type: Task Reporter: Michael Busch Priority: Minor Fix For: Realtime Branch We should run indexing performance tests with the DWPT changes and compare to trunk. We need to test both single-threaded and multi-threaded performance. NOTE: flush by RAM isn't implemented just yet, so either we wait with the tests or flush by doc count. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Commented: (LUCENE-2324) Per thread DocumentsWriters that write their own private segments
[ https://issues.apache.org/jira/browse/LUCENE-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12892924#action_12892924 ] Michael Busch commented on LUCENE-2324: --- {quote} Is it possible that as part of this issue (or this effort), you'll think of opening PTDW for easier extensions (such as Parallel Indexing)? {quote} Yeah I'd like to make some progress on parallel indexing too. I think now that DWPT is roughly working I can start thinking about what further changes are necessary in the indexer. Per thread DocumentsWriters that write their own private segments - Key: LUCENE-2324 URL: https://issues.apache.org/jira/browse/LUCENE-2324 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: lucene-2324.patch, lucene-2324.patch, LUCENE-2324.patch See LUCENE-2293 for motivation and more details. I'm copying here Mike's summary he posted on 2293: Change the approach for how we buffer in RAM to a more isolated approach, whereby IW has N fully independent RAM segments in-process and when a doc needs to be indexed it's added to one of them. Each segment would also write its own doc stores and normal segment merging (not the inefficient merge we now do on flush) would merge them. This should be a good simplification in the chain (eg maybe we can remove the *PerThread classes). The segments can flush independently, letting us make much better concurrent use of IO CPU. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org
[jira] Updated: (LUCENE-2555) Remove shared doc stores
[ https://issues.apache.org/jira/browse/LUCENE-2555?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Busch updated LUCENE-2555: -- Attachment: lucene-2555.patch Changed the patch to also remove PerDocBuffer. It changes StoredFieldsWriter and TermVectorsTermsWriter to write the data directly to the final IndexOutput, without buffering it in a temporary PerDocBuffer. Several tests still fail due to exception handling or thread-safety problems (which is expected - haven't tried very hard to fix them yet). I will commit this patch to the realtime branch and work on fixing the tests with a separate issue. Remove shared doc stores Key: LUCENE-2555 URL: https://issues.apache.org/jira/browse/LUCENE-2555 Project: Lucene - Java Issue Type: Improvement Components: Index Reporter: Michael Busch Assignee: Michael Busch Priority: Minor Fix For: Realtime Branch Attachments: lucene-2555.patch, lucene-2555.patch With per-thread DocumentsWriters sharing doc stores across segments doesn't make much sense anymore. See also LUCENE-2324. -- 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-unsubscr...@lucene.apache.org For additional commands, e-mail: dev-h...@lucene.apache.org