SOLR-9784: Refactor CloudSolrClient to eliminate direct dependency on ZK

SOLR-9512: CloudSolrClient's cluster state cache can break direct updates to leaders
This commit is contained in:
Noble Paul 2016-11-25 00:27:16 +05:30
parent eb51ebc0e9
commit e309f90589
7 changed files with 595 additions and 120 deletions

View File

@ -142,7 +142,9 @@ Bug Fixes
* SOLR-9729: JDBCStream improvements (Kevin Risden)
* SOLR-9626: new Admin UI now also highlights matched terms in the Analysis screen. (Alexandre Rafalovitch)
* SOLR-9512: CloudSolrClient's cluster state cache can break direct updates to leaders (noble)
Other Changes
----------------------
@ -168,6 +170,8 @@ Other Changes
* SOLR-8785: Use Dropwizard Metrics library for core metrics. The copied over code in
org.apache.solr.util.stats has been removed. (Jeff Wartes, Kelvin Wong, Christine Poerschke, shalin)
* SOLR-9784: Refactor CloudSolrClient to eliminate direct dependency on ZK (noble)
================== 6.3.0 ==================
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

View File

@ -40,6 +40,10 @@
<dependency org="org.slf4j" name="jcl-over-slf4j" rev="${/org.slf4j/jcl-over-slf4j}" conf="compile"/>
<dependency org="org.slf4j" name="slf4j-log4j12" rev="${/org.slf4j/slf4j-log4j12}" conf="test"/>
<dependency org="org.easymock" name="easymock" rev="${/org.easymock/easymock}" conf="test"/>
<dependency org="cglib" name="cglib-nodep" rev="${/cglib/cglib-nodep}" conf="test"/>
<dependency org="org.objenesis" name="objenesis" rev="${/org.objenesis/objenesis}" conf="test"/>
<exclude org="*" ext="*" matcher="regexp" type="${ivy.exclude.types}"/>
</dependencies>

View File

