fix cache populate incorrect content when numBackgroundThreads>1#3943
Merged
nishantmonu51 merged 3 commits intoapache:masterfrom Feb 17, 2017
Merged
fix cache populate incorrect content when numBackgroundThreads>1#3943nishantmonu51 merged 3 commits intoapache:masterfrom
nishantmonu51 merged 3 commits intoapache:masterfrom
Conversation
b93df60 to
d011d91
Compare
gianm
reviewed
Feb 16, 2017
Contributor
gianm
left a comment
There was a problem hiding this comment.
LGTM, just a question on the tests.
thx for the fix @kaijianding!
| Assert.assertEquals(expectedRes.toString(), results.toString()); | ||
|
|
||
| // wait for background caching finish | ||
| Thread.sleep(500); |
Contributor
There was a problem hiding this comment.
Is it possible to do this with a CountDownLatch or similar so we are guaranteed it finishes?
drcrallen
reviewed
Feb 16, 2017
| try { | ||
| Futures.allAsList(cacheFutures).get(); | ||
| CacheUtil.populate(cache, mapper, key, cacheResults); | ||
| CacheUtil.populate(cache, mapper, key, Iterables.transform( |
Contributor
There was a problem hiding this comment.
I believe there's a Futures.allAsList method in guava
Contributor
|
At one point this had problems with HLL buffers, I don't remember what the issue was off hand. I think the buffer was being released before the cache was calculated or something. |
KurtYoung
reviewed
Feb 17, 2017
| new DateTime("2011-01-09"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983 | ||
| }; | ||
|
|
||
| private ExecutorService backgroundExecutorService; |
Contributor
There was a problem hiding this comment.
please add a new line after this.
drcrallen
reviewed
Feb 17, 2017
|
|
||
| // wait for background caching finish | ||
| // wait at most 10 seconds to fail the test to avoid block overall tests | ||
| cacheMustBePutOnce.await(10, TimeUnit.SECONDS); |
Contributor
There was a problem hiding this comment.
I'm on my phone right now but doesn't await just return a boolean and not actually throw an error on timeout?
KurtYoung
approved these changes
Feb 17, 2017
drcrallen
approved these changes
Feb 17, 2017
2 tasks
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
The origin code can cause:
the sequence order is
but the cacheResults order can be
Basically, this issue is caused by the cacheResults.add() called order is not promised as same as backgroundExecutorService.submit() called order when numBackgroundThreads>1
This difference can cause incorrect merge sort result on broker.