[GitHub] [lucene] javanna commented on a diff in pull request #12523: TaskExecutor waits for all tasks to complete before returning
javanna commented on code in PR #12523: URL: https://github.com/apache/lucene/pull/12523#discussion_r1315463865 ## lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java: ## @@ -50,16 +51,21 @@ final List invokeAll(Collection> tasks) { for (Runnable task : tasks) { executor.execute(task); } + +RuntimeException exc = null; final List results = new ArrayList<>(); for (Future future : tasks) { try { results.add(future.get()); } catch (InterruptedException e) { -throw new ThreadInterruptedException(e); +exc = new ThreadInterruptedException(e); } catch (ExecutionException e) { -throw new RuntimeException(e.getCause()); +exc = new RuntimeException(e.getCause()); Review Comment: I wonder if we should add the previous exception as suppressed exception or something along those lines, rather than throwing only the last ones and not letting users know about the others. This can totally be a follow-up change though. ## lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java: ## @@ -267,11 +266,130 @@ protected LeafSlice[] slices(List leaves) { return slices.toArray(new LeafSlice[0]); } }; -searcher.search(new MatchAllDocsQuery(), 10); +TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), 10); +assertTrue(topDocs.totalHits.value > 0); if (leaves.size() <= 1) { assertEquals(0, numExecutions.get()); } else { assertEquals(leaves.size(), numExecutions.get()); } } + + /** + * Tests that when IndexerSearcher runs concurrent searches on multiple slices if any Exception is + * thrown by one of the slice tasks, it will not return until all tasks have completed. + * + * Without a larger refactoring of the Lucene IndexSearcher and/or TaskExecutor there isn't a + * clean deterministic way to test this. This test is probabilistic using short timeouts in the + * tasks that do not throw an Exception. + */ + public void testMultipleSegmentsOnTheExecutorWithException() { +List leaves = reader.leaves(); +int fixedThreads = leaves.size() == 1 ? 1 : leaves.size() / 2; + +ExecutorService fixedThreadPoolExecutor = +Executors.newFixedThreadPool(fixedThreads, new NamedThreadFactory("concurrent-slices")); + +IndexSearcher searcher = +new IndexSearcher(reader, fixedThreadPoolExecutor) { + @Override + protected LeafSlice[] slices(List leaves) { +ArrayList slices = new ArrayList<>(); +for (LeafReaderContext ctx : leaves) { + slices.add(new LeafSlice(Arrays.asList(ctx))); +} +return slices.toArray(new LeafSlice[0]); + } +}; + +try { + AtomicInteger callsToScorer = new AtomicInteger(0); + int numExceptions = leaves.size() == 1 ? 1 : RandomizedTest.randomIntBetween(1, 2); + MatchAllOrThrowExceptionQuery query = + new MatchAllOrThrowExceptionQuery(numExceptions, callsToScorer); + RuntimeException exc = expectThrows(RuntimeException.class, () -> searcher.search(query, 10)); + // if the TaskExecutor didn't wait for all tasks to finish, this assert would frequently fail + assertEquals(leaves.size(), callsToScorer.get()); + assertThat( + exc.getMessage(), Matchers.containsString("MatchAllOrThrowExceptionQuery Exception")); +} finally { + TestUtil.shutdownExecutorService(fixedThreadPoolExecutor); +} + } + + private static class MatchAllOrThrowExceptionQuery extends Query { + +private final AtomicInteger numExceptionsToThrow; +private final Query delegate; +private final AtomicInteger callsToScorer; + +/** + * Throws an Exception out of the {@code scorer} method the first {@code numExceptions} times it + * is called. Otherwise, it delegates all calls to the MatchAllDocsQuery. + * + * @param numExceptions number of exceptions to throw from scorer method + * @param callsToScorer where to record the number of times the {@code scorer} method has been + * called + */ +public MatchAllOrThrowExceptionQuery(int numExceptions, AtomicInteger callsToScorer) { + this.numExceptionsToThrow = new AtomicInteger(numExceptions); + this.callsToScorer = callsToScorer; + this.delegate = new MatchAllDocsQuery(); +} + +@Override +public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) +throws IOException { + Weight matchAllWeight = delegate.createWeight(searcher, scoreMode, boost); + + return new Weight(delegate) { +@Override +public boolean isCacheable(LeafReaderContext ctx) { + return matchAllWeight.isCacheable(ctx); +} + +@Override +public Explanation explain(LeafReaderContext context, int doc) throws IOException { + return ma
[GitHub] [lucene] javanna commented on a diff in pull request #12516: Unwrap execution exceptions cause and rethrow as is when possible
javanna commented on code in PR #12516: URL: https://github.com/apache/lucene/pull/12516#discussion_r1315559947 ## lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java: ## @@ -57,6 +58,12 @@ final List invokeAll(Collection> tasks) { } catch (InterruptedException e) { throw new ThreadInterruptedException(e); } catch (ExecutionException e) { +if (e.getCause() instanceof IOException ioException) { + throw ioException; +} +if (e.getCause() instanceof RuntimeException runtimeException) { + throw runtimeException; +} throw new RuntimeException(e.getCause()); Review Comment: yep, thanks for the suggestion, makes sense to me. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] javanna commented on pull request #12515: Offload single slice to executor
javanna commented on PR #12515: URL: https://github.com/apache/lucene/pull/12515#issuecomment-1706176412 I added the entry to the changelog, I now have a test failure around exception handling which is going to be addressed by #12516 , so I'd merge that one first and only then this PR. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] javanna commented on a diff in pull request #12516: Unwrap execution exceptions cause and rethrow as is when possible
javanna commented on code in PR #12516: URL: https://github.com/apache/lucene/pull/12516#discussion_r1315608401 ## lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java: ## @@ -95,20 +95,17 @@ public Query rewrite(IndexSearcher indexSearcher) throws IOException { } private TopDocs[] sequentialSearch( - List leafReaderContexts, Weight filterWeight) { -try { - TopDocs[] perLeafResults = new TopDocs[leafReaderContexts.size()]; - for (LeafReaderContext ctx : leafReaderContexts) { -perLeafResults[ctx.ord] = searchLeaf(ctx, filterWeight); - } - return perLeafResults; -} catch (Exception e) { - throw new RuntimeException(e); Review Comment: This looks like a way to compensate for the exception wrapping when doing concurrent search, by wrapping when doing sequential search do. This made it easier to test knn vector queries so far, regardless of whether rewrite is done concurrently or not. If we fix the root cause and remove the wrapping, we no longer need to artificially wrap here. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] mikemccand commented on issue #12536: Remove `lastPosBlockOffset` from term metadata for Lucene90PostingsFormat
mikemccand commented on issue #12536: URL: https://github.com/apache/lucene/issues/12536#issuecomment-1706251517 Sorry, what went wrong when you tried to remove `lastPosBlockOffset`? Skipping is (currently) only at every 8 blocks boundaries right? So doesn't the skip data already know how many positions were skipped when it skips into a spot? Hmm I think I see -- there are in general more positions blocks (since there can of course be many more positions than doc/freq) than doc/freq blocks, and so as `PostingsEnum` is skipping or reading positions, it must know when it transitions into the final (`vInt` encoded) chunk of positions? Hmm but again it seems like tracking number of positions skipped/read vs `totalTermFreq` should be enough? Maybe make a PR adding a nice comment explaining why we really do need this `int lastPosBlockOffset`? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] jpountz commented on a diff in pull request #12516: Unwrap execution exceptions cause and rethrow as is when possible
jpountz commented on code in PR #12516: URL: https://github.com/apache/lucene/pull/12516#discussion_r1315641242 ## lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java: ## @@ -57,6 +58,12 @@ final List invokeAll(Collection> tasks) { } catch (InterruptedException e) { throw new ThreadInterruptedException(e); } catch (ExecutionException e) { +if (e.getCause() instanceof IOException ioException) { + throw ioException; +} +if (e.getCause() instanceof RuntimeException runtimeException) { + throw runtimeException; +} throw new RuntimeException(e.getCause()); Review Comment: Maybe reuse `IOUtils#rethrowAlways` here? `throw IOUtils.rethrowAlways(e.getCause());` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] mikemccand commented on issue #12513: Try out a tantivy's term dictionary format
mikemccand commented on issue #12513: URL: https://github.com/apache/lucene/issues/12513#issuecomment-1706283574 > Yes, I actually tried to use FSTPostingsFormat in the benchmarks game and I had to increase the heap size from 4g to 32g to workaround the in-heap memory demand. Do you know whether Tantivy is producing a truly minimal FST? Doing so (which Lucene does, by default, I think!) requires a big hash map during indexing to keep track of already-seen suffixes and share them, making a minimal FST that is like a combined prefix and suffix trie. If you disable this entirely, the FST becomes a prefix trie. You can play with Lucene NOT trying so hard to make a minimal FST by increasing the `minSuffixCount` and `minSuffixCount2` in the `FSTCompiler.Builder` from 0 to 2 or 3 -- this should make FST compilation faster, less RAM hungry, and increase its size somewhat (perhaps negligibly in practice; maybe we should change `Lucene90PostingsWriter`'s defaults here?). Maybe we could try a standalone test to build an FST with both Tantivy and Lucene and compare the byte size? I'll open an issue over in [`search-benchmark-game`](https://github.com/Tony-X/search-benchmark-game)! > Search-wise, the performance got slightly bit worse. This is curious -- I would expect the terms dict lookup cost to be unimportant for queries that visit many hits. The terms dict cost should be dwarfed by the cost of iterating the postings. Did you see the slowdown only for terms dict intensive queries? We should really test e.g. `FuzzyQuery` or `RegexpQuery` or maybe `TermInSetQuery` with a large set of primary key values, to suss out the impact here. Or maybe the performance drop was under the noise floor of the benchy...? > > Term dictionary heavy queries, e.g. FuzzyQuery or RegexpQuery, might become faster? Maybe this eventually becomes Lucene's default terms dictionary! > > Yes, this can be very promising :) The fact that it is FST and contains all terms makes it efficient to skip no-existent terms. +1, this is an exciting exploration! Note that there are certain cases (perhaps rare in practice, not sure) where even Lucene's "prefix" FST can skip a segment without checking the on-disk terms blocks. It happens when even the prefix for a given term never occurs in any other terms in that segment, which might be frequent if say documents are indexed in sorted order by their primary keys. This would cause certain "dense" regions of primary key space to be indexed into each segment and might then mean on lookup that the prefix FST can know that a given PK cannot occur in the segment without even scanning the on-disk blocks. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] mikemccand commented on issue #12513: Try out a tantivy's term dictionary format
mikemccand commented on issue #12513: URL: https://github.com/apache/lucene/issues/12513#issuecomment-1706307210 > 2. random-addressing term information given an ordinal. again no additional scan; Hmm indeed this would require a fixed block size for every term's metadata. Does Tantivy do pulsing (inlining postings for a singleton terms into the terms dictionary)? Another option might be to have each term block have some sort of header array to quickly map a term ordinal (within the block) to its corresponding file pointer location? Or perhaps we keep the scanning within a block when looking for an ord within that block? The FST would still be definitive about whether a term exists or not, but then when it exists, we would still need to do some scanning. Or, for starters, just make all terms metadata fixed width (yes, wasting bytes for those terms that don't need the extra stuff). It'd be a start just to simplify playing with this idea, which we could then iterate from? > I have not gone to the full details of the [paper](https://citeseerx.ist.psu.edu/doc/10.1.1.24.3698) that underpins FSTCompiler implementation but I believe mapping to 8-byte ordinals (monotonically increasing) are much easier than mapping to variable-length and unordered byte[] blobs. Yeah -- this is very true! FST is very efficient at encoding monotonically increasing int/long outputs, much more so than semi-random looking `byte[]` blobs that don't often share prefixes. Also, with a monotonically increasing output, reverse lookup (ord -> term) becomes possible! I think `FSTOrdPostingsFormat` does that (implements the optional `TermsEnum.seekExact(long ord)` API)? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] SevenCss commented on issue #7820: CheckIndex cannot "fix" indexes that have individual segments with missing or corrupt .si files because sanity checks will fail trying to read the
SevenCss commented on issue #7820: URL: https://github.com/apache/lucene/issues/7820#issuecomment-1706310673 @mikemccand Appreciated for your response. Glad to know that. I will have a try. :) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] fulmicoton commented on issue #12513: Try out a tantivy's term dictionary format
fulmicoton commented on issue #12513: URL: https://github.com/apache/lucene/issues/12513#issuecomment-1706314422 > Does Tantivy do pulsing (inlining postings for a singleton terms into the terms dictionary)? No but we should. It has been on my task list for a long time. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] SevenCss commented on issue #7820: CheckIndex cannot "fix" indexes that have individual segments with missing or corrupt .si files because sanity checks will fail trying to read the
SevenCss commented on issue #7820: URL: https://github.com/apache/lucene/issues/7820#issuecomment-1706323759 > > Third off, there is possibly a separate improvement we could make to IndexWriter, to remove segments_N files before removing all other files when a commit point is deleted, to try to reduce the chance of an index getting into this state. That has a nice symmetry with how we write a commit (write various files first, and only when that succeeds do we write and fsync the segments_N referencing them). I'll open a follow-on issue for that. Let's focus for this issue on fixing this bug in CheckIndex. > > Oh, nevermind about this improvement -- this is [already how Lucene does it](https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/util/FileDeleter.java#L212-L220)! Which makes it all the more mystifying how an index can get into this exotic form of corruption... @mikemccand Appreciated for your response. Glad to know that. I will have a try. :) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] javanna commented on a diff in pull request #12516: Unwrap execution exceptions cause and rethrow as is when possible
javanna commented on code in PR #12516: URL: https://github.com/apache/lucene/pull/12516#discussion_r1315852459 ## lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java: ## @@ -57,6 +58,12 @@ final List invokeAll(Collection> tasks) { } catch (InterruptedException e) { throw new ThreadInterruptedException(e); } catch (ExecutionException e) { +if (e.getCause() instanceof IOException ioException) { + throw ioException; +} +if (e.getCause() instanceof RuntimeException runtimeException) { + throw runtimeException; +} throw new RuntimeException(e.getCause()); Review Comment: that's a really good idea, thanks, I did not know this method existed, and it's exactly what I needed. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] javanna merged pull request #12516: Unwrap execution exceptions cause and rethrow as is when possible
javanna merged PR #12516: URL: https://github.com/apache/lucene/pull/12516 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] javanna merged pull request #12515: Offload single slice to executor
javanna merged PR #12515: URL: https://github.com/apache/lucene/pull/12515 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] javanna closed issue #12498: Simplify task executor for concurrent operations
javanna closed issue #12498: Simplify task executor for concurrent operations URL: https://github.com/apache/lucene/issues/12498 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] romseygeek commented on issue #12538: clarify QueryVisitor.visitLeaf interaction with QueryVisitor.acceptField
romseygeek commented on issue #12538: URL: https://github.com/apache/lucene/issues/12538#issuecomment-1706747711 I would adjust the docs on `acceptField` to just say "Whether or not this field is of interest to the visitor". It shouldn't be term-specific. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] rmuir commented on pull request #12535: LockVerifyServer does not need to reuse addresses nor set accept timeout
rmuir commented on PR #12535: URL: https://github.com/apache/lucene/pull/12535#issuecomment-1706783612 The 500ms connect timeout on the client: extremely suspicious (as far as jenkins failure). Make it at least 3 seconds or something? Doesn't matter if its being used by jenkins or not, there's sorta no sense in extremely low timeout values like that when using TCP... -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] uschindler commented on pull request #12535: LockVerifyServer does not need to reuse addresses nor set accept timeout
uschindler commented on PR #12535: URL: https://github.com/apache/lucene/pull/12535#issuecomment-1706816121 > The 500ms connect timeout on the client: extremely suspicious (as far as jenkins failure). Make it at least 3 seconds or something? Doesn't matter if its being used by jenkins or not, there's sorta no sense in extremely low timeout values like that when using TCP... Thanks! Maybe that is the reason for the problem, I haven't seen the timeout on client! If one of the clients times out, the server will wait for that client forever. As we had not stackdump of the client we can't see that. I agree: - lets raise the connect timeout - DO NOT remove the server timeout, because a fileing client will otherwise make the server wait forever for the missing client after accepting connections of all others Uwe -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] Tony-X commented on issue #12536: Remove `lastPosBlockOffset` from term metadata for Lucene90PostingsFormat
Tony-X commented on issue #12536: URL: https://github.com/apache/lucene/issues/12536#issuecomment-1706979791 Today when it skips, the skipper can tell us 1) the offset of the position block we should seek to 2) how many positions it needs to skip within this block. This is because post-skipping the first doc's position is not always at the start of a positions block (that's actually the most common case). Additionally, advancing to a specific doc that is not the start of a block means we need to skip those docs as well as their positions. In theory, if the skipper can tell us how many positions it has skipped that would work. This will require storing more information in the skip data than the current scheme. > Maybe make a PR adding a nice comment explaining why we really do need this int lastPosBlockOffset? Sure, I can enhance the documentation of `IntBlockTermState` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org
[GitHub] [lucene] Tony-X commented on issue #12513: Try out a tantivy's term dictionary format
Tony-X commented on issue #12513: URL: https://github.com/apache/lucene/issues/12513#issuecomment-1707066423 > Do you know whether Tantivy is producing a truly minimal FST? Maybe @fulmicoton can shed more light on this topic :) A related question: can Tantivy read a Lucene-built FST and vice-versa? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org