Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Poor multi-threaded indexing performance #935

Open
1 task done
superkelvint opened this issue Apr 25, 2024 · 21 comments · May be fixed by #940
Open
1 task done

Poor multi-threaded indexing performance #935

superkelvint opened this issue Apr 25, 2024 · 21 comments · May be fixed by #940
Labels

Comments

@superkelvint
Copy link

superkelvint commented Apr 25, 2024

Is there an existing issue for this?

  • I have searched the existing issues

Describe the bug

I wrote a multithreaded indexer in Java for Lucene 4.8 and one for Lucene.net 4.8. I tried to do pretty much a line-for-line port. However, there is a substantial difference in the performance characteristics of the two. In Java, I used a ThreadPoolExecutor with CallerRunsPolicy, and in C# I used Parallel.ForEach.

  1. The Java program scales more or less linearly to the number of CPUs.
  2. The C# program doesn't seem to fundamentally speed up when increasing the number of threads.
  3. The Java program completes in about 8 minutes with 4 threads, the Lucene.net version takes about 65 minutes with 4, 10 or even 20 threads.
  4. It was very easy to max out CPU utilization in Java and pretty much impossible to do so in C#.

I can provide both Java and .NET files, but before doing so, I wanted to understand if this is expected behavior and therefore not a bug.

I am observing identical behavior in .NET 8 as well as .NET 6, so I don't think its related to #933

Expected Behavior

Multi-threaded indexing performance scales with the number of threads used.

Steps To Reproduce

No response

Exceptions (if any)

No response

Lucene.NET Version

4.8

.NET Version

8.0.204 and 6.0.29

Operating System

Windows 10

Anything else?

No response

@paulirwin
Copy link
Contributor

I would say this is very much not expected behavior. Having your benchmark code would be very helpful! Thanks for the report.

@eladmarg
Copy link
Contributor

this sounds too much difference between java / c#.
do you add the documents and commit only in the end?

a test program would be great to analyze this.
thanks

@NightOwl888
Copy link
Contributor

Concurrent document writing support is currently broken due to a bug mentioned in #886 (comment). No objections if someone wants to contribute a solution to that issue, but this isn't considered a blocker for the Lucene.NET 4.8.0 release.

@eladmarg
Copy link
Contributor

@NightOwl888 I remember this was an old bug relevant only to older dotnet core.
so if its not reproducible on the new dotnet, and only happen on versions that out of support anyway, maybe we shouldn't too

@jeme
Copy link
Contributor

jeme commented Apr 25, 2024

@NightOwl888 If you apply the changes to DocumentsWriterFlushControl from that branch on top of the latest can you still reproduce it? I have no luck on 5.0, 6.0 or 7.0 - but asking because maybe I am doing something wrong >.<

(I have not run the entire test suite yet, there is allot of test and I have problems compiling the IKVM stuff)

@superkelvint Sharing the source code would certainly be relevant, Parallel.ForEach would not be my first choice here.

@superkelvint
Copy link
Author

@superkelvint Sharing the source code would certainly be relevant, Parallel.ForEach would not be my first choice here.

Yes, I just got approval to do so, and will be providing that shortly.

@superkelvint
Copy link
Author

Here is the MultithreadedIndexer.cs file.
MultithreadedIndexer.cs.zip

I'm not a C# programmer so perhaps I am doing something silly that's slowing things down.

@NightOwl888
Copy link
Contributor

@NightOwl888 I remember this was an old bug relevant only to older dotnet core. so if its not reproducible on the new dotnet, and only happen on versions that out of support anyway, maybe we shouldn't too

Nope. This happens on all operating systems and .NET versions. I got it to fail on .NET 7 after that post.

We cannot rule out that the problem lies in the test or the test framework, but it almost certainly is in our codebase. We have a configuration that never existed in Lucene. We are somewhere between 4.8.0 and 4.8.1 on the documents writer stuff. We should go through the JIRA issues in the Changes.txt to determine if we have all of the changes that are relevant to the documents writer. That may or may not be contributing to this.

@NightOwl888 If you apply the changes to DocumentsWriterFlushControl from that branch on top of the latest can you still reproduce it? I have no luck on 5.0, 6.0 or 7.0 - but asking because maybe I am doing something wrong >.<

(I have not run the entire test suite yet, there is allot of test and I have problems compiling the IKVM stuff)

IKVM can be tricky to get to compile the first time because it doesn't properly clean the build. I suggest closing the IDE and then doing the following:

  • Delete all of your /bin and /obj folders (be careful not to delete the objects folder below /.git)
  • Clear the IKVM cache in the ~\AppData\Local\Temp\ikvm\cache folder
  • Clear the ~\.m2 cache

