-
Notifications
You must be signed in to change notification settings - Fork 656
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
SOLR-16116: Use apache curator to manage the Solr Zookeeper interactions #760
base: main
Are you sure you want to change the base?
Conversation
solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerRoleCmd.java
Outdated
Show resolved
Hide resolved
leader election code is not broken, it was that the sessionTimeout was not being populated in SolrZkClient. |
@HoustonPutman I haven't looked super closely at this, but I think that Hadoop shades curator when needed. Its possible that the Solr Hadoop classes just need to have the imports updated to use the Hadoop specific shaded curator paths. This is new with the Hadoop auth and HDFS modules in Solr when we upgraded Hadoop. I tried to switch to the Hadoop self contained shaded dependencies to avoid issues like this. |
@risdenk I believe the issue currently is that for hadoop-auth we are not able to rely solely on |
I propose we make the loss of the reconnect boolean its own PR, plus the methods that shielded use of ZkCmdExecutor. That will make this effort here more independently reviewable and would be nice in its own right. |
var clientBuilder = CuratorFrameworkFactory.builder() | ||
.ensembleProvider(new FixedEnsembleProvider(zkHost)) | ||
.namespace(chroot) | ||
.sessionTimeoutMs(zkClientTimeout) | ||
.connectionTimeoutMs(clientConnectTimeout) | ||
.aclProvider(this.aclProvider) | ||
.authorization(zkCredentialsProvider.getCredentials()) | ||
.retryPolicy(retryPolicy); | ||
|
||
client = clientBuilder.build(); |
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 recently worked on an experimental hack for some days on 8x to have the Overseer use Curator for elections. One thing that I got stuck on was that Solr's tests were finding some threads lingering. Eventually I discovered that Curator has an internal executor and a way to set it on the builder runSafeService
. Once I did that and tended to ensuring this executor got shot down, I didn't have that problem anymore. Just sharing this with you now in hopes it helps.
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.
FWIW #1743 used this runSafeService stuff for the hadoop-auth changes. I think it might only come into play when there is an external zkClient provided but good to know either way.
hadoop 3.3.6 upgraded curator to 5.2.0 - #1743. I'm working on getting that in. chatted w/ @HoustonPutman about it as well |
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.
Overall looks great. I know it needs to be brought up to main again but 🥳
solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
Outdated
Show resolved
Hide resolved
#1743 is in main and so this PR should be ready to update :D |
I merged main (after hadoop 3.3.6/curator 5.x) and then fixed up different things after that to get all the tests passing including nightly. Would appreciate another set of eyes @HoustonPutman @dsmiley I'll look through this again to see if things can be cleaned up more, but hopefully its progress. |
public void command(); | ||
|
||
@Override | ||
default void stateChanged(CuratorFramework client, ConnectionState newState) { |
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 really don't know if this is right. Its just like OnDisconnect which seems wrong but tests pass :/
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'm not really sure about any of these classes. It seems like OnDisconnect
should do !newState.isConnected()
instead of newState == ConnectionState.LOST
... I have a slight memory about doing this stuff and there being reasons, but I can't remember why I did this... It definitely warrants a much more thorough look. I'm not even sure having a BeforeReconnect
is a good idea. The curator APIs don't really give us a good way of implementing it.
AFAIK, ever since #1743 landed, we're no longer blocked on Hadoop-Auth |
Yeah let's get this in main. Probably best to not introduce it in a late 9x version |
@HoustonPutman @dsmiley I'm happy to merge main into this again and resolve any conflicts. can you do another review to make sure all is good before merging to main? |
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.
Wow this is a doozy
@@ -121,6 +121,16 @@ dependencies { | |||
implementation 'org.eclipse.jetty:jetty-io' | |||
implementation 'org.eclipse.jetty.toolchain:jetty-servlet-api' | |||
|
|||
implementation('org.apache.curator:curator-framework', { |
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.
Rather minor but I think moving this below so that we have a "ZooKeeper / Curator" section is more organized.
@@ -366,7 +361,6 @@ public ZkController( | |||
.withUrl(zkServerAddress) | |||
.withTimeout(clientTimeout, TimeUnit.MILLISECONDS) | |||
.withConnTimeOut(zkClientConnectTimeout, TimeUnit.MILLISECONDS) | |||
.withConnStrategy(strat) |
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.
What happened with this?
Op.create( | ||
nodePath, | ||
null, | ||
zkClient.getZkACLProvider().getACLsToAdd(nodePath), |
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'm wondering where ACLs are handled
zookeeper.delete(ops.get(j).getPath(), -1, true); | ||
} catch (KeeperException.NoNodeException e2) { | ||
if (log.isDebugEnabled()) { | ||
log.debug("Can not remove node which is not exist : {}", ops.get(j).getPath()); |
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.
Did you remove because you saw too much of these?
@@ -398,16 +397,6 @@ private void retryRegisterWatcher() { | |||
return; | |||
} catch (KeeperException e) { | |||
log.warn("Failed watching shard term for collection: {}, retrying!", collection, e); | |||
try { |
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.
why remove?
} | ||
|
||
protected <T> T runWithCorrectThrows(String action, SupplierWithException<T> func) | ||
throws KeeperException, InterruptedException { |
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.
but we catch InterruptedException now
@@ -1048,6 +1125,11 @@ public void downloadFromZK(String zkPath, Path dir) throws IOException { | |||
ZkMaintenanceUtils.downloadFromZK(this, zkPath, dir); | |||
} | |||
|
|||
@FunctionalInterface | |||
public interface IsClosed { |
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.
Does this new interface need to be public?
@@ -43,6 +43,17 @@ dependencies { | |||
var zkExcludes = { | |||
exclude group: "org.apache.yetus", module: "audience-annotations" | |||
} | |||
api('org.apache.curator:curator-client', { |
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.
Debatable if "api" is appropriate here
api('org.apache.curator:curator-framework', { | ||
exclude group: 'org.apache.zookeeper', module: 'zookeeper' | ||
}) | ||
api('org.apache.curator:curator-test') { |
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.
Thus all solr-test-framework users (plugin users outside of the project) will need curator-test. Ah; I see this is because ChaosMonkey is here and uses the KillSession thing. Okay; a little comment might be nice.
long sessionId = zkClient.getZkSessionId(); | ||
zkServer.expire(sessionId); | ||
try { | ||
KillSession.kill(zkClient.getCuratorFramework().getZookeeperClient().getZooKeeper()); |
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.
didn't you add a convenience method for that? If I'm wrong; never mind.
This is an important initiative. If you've given up on it @risdenk, let us know so someone can take over. |
I'm not actively working on it. @HoustonPutman started this and I just tried to push it a bit further. I might get back to it but nothing immediately. Someone else can definitely push it further. |
Thanks for putting in the work @risdenk ! I will push to add this for 10x only this summer. |
This PR had no visible activity in the past 60 days, labeling it as stale. Any new activity will remove the stale label. To attract more reviewers, please tag someone or notify the dev@solr.apache.org mailing list. Thank you for your contribution! |
This PR is now closed due to 60 days of inactivity after being marked as stale. Re-opening this PR is still possible, in which case it will be marked as active again. |
Thanks for reopening this @dsmiley and would be great to get it in now that summer is over ;-) |
https://issues.apache.org/jira/browse/SOLR-16116
This is still a WIP there are two outstanding issues:
Might look into using curator recipes for persistent watchers and leader election, but this is a fairly complete migration to start with.