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
FLUO-1000 OracleServer race conditions #1004
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@kpm1985 nice job on tracking these two issues down. The lack of spurious error messages made viewing the Travis CI output for this PR quite pleasant.
@@ -195,18 +195,25 @@ public OracleServer(Environment env) throws Exception { | |||
|
|||
private void allocateTimestamp() throws Exception { | |||
Stat stat = new Stat(); | |||
byte[] d = curatorFramework.getData().storingStatIn(stat).forPath(maxTsPath); | |||
// FLUO-1000 | |||
synchronized (this) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think adding this synchronized block has any effect because this method is only called from a synchronized method.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please correct me if I am wrong but I was under the impression that object locks do not inherit from my reading of Doug Lea's Concurrent Programming.
https://stackoverflow.com/questions/9525882/if-a-synchronized-method-calls-another-non-synchronized-method-is-there-a-lock
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
That post is correct. However in this situation, the method is private and only called by a synchronized public method. Therefore nothing external can directly call the method in an unsynchronized way. It would be fine to make the whole method synchronized if you think that makes the code more clear.
@@ -410,15 +417,18 @@ public void takeLeadership(CuratorFramework curatorFramework) throws Exception { | |||
} | |||
|
|||
synchronized (this) { | |||
// FLUO-1000 | |||
isLeader = true; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Was this moved to avoid race condition with updateAdvertisedGcTimestamp()
and/or isLeader()
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@kpm1985 after 9b5277c, I think this may be the only change in this PR that still need to be made. It still makes sense to set isLeader before starting gcTsTracker.
@@ -410,15 +417,18 @@ public void takeLeadership(CuratorFramework curatorFramework) throws Exception { | |||
} | |||
|
|||
synchronized (this) { | |||
// FLUO-1000 | |||
isLeader = true; | |||
while (!curatorFramework.getState().equals(CuratorFrameworkState.STARTED)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Would it make sense to move this loop to the OracleServer.start()
method after curatorFramework.start()
is called?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Tried that, it failed. @keith-turner Thank you for looking at this, I've put a lot of invisible work into it and am greatful for your review.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could take another look at that, and try it again though because I did it that way because likewise that makes the best sense.
Looking at the latest travis output, I am still seeing some error messages like the following.
I suspect this is happening because the CuratorFramework was stopped, however I am not sure. I opened CURATOR-448 because looking into this I found the error message confusing. The error message leads one to believe that curator was not started yet, however I think you could see the error message when it was stopped. Looking at Fluo's code it closes the leaderSelector before closing the curatorFramework. I looked at the implementation of the leaderSelector close method and it does not wait for thread it created to terminate. So its possible that when leaderSelector is closed and then the curatorFramework is closed that the thread created by the leaderSelector is till running. It would be good to verify that the state is STOPPED when we see this error message. If it is I think one possible approach is to do something like the following in the takeLeadership method. However I am not sure how to have strong check to ensure the exception came from curator because of the wrong state. @Override
public void takeLeadership(CuratorFramework curatorFramework) throws Exception {
try {
} catch (IllegalStateException e) {
//TODO how can we verify this exception came from Curator???? Don't want to suppress other illegal state exceptions.
if(curatorFramework.getState() == STOPPED) {
log.debug(...); //log a debug message that this happened
} else {
throw e;
}
} finally {
isLeader = false;
if (started) {
// if we stopped the server manually, we shouldn't halt
Halt.halt("Oracle has lost leadership unexpectedly and is now halting.");
}
}
} |
Maybe we don't need to verify that the illegal state came from curator. We could make the check stronger, something like the following. } catch (IllegalStateException e) {
if(curatorFramework.getState() == STOPPED && !started) {
// the oracle server had been stopped, do not care much about errors... lets suppress the exception
log.debug(...); //log a debug message that this happened
} else {
throw e;
}
} |
Hi Keith,
About the errors that show during my verify. My original solution I sit and
check curator state inside of the function takeLadership and wait until its
ready.
I tried moving this check to the start method and I ran Maven verify on
this a few times and to my surprise I kept getting some exceptions on the
state of that curator framework again.
So what I did is I left it in that method that is called by leader selector
and when the check is in that method with a couple other checks like this
synchronized on the leader variable then the errors are not present.
This made me wonder after tracing the flow of what I thought was going on
in Oracle server, well what the heck is happening somebody else is screwing
with something in the background thread somewhere so, I still didn't really
understand what was going on but I left the check inside of the i, I still
didn't really understand what was going on but I left the check inside of
the is take leadership method.
My only thought is possibly multiple instances but I thought those were
private variables so not exactly sure what's going on with that.
Anyways I really appreciate you responding and looking into this with me
this is the help I totally needed. I'm on break with the phone so I
apologize for not highlighting code and doing other things that you can do
on a computer.
…On Jan 23, 2018 8:42 AM, "Keith Turner" ***@***.***> wrote:
Maybe we don't need to verify that the illegal state came from curator. We
could make the check stronger, something like the following.
} catch (IllegalStateException e) {
if(curatorFramework.getState() == STOPPED && !started) {
// the oracle server had been stopped, do not care much about errors... lets suppress the exception
log.debug(...); log a debug message that this happened
} else {
throw e;
}
}
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#1004 (comment)>, or mute
the thread
<https://github.com/notifications/unsubscribe-auth/Acg-LI5CI5WUf9Avp_CQOuOa08pi5s8Cks5tNgv9gaJpZM4Rh_58>
.
|
For what it is worth @keith-turner part of the debugging process I put a synchronized method into the code that dumped state during the catch block and there are definitely CuratorFramework.STOPPED causing errors. I think this is happening in the OracleIT fifth test, where a lot of OracleServer.start/stop is happening. |
@@ -195,11 +195,16 @@ public OracleServer(Environment env) throws Exception { | |||
|
|||
private void allocateTimestamp() throws Exception { | |||
Stat stat = new Stat(); | |||
// FLUO-1000 | |||
while (!curatorFramework.getState().equals(CuratorFrameworkState.STARTED)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In 9b5277c I changed the order in which things started. I started curator before starting the thrift server. Given those changes, this is probably no longer needed.
@@ -410,15 +417,18 @@ public void takeLeadership(CuratorFramework curatorFramework) throws Exception { | |||
} | |||
|
|||
synchronized (this) { | |||
// FLUO-1000 | |||
isLeader = true; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@kpm1985 after 9b5277c, I think this may be the only change in this PR that still need to be made. It still makes sense to set isLeader before starting gcTsTracker.
@@ -318,6 +324,10 @@ public synchronized void start() throws Exception { | |||
curatorFramework.getConnectionStateListenable().addListener(cnxnListener); | |||
curatorFramework.start(); | |||
|
|||
while (!curatorFramework.getState().equals(CuratorFrameworkState.STARTED)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think this is needed because curator immediately changes the state in start()
. After start returns, the state will be STARTED
.
@kpm1985 |
This pull request is motivated by issue #1000 and is a work in progress. There are two main issues here that I've identified, they are both in OracleServer.
isLeader has a race condition, it is a volatile var so I've set the flag at the beginning of the LeaderSelector callback method takeLeadership.
There are two curator frameworks in OracleServer. One comes from sharedResources and doesn't seem to cause any issues, but the one created during the start method does cause issues. Specifically when takeLeadership is called, the curatorFramework may be in a state that is not CuratorFrameworkState.STARTED. One would think blockUntilConnected() would resolve this problem, but if you dig into the curator code, the state.started is not checked. To be clear, blockUntilConnected does not solve the problem. I have found that if you spin on CuratorFrameworkState.STARTED these exceptions disappear.
I'd welcome some analysis when everyone gets a little time. In the meanwhile I'll continue to post on #1000 and leave this section for the code changes.