If upon reopening you still have issues compiling, report the issue to ikvm-maven. The team is pretty responsive.


On Azure DevOps, I was able to reproduce the test failure, but this time it took 60 runs and it only happened once: https://dev.azure.com/lucene-net-temp4/Main/_build/results?buildId=266&view=ms.vss-test-web.build-test-results-tab&runId=191370&resultId=100202&paneView=debug

I was also able to get it to fail locally with the seed from #886. Using the [Repeat(1000)] attribute on the test, it failed on the first run on net6.0 on Windows.

image

I just put the attribute in the TestRollingUpdates.cs file, but be sure to remove it when you are done debugging.

image

image

It also failed on the first run with the seed from the above Azure DevOps failure on net6.0 on Windows.

[assembly: Lucene.Net.Util.RandomSeed("0x9f28125320b5e6c3")]
[assembly: NUnit.Framework.SetCulture("lrc-IR")]

I also got it to fail on net7.0 with that seed on the first run, but it took significantly longer.

image

@pc-LMS
Copy link

pc-LMS commented May 7, 2024

@jeme thoughts on the included code within MultithreadedIndexer.cs file?

@jeme
Copy link
Contributor

jeme commented May 14, 2024

@pc-LMS I have actually not looked further into it as I have used the time I have for Lucene (here and there) to attempt to reproduce the problem linked by @NightOwl888 locally (as that would ease things hugely for correcting it) or just trying to unravel the code in an attempt to get a theoretical idea of what might be wrong, so far no luck on reproducing it though.

@NightOwl888
Copy link
Contributor

@jeme - I am a bit perplexed why you cannot reproduce this. While it is not trivial to do, it also is not that hard and I have reproduced it on multiple machines.

It is possible that if the problem is a timing issue, perhaps the particular machine that you are experimenting with is somehow perfectly synchronized to miss the error, too fast/slow, or the timing accuracy of the underlying OS is sufficient to where it doesn't happen. Have you attempted to reproduce on more than one machine and/or OS?

@jeme
Copy link
Contributor

jeme commented May 15, 2024

@NightOwl888 No, I have not tried other machines or OS'es for now, I have tried running the tests under various .NET versions with the seeds provided, I would have to install VM's for trying other OS'es for now (or dust off my old 2012 Mac book PRO and boot that into OSX, again it would take some time).

But I could perhaps find a few other machines to run it on, my machine is a bit dated (i9-7900X, 128GB Ram and M.2 SSD's). If you can reproduce it locally, what are you running on? (OS etc).

Also, are you running all the test suites, just that project containing that test, just that suite (file) or can you reproduce it with just running that one test alone?

Because if this is timing and mine just runs slower pr. thread (10 cores means lower clock speed after all + the IPC has increased a bit since then I guess).

@NightOwl888
Copy link
Contributor

I am on Windows 10 x64 running with an I7-8850H, 2.60Ghz, 6 Cores, 12 Logical Processors, 32GB RAM, and M.2 SSD's.

I am unable to reproduce the failure locally without the [Repeat(1000)] attribute on the TestRollingUpdates.TestUpdateSameDoc() method since the failure happens so rarely. It usually fails within the first 3 runs with that attribute applied. And I am explicitly running just that one test using Visual Studio 2022. But I have also had it fail when running the entire test project that it is in.

FYI - I did a new manual revert that is up to date with master here: https://github.com/NightOwl888/lucenenet/tree/fix/GH-935-documentswriter-concurrency

@jeme
Copy link
Contributor

jeme commented May 16, 2024

Ok, I was actually in the thinking that this was because my CPU was a slow old one, but maybe it's the other way around. It passes just fine, takes a while with repeat:1000 obviously, but since it at max starts 6 threads I tried upping that seeing as it would never occupy all Cores(10)/Logical(20) on my system. Thinking that it could also be if the system was put under stress, even with a random between 32 to 64 threads, my system is just not stressed at all, but I still wonder if stressing things could make it easier to reproduce, so my plan is to do 2 things after work today.

A) Move the code to my old MacBook (with windows) and run the tests there, that would run on a Core i7 i7 3820QM.
B) Write a stress test suite that tries to push things as high as I can.

Thanks for the updated source/branch.

So status is that the test passes for me with:

[assembly: Lucene.Net.Util.RandomSeed("0x9f28125320b5e6c3")]
[assembly: NUnit.Framework.SetCulture("lrc-IR")]
[Repeat(1000)]

(what you can reproduce it with)

And it also passes with in addition to the above:

int numThreads = TestUtil.NextInt32(Random, 32, 64);

@NightOwl888
Copy link
Contributor

Interesting. I tried with net8.0 (which has been changed as the default IDE test platform since the last time I checked) with the same configuration you used and it doesn't fail there. But then I changed the test target to net6.0 and it failed on the first run. So, that probably explains why you are not reproducing it.

