Location via proxy:   [ UP ]  
[Report a bug]   [Manage cookies]                
Skip to content
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-14371 Zk StatusHandler should know about dynamic zk config #1392

Merged
merged 18 commits into from
Apr 17, 2020
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
18 commits
Select commit Hold shift + click to select a range
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Next Next commit
SOLR-14371: Zk StatusHandler should know about dynamic zk config
  • Loading branch information
janhoy committed Mar 30, 2020
commit 987cbe3bb486619c3b96994335e17f3e344bc439
3 changes: 3 additions & 0 deletions solr/CHANGES.txt
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,9 @@ Bug Fixes

* SOLR-14317: HttpClusterStateProvider throws exception when only one node down. (Lyle Wang via Ishan Chattopadhyaya)

* SOLR-14371: Zk StatusHandler now parse dynamic zk server config if supported, fixing Admin UI Zookeeper Status
screen in case dynamic reconfig host list differs from static zkHost string (janhoy)

Other Changes
---------------------
* SOLR-14197: SolrResourceLoader: marked many methods as deprecated, and in some cases rerouted exiting logic to avoid
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ public Category getCategory() {
public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
NamedList values = rsp.getValues();
if (cores.isZooKeeperAware()) {
values.add("zkStatus", getZkStatus(cores.getZkController().getZkServerAddress()));
values.add("zkStatus", getZkStatus());
} else {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The Zookeeper status API is only available in Cloud mode");
}
Expand All @@ -84,9 +84,14 @@ public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throw
/*
Gets all info from ZK API and returns as a map
*/
protected Map<String, Object> getZkStatus(String zkHost) {
protected Map<String, Object> getZkStatus() {
String zkHost = cores.getZkController().getZkServerAddress();
List<String> zookeepers = cores.getZkController().getZkClient().getZkServerListDynamic();
if (zookeepers.size() == 0) {
// Fallback to parsing zkHost for older zk servers without support for dynamic reconfiguration
zookeepers = Arrays.asList(zkHost.split("/")[0].split(","));
}
Map<String, Object> zkStatus = new HashMap<>();
List<String> zookeepers = Arrays.asList(zkHost.split("/")[0].split(","));
List<Object> details = new ArrayList<>();
int numOk = 0;
String status = STATUS_NA;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -269,6 +269,8 @@ When starting Solr, you must provide an address for ZooKeeper or Solr won't know

When referring to the location of ZooKeeper within Solr, it's best to use the addresses of all the servers in the ensemble. If one happens to be down, Solr will automatically be able to send its request to another server in the list.

Zookeeper version 3.5 and later supports dynamic reconfiguration of server addresses. Solr's Zookeeper Client will then automatically discover the list of Zookeeper servers from one of the servers provided in the _connect string_. This makes it possible to configure only a subset of addresses in the initial list.

=== Using a chroot

If your ensemble is or will be shared among other systems besides Solr, you should consider defining application-specific _znodes_, or a hierarchical namespace that will only include Solr's files.
Expand Down
90 changes: 61 additions & 29 deletions solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
Original file line number Diff line number Diff line change
Expand Up @@ -16,51 +16,42 @@
*/
package org.apache.solr.common.cloud;

import org.apache.commons.io.FileUtils;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.StringUtils;
import org.apache.solr.common.cloud.ConnectionManager.IsClosed;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.ObjectReleaseTracker;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.zookeeper.*;
import org.apache.zookeeper.KeeperException.NoAuthException;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.xml.transform.OutputKeys;
import javax.xml.transform.Source;
import javax.xml.transform.Transformer;
import javax.xml.transform.TransformerFactory;
import javax.xml.transform.stream.StreamResult;
import javax.xml.transform.stream.StreamSource;
import java.io.Closeable;
import java.io.File;
import java.io.IOException;
import java.io.PrintStream;
import java.io.StringReader;
import java.io.StringWriter;
import java.io.*;
import java.lang.invoke.MethodHandles;
import java.nio.charset.StandardCharsets;
import java.nio.file.Path;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.RejectedExecutionException;
import java.util.function.BiFunction;
import java.util.function.Function;
import java.util.function.Predicate;
import java.util.regex.Pattern;

import org.apache.commons.io.FileUtils;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.StringUtils;
import org.apache.solr.common.cloud.ConnectionManager.IsClosed;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.ObjectReleaseTracker;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoAuthException;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.apache.zookeeper.Op;
import org.apache.zookeeper.OpResult;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.stream.Collectors;

/**
*
Expand Down Expand Up @@ -744,6 +735,47 @@ public String getZkServerAddress() {
return zkServerAddress;
}

/**
* Gets the list of server:port addresses by querying dynamic config from server.
* This list may be different from the initial {@link #getZkServerAddress()} string.
* If the server is pre-3.5 this may return an empty list.
* @return list of strings for zk servers, including client port, or empty list if none
*/
public List<String> getZkServerListDynamic() {
String config = getConfig();
if (config == null) {
return Collections.emptyList();
}
return config.lines().filter(l -> l.startsWith("server."))
.filter(l -> l.contains(";"))
.map(l -> l.split(";")[1])
.collect(Collectors.toList());
}

/**
* Gets the raw config node /zookeeper/config as returned by server. Response may look like
* <pre>
* server.1=localhost:2780:2783:participant;localhost:2791
* server.2=localhost:2781:2784:participant;localhost:2792
* server.3=localhost:2782:2785:participant;localhost:2793
* version=400000003
* </pre>
* @return Multi line string representing the config or null if no data (pre 3.5 zk server)
*/
public String getConfig() {
try {
Stat stat = new Stat();
keeper.sync(ZooDefs.CONFIG_NODE, null, null);
byte[] data = keeper.getConfig(false, stat);
if (data == null) {
return null;
}
return new String(data, StandardCharsets.UTF_8);
} catch (KeeperException|InterruptedException ex) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Failed to get config from zookeeper", ex);
}
}

public ZkACLProvider getZkACLProvider() {
return zkACLProvider;
}
Expand Down