-
Notifications
You must be signed in to change notification settings - Fork 676
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-17519: CloudSolrClient with HTTP ClusterState can forget live nodes and then fail #2935
base: main
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -21,8 +21,13 @@ | |
|
||
import java.io.IOException; | ||
import java.lang.invoke.MethodHandles; | ||
import java.net.MalformedURLException; | ||
import java.net.URI; | ||
import java.net.URISyntaxException; | ||
import java.net.URL; | ||
import java.time.Instant; | ||
import java.util.Collections; | ||
import java.util.HashSet; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.Map.Entry; | ||
|
@@ -51,6 +56,7 @@ public abstract class BaseHttpClusterStateProvider implements ClusterStateProvid | |
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); | ||
|
||
private String urlScheme; | ||
private Set<String> initialNodes; | ||
volatile Set<String> liveNodes; | ||
long liveNodesTimestamp = 0; | ||
volatile Map<String, List<String>> aliases; | ||
|
@@ -61,6 +67,7 @@ public abstract class BaseHttpClusterStateProvider implements ClusterStateProvid | |
private int cacheTimeout = EnvUtils.getPropertyAsInteger("solr.solrj.cache.timeout.sec", 5); | ||
|
||
public void init(List<String> solrUrls) throws Exception { | ||
this.initialNodes = getNodeNamesFromSolrUrls(solrUrls); | ||
for (String solrUrl : solrUrls) { | ||
urlScheme = solrUrl.startsWith("https") ? "https" : "http"; | ||
try (SolrClient initialClient = getSolrClient(solrUrl)) { | ||
|
@@ -136,7 +143,7 @@ private ClusterState fetchClusterState(SolrClient client) | |
|
||
List<String> liveNodesList = (List<String>) cluster.get("live_nodes"); | ||
if (liveNodesList != null) { | ||
this.liveNodes = Set.copyOf(liveNodesList); | ||
setLiveNodes(Set.copyOf(liveNodesList)); | ||
liveNodesTimestamp = System.nanoTime(); | ||
} | ||
|
||
|
@@ -229,10 +236,9 @@ > getCacheTimeout()) { | |
for (String nodeName : liveNodes) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we exhaust liveNodes, shouldn't we then try the initial configured nodes, and then only failing that, throw an exception? |
||
String baseUrl = Utils.getBaseUrlForNodeName(nodeName, urlScheme); | ||
try (SolrClient client = getSolrClient(baseUrl)) { | ||
Set<String> liveNodes = fetchLiveNodes(client); | ||
this.liveNodes = (liveNodes); | ||
setLiveNodes(fetchLiveNodes(client)); | ||
liveNodesTimestamp = System.nanoTime(); | ||
return liveNodes; | ||
return this.liveNodes; | ||
} catch (Exception e) { | ||
log.warn("Attempt to fetch cluster state from {} failed.", baseUrl, e); | ||
} | ||
|
@@ -246,7 +252,7 @@ > getCacheTimeout()) { | |
+ " you could try re-creating a new CloudSolrClient using working" | ||
+ " solrUrl(s) or zkHost(s)."); | ||
} else { | ||
return liveNodes; // cached copy is fresh enough | ||
return this.liveNodes; // cached copy is fresh enough | ||
} | ||
} | ||
|
||
|
@@ -358,7 +364,6 @@ public ClusterState getClusterState() { | |
@SuppressWarnings("unchecked") | ||
@Override | ||
public Map<String, Object> getClusterProperties() { | ||
// Map<String, Object> clusterPropertiesMap = new HashMap<>(); | ||
for (String nodeName : liveNodes) { | ||
String baseUrl = Utils.getBaseUrlForNodeName(nodeName, urlScheme); | ||
try (SolrClient client = getSolrClient(baseUrl)) { | ||
|
@@ -413,6 +418,29 @@ public String getQuorumHosts() { | |
return String.join(",", this.liveNodes); | ||
} | ||
|
||
/** Live nodes should always have the latest set of live nodes but never remove initial set */ | ||
private void setLiveNodes(Set<String> nodes) { | ||
Set<String> liveNodes = new HashSet<>(nodes); | ||
liveNodes.addAll(this.initialNodes); | ||
this.liveNodes = Set.copyOf(liveNodes); | ||
Comment on lines
+423
to
+425
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why is this 3 lines of extra copy-ing instead of nothing more than: There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. My idea was to not do another loop through if liveNodes was exhausted. |
||
} | ||
|
||
public Set<String> getNodeNamesFromSolrUrls(List<String> urls) | ||
throws URISyntaxException, MalformedURLException { | ||
Set<String> urlSet = new HashSet<>(); | ||
for (String url : urls) { | ||
urlSet.add(getNodeNameFromSolrUrl(url)); | ||
} | ||
return Collections.unmodifiableSet(urlSet); | ||
} | ||
|
||
/** URL to cluster state node name (http://127.0.0.1:12345/solr to 127.0.0.1:12345_solr) */ | ||
public String getNodeNameFromSolrUrl(String solrUrl) | ||
throws MalformedURLException, URISyntaxException { | ||
URL url = new URI(solrUrl).toURL(); | ||
return url.getAuthority() + url.getPath().replace('/', '_'); | ||
} | ||
Comment on lines
+437
to
+442
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can you find other code in Solr doing this? Surely it's somewhere. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. should be a static method that with a unit test There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hmm maybe... I found |
||
|
||
private enum ClusterStateRequestType { | ||
FETCH_LIVE_NODES, | ||
FETCH_CLUSTER_PROP, | ||
|
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 makes sense. This leaves me with a few questions then. Shouldn't this take a list of
URL
orURI
java object to verify actual non malformed URLs instead of a list of strings? I created the functions below to convert these Strings into cluster state nodeNames forliveNodes
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.
great idea to do basic URL malformed checks like this