Skip to content

fix cache populate incorrect content when numBackgroundThreads>1#3943

Merged
nishantmonu51 merged 3 commits intoapache:masterfrom
kaijianding:cachebug
Feb 17, 2017
Merged

fix cache populate incorrect content when numBackgroundThreads>1#3943
nishantmonu51 merged 3 commits intoapache:masterfrom
kaijianding:cachebug

Conversation

@kaijianding
Copy link
Copy Markdown
Contributor

The origin code can cause:

  1. concurrency issue
  2. sequence order and cacheResults order maybe different, eg: for timeSeries query,
    the sequence order is
[{
  "timestamp" : "2017-02-16T09:00:48.000+08:00",
  "result" : {
    "__result" : 1.0
  }
}, {
  "timestamp" : "2017-02-16T09:01:04.000+08:00",
  "result" : {
    "__result" : 1.0
  }
}, {
  "timestamp" : "2017-02-16T09:01:20.000+08:00",
  "result" : {
    "__result" : 1.0
  }
}]

but the cacheResults order can be

[{
  "timestamp" : "2017-02-16T09:00:48.000+08:00",
  "result" : {
    "__result" : 1.0
  }
}, {
  "timestamp" : "2017-02-16T09:01:20.000+08:00",
  "result" : {
    "__result" : 1.0
  }
}, {
  "timestamp" : "2017-02-16T09:01:04.000+08:00",
  "result" : {
    "__result" : 1.0
  }
}]

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.

Copy link
Copy Markdown
Contributor

@gianm gianm left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it possible to do this with a CountDownLatch or similar so we are guaranteed it finishes?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Modified, @gianm

@gianm gianm added this to the 0.10.0 milestone Feb 16, 2017
@gianm gianm added the Bug label Feb 16, 2017
try {
Futures.allAsList(cacheFutures).get();
CacheUtil.populate(cache, mapper, key, cacheResults);
CacheUtil.populate(cache, mapper, key, Iterables.transform(
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I believe there's a Futures.allAsList method in guava

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Modified, @drcrallen

@drcrallen
Copy link
Copy Markdown
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.

new DateTime("2011-01-09"), "a", 50, 4985, "b", 50, 4984, "c", 50, 4983
};

private ExecutorService backgroundExecutorService;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

please add a new line after this.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Modefied, @KurtYoung


// wait for background caching finish
// wait at most 10 seconds to fail the test to avoid block overall tests
cacheMustBePutOnce.await(10, TimeUnit.SECONDS);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm on my phone right now but doesn't await just return a boolean and not actually throw an error on timeout?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Modified, @drcrallen

Copy link
Copy Markdown
Member

@nishantmonu51 nishantmonu51 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍

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 this pull request may close these issues.

5 participants