You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2021/03/09 19:05:15 UTC
[lucene-solr] 07/09: @1431 std lazy fetch again.
This is an automated email from the ASF dual-hosted git repository.
markrmiller pushed a commit to branch reference_impl_dev
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git
commit 3ed5d3b657a595b662f7d1d38cbb4c66a3dcc581
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Sat Mar 6 18:08:10 2021 -0600
@1431 std lazy fetch again.
Took 19 minutes
Took 5 minutes
---
.../apache/solr/common/cloud/ZkStateReader.java | 45 ++++++++++------------
1 file changed, 21 insertions(+), 24 deletions(-)
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index d65c10f..4d13f11 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -87,7 +87,7 @@ import static java.util.Collections.emptySortedSet;
import static org.apache.solr.common.util.Utils.fromJSON;
public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
- // public static final int STATE_UPDATE_DELAY = Integer.getInteger("solr.OverseerStateUpdateDelay", 2000); // delay between cloud state updates
+ public static final int STATE_UPDATE_DELAY = Integer.getInteger("solr.OverseerStateUpdateDelay", 2000); // delay between cloud state updates
public static final String STRUCTURE_CHANGE_NOTIFIER = "_scn";
public static final String STATE_UPDATES = "_statupdates";
@@ -234,6 +234,8 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
private final Set<ClusterPropertiesListener> clusterPropertiesListeners = ConcurrentHashMap.newKeySet();
+ private static final long LAZY_CACHE_TIME = TimeUnit.NANOSECONDS.convert(STATE_UPDATE_DELAY, TimeUnit.MILLISECONDS);
+
private volatile Future<?> collectionPropsCacheCleaner; // only kept to identify if the cleaner has already been started.
private volatile String node = null;
private volatile LiveNodeWatcher liveNodesWatcher;
@@ -662,7 +664,7 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
private void refreshCollectionList() throws KeeperException, InterruptedException {
List<String> children = null;
try {
- children = zkClient.getChildren(COLLECTIONS_ZKNODE, null, null,true, false);
+ children = zkClient.getChildren(COLLECTIONS_ZKNODE, null, null, true, true);
} catch (KeeperException.NoNodeException e) {
log.warn("Error fetching collection names: [{}]", e.getMessage());
// fall through
@@ -745,40 +747,35 @@ public class ZkStateReader implements SolrCloseable, Replica.NodeNameToBaseUrl {
private class LazyCollectionRef extends ClusterState.CollectionRef {
private final String collName;
- private volatile DocCollection cachedDocCollection;
+ private long lastUpdateTime;
+ private DocCollection cachedDocCollection;
public LazyCollectionRef(String collName) {
super(null);
this.collName = collName;
+ this.lastUpdateTime = -1;
}
@Override
public DocCollection get(boolean allowCached) {
gets.incrementAndGet();
-
- boolean shouldFetch = true;
- DocCollection cached = cachedDocCollection;
- if (cached != null) {
- Stat exists = null;
- try {
- exists = zkClient.exists(getCollectionPath(collName), null, true);
- } catch (Exception e) {
- ParWork.propagateInterrupt(e);
- throw new SolrException(ErrorCode.SERVER_ERROR, e);
+ if (!allowCached || lastUpdateTime < 0 || System.nanoTime() - lastUpdateTime > LAZY_CACHE_TIME) {
+ boolean shouldFetch = true;
+ if (cachedDocCollection != null) {
+ Stat exists = null;
+ try {
+ exists = zkClient.exists(getCollectionPath(collName), null, true);
+ } catch (Exception e) {
+ }
+ if (exists != null && exists.getVersion() == cachedDocCollection.getZNodeVersion()) {
+ shouldFetch = false;
+ }
}
- if (exists != null && exists.getVersion() == cached.getZNodeVersion()) {
- shouldFetch = false;
+ if (shouldFetch) {
+ cachedDocCollection = getCollectionLive(ZkStateReader.this, collName);
+ lastUpdateTime = System.nanoTime();
}
}
- if (shouldFetch) {
- cached = getCollectionLive(ZkStateReader.this, collName);
- cachedDocCollection = cached;
- return cached;
- }
-
- if (log.isDebugEnabled() && cachedDocCollection == null) {
- log.debug("cached collection is null");
- }
return cachedDocCollection;
}