@ -16,6 +16,7 @@
*/
package org.apache.solr.client.solrj.impl;
import java.io.Closeable;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.net.ConnectException;
@ -37,6 +38,7 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
@ -56,7 +58,6 @@ import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.ToleratedUpdateError;
import org.apache.solr.common.cloud.Aliases;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.CollectionStatePredicate;
import org.apache.solr.common.cloud.CollectionStateWatcher;
@ -68,7 +69,6 @@ import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.ShardParams;
import org.apache.solr.common.params.SolrParams;
@ -79,7 +79,6 @@ import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.common.util.StrUtils;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.slf4j.MDC;
@ -101,10 +100,7 @@ public class CloudSolrClient extends SolrClient {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private volatile ZkStateReader zkStateReader;
private String zkHost; // the zk server connect string
private int zkConnectTimeout = 10000;
private int zkClientTimeout = 10000;
private final ClusterStateProvider stateProvider;
private volatile String defaultCollection;
private final LBHttpSolrClient lbClient;
private final boolean shutdownLBHttpSolrServer;
@ -122,6 +118,7 @@ public class CloudSolrClient extends SolrClient {
"CloudSolrClient ThreadPool"));
private String idField = "id";
public static final String STATE_VERSION = "_stateVer_";
private long retryExpiryTime = TimeUnit.NANOSECONDS.convert(3, TimeUnit.SECONDS);//3 seconds or 3 million nanos
private final Set<String> NON_ROUTABLE_PARAMS;
{
NON_ROUTABLE_PARAMS = new HashSet<>();
@ -139,12 +136,15 @@ public class CloudSolrClient extends SolrClient {
// NON_ROUTABLE_PARAMS.add(UpdateParams.ROLLBACK);
}
private volatile long timeToLive = 60* 1000L;
private volatile List<Object> locks = objectList(3);
protected final Map<String, ExpiringCachedDocCollection> collectionStateCache = new ConcurrentHashMap<String, ExpiringCachedDocCollection>(){
static class StateCache extends ConcurrentHashMap<String, ExpiringCachedDocCollection> {
final AtomicLong puts = new AtomicLong();
final AtomicLong hits = new AtomicLong();
final Lock evictLock = new ReentrantLock(true);
private volatile long timeToLive = 60 * 1000L;
@Override
public ExpiringCachedDocCollection get(Object key) {
ExpiringCachedDocCollection val = super.get(key);
@ -158,9 +158,16 @@ public class CloudSolrClient extends SolrClient {
super.remove(key);
return null;
}
hits.incrementAndGet();
return val;
}
@Override
public ExpiringCachedDocCollection put(String key, ExpiringCachedDocCollection value) {
puts.incrementAndGet();
return super.put(key, value);
}
void evictStale() {
if(!evictLock.tryLock()) return;
try {
@ -174,15 +181,30 @@ public class CloudSolrClient extends SolrClient {
}
}
};
}
/**
* This is the time to wait to refetch the state
* after getting the same state version from ZK
* <p>
* secs
*/
public void setRetryExpiryTime(int secs) {
this.retryExpiryTime = TimeUnit.NANOSECONDS.convert(secs, TimeUnit.SECONDS);
}
public void setSoTimeout(int timeout) {
lbClient.setSoTimeout(timeout);
}
protected final StateCache collectionStateCache = new StateCache();
class ExpiringCachedDocCollection {
final DocCollection cached;
long cachedAt;
final long cachedAt;
//This is the time at which the collection is retried and got the same old version
long retriedAt = -1;
//flag that suggests that this is potentially to be rechecked
boolean maybeStale = false;
ExpiringCachedDocCollection(DocCollection cached) {
this.cached = cached;
@ -193,6 +215,21 @@ public class CloudSolrClient extends SolrClient {
return (System.nanoTime() - cachedAt)
> TimeUnit.NANOSECONDS.convert(timeToLiveMs, TimeUnit.MILLISECONDS);
}
boolean shoulRetry() {
if (maybeStale) {// we are not sure if it is stale so check with retry time
if ((retriedAt == -1 ||
(System.nanoTime() - retriedAt) > retryExpiryTime)) {
return true;// we retried a while back. and we could not get anything new.
//it's likely that it is not going to be available now also.
}
}
return false;
}
void setRetriedAt() {
retriedAt = System.nanoTime();
}
}
/**
@ -219,7 +256,7 @@ public class CloudSolrClient extends SolrClient {
*/
@Deprecated
public CloudSolrClient(String zkHost) {
this.zkHost = zkHost;
this.stateProvider = new ZkClientClusterStateProvider(zkHost);
this.clientIsInternal = true;
this.myClient = HttpClientUtil.createClient(null);
this.lbClient = new LBHttpSolrClient.Builder()
@ -259,8 +296,8 @@ public class CloudSolrClient extends SolrClient {
* @deprecated use {@link Builder} instead.
*/
@Deprecated
public CloudSolrClient(String zkHost, HttpClient httpClient) {
this.zkHost = zkHost;
public CloudSolrClient(String zkHost, HttpClient httpClient) {
this.stateProvider = new ZkClientClusterStateProvider(zkHost);
this.clientIsInternal = httpClient == null;
this.myClient = httpClient == null ? HttpClientUtil.createClient(null) : httpClient;
this.lbClient = createLBHttpSolrClient(myClient);
@ -318,7 +355,7 @@ public class CloudSolrClient extends SolrClient {
*/
@Deprecated
public CloudSolrClient(Collection<String> zkHosts, String chroot, HttpClient httpClient) {
this.zkHost = buildZkHostString(zkHosts, chroot);
this.stateProvider = new ZkClientClusterStateProvider(zkHosts, chroot);
this.clientIsInternal = httpClient == null;
this.myClient = httpClient == null ? HttpClientUtil.createClient(null) : httpClient;
this.lbClient = createLBHttpSolrClient(myClient);
@ -354,7 +391,7 @@ public class CloudSolrClient extends SolrClient {
*/
@Deprecated
public CloudSolrClient(Collection<String> zkHosts, String chroot, HttpClient httpClient, LBHttpSolrClient lbSolrClient, boolean updatesToLeaders) {
this(zkHosts, chroot, httpClient, lbSolrClient, null, updatesToLeaders, false);
this(zkHosts, chroot, httpClient, lbSolrClient, null, updatesToLeaders, false, null);
}
/**
@ -389,8 +426,15 @@ public class CloudSolrClient extends SolrClient {
LBHttpSolrClient lbSolrClient,
LBHttpSolrClient.Builder lbHttpSolrClientBuilder,
boolean updatesToLeaders,
boolean directUpdatesToLeadersOnly) {
this.zkHost = buildZkHostString(zkHosts, chroot);
boolean directUpdatesToLeadersOnly,
ClusterStateProvider stateProvider
) {
if (stateProvider == null) {
this.stateProvider = new ZkClientClusterStateProvider(zkHosts, chroot);
} else {
this.stateProvider = stateProvider;
}
this.clientIsInternal = httpClient == null;
this.shutdownLBHttpSolrServer = lbSolrClient == null;
if(lbHttpSolrClientBuilder != null) lbSolrClient = lbHttpSolrClientBuilder.build();
@ -428,7 +472,7 @@ public class CloudSolrClient extends SolrClient {
*/
@Deprecated
public CloudSolrClient(String zkHost, boolean updatesToLeaders, HttpClient httpClient) {
this.zkHost = zkHost;
this.stateProvider = new ZkClientClusterStateProvider(zkHost);
this.clientIsInternal = httpClient == null;
this.myClient = httpClient == null ? HttpClientUtil.createClient(null) : httpClient;
this.lbClient = new LBHttpSolrClient.Builder()
@ -447,7 +491,7 @@ public class CloudSolrClient extends SolrClient {
*/
public void setCollectionCacheTTl(int seconds){
assert seconds > 0;
timeToLive = seconds*1000L;
this.collectionStateCache.timeToLive = seconds * 1000L;
}
/**
@ -475,8 +519,8 @@ public class CloudSolrClient extends SolrClient {
*/
@Deprecated
public CloudSolrClient(String zkHost, LBHttpSolrClient lbClient, boolean updatesToLeaders) {
this.zkHost = zkHost;
this.lbClient = lbClient;
this.stateProvider = new ZkClientClusterStateProvider(zkHost);
this.updatesToLeaders = updatesToLeaders;
this.directUpdatesToLeadersOnly = false;
shutdownLBHttpSolrServer = false;
@ -512,11 +556,15 @@ public class CloudSolrClient extends SolrClient {
* @return the zkHost value used to connect to zookeeper.
*/
public String getZkHost() {
return zkHost;
return assertZKStateProvider().zkHost;
}
public ZkStateReader getZkStateReader() {
return zkStateReader;
if (stateProvider instanceof ZkClientClusterStateProvider) {
ZkClientClusterStateProvider provider = (ZkClientClusterStateProvider) stateProvider;
return provider.zkStateReader;
}
throw new IllegalStateException("This has no Zk stateReader");
}
/**
@ -545,12 +593,12 @@ public class CloudSolrClient extends SolrClient {
/** Set the connect timeout to the zookeeper ensemble in ms */
public void setZkConnectTimeout(int zkConnectTimeout) {
this.zkConnectTimeout = zkConnectTimeout;
assertZKStateProvider().zkConnectTimeout = zkConnectTimeout;
}
/** Set the timeout to the zookeeper ensemble in ms */
public void setZkClientTimeout(int zkClientTimeout) {
this.zkClientTimeout = zkClientTimeout;
assertZKStateProvider().zkClientTimeout = zkClientTimeout;
}
/**
@ -559,29 +607,7 @@ public class CloudSolrClient extends SolrClient {
*
*/
public void connect() {
if (zkStateReader == null) {
synchronized (this) {
if (zkStateReader == null) {
ZkStateReader zk = null;
try {
zk = new ZkStateReader(zkHost, zkClientTimeout, zkConnectTimeout);
zk.createClusterStateWatchersAndUpdate();
zkStateReader = zk;
} catch (InterruptedException e) {
zk.close();
Thread.currentThread().interrupt();
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
} catch (KeeperException e) {
zk.close();
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
} catch (Exception e) {
if (zk != null) zk.close();
// do not wrap because clients may be relying on the underlying exception being thrown
throw e;
}
}
}
}
stateProvider.connect();
}
/**
@ -592,12 +618,12 @@ public class CloudSolrClient extends SolrClient {
* @throws InterruptedException if the wait is interrupted
*/
public void connect(long duration, TimeUnit timeUnit) throws TimeoutException, InterruptedException {
log.info("Waiting for {} {} for cluster at {} to be ready", duration, timeUnit, zkHost);
log.info("Waiting for {} {} for cluster at {} to be ready", duration, timeUnit, stateProvider);
long timeout = System.nanoTime() + timeUnit.toNanos(duration);
while (System.nanoTime() < timeout) {
try {
connect();
log.info("Cluster at {} ready", zkHost);
log.info("Cluster at {} ready", stateProvider);
return;
}
catch (RuntimeException e) {
@ -624,8 +650,16 @@ public class CloudSolrClient extends SolrClient {
* @throws IOException if an IO error occurs
*/
public void uploadConfig(Path configPath, String configName) throws IOException {
connect();
zkStateReader.getConfigManager().uploadConfigDir(configPath, configName);
stateProvider.connect();
assertZKStateProvider().uploadConfig(configPath, configName);
}
private ZkClientClusterStateProvider assertZKStateProvider() {
if (stateProvider instanceof ZkClientClusterStateProvider) {
return (ZkClientClusterStateProvider) stateProvider;
}
throw new IllegalArgumentException("This client does not use ZK");
}
/**
@ -635,8 +669,7 @@ public class CloudSolrClient extends SolrClient {
* @throws IOException if an I/O exception occurs
*/
public void downloadConfig(String configName, Path downloadPath) throws IOException {
connect();
zkStateReader.getConfigManager().downloadConfigDir(configName, downloadPath);
assertZKStateProvider().downloadConfig(configName, downloadPath);
}
/**
@ -654,8 +687,8 @@ public class CloudSolrClient extends SolrClient {
*/
public void waitForState(String collection, long wait, TimeUnit unit, CollectionStatePredicate predicate)
throws InterruptedException, TimeoutException {
connect();
zkStateReader.waitForState(collection, wait, unit, predicate);
stateProvider.connect();
assertZKStateProvider().zkStateReader.waitForState(collection, wait, unit, predicate);
}
/**
@ -669,11 +702,11 @@ public class CloudSolrClient extends SolrClient {
* @param watcher a watcher that will be called when the state changes
*/
public void registerCollectionStateWatcher(String collection, CollectionStateWatcher watcher) {
connect();
zkStateReader.registerCollectionStateWatcher(collection, watcher);
stateProvider.connect();
assertZKStateProvider().zkStateReader.registerCollectionStateWatcher(collection, watcher);
}
private NamedList<Object> directUpdate(AbstractUpdateRequest request, String collection, ClusterState clusterState) throws SolrServerException {
private NamedList<Object> directUpdate(AbstractUpdateRequest request, String collection) throws SolrServerException {
UpdateRequest updateRequest = (UpdateRequest) request;
ModifiableSolrParams params = (ModifiableSolrParams) request.getParams();
ModifiableSolrParams routableParams = new ModifiableSolrParams();
@ -693,15 +726,9 @@ public class CloudSolrClient extends SolrClient {
//Check to see if the collection is an alias.
Aliases aliases = zkStateReader.getAliases();
if(aliases != null) {
Map<String, String> collectionAliases = aliases.getCollectionAliasMap();
if(collectionAliases != null && collectionAliases.containsKey(collection)) {
collection = collectionAliases.get(collection);
}
}
collection = stateProvider.getCollectionName(collection);
DocCollection col = getDocCollection(clusterState, collection,null);
DocCollection col = getDocCollection(collection, null);
DocRouter router = col.getRouter();
@ -1022,12 +1049,12 @@ public class CloudSolrClient extends SolrClient {
List<DocCollection> requestedCollections = null;
boolean isAdmin = ADMIN_PATHS.contains(request.getPath());
if (collection != null && !isAdmin) { // don't do _stateVer_ checking for admin requests
Set<String> requestedCollectionNames = getCollectionNames(getZkStateReader().getClusterState(), collection);
Set<String> requestedCollectionNames = getCollectionNames(collection);
StringBuilder stateVerParamBuilder = null;
for (String requestedCollection : requestedCollectionNames) {
// track the version of state we're using on the client side using the _stateVer_ param
DocCollection coll = getDocCollection(getZkStateReader().getClusterState(), requestedCollection,null);
DocCollection coll = getDocCollection(requestedCollection, null);
int collVer = coll.getZNodeVersion();
if (coll.getStateFormat()>1) {
if(requestedCollections == null) requestedCollections = new ArrayList<>(requestedCollectionNames.size());
@ -1068,7 +1095,7 @@ public class CloudSolrClient extends SolrClient {
Map invalidStates = (Map) o;
for (Object invalidEntries : invalidStates.entrySet()) {
Map.Entry e = (Map.Entry) invalidEntries;
getDocCollection(getZkStateReader().getClusterState(),(String)e.getKey(), (Integer)e.getValue());
getDocCollection((String) e.getKey(), (Integer) e.getValue());
}
}
@ -1101,6 +1128,26 @@ public class CloudSolrClient extends SolrClient {
rootCause instanceof NoHttpResponseException ||
rootCause instanceof SocketException);
if (wasCommError) {
// it was a communication error. it is likely that
// the node to which the request to be sent is down . So , expire the state
// so that the next attempt would fetch the fresh state
// just re-read state for all of them, if it has not been retired
// in retryExpiryTime time
for (DocCollection ext : requestedCollections) {
ExpiringCachedDocCollection cacheEntry = collectionStateCache.get(ext.getName());
if (cacheEntry == null) continue;
cacheEntry.maybeStale = true;
}
if (retryCount < MAX_STALE_RETRIES) {//if it is a communication error , we must try again
//may be, we have a stale version of the collection state
// and we could not get any information from the server
//it is probably not worth trying again and again because
// the state would not have been updated
return requestWithRetryOnStaleState(request, retryCount + 1, collection);
}
}
boolean stateWasStale = false;
if (retryCount < MAX_STALE_RETRIES &&
requestedCollections != null &&
@ -1125,7 +1172,7 @@ public class CloudSolrClient extends SolrClient {
!requestedCollections.isEmpty() &&
wasCommError) {
for (DocCollection ext : requestedCollections) {
DocCollection latestStateFromZk = getDocCollection(zkStateReader.getClusterState(), ext.getName(),null);
DocCollection latestStateFromZk = getDocCollection(ext.getName(), null);
if (latestStateFromZk.getZNodeVersion() != ext.getZNodeVersion()) {
// looks like we couldn't reach the server because the state was stale == retry
stateWasStale = true;
@ -1163,14 +1210,12 @@ public class CloudSolrClient extends SolrClient {
throws SolrServerException, IOException {
connect();
ClusterState clusterState = zkStateReader.getClusterState();
boolean sendToLeaders = false;
List<String> replicas = null;
if (request instanceof IsUpdateRequest) {
if (request instanceof UpdateRequest) {
NamedList<Object> response = directUpdate((AbstractUpdateRequest) request, collection, clusterState);
NamedList<Object> response = directUpdate((AbstractUpdateRequest) request, collection);
if (response != null) {
return response;
}
@ -1185,9 +1230,10 @@ public class CloudSolrClient extends SolrClient {
}
List<String> theUrlList = new ArrayList<>();
if (ADMIN_PATHS.contains(request.getPath())) {
Set<String> liveNodes = clusterState.getLiveNodes();
Set<String> liveNodes = getLiveNodes();
for (String liveNode : liveNodes) {
theUrlList.add(zkStateReader.getBaseUrlForNodeName(liveNode));
theUrlList.add(ZkStateReader.getBaseUrlForNodeName(liveNode,
(String) stateProvider.getClusterProperties().getOrDefault(ZkStateReader.URL_SCHEME,"http")));
}
} else {
@ -1195,8 +1241,8 @@ public class CloudSolrClient extends SolrClient {
throw new SolrServerException(
"No collection param specified on request and no default collection has been set.");
}
Set<String> collectionNames = getCollectionNames(clusterState, collection);
Set<String> collectionNames = getCollectionNames(collection);
if (collectionNames.size() == 0) {
throw new SolrException(ErrorCode.BAD_REQUEST,
"Could not find collection: " + collection);
@ -1213,11 +1259,11 @@ public class CloudSolrClient extends SolrClient {
// add it to the Map of slices.
Map<String,Slice> slices = new HashMap<>();
for (String collectionName : collectionNames) {
DocCollection col = getDocCollection(clusterState, collectionName, null);
DocCollection col = getDocCollection(collectionName, null);
Collection<Slice> routeSlices = col.getRouter().getSearchSlices(shardKeys, reqParams , col);
ClientUtils.addSlices(slices, collectionName, routeSlices, true);
}
Set<String> liveNodes = clusterState.getLiveNodes();
Set<String> liveNodes = getLiveNodes();
List<String> leaderUrlList = null;
List<String> urlList = null;
@ -1293,16 +1339,18 @@ public class CloudSolrClient extends SolrClient {
return rsp.getResponse();
}
private Set<String> getCollectionNames(ClusterState clusterState,
String collection) {
private Set<String> getLiveNodes() {
return getZkStateReader().getClusterState().getLiveNodes();
}
Set<String> getCollectionNames(String collection) {
// Extract each comma separated collection name and store in a List.
List<String> rawCollectionsList = StrUtils.splitSmart(collection, ",", true);
Set<String> collectionNames = new HashSet<>();
// validate collections
for (String collectionName : rawCollectionsList) {
if (!clusterState.hasCollection(collectionName)) {
Aliases aliases = zkStateReader.getAliases();
String alias = aliases.getCollectionAlias(collectionName);
if (stateProvider.getState(collectionName) == null) {
String alias = stateProvider.getAlias(collection);
if (alias != null) {
List<String> aliasList = StrUtils.splitSmart(alias, ",", true);
collectionNames.addAll(aliasList);
@ -1319,13 +1367,7 @@ public class CloudSolrClient extends SolrClient {
@Override
public void close() throws IOException {
if (zkStateReader != null) {
synchronized(this) {
if (zkStateReader!= null)
zkStateReader.close();
zkStateReader = null;
}
}
stateProvider.close();
if (shutdownLBHttpSolrServer) {
lbClient.close();
@ -1371,15 +1413,17 @@ public class CloudSolrClient extends SolrClient {
}
protected DocCollection getDocCollection(ClusterState clusterState, String collection, Integer expectedVersion) throws SolrException {
protected DocCollection getDocCollection(String collection, Integer expectedVersion) throws SolrException {
if (expectedVersion == null) expectedVersion = -1;
if (collection == null) return null;
DocCollection col = getFromCache(collection);
ExpiringCachedDocCollection cacheEntry = collectionStateCache.get(collection);
DocCollection col = cacheEntry == null ? null : cacheEntry.cached;
if (col != null) {
if (expectedVersion == null) return col;
if (expectedVersion.intValue() == col.getZNodeVersion()) return col;
if (expectedVersion <= col.getZNodeVersion()
&& !cacheEntry.shoulRetry()) return col;
}
ClusterState.CollectionRef ref = clusterState.getCollectionRef(collection);
ClusterState.CollectionRef ref = getCollectionRef(collection);
if (ref == null) {
//no such collection exists
return null;
@ -1390,30 +1434,34 @@ public class CloudSolrClient extends SolrClient {
}
List locks = this.locks;
final Object lock = locks.get(Math.abs(Hash.murmurhash3_x86_32(collection, 0, collection.length(), 0) % locks.size()));
DocCollection fetchedCol = null;
synchronized (lock) {
//we have waited for sometime just check once again
col = getFromCache(collection);
/*we have waited for sometime just check once again*/
cacheEntry = collectionStateCache.get(collection);
col = cacheEntry == null ? null : cacheEntry.cached;
if (col != null) {
if (expectedVersion == null) return col;
if (expectedVersion.intValue() == col.getZNodeVersion()) {
return col;
} else {
collectionStateCache.remove(collection);
}
if (expectedVersion <= col.getZNodeVersion()
&& !cacheEntry.shoulRetry()) return col;
}
col = ref.get();//this is a call to ZK
// We are going to fetch a new version
// we MUST try to get a new version
fetchedCol = ref.get();//this is a call to ZK
if (fetchedCol == null) return null;// this collection no more exists
if (col != null && fetchedCol.getZNodeVersion() == col.getZNodeVersion()) {
cacheEntry.setRetriedAt();//we retried and found that it is the same version
cacheEntry.maybeStale = false;
} else {
if (fetchedCol.getStateFormat() > 1)
collectionStateCache.put(collection, new ExpiringCachedDocCollection(fetchedCol));
}
return fetchedCol;
}
if (col == null) return null;
if (col.getStateFormat() > 1) collectionStateCache.put(collection, new ExpiringCachedDocCollection(col));
return col;
}
private DocCollection getFromCache(String c){
ExpiringCachedDocCollection cachedState = collectionStateCache.get(c);
return cachedState != null ? cachedState.cached : null;
ClusterState.CollectionRef getCollectionRef(String collection) {
return stateProvider.getState(collection);
}
/**
* Useful for determining the minimum achieved replication factor across
* all shards involved in processing an update request, typically useful
@ -1449,9 +1497,9 @@ public class CloudSolrClient extends SolrClient {
Map<String,Integer> results = new HashMap<String,Integer>();
if (resp instanceof CloudSolrClient.RouteResponse) {
NamedList routes = ((CloudSolrClient.RouteResponse)resp).getRouteResponses();
ClusterState clusterState = zkStateReader.getClusterState();
DocCollection coll = getDocCollection(collection, null);
Map<String,String> leaders = new HashMap<String,String>();
for (Slice slice : clusterState.getActiveSlices(collection)) {
for (Slice slice : coll.getActiveSlices()) {
Replica leader = slice.getLeader();
if (leader != null) {
ZkCoreNodeProps zkProps = new ZkCoreNodeProps(leader);
@ -1560,6 +1608,8 @@ public class CloudSolrClient extends SolrClient {
private LBHttpSolrClient.Builder lbClientBuilder;
private boolean shardLeadersOnly;
private boolean directUpdatesToLeadersOnly;
private ClusterStateProvider stateProvider;
public Builder() {
this.zkHosts = new ArrayList();
@ -1662,12 +1712,35 @@ public class CloudSolrClient extends SolrClient {
return this;
}
public Builder withClusterStateProvider(ClusterStateProvider stateProvider) {
this.stateProvider = stateProvider;
return this;
}
/**
* Create a {@link CloudSolrClient} based on the provided configuration.
*/
public CloudSolrClient build() {
if (stateProvider == null) {
stateProvider = new ZkClientClusterStateProvider(zkHosts, zkChroot);
}
return new CloudSolrClient(zkHosts, zkChroot, httpClient, loadBalancedSolrClient, lbClientBuilder,
shardLeadersOnly, directUpdatesToLeadersOnly);
shardLeadersOnly, directUpdatesToLeadersOnly, stateProvider);
}
}
interface ClusterStateProvider extends Closeable {
ClusterState.CollectionRef getState(String collection);
Set<String> liveNodes();
String getAlias(String collection);
String getCollectionName(String name);
Map<String, Object> getClusterProperties();
void connect();
}
}

View File

@ -0,0 +1,177 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.client.solrj.impl;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.nio.file.Path;
import java.util.Collection;
import java.util.Map;
import java.util.Set;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.Aliases;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
class ZkClientClusterStateProvider implements CloudSolrClient.ClusterStateProvider {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
ZkStateReader zkStateReader;
String zkHost;
int zkConnectTimeout = 10000;
int zkClientTimeout = 10000;
public ZkClientClusterStateProvider(Collection<String> zkHosts, String chroot) {
zkHost = buildZkHostString(zkHosts,chroot);
}
public ZkClientClusterStateProvider(String zkHost){
this.zkHost = zkHost;
}
@Override
public ClusterState.CollectionRef getState(String collection) {
return zkStateReader.getClusterState().getCollectionRef(collection);
}
@Override
public Set<String> liveNodes() {
return zkStateReader.getClusterState().getLiveNodes();
}
@Override
public String getAlias(String collection) {
Aliases aliases = zkStateReader.getAliases();
return aliases.getCollectionAlias(collection);
}
@Override
public Map<String, Object> getClusterProperties() {
return zkStateReader.getClusterProperties();
}
@Override
public String getCollectionName(String name) {
Aliases aliases = zkStateReader.getAliases();
if (aliases != null) {
Map<String, String> collectionAliases = aliases.getCollectionAliasMap();
if (collectionAliases != null && collectionAliases.containsKey(name)) {
name = collectionAliases.get(name);
}
}
return name;
}
/**
* Download a named config from Zookeeper to a location on the filesystem
* @param configName the name of the config
* @param downloadPath the path to write config files to
* @throws IOException if an I/O exception occurs
*/
public void downloadConfig(String configName, Path downloadPath) throws IOException {
connect();
zkStateReader.getConfigManager().downloadConfigDir(configName, downloadPath);
}
public void uploadConfig(Path configPath, String configName) throws IOException {
connect();
zkStateReader.getConfigManager().uploadConfigDir(configPath, configName);
}
@Override
public void connect() {
if (zkStateReader == null) {
synchronized (this) {
if (zkStateReader == null) {
ZkStateReader zk = null;
try {
zk = new ZkStateReader(zkHost, zkClientTimeout, zkConnectTimeout);
zk.createClusterStateWatchersAndUpdate();
zkStateReader = zk;
log.info("Cluster at {} ready", zkHost);
} catch (InterruptedException e) {
zk.close();
Thread.currentThread().interrupt();
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
} catch (KeeperException e) {
zk.close();
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
} catch (Exception e) {
if (zk != null) zk.close();
// do not wrap because clients may be relying on the underlying exception being thrown
throw e;
}
}
}
}
}
@Override
public void close() throws IOException {
if (zkStateReader != null) {
synchronized (this) {
if (zkStateReader != null)
zkStateReader.close();
zkStateReader = null;
}
}
}
static String buildZkHostString(Collection<String> zkHosts, String chroot) {
if (zkHosts == null || zkHosts.isEmpty()) {
throw new IllegalArgumentException("Cannot create CloudSearchClient without valid ZooKeeper host; none specified!");
}
StringBuilder zkBuilder = new StringBuilder();
int lastIndexValue = zkHosts.size() - 1;
int i = 0;
for (String zkHost : zkHosts) {
zkBuilder.append(zkHost);
if (i < lastIndexValue) {
zkBuilder.append(",");
}
i++;
}
if (chroot != null) {
if (chroot.startsWith("/")) {
zkBuilder.append(chroot);
} else {
throw new IllegalArgumentException(
"The chroot must start with a forward slash.");
}
}
/* Log the constructed connection string and then initialize. */
final String zkHostString = zkBuilder.toString();
log.debug("Final constructed zkHost string: " + zkHostString);
return zkHostString;
}
@Override
public String toString() {
return zkHost;
}
}

View File

@ -24,6 +24,7 @@ import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
@ -442,13 +443,19 @@ public class ClusterState implements JSONWriter.Writable {
}
public static class CollectionRef {
protected final AtomicInteger gets = new AtomicInteger();
private final DocCollection coll;
public int getCount(){
return gets.get();
}
public CollectionRef(DocCollection coll) {
this.coll = coll;
}
public DocCollection get(){
gets.incrementAndGet();
return coll;
}

View File

@ -633,6 +633,7 @@ public class ZkStateReader implements Closeable {
@Override
public DocCollection get() {
gets.incrementAndGet();
// TODO: consider limited caching
return getCollectionLive(ZkStateReader.this, collName);
}
@ -915,14 +916,18 @@ public class ZkStateReader implements Closeable {
}
return null;
}
/**
* Returns the baseURL corresponding to a given node's nodeName --
* NOTE: does not (currently) imply that the nodeName (or resulting
* NOTE: does not (currently) imply that the nodeName (or resulting
* baseURL) exists in the cluster.
* @lucene.experimental
*/
public String getBaseUrlForNodeName(final String nodeName) {
return getBaseUrlForNodeName(nodeName, getClusterProperty(URL_SCHEME, "http"));
}
public static String getBaseUrlForNodeName(final String nodeName, String urlScheme) {
final int _offset = nodeName.indexOf("_");
if (_offset < 0) {
throw new IllegalArgumentException("nodeName does not contain expected '_' seperator: " + nodeName);
@ -930,7 +935,6 @@ public class ZkStateReader implements Closeable {
final String hostAndPort = nodeName.substring(0,_offset);
try {
final String path = URLDecoder.decode(nodeName.substring(1+_offset), "UTF-8");
String urlScheme = getClusterProperty(URL_SCHEME, "http");
return urlScheme + "://" + hostAndPort + (path.isEmpty() ? "" : ("/" + path));
} catch (UnsupportedEncodingException e) {
throw new IllegalStateException("JVM Does not seem to support UTF-8", e);

View File

@ -0,0 +1,206 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.client.solrj.impl;
import java.io.IOException;
import java.net.ConnectException;
import java.net.SocketException;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Function;
import com.google.common.collect.ImmutableSet;
import org.apache.http.NoHttpResponseException;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.NamedList;
import org.easymock.EasyMock;
import static java.nio.charset.StandardCharsets.UTF_8;
public class CloudSolrClientCacheTest extends SolrTestCaseJ4 {
public void testCaching() throws Exception {
String collName = "gettingstarted";
Set<String> livenodes = new HashSet<>();
Map<String, ClusterState.CollectionRef> refs = new HashMap<>();
Map<String, DocCollection> colls = new HashMap<>();
class Ref extends ClusterState.CollectionRef {
private String c;
public Ref(String c) {
super(null);
this.c = c;
}
@Override
public boolean isLazilyLoaded() {
return true;
}
@Override
public DocCollection get() {
gets.incrementAndGet();
return colls.get(c);
}
}
Map<String, Function> responses = new HashMap<>();
NamedList okResponse = new NamedList();
okResponse.add("responseHeader", new NamedList<>(Collections.singletonMap("status", 0)));
LBHttpSolrClient mockLbclient = getMockLbHttpSolrClient(responses);
AtomicInteger lbhttpRequestCount = new AtomicInteger();
try (CloudSolrClient cloudClient = new CloudSolrClient.Builder()
.withLBHttpSolrClient(mockLbclient)
.withClusterStateProvider(getStateProvider(livenodes, refs))
.build()) {
livenodes.addAll(ImmutableSet.of("192.168.1.108:7574_solr", "192.168.1.108:8983_solr"));
ClusterState cs = ClusterState.load(1, coll1State.getBytes(UTF_8),
Collections.emptySet(), "/collections/gettingstarted/state.json");
refs.put(collName, new Ref(collName));
colls.put(collName, cs.getCollectionOrNull(collName));
responses.put("request", o -> {
int i = lbhttpRequestCount.incrementAndGet();
if (i == 1) return new ConnectException("TEST");
if (i == 2) return new SocketException("TEST");
if (i == 3) return new NoHttpResponseException("TEST");
return okResponse;
});
UpdateRequest update = new UpdateRequest()
.add("id", "123", "desc", "Something 0");
cloudClient.request(update, collName);
assertEquals(2, refs.get(collName).getCount());
}
}
private LBHttpSolrClient getMockLbHttpSolrClient(Map<String, Function> responses) throws Exception {
LBHttpSolrClient mockLbclient = EasyMock.createMock(LBHttpSolrClient.class);
EasyMock.reset(mockLbclient);
mockLbclient.request(EasyMock.anyObject(LBHttpSolrClient.Req.class));
EasyMock.expectLastCall().andAnswer(() -> {
LBHttpSolrClient.Req req = (LBHttpSolrClient.Req) EasyMock.getCurrentArguments()[0];
Function f = responses.get("request");
if (f == null) return null;
Object res = f.apply(null);
if (res instanceof Exception) throw (Throwable) res;
LBHttpSolrClient.Rsp rsp = new LBHttpSolrClient.Rsp();
rsp.rsp = (NamedList<Object>) res;
rsp.server = req.servers.get(0);
return rsp;
}).anyTimes();
mockLbclient.getHttpClient();
EasyMock.expectLastCall().andAnswer(() -> null).anyTimes();
EasyMock.replay(mockLbclient);
return mockLbclient;
}
private CloudSolrClient.ClusterStateProvider getStateProvider(Set<String> livenodes,
Map<String, ClusterState.CollectionRef> colls) {
return new CloudSolrClient.ClusterStateProvider() {
@Override
public ClusterState.CollectionRef getState(String collection) {
return colls.get(collection);
}
@Override
public Set<String> liveNodes() {
return livenodes;
}
@Override
public Map<String, Object> getClusterProperties() {
return Collections.EMPTY_MAP;
}
@Override
public String getAlias(String collection) {
return collection;
}
@Override
public String getCollectionName(String name) {
return name;
}
@Override
public void connect() { }
@Override
public void close() throws IOException {
}
};
}
private String coll1State = "{'gettingstarted':{\n" +
" 'replicationFactor':'2',\n" +
" 'router':{'name':'compositeId'},\n" +
" 'maxShardsPerNode':'2',\n" +
" 'autoAddReplicas':'false',\n" +
" 'shards':{\n" +
" 'shard1':{\n" +
" 'range':'80000000-ffffffff',\n" +
" 'state':'active',\n" +
" 'replicas':{\n" +
" 'core_node2':{\n" +
" 'core':'gettingstarted_shard1_replica1',\n" +
" 'base_url':'http://192.168.1.108:8983/solr',\n" +
" 'node_name':'192.168.1.108:8983_solr',\n" +
" 'state':'active',\n" +
" 'leader':'true'},\n" +
" 'core_node4':{\n" +
" 'core':'gettingstarted_shard1_replica2',\n" +
" 'base_url':'http://192.168.1.108:7574/solr',\n" +
" 'node_name':'192.168.1.108:7574_solr',\n" +
" 'state':'active'}}},\n" +
" 'shard2':{\n" +
" 'range':'0-7fffffff',\n" +
" 'state':'active',\n" +
" 'replicas':{\n" +
" 'core_node1':{\n" +
" 'core':'gettingstarted_shard2_replica1',\n" +
" 'base_url':'http://192.168.1.108:8983/solr',\n" +
" 'node_name':'192.168.1.108:8983_solr',\n" +
" 'state':'active',\n" +
" 'leader':'true'},\n" +
" 'core_node3':{\n" +
" 'core':'gettingstarted_shard2_replica2',\n" +
" 'base_url':'http://192.168.1.108:7574/solr',\n" +
" 'node_name':'192.168.1.108:7574_solr',\n" +
" 'state':'active'}}}}}}";
}