I think I worked out what is happening, though.

Issue 1

The error indicates that dwpt is null on this line:

https://github.com/NightOwl888/lucenenet/blob/878261fa920a961288639863da28d7c9f5d0d800/src/Lucene.Net/Index/DocumentsWriterFlushControl.cs#L471

Digging into the DocumentsWriterPerThreadPool, there is only 1 way it could return null, and that is if this method is called: https://github.com/NightOwl888/lucenenet/blob/878261fa920a961288639863da28d7c9f5d0d800/src/Lucene.Net/Index/DocumentsWriterPerThreadPool.cs#L91

We cannot pass this line unless perThread.IsInitialized returns true. And that can only happen if dwpt is not null.

Therefore, to fail in this way, DocumentsWriterPerThreadPool.Reset(perThread, closed) must be called on a different thread while these lines are executing. This explains why it is an exceedingly rare event.

I checked and there are 4 calls to DocumentsWriterPerThreadPool.Reset(perThread, closed). All of them are called within UninterruptableMonitor.Enter(this) except for this one:

https://github.com/NightOwl888/lucenenet/blob/878261fa920a961288639863da28d7c9f5d0d800/src/Lucene.Net/Index/DocumentsWriterFlushControl.cs#L818

So, moving the UninterruptableMonitor.Enter(this) outside of that call should fix the problem.

Issue 2

After that fix, we still got some failures. But it was also clear from the stack trace what was happening. If TryLock() returns false, we end up on this line:

https://github.com/NightOwl888/lucenenet/blob/fix/GH-935-documentswriter-concurrency/src/Lucene.Net/Index/DocumentsWriterFlushControl.cs#L484

The documentation for ReentrantLock.tryLock() states:

Even when this lock has been set to use a fair ordering policy, a call to tryLock() will immediately acquire the lock if it is available, whether or not other threads are currently waiting for the lock. This "barging" behavior can be useful in certain circumstances, even though it breaks fairness. If you want to honor the fairness setting for this lock, then use tryLock(0, TimeUnit.SECONDS) which is almost equivalent (it also detects interruption).

If the current thread already holds this lock then the hold count is incremented by one and the method returns true.

If the lock is held by another thread then this method will return immediately with the value false.

So, it can return false, but I don't think it ever does in practice since a null return value from InternalTryCheckOutForFlush() will cause an error (assert in test, NullReferenceException in production). So, apparently Lucene relies on the "barging" behavior of this method to be put in the front of the queue.

.NET has no such behavior, but I don't think it really matters which order the locks are taken in as long as ReentrantLock.TryLock() returns true. So, I changed it to call UninterruptableMonitor.Enter(object, ref bool) instead of UninterruptableMonitor.TryEnter(object), which will always acquire the lock and return true.

This probably won't perform as well because the method is supposed to return immediately and may end up deadlocking, but so far it seems to perform better and never deadlock.


I am running the tests now to see if the above changes work on all platforms, but locally this seems promising.

NightOwl888 added a commit to NightOwl888/lucenenet that referenced this issue May 23, 2024
…): The DocumentsWriterThreadPool.Reset() method must be called within the context of a lock because there is a possible race condition with other callers of DocumentsWriterThreadPool.Reset(). See apache#935.
NightOwl888 added a commit to NightOwl888/lucenenet that referenced this issue May 23, 2024
…ntLock.tryLock() method barges to the front of the queue instead of returning false like Monitor.TryEnter(). Use Monitor.Enter(object, ref bool) instead, which always returns true. We get locks in a different order, but I am not sure whether that matters. Fixes apache#935. Closes apache#886.
@NightOwl888
Copy link
Contributor

I have just submitted #940 with a patch. Well, there were 3 different attempts.

While identifying the problem wasn't too difficult, finding a solution for Issue 2 turned out to be quite challenging due to how performance sensitive of an area this is.

If interested parties could run some benchmarks and/or review the PR to provide some feedback on how to improve this approach, suggestions would be welcomed. I have already attempted Thread.Sleep(1), Thread.SpinWait(1) and the SpinWait struct as part of the retry loop, but all of these seem to slow the tests down a lot. I don't think it would work right if Thread.Yield() were not also used, but it seems to perform best with only Thread.Yield() and nothing else.

I also gave SpinLock a try. Although it has all of the API members we need, it is not a reentrant implementation, so no dice.

@jeme
Copy link
Contributor

jeme commented May 24, 2024

Interesting. I tried with net8.0 (which has been changed as the default IDE test platform since the last time I checked) with the same configuration you used and it doesn't fail there. But then I changed the test target to net6.0 and it failed on the first run. So, that probably explains why you are not reproducing it.

