mirror of https://github.com/apache/lucene.git
SOLR-6521 CloudSolrServer should synchronize cache cluster state loading
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1653822 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
3af6787e55
commit
1eda04b2cf
|
@ -794,6 +794,9 @@ Other Changes
|
|||
* SOLR-6976: Remove classes and methods deprecated in 4.x (Alan Woodward, Noble
|
||||
Paul, Chris Hostetter)
|
||||
|
||||
* SOLR-6521: CloudSolrClient should synchronize cache cluster state loading
|
||||
( Noble Paul, Jessica Cheng Mallet)
|
||||
|
||||
================== 4.10.3 ==================
|
||||
|
||||
Bug Fixes
|
||||
|
|
|
@ -126,9 +126,10 @@ public class CloudSolrClient extends SolrClient {
|
|||
|
||||
}
|
||||
private volatile long timeToLive = 60* 1000L;
|
||||
private volatile List<Object> locks = objectList(3);
|
||||
|
||||
|
||||
protected Map<String, ExpiringCachedDocCollection> collectionStateCache = new ConcurrentHashMap<String, ExpiringCachedDocCollection>(){
|
||||
protected final Map<String, ExpiringCachedDocCollection> collectionStateCache = new ConcurrentHashMap<String, ExpiringCachedDocCollection>(){
|
||||
@Override
|
||||
public ExpiringCachedDocCollection get(Object key) {
|
||||
ExpiringCachedDocCollection val = super.get(key);
|
||||
|
@ -143,7 +144,7 @@ public class CloudSolrClient extends SolrClient {
|
|||
};
|
||||
|
||||
class ExpiringCachedDocCollection {
|
||||
DocCollection cached;
|
||||
final DocCollection cached;
|
||||
long cachedAt;
|
||||
|
||||
ExpiringCachedDocCollection(DocCollection cached) {
|
||||
|
@ -1060,18 +1061,50 @@ public class CloudSolrClient extends SolrClient {
|
|||
return updatesToLeaders;
|
||||
}
|
||||
|
||||
protected DocCollection getDocCollection(ClusterState clusterState, String collection) throws SolrException {
|
||||
ExpiringCachedDocCollection cachedState = collectionStateCache != null ? collectionStateCache.get(collection) : null;
|
||||
if (cachedState != null && cachedState.cached != null) {
|
||||
return cachedState.cached;
|
||||
}
|
||||
/**If caches are expired they are refreshed after acquiring a lock.
|
||||
* use this to set the number of locks
|
||||
*/
|
||||
public void setParallelCacheRefreshes(int n){ locks = objectList(n); }
|
||||
|
||||
DocCollection col = clusterState.getCollectionOrNull(collection);
|
||||
private static ArrayList<Object> objectList(int n) {
|
||||
ArrayList<Object> l = new ArrayList<>(n);
|
||||
for(int i=0;i<n;i++) l.add(new Object());
|
||||
return l;
|
||||
}
|
||||
|
||||
|
||||
protected DocCollection getDocCollection(ClusterState clusterState, String collection) throws SolrException {
|
||||
if(collection == null) return null;
|
||||
DocCollection col = getFromCache(collection);
|
||||
if(col != null) return col;
|
||||
|
||||
ClusterState.CollectionRef ref = clusterState.getCollectionRef(collection);
|
||||
if(ref == null){
|
||||
//no such collection exists
|
||||
return null;
|
||||
}
|
||||
if(!ref.isLazilyLoaded()) {
|
||||
//it is readily available just return it
|
||||
return ref.get();
|
||||
}
|
||||
List locks = this.locks;
|
||||
final Object lock = locks.get(collection.hashCode() % locks.size());
|
||||
synchronized (lock){
|
||||
//we have waited for sometime just check once again
|
||||
col = getFromCache(collection);
|
||||
if(col !=null) return col;
|
||||
col = ref.get();
|
||||
}
|
||||
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;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Useful for determining the minimum achieved replication factor across
|
||||
|
|
|
@ -165,6 +165,9 @@ public class ClusterState implements JSONWriter.Writable {
|
|||
return coll;
|
||||
}
|
||||
|
||||
public CollectionRef getCollectionRef(String coll) {
|
||||
return collectionStates.get(coll);
|
||||
}
|
||||
|
||||
public DocCollection getCollectionOrNull(String coll) {
|
||||
CollectionRef ref = collectionStates.get(coll);
|
||||
|
@ -397,6 +400,8 @@ public class ClusterState implements JSONWriter.Writable {
|
|||
return coll;
|
||||
}
|
||||
|
||||
public boolean isLazilyLoaded() { return false; }
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -471,6 +471,9 @@ public class ZkStateReader implements Closeable {
|
|||
public DocCollection get() {
|
||||
return getCollectionLive(ZkStateReader.this, collName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isLazilyLoaded() { return true; }
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue