SOLR-12067: Remove support for autoReplicaFailoverWaitAfterExpiration

This closes #1402.
This commit is contained in:
Shalin Shekhar Mangar 2020-04-06 07:25:14 +05:30
parent e916056c8e
commit 9322a7b375
13 changed files with 44 additions and 42 deletions

View File

@ -53,6 +53,8 @@ Other Changes
* SOLR-13893: Remove support to read BlobRepository's max jar size from deprecated `runtme.lib.size` system property
(Erick Erickson, Kesharee Nandan Vishwakarma, Munendra S N)
* SOLR-12067: Remove support for `autoReplicaFailoverWaitAfterExpiration`. (marcussorealheis, shalin)
================== 8.6.0 ==================
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

View File

@ -578,7 +578,7 @@ public class Overseer implements SolrCloseable {
ThreadGroup triggerThreadGroup = new ThreadGroup("Overseer autoscaling triggers");
OverseerTriggerThread trigger = new OverseerTriggerThread(zkController.getCoreContainer().getResourceLoader(),
zkController.getSolrCloudManager(), config);
zkController.getSolrCloudManager());
triggerThread = new OverseerThread(triggerThreadGroup, trigger, "OverseerAutoScalingTriggerThread-" + id);
updaterThread.start();

View File

@ -39,7 +39,6 @@ import org.apache.solr.common.SolrCloseable;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.CloudConfig;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
@ -60,12 +59,11 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
public static final String MARKER_STATE = "state";
public static final String MARKER_ACTIVE = "active";
public static final String MARKER_INACTIVE = "inactive";
public static final int DEFAULT_AUTO_ADD_REPLICA_WAIT_FOR_SECONDS = 120;
private final SolrCloudManager cloudManager;
private final CloudConfig cloudConfig;
private final ScheduledTriggers scheduledTriggers;
private final AutoScaling.TriggerFactory triggerFactory;
@ -87,9 +85,8 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
private AutoScalingConfig autoScalingConfig;
public OverseerTriggerThread(SolrResourceLoader loader, SolrCloudManager cloudManager, CloudConfig cloudConfig) {
public OverseerTriggerThread(SolrResourceLoader loader, SolrCloudManager cloudManager) {
this.cloudManager = cloudManager;
this.cloudConfig = cloudConfig;
scheduledTriggers = new ScheduledTriggers(loader, cloudManager);
triggerFactory = new AutoScaling.TriggerFactoryImpl(loader, cloudManager);
}
@ -366,7 +363,7 @@ public class OverseerTriggerThread implements Runnable, SolrCloseable {
}
}
// need to add
triggerProps.computeIfPresent("waitFor", (k, v) -> (long) (cloudConfig.getAutoReplicaFailoverWaitAfterExpiration() / 1000));
triggerProps.computeIfPresent("waitFor", (k, v) -> (long) (DEFAULT_AUTO_ADD_REPLICA_WAIT_FOR_SECONDS));
AutoScalingConfig.TriggerConfig config = new AutoScalingConfig.TriggerConfig(triggerName, triggerProps);
autoScalingConfig = autoScalingConfig.withTriggerConfig(config);
// need to add SystemLogListener explicitly here

View File

@ -89,7 +89,6 @@ import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.ObjectCache;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.TimeSource;
import org.apache.solr.core.CloudConfig;
import org.apache.solr.core.SolrInfoBean;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.handler.admin.MetricsHandler;
@ -268,8 +267,7 @@ public class SimCloudManager implements SolrCloudManager {
triggerThreadGroup = new ThreadGroup("Simulated Overseer autoscaling triggers");
OverseerTriggerThread trigger = new OverseerTriggerThread(loader, this,
new CloudConfig.CloudConfigBuilder("nonexistent", 0, "sim").build());
OverseerTriggerThread trigger = new OverseerTriggerThread(loader, this);
triggerThread = new Overseer.OverseerThread(triggerThreadGroup, trigger, "Simulated OverseerAutoScalingTriggerThread");
triggerThread.start();
}
@ -613,8 +611,7 @@ public class SimCloudManager implements SolrCloudManager {
}
simCloudManagerPool = ExecutorUtil.newMDCAwareFixedThreadPool(200, new DefaultSolrThreadFactory("simCloudManagerPool"));
OverseerTriggerThread trigger = new OverseerTriggerThread(loader, this,
new CloudConfig.CloudConfigBuilder("nonexistent", 0, "sim").build());
OverseerTriggerThread trigger = new OverseerTriggerThread(loader, this);
triggerThread = new Overseer.OverseerThread(triggerThreadGroup, trigger, "Simulated OverseerAutoScalingTriggerThread");
triggerThread.start();

View File