Not really as I actually tested on the various .net versions, other things came up as we had a break down in one of our systems due to a windows update, so I have sadly been busy elsewhere.

But I am actually having another timing related bug in another codebase which I can easily reproduce at work with a single run of a test (always fails after ~max 2min, a run is 10min), but it's far harder to reproduce the same bug at home on the same PC as I am having trouble reproducing this issue (takes 2-3 runs). So I don't know.

Running the Lucene tests at work has proven to be a challenge though, I don't know if it's because I use ReSharper that the Lucene solution seems to be very slugish for me to work with, or it's something else. But I always have challenges getting started working on anything.

But great if you found a fix.

@rclabo
Copy link
Contributor

rclabo commented Jun 5, 2024

Here is the MultithreadedIndexer.cs file. MultithreadedIndexer.cs.zip

I'm not a C# programmer so perhaps I am doing something silly that's slowing things down.

This is super helpful. I'll work to reproduce the behavior you observed in Luncene.NET and then test to see PR #940 shows improved performance.

I'm curious, for your StringFields, how many did the typical doc have, and what was the typical length of these fields?

@sgf
Copy link

sgf commented Jun 11, 2024

If it is difficult to reproduce, can try to remote and debug the bug using DNSpy on the user's computer.

@rclabo
Copy link
Contributor

rclabo commented Jun 17, 2024

@superkelvint Which specific version of Java Lucene did you use in your comparison to Lucene.NET?

@rclabo
Copy link
Contributor

rclabo commented Jun 17, 2024

@superkelvint Thank you for providing some code to show how you are using Lucene.NET. That helps a lot. Maybe we can work together to make sense out of what you are seeing.

One challenge in creating any kind of benchmark is having enough data to index so that it takes some time for LuceneNET to complete. LuceneNET is very fast, so thousands of records of data are needed. I went on a hunt for an open-source data set that we could use. Ultimately I chose the Book Dataset from Kaggle.com, which is available under a CC0 public domain license.

The dataset has approx. 271K records and each record contains several fields (Book Title, Author, URL to cover photo, etc.). To allow the program to load data in parallel, I created 21 copies of this data which is 5.4 million records total. Because sometimes it’s nice to be able to run the program more quickly when tweaking settings to see if they matter much, I also create 21 “small” data files, which contain about 50,000 records each for a total of 1.05 million records total.

A single flag at the top of the program causes it to index big or small files. Either way, it’s indexing 21 files.
I wrote a DataReader class that can read one record at a time from one of these data files. It uses a Stream, which in .NET is a performant approach. The implementation uses synchronous IO, but I tried a version of the app with async IO (async await) for reading these data files, and it made no difference from a performance perspective.

Using this console app I tested the performance of Lucene.NET Beta 16, vs, the current master vs, PR#940 and I get more or less the same results from each.

I tried different levels of parallelism to see how that affected performance and also tried different levels of RAMBufferSizeMB. In the process, I came to realize that by default Lucen.NET 4.8 will use a MAX of 8 threads for creating index segments via DWTP threads when using the IndexWriter. At first, I thought perhaps this was the issue and a larger value just needed to be passed via the IndexWriterConfig.MaxThreadStates when the IndexWriter is created. But that didn’t turn out to be true. In fact, on my hardware I ultimately got faster performance by passing a SMALLER value then the default for the indexWriterConfig.MaxThreadStates value.

I’m providing this full working .NET console app that uses .NET 8 including the data used and solution file and structure. So you can easily run this on your hardware and try different values for the number of threads used to read the data files, the MaxThreadStates, and the RAMBufferSizeMB value. What I’m most interested in, is having you port this code to java and run it using the exact same dataset in a java environment. The solution with all the data was 412MB which GitHub wouldn't let me upload (it has a 25MB limit) so I trimmed the solution down to one small data file and one big one and then when the app runs it will create the other 20 small data files and 20 big ones on the fly. This got the app just barely under the upload limit. :-)

Ideally run the console using the big data set, which is how I have the app configured by default. The app auto-deletes the LuceneIndex folder before each run. So you can run it multiple times in a row and you will be left with just the LuceneIndex from the last run in the LuceneIndex folder.

I would love to hear back your Lucene.NET 4.8 results vs Java results and have you contribute back your java code port of this console app. I'd also like to understand exactly which version of Java Lucene you are benchmarking against.

On my machine, Lucene.NET can index the 5.65 million book records in 47 seconds, which is a rate of approximately 119K records per second. That average time includes the final commit time.

To my eyes that seems pretty darn fast. But if the Java version can really index these same documents much faster that would be great to know.

LuceneIndexingPerformane.zip

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging a pull request may close this issue.

8 participants