mirror of https://github.com/apache/lucene.git
SOLR-14340: Remove unnecessary configset verification checks
Improves CLUSTERSTATUS times for massive clusters. Closes #1373
This commit is contained in:
parent
674aba6a85
commit
aad814ba63
|
@ -64,6 +64,9 @@ Optimizations
|
|||
---------------------
|
||||
* SOLR-8306: Do not collect expand documents when expand.rows=0 (Marshall Sanders, Amelia Henderson)
|
||||
|
||||
* SOLR-14340: Remove unnecessary configset verification checks when SolrCloud merely wants to know which configset a
|
||||
collection is using. Improves CLUSTERSTATUS times for massive clusters. (Mathieu Marie, David Smiley)
|
||||
|
||||
Bug Fixes
|
||||
---------------------
|
||||
* SOLR-13264: IndexSizeTrigger aboveOp / belowOp properties not in valid properties.
|
||||
|
|
|
@ -68,7 +68,6 @@ public class CloudConfigSetService extends ConfigSetService {
|
|||
// The configSet is read from ZK and populated. Ignore CD's pre-existing configSet; only populated in standalone
|
||||
final String configSetName;
|
||||
try {
|
||||
//TODO readConfigName() also validates the configSet exists but seems needless. We'll get errors soon enough.
|
||||
configSetName = zkController.getZkStateReader().readConfigName(colName);
|
||||
cd.setConfigSet(configSetName);
|
||||
} catch (KeeperException ex) {
|
||||
|
|
|
@ -269,7 +269,6 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
|
|||
NamedList<Object> collections = (NamedList<Object>) cluster.get("collections");
|
||||
assertNotNull("Collections should not be null in cluster state", collections);
|
||||
assertNotNull("Testing to insure collections are returned", collections.get(COLLECTION_NAME1));
|
||||
assertNull("Should have failed to find: " + collection + " because the configset was delted. ", collections.get(collection));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -283,6 +283,7 @@ public class ZkStateReader implements SolrCloseable {
|
|||
|
||||
/**
|
||||
* Returns config set name for collection.
|
||||
* TODO move to DocCollection (state.json).
|
||||
*
|
||||
* @param collection to return config set name for
|
||||
*/
|
||||
|
@ -294,10 +295,6 @@ public class ZkStateReader implements SolrCloseable {
|
|||
log.debug("Loading collection config from: [{}]", path);
|
||||
|
||||
try {
|
||||
if (zkClient.exists(path, true) == false) {
|
||||
log.warn("No collection found at path {}.", path);
|
||||
throw new KeeperException.NoNodeException("No collection found at path: " + path);
|
||||
}
|
||||
byte[] data = zkClient.getData(path, null, null, true);
|
||||
if (data == null) {
|
||||
log.warn("No config data found at path {}.", path);
|
||||
|
@ -311,14 +308,6 @@ public class ZkStateReader implements SolrCloseable {
|
|||
log.warn("No config data found at path{}. ", path);
|
||||
throw new KeeperException.NoNodeException("No config data found at path: " + path);
|
||||
}
|
||||
|
||||
String configPath = CONFIGS_ZKNODE + "/" + configName;
|
||||
if (zkClient.exists(configPath, true) == false) {
|
||||
log.error("Specified config=[{}] does not exist in ZooKeeper at location=[{}]", configName, configPath);
|
||||
throw new KeeperException.NoNodeException("Specified config=[" + configName + "] does not exist in ZooKeeper at location=[" + configPath + "]");
|
||||
} else {
|
||||
log.debug("path=[{}] [{}]=[{}] specified config exists in ZooKeeper", configPath, CONFIGNAME_PROP, configName);
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
SolrZkClient.checkInterrupted(e);
|
||||
log.warn("Thread interrupted when loading config name for collection {}", collection);
|
||||
|
|
Loading…
Reference in New Issue