@ -36,8 +36,6 @@ public class CloudConfig {
private final int leaderConflictResolveWait;
private final int autoReplicaFailoverWaitAfterExpiration;
private final String zkCredentialsProviderClass;
private final String zkACLProviderClass;
@ -51,9 +49,9 @@ public class CloudConfig {
private final String pkiHandlerPublicKeyPath;
CloudConfig(String zkHost, int zkClientTimeout, int hostPort, String hostName, String hostContext, boolean useGenericCoreNames,
int leaderVoteWait, int leaderConflictResolveWait, int autoReplicaFailoverWaitAfterExpiration,
String zkCredentialsProviderClass, String zkACLProviderClass, int createCollectionWaitTimeTillActive,
boolean createCollectionCheckLeaderActive, String pkiHandlerPrivateKeyPath, String pkiHandlerPublicKeyPath) {
int leaderVoteWait, int leaderConflictResolveWait, String zkCredentialsProviderClass, String zkACLProviderClass,
int createCollectionWaitTimeTillActive, boolean createCollectionCheckLeaderActive, String pkiHandlerPrivateKeyPath,
String pkiHandlerPublicKeyPath) {
this.zkHost = zkHost;
this.zkClientTimeout = zkClientTimeout;
this.hostPort = hostPort;
@ -62,7 +60,6 @@ public class CloudConfig {
this.useGenericCoreNames = useGenericCoreNames;
this.leaderVoteWait = leaderVoteWait;
this.leaderConflictResolveWait = leaderConflictResolveWait;
this.autoReplicaFailoverWaitAfterExpiration = autoReplicaFailoverWaitAfterExpiration;
this.zkCredentialsProviderClass = zkCredentialsProviderClass;
this.zkACLProviderClass = zkACLProviderClass;
this.createCollectionWaitTimeTillActive = createCollectionWaitTimeTillActive;
@ -112,10 +109,6 @@ public class CloudConfig {
return leaderConflictResolveWait;
}
public int getAutoReplicaFailoverWaitAfterExpiration() {
return autoReplicaFailoverWaitAfterExpiration;
}
public boolean getGenericCoreNodeNames() {
return useGenericCoreNames;
}
@ -144,8 +137,6 @@ public class CloudConfig {
private static final int DEFAULT_CREATE_COLLECTION_ACTIVE_WAIT = 45; // 45 seconds
private static final boolean DEFAULT_CREATE_COLLECTION_CHECK_LEADER_ACTIVE = false;
private static final int DEFAULT_AUTO_REPLICA_FAILOVER_WAIT_AFTER_EXPIRATION = 120000;
private String zkHost = System.getProperty("zkHost");
private int zkClientTimeout = Integer.getInteger("zkClientTimeout", DEFAULT_ZK_CLIENT_TIMEOUT);
private final int hostPort;
@ -154,7 +145,6 @@ public class CloudConfig {
private boolean useGenericCoreNames;
private int leaderVoteWait = DEFAULT_LEADER_VOTE_WAIT;
private int leaderConflictResolveWait = DEFAULT_LEADER_CONFLICT_RESOLVE_WAIT;
private int autoReplicaFailoverWaitAfterExpiration = DEFAULT_AUTO_REPLICA_FAILOVER_WAIT_AFTER_EXPIRATION;
private String zkCredentialsProviderClass;
private String zkACLProviderClass;
private int createCollectionWaitTimeTillActive = DEFAULT_CREATE_COLLECTION_ACTIVE_WAIT;
@ -196,12 +186,7 @@ public class CloudConfig {
this.leaderConflictResolveWait = leaderConflictResolveWait;
return this;
}
public CloudConfigBuilder setAutoReplicaFailoverWaitAfterExpiration(int autoReplicaFailoverWaitAfterExpiration) {
this.autoReplicaFailoverWaitAfterExpiration = autoReplicaFailoverWaitAfterExpiration;
return this;
}
public CloudConfigBuilder setZkCredentialsProviderClass(String zkCredentialsProviderClass) {
this.zkCredentialsProviderClass = zkCredentialsProviderClass;
return this;
@ -234,7 +219,7 @@ public class CloudConfig {
public CloudConfig build() {
return new CloudConfig(zkHost, zkClientTimeout, hostPort, hostName, hostContext, useGenericCoreNames, leaderVoteWait,
leaderConflictResolveWait, autoReplicaFailoverWaitAfterExpiration, zkCredentialsProviderClass, zkACLProviderClass, createCollectionWaitTimeTillActive,
leaderConflictResolveWait, zkCredentialsProviderClass, zkACLProviderClass, createCollectionWaitTimeTillActive,
createCollectionCheckLeaderActive, pkiHandlerPrivateKeyPath, pkiHandlerPublicKeyPath);
}
}

View File

@ -397,9 +397,6 @@ public class SolrXmlConfig {
case "zkClientTimeout":
builder.setZkClientTimeout(parseInt(name, value));
break;
case "autoReplicaFailoverWaitAfterExpiration":
builder.setAutoReplicaFailoverWaitAfterExpiration(parseInt(name, value));
break;
case "zkHost":
builder.setZkHost(value);
break;

View File

@ -32,7 +32,6 @@
<int name="leaderVoteWait">${leaderVoteWait:10000}</int>
<int name="distribUpdateConnTimeout">${distribUpdateConnTimeout:45000}</int>
<int name="distribUpdateSoTimeout">${distribUpdateSoTimeout:340000}</int>
<int name="autoReplicaFailoverWaitAfterExpiration">${autoReplicaFailoverWaitAfterExpiration:10000}</int>
<int name="createCollectionWaitTimeTillActive">${createCollectionWaitTimeTillActive:30}</int>
</solrcloud>

View File

@ -32,7 +32,6 @@
<int name="leaderVoteWait">${leaderVoteWait:10000}</int>
<int name="distribUpdateConnTimeout">${distribUpdateConnTimeout:45000}</int>
<int name="distribUpdateSoTimeout">${distribUpdateSoTimeout:340000}</int>
<int name="autoReplicaFailoverWaitAfterExpiration">${autoReplicaFailoverWaitAfterExpiration:10000}</int>
</solrcloud>
<metrics>

View File

@ -33,7 +33,6 @@
<bool name="genericCoreNodeNames">${genericCoreNodeNames:true}</bool>
<int name="distribUpdateConnTimeout">${distribUpdateConnTimeout:45000}</int>
<int name="distribUpdateSoTimeout">${distribUpdateSoTimeout:340000}</int>
<int name="autoReplicaFailoverWaitAfterExpiration">${autoReplicaFailoverWaitAfterExpiration:10000}</int>
</solrcloud>
<shardHandlerFactory name="shardHandlerFactory"

View File

@ -47,7 +47,6 @@
<int name="leaderConflictResolveWait">${leaderConflictResolveWait:45000}</int>
<int name="distribUpdateConnTimeout">${distribUpdateConnTimeout:5000}</int>
<int name="distribUpdateSoTimeout">${distribUpdateSoTimeout:30000}</int> <!-- We are running tests - the default should be low, not like production -->
<int name="autoReplicaFailoverWaitAfterExpiration">${autoReplicaFailoverWaitAfterExpiration:10000}</int>
</solrcloud>
</solr>

View File

@ -73,6 +73,12 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
.withPayload("{set-obj-property:{defaults : {cluster: {useLegacyReplicaAssignment:true}}}}")
.build()
.process(cluster.getSolrClient());
new V2Request.Builder("/cluster/autoscaling")
.withMethod(SolrRequest.METHOD.POST)
.withPayload("{'set-trigger':{'name':'.auto_add_replicas','event':'nodeLost','waitFor':'5s','enabled':'true','actions':[{'name':'auto_add_replicas_plan','class':'solr.AutoAddReplicasPlanAction'},{'name':'auto_add_replicas_plan','class':'solr.ExecutePlanAction'}]}}")
.build()
.process(cluster.getSolrClient());
}
@After

View File

@ -100,6 +100,9 @@ _(raw; not yet edited)_
* SOLR-14344: Remove Deprecated HttpSolrClient.RemoteSolrException and HttpSolrClient.RemoteExcecutionException.
All the usages are replaced by BaseHttpSolrClient.RemoteSolrException and BaseHttpSolrClient.RemoteExcecutionException.
(Munendra S N)
* SOLR-12720: To change the auto add replica wait period modify the `waitFor` attribute of the `.auto_add_replicas` trigger.
(marcussorealheis, shalin)
=== Upgrade Prerequisites in Solr 9

View File

@ -205,10 +205,29 @@ to add replicas on the live nodes to maintain the expected replication factor).
Refer to the section <<solrcloud-autoscaling-auto-add-replicas.adoc#solrcloud-autoscaling-auto-add-replicas, Autoscaling Automatically Adding Replicas>> to learn more about how the `.autoAddReplicas` trigger works.
In addition to the parameters described at <<Trigger Configuration>>, this trigger supports one parameter, which is defined in the `<solrcloud>` section of `solr.xml`:
If you would like to change the value of `.autoAddReplicas` trigger, you need to call the autoscaling API and use the `set-trigger` command to add a value for `waitFor`.
`autoReplicaFailoverWaitAfterExpiration`::
The minimum time in milliseconds to wait for initiating replacement of a replica after first noticing it not being live. This is important to prevent false positives while stopping or starting the cluster. The default is `120000` (2 minutes). The value provided for this parameter is used as the value for the `waitFor` parameter in the `.auto_add_replicas` trigger.
.Example: Updating Trigger with wait for 5 seconds
----
{
"set-trigger": {
"name": ".auto_add_replicas",
"event": "nodeLost,
"waitFor": "5s",
"enabled": true,
"actions": [
{
"name": "auto_add_replicas_plan",
"class": "solr.AutoAddReplicasPlanAction"
},
{
"name": "auto_add_replicas_plan",
"class": "solr.ExecutePlanAction"
}
]
}
}
----
TIP: See <<format-of-solr-xml.adoc#the-solrcloud-element,The <solrcloud> Element>> for more details about how to work with `solr.xml`.