You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2017/12/11 23:43:23 UTC
[2/2] lucene-solr:jira/solr-11702: SOLR-11702: Redesign current LIR
implementation
SOLR-11702: Redesign current LIR implementation
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/93767bb4
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/93767bb4
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/93767bb4
Branch: refs/heads/jira/solr-11702
Commit: 93767bb4599c13abfcab5a3de85d7d46d883af2b
Parents: ca84ca2
Author: Cao Manh Dat <da...@apache.org>
Authored: Tue Dec 12 06:41:22 2017 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Tue Dec 12 06:41:22 2017 +0700
----------------------------------------------------------------------
.../org/apache/solr/cloud/ElectionContext.java | 113 +-----
.../cloud/LeaderInitiatedRecoveryThread.java | 347 -----------------
.../solr/cloud/RecoveringCoreTermWatcher.java | 62 ++++
.../org/apache/solr/cloud/RecoveryStrategy.java | 6 +-
.../apache/solr/cloud/ZkCollectionTerms.java | 49 +++
.../org/apache/solr/cloud/ZkController.java | 368 ++-----------------
.../org/apache/solr/cloud/ZkShardTerms.java | 251 +++++++++++++
.../solr/handler/admin/CollectionsHandler.java | 9 -
.../solr/update/DefaultSolrCoreState.java | 19 +-
.../processor/DistributedUpdateProcessor.java | 64 +++-
.../org/apache/solr/cloud/ForceLeaderTest.java | 247 -------------
.../solr/cloud/HttpPartitionOnCommitTest.java | 178 +++++++++
.../apache/solr/cloud/HttpPartitionTest.java | 60 +--
.../LeaderInitiatedRecoveryOnCommitTest.java | 178 ---------
...aderInitiatedRecoveryOnShardRestartTest.java | 186 ----------
.../TestLeaderInitiatedRecoveryThread.java | 242 ------------
.../org/apache/solr/cloud/ZkShardTermsTest.java | 195 ++++++++++
.../solr/update/TestInPlaceUpdatesDistrib.java | 32 +-
.../cloud/AbstractFullDistribZkTestBase.java | 5 +-
19 files changed, 849 insertions(+), 1762 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93767bb4/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
index 7169ea8..d659c73 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
@@ -302,6 +302,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
InterruptedException, IOException {
String coreName = leaderProps.getStr(ZkStateReader.CORE_NAME_PROP);
ActionThrottle lt;
+ String coreNodeName;
try (SolrCore core = cc.getCore(coreName)) {
if (core == null ) {
if (cc.isShutDown()) {
@@ -310,6 +311,14 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
throw new SolrException(ErrorCode.SERVER_ERROR, "SolrCore not found:" + coreName + " in " + cc.getLoadedCoreNames());
}
}
+ coreNodeName = core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName();
+ if (!zkController.getShardTerms(collection, shardId).canBecomeLeader(coreNodeName)) {
+ // no need to do recovery here, we already has term less than leader term, the recovery will be triggered later
+ log.info("Can not become leader, this core has term less than leader's term");
+ cancelElection();
+ leaderElector.joinElection(this, true);
+ return;
+ }
MDCLoggingContext.setCore(core);
lt = core.getUpdateHandler().getSolrCoreState().getLeaderThrottle();
}
@@ -442,8 +451,6 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
boolean isLeader = true;
if (!isClosed) {
try {
- // we must check LIR before registering as leader
- checkLIR(coreName, allReplicasInLine);
if (replicaType == Replica.Type.TLOG) {
// stop replicate from old leader
zkController.stopReplicationFromLeader(coreName);
@@ -491,16 +498,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
rejoinLeaderElection(core);
}
}
-
- if (isLeader) {
- // check for any replicas in my shard that were set to down by the previous leader
- try {
- startLeaderInitiatedRecoveryOnReplicas(coreName);
- } catch (Exception exc) {
- // don't want leader election to fail because of
- // an error trying to tell others to recover
- }
- }
+
} else {
cancelElection();
}
@@ -530,97 +528,6 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
return docCollection.getReplica(replicaName);
}
- public void checkLIR(String coreName, boolean allReplicasInLine)
- throws InterruptedException, KeeperException, IOException {
- if (allReplicasInLine) {
- log.info("Found all replicas participating in election, clear LIR");
- // SOLR-8075: A bug may allow the proper leader to get marked as LIR DOWN and
- // if we are marked as DOWN but were able to become the leader, we remove
- // the DOWN entry here so that we don't fail publishing ACTIVE due to being in LIR.
- // We only do this if all the replicas participated in the election just in case
- // this was a valid LIR entry and the proper leader replica is missing.
- try (SolrCore core = cc.getCore(coreName)) {
- final Replica.State lirState = zkController.getLeaderInitiatedRecoveryState(collection, shardId,
- core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName());
- if (lirState == Replica.State.DOWN) {
- // We can do this before registering as leader because only setting DOWN requires that
- // we are already registered as leader, and here we are setting ACTIVE
- // The fact that we just won the zk leader election provides a quasi lock on setting this state, but
- // we should improve this: see SOLR-8075 discussion
- zkController.updateLeaderInitiatedRecoveryState(collection, shardId,
- leaderProps.getStr(ZkStateReader.CORE_NODE_NAME_PROP), Replica.State.ACTIVE, core.getCoreDescriptor(), true);
- }
- }
-
- } else {
- try (SolrCore core = cc.getCore(coreName)) {
- if (core != null) {
- final Replica.State lirState = zkController.getLeaderInitiatedRecoveryState(collection, shardId,
- core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName());
- if (lirState == Replica.State.DOWN || lirState == Replica.State.RECOVERING) {
- log.warn("The previous leader marked me " + core.getName()
- + " as " + lirState.toString() + " and I haven't recovered yet, so I shouldn't be the leader.");
-
- throw new SolrException(ErrorCode.SERVER_ERROR, "Leader Initiated Recovery prevented leadership");
- }
- }
- }
- }
- }
-
- private void startLeaderInitiatedRecoveryOnReplicas(String coreName) throws Exception {
- try (SolrCore core = cc.getCore(coreName)) {
- CloudDescriptor cloudDesc = core.getCoreDescriptor().getCloudDescriptor();
- String coll = cloudDesc.getCollectionName();
- String shardId = cloudDesc.getShardId();
- String coreNodeName = cloudDesc.getCoreNodeName();
-
- if (coll == null || shardId == null) {
- log.error("Cannot start leader-initiated recovery on new leader (core="+
- coreName+",coreNodeName=" + coreNodeName + ") because collection and/or shard is null!");
- return;
- }
-
- String znodePath = zkController.getLeaderInitiatedRecoveryZnodePath(coll, shardId);
- List<String> replicas = null;
- try {
- replicas = zkClient.getChildren(znodePath, null, false);
- } catch (NoNodeException nne) {
- // this can be ignored
- }
-
- if (replicas != null && replicas.size() > 0) {
- for (String replicaCoreNodeName : replicas) {
-
- if (coreNodeName.equals(replicaCoreNodeName))
- continue; // added safe-guard so we don't mark this core as down
-
- final Replica.State lirState = zkController.getLeaderInitiatedRecoveryState(coll, shardId, replicaCoreNodeName);
- if (lirState == Replica.State.DOWN || lirState == Replica.State.RECOVERY_FAILED) {
- log.info("After core={} coreNodeName={} was elected leader, a replica coreNodeName={} was found in state: "
- + lirState.toString() + " and needing recovery.", coreName, coreNodeName, replicaCoreNodeName);
- List<ZkCoreNodeProps> replicaProps =
- zkController.getZkStateReader().getReplicaProps(collection, shardId, coreNodeName);
-
- if (replicaProps != null && replicaProps.size() > 0) {
- ZkCoreNodeProps coreNodeProps = null;
- for (ZkCoreNodeProps p : replicaProps) {
- if (((Replica)p.getNodeProps()).getName().equals(replicaCoreNodeName)) {
- coreNodeProps = p;
- break;
- }
- }
-
- zkController.ensureReplicaInLeaderInitiatedRecovery(cc,
- collection, shardId, coreNodeProps, core.getCoreDescriptor(),
- false /* forcePublishState */);
- }
- }
- }
- }
- } // core gets closed automagically
- }
-
// returns true if all replicas are found to be up, false if not
private boolean waitForReplicasToComeUp(int timeoutms) throws InterruptedException {
long timeoutAt = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeoutms, TimeUnit.MILLISECONDS);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93767bb4/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java b/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java
deleted file mode 100644
index 8c892ce..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java
+++ /dev/null
@@ -1,347 +0,0 @@
-/*
- * 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.cloud;
-
-import org.apache.http.NoHttpResponseException;
-import org.apache.http.conn.ConnectTimeoutException;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
-import org.apache.solr.client.solrj.request.CoreAdminRequest.RequestRecovery;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.common.cloud.Replica;
-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.params.CoreAdminParams.CoreAdminAction;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.CoreContainer;
-import org.apache.solr.core.CoreDescriptor;
-import org.apache.zookeeper.KeeperException;
-import org.apache.solr.util.RTimer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.lang.invoke.MethodHandles;
-import java.net.ConnectException;
-import java.net.SocketException;
-import java.util.List;
-
-/**
- * Background daemon thread that tries to send the REQUESTRECOVERY to a downed
- * replica; used by a shard leader to nag a replica into recovering after the
- * leader experiences an error trying to send an update request to the replica.
- */
-public class LeaderInitiatedRecoveryThread extends Thread {
-
- private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- protected ZkController zkController;
- protected CoreContainer coreContainer;
- protected String collection;
- protected String shardId;
- protected ZkCoreNodeProps nodeProps;
- protected int maxTries;
- private CoreDescriptor leaderCd;
-
- public LeaderInitiatedRecoveryThread(ZkController zkController,
- CoreContainer cc,
- String collection,
- String shardId,
- ZkCoreNodeProps nodeProps,
- int maxTries,
- CoreDescriptor leaderCd)
- {
- super("LeaderInitiatedRecoveryThread-"+nodeProps.getCoreName());
- this.zkController = zkController;
- this.coreContainer = cc;
- this.collection = collection;
- this.shardId = shardId;
- this.nodeProps = nodeProps;
- this.maxTries = maxTries;
- this.leaderCd = leaderCd;
- setDaemon(true);
- }
-
- public void run() {
- RTimer timer = new RTimer();
-
- String replicaCoreName = nodeProps.getCoreName();
- String replicaCoreNodeName = ((Replica) nodeProps.getNodeProps()).getName();
- String replicaNodeName = nodeProps.getNodeName();
- final String replicaUrl = nodeProps.getCoreUrl();
-
- if (!zkController.isReplicaInRecoveryHandling(replicaUrl)) {
- throw new SolrException(ErrorCode.INVALID_STATE, "Replica: " + replicaUrl + " should have been marked under leader initiated recovery in ZkController but wasn't.");
- }
-
- boolean sendRecoveryCommand = publishDownState(replicaCoreName, replicaCoreNodeName, replicaNodeName, replicaUrl, false);
-
- if (sendRecoveryCommand) {
- try {
- sendRecoveryCommandWithRetry();
- } catch (Exception exc) {
- log.error(getName()+" failed due to: "+exc, exc);
- if (exc instanceof SolrException) {
- throw (SolrException)exc;
- } else {
- throw new SolrException(ErrorCode.SERVER_ERROR, exc);
- }
- } finally {
- zkController.removeReplicaFromLeaderInitiatedRecoveryHandling(replicaUrl);
- }
- } else {
- // replica is no longer in recovery on this node (may be handled on another node)
- zkController.removeReplicaFromLeaderInitiatedRecoveryHandling(replicaUrl);
- }
- log.info("{} completed successfully after running for {}ms", getName(), timer.getTime());
- }
-
- public boolean publishDownState(String replicaCoreName, String replicaCoreNodeName, String replicaNodeName, String replicaUrl, boolean forcePublishState) {
- boolean sendRecoveryCommand = true;
- boolean publishDownState = false;
-
- if (zkController.getZkStateReader().getClusterState().liveNodesContain(replicaNodeName)) {
- try {
- // create a znode that requires the replica needs to "ack" to verify it knows it was out-of-sync
- updateLIRState(replicaCoreNodeName);
-
- log.info("Put replica core={} coreNodeName={} on " +
- replicaNodeName + " into leader-initiated recovery.", replicaCoreName, replicaCoreNodeName);
- publishDownState = true;
- } catch (Exception e) {
- Throwable setLirZnodeFailedCause = SolrException.getRootCause(e);
- log.error("Leader failed to set replica " +
- nodeProps.getCoreUrl() + " state to DOWN due to: " + setLirZnodeFailedCause, setLirZnodeFailedCause);
- if (setLirZnodeFailedCause instanceof KeeperException.SessionExpiredException
- || setLirZnodeFailedCause instanceof KeeperException.ConnectionLossException
- || setLirZnodeFailedCause instanceof ZkController.NotLeaderException) {
- // our session is expired, which means our state is suspect, so don't go
- // putting other replicas in recovery (see SOLR-6511)
- sendRecoveryCommand = false;
- forcePublishState = false; // no need to force publish any state in this case
- } // else will go ahead and try to send the recovery command once after this error
- }
- } else {
- log.info("Node " + replicaNodeName +
- " is not live, so skipping leader-initiated recovery for replica: core={} coreNodeName={}",
- replicaCoreName, replicaCoreNodeName);
- // publishDownState will be false to avoid publishing the "down" state too many times
- // as many errors can occur together and will each call into this method (SOLR-6189)
- forcePublishState = false; // no need to force publish the state because replica is not live
- sendRecoveryCommand = false; // no need to send recovery messages as well
- }
-
- try {
- if (publishDownState || forcePublishState) {
- ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, "state",
- ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
- ZkStateReader.BASE_URL_PROP, nodeProps.getBaseUrl(),
- ZkStateReader.CORE_NAME_PROP, nodeProps.getCoreName(),
- ZkStateReader.NODE_NAME_PROP, nodeProps.getNodeName(),
- ZkStateReader.SHARD_ID_PROP, shardId,
- ZkStateReader.COLLECTION_PROP, collection);
- log.warn("Leader is publishing core={} coreNodeName ={} state={} on behalf of un-reachable replica {}",
- replicaCoreName, replicaCoreNodeName, Replica.State.DOWN.toString(), replicaUrl);
- zkController.getOverseerJobQueue().offer(Utils.toJSON(m));
- }
- } catch (Exception e) {
- log.error("Could not publish 'down' state for replicaUrl: {}", replicaUrl, e);
- }
-
- return sendRecoveryCommand;
- }
-
- /*
- protected scope for testing purposes
- */
- protected void updateLIRState(String replicaCoreNodeName) {
- zkController.updateLeaderInitiatedRecoveryState(collection,
- shardId,
- replicaCoreNodeName, Replica.State.DOWN, leaderCd, true);
- }
-
- protected void sendRecoveryCommandWithRetry() throws Exception {
- int tries = 0;
- long waitBetweenTriesMs = 5000L;
- boolean continueTrying = true;
-
- String replicaCoreName = nodeProps.getCoreName();
- String recoveryUrl = nodeProps.getBaseUrl();
- String replicaNodeName = nodeProps.getNodeName();
- String coreNeedingRecovery = nodeProps.getCoreName();
- String replicaCoreNodeName = ((Replica) nodeProps.getNodeProps()).getName();
- String replicaUrl = nodeProps.getCoreUrl();
-
- log.info(getName()+" started running to send REQUESTRECOVERY command to "+replicaUrl+
- "; will try for a max of "+(maxTries * (waitBetweenTriesMs/1000))+" secs");
-
- RequestRecovery recoverRequestCmd = new RequestRecovery();
- recoverRequestCmd.setAction(CoreAdminAction.REQUESTRECOVERY);
- recoverRequestCmd.setCoreName(coreNeedingRecovery);
-
- while (continueTrying && ++tries <= maxTries) {
- if (tries > 1) {
- log.warn("Asking core={} coreNodeName={} on " + recoveryUrl +
- " to recover; unsuccessful after "+tries+" of "+maxTries+" attempts so far ...", coreNeedingRecovery, replicaCoreNodeName);
- } else {
- log.info("Asking core={} coreNodeName={} on " + recoveryUrl + " to recover", coreNeedingRecovery, replicaCoreNodeName);
- }
-
- try (HttpSolrClient client = new HttpSolrClient.Builder(recoveryUrl)
- .withConnectionTimeout(15000)
- .withSocketTimeout(60000)
- .build()) {
- try {
- client.request(recoverRequestCmd);
-
- log.info("Successfully sent " + CoreAdminAction.REQUESTRECOVERY +
- " command to core={} coreNodeName={} on " + recoveryUrl, coreNeedingRecovery, replicaCoreNodeName);
-
- continueTrying = false; // succeeded, so stop looping
- } catch (Exception t) {
- Throwable rootCause = SolrException.getRootCause(t);
- boolean wasCommError =
- (rootCause instanceof ConnectException ||
- rootCause instanceof ConnectTimeoutException ||
- rootCause instanceof NoHttpResponseException ||
- rootCause instanceof SocketException);
-
- SolrException.log(log, recoveryUrl + ": Could not tell a replica to recover", t);
-
- if (!wasCommError) {
- continueTrying = false;
- }
- }
- }
-
- // wait a few seconds
- if (continueTrying) {
- try {
- Thread.sleep(waitBetweenTriesMs);
- } catch (InterruptedException ignoreMe) {
- Thread.currentThread().interrupt();
- }
-
- if (coreContainer.isShutDown()) {
- log.warn("Stop trying to send recovery command to downed replica core={} coreNodeName={} on "
- + replicaNodeName + " because my core container is closed.", coreNeedingRecovery, replicaCoreNodeName);
- continueTrying = false;
- break;
- }
-
- // see if the replica's node is still live, if not, no need to keep doing this loop
- ZkStateReader zkStateReader = zkController.getZkStateReader();
- if (!zkStateReader.getClusterState().liveNodesContain(replicaNodeName)) {
- log.warn("Node "+replicaNodeName+" hosting core "+coreNeedingRecovery+
- " is no longer live. No need to keep trying to tell it to recover!");
- continueTrying = false;
- break;
- }
-
- String leaderCoreNodeName = leaderCd.getCloudDescriptor().getCoreNodeName();
- // stop trying if I'm no longer the leader
- if (leaderCoreNodeName != null && collection != null) {
- String leaderCoreNodeNameFromZk = null;
- try {
- leaderCoreNodeNameFromZk = zkController.getZkStateReader().getLeaderRetry(collection, shardId, 1000).getName();
- } catch (Exception exc) {
- log.error("Failed to determine if " + leaderCoreNodeName + " is still the leader for " + collection +
- " " + shardId + " before starting leader-initiated recovery thread for " + replicaUrl + " due to: " + exc);
- }
- if (!leaderCoreNodeName.equals(leaderCoreNodeNameFromZk)) {
- log.warn("Stop trying to send recovery command to downed replica core=" + coreNeedingRecovery +
- ",coreNodeName=" + replicaCoreNodeName + " on " + replicaNodeName + " because " +
- leaderCoreNodeName + " is no longer the leader! New leader is " + leaderCoreNodeNameFromZk);
- continueTrying = false;
- break;
- }
- if (!leaderCd.getCloudDescriptor().isLeader()) {
- log.warn("Stop trying to send recovery command to downed replica core=" + coreNeedingRecovery +
- ",coreNodeName=" + replicaCoreNodeName + " on " + replicaNodeName + " because " +
- leaderCoreNodeName + " is no longer the leader!");
- continueTrying = false;
- break;
- }
- }
-
- // additional safeguard against the replica trying to be in the active state
- // before acknowledging the leader initiated recovery command
- if (collection != null && shardId != null) {
- try {
- // call out to ZooKeeper to get the leader-initiated recovery state
- final Replica.State lirState = zkController.getLeaderInitiatedRecoveryState(collection, shardId, replicaCoreNodeName);
-
- if (lirState == null) {
- log.warn("Stop trying to send recovery command to downed replica core="+coreNeedingRecovery+
- ",coreNodeName=" + replicaCoreNodeName + " on "+replicaNodeName+" because the znode no longer exists.");
- continueTrying = false;
- break;
- }
-
- if (lirState == Replica.State.RECOVERING) {
- // replica has ack'd leader initiated recovery and entered the recovering state
- // so we don't need to keep looping to send the command
- continueTrying = false;
- log.info("Replica "+coreNeedingRecovery+
- " on node "+replicaNodeName+" ack'd the leader initiated recovery state, "
- + "no need to keep trying to send recovery command");
- } else {
- String lcnn = zkStateReader.getLeaderRetry(collection, shardId, 5000).getName();
- List<ZkCoreNodeProps> replicaProps =
- zkStateReader.getReplicaProps(collection, shardId, lcnn);
- if (replicaProps != null && replicaProps.size() > 0) {
- for (ZkCoreNodeProps prop : replicaProps) {
- final Replica replica = (Replica) prop.getNodeProps();
- if (replicaCoreNodeName.equals(replica.getName())) {
- if (replica.getState() == Replica.State.ACTIVE) {
- // replica published its state as "active",
- // which is bad if lirState is still "down"
- if (lirState == Replica.State.DOWN) {
- // OK, so the replica thinks it is active, but it never ack'd the leader initiated recovery
- // so its state cannot be trusted and it needs to be told to recover again ... and we keep looping here
- log.warn("Replica core={} coreNodeName={} set to active but the leader thinks it should be in recovery;"
- + " forcing it back to down state to re-run the leader-initiated recovery process; props: " + replicaProps.get(0), coreNeedingRecovery, replicaCoreNodeName);
- publishDownState(replicaCoreName, replicaCoreNodeName, replicaNodeName, replicaUrl, true);
- }
- }
- break;
- }
- }
- }
- }
- } catch (Exception ignoreMe) {
- log.warn("Failed to determine state of core={} coreNodeName={} due to: "+ignoreMe, coreNeedingRecovery, replicaCoreNodeName);
- // eventually this loop will exhaust max tries and stop so we can just log this for now
- }
- }
- }
- }
-
- // replica is no longer in recovery on this node (may be handled on another node)
- zkController.removeReplicaFromLeaderInitiatedRecoveryHandling(replicaUrl);
-
- if (continueTrying) {
- // ugh! this means the loop timed out before the recovery command could be delivered
- // how exotic do we want to get here?
- log.error("Timed out after waiting for "+(tries * (waitBetweenTriesMs/1000))+
- " secs to send the recovery request to: "+replicaUrl+"; not much more we can do here?");
-
- // TODO: need to raise a JMX event to allow monitoring tools to take over from here
-
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93767bb4/solr/core/src/java/org/apache/solr/cloud/RecoveringCoreTermWatcher.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveringCoreTermWatcher.java b/solr/core/src/java/org/apache/solr/cloud/RecoveringCoreTermWatcher.java
new file mode 100644
index 0000000..82b5cce
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveringCoreTermWatcher.java
@@ -0,0 +1,62 @@
+/*
+ * 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.cloud;
+
+import java.lang.invoke.MethodHandles;
+
+import org.apache.solr.core.CoreDescriptor;
+import org.apache.solr.core.SolrCore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RecoveringCoreTermWatcher implements ZkShardTerms.CoreTermWatcher {
+ private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+ private final SolrCore solrCore;
+
+ public RecoveringCoreTermWatcher(SolrCore solrCore) {
+ this.solrCore = solrCore;
+ }
+
+ @Override
+ public boolean onTermChanged(ZkShardTerms zkShardTerms) {
+ if (solrCore.isClosed()) {
+ return false;
+ }
+ String coreNodeName = solrCore.getCoreDescriptor().getCloudDescriptor().getCoreNodeName();
+ if (!zkShardTerms.canBecomeLeader(coreNodeName)) {
+ log.info("Start recovery on {} because core's term is less than leader's term", coreNodeName);
+ solrCore.getUpdateHandler().getSolrCoreState().doRecovery(solrCore.getCoreContainer(), solrCore.getCoreDescriptor());
+ }
+ return true;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+
+ RecoveringCoreTermWatcher that = (RecoveringCoreTermWatcher) o;
+
+ return solrCore.equals(that.solrCore);
+ }
+
+ @Override
+ public int hashCode() {
+ return solrCore.hashCode();
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93767bb4/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
index 3ab4eca..944ae4f 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
@@ -458,7 +458,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
core.getCoreDescriptor());
return;
}
-
+
// we temporary ignore peersync for tlog replicas
boolean firstTime = replicaType != Replica.Type.TLOG;
@@ -541,7 +541,9 @@ public class RecoveryStrategy implements Runnable, Closeable {
zkController.publish(core.getCoreDescriptor(), Replica.State.ACTIVE);
return;
}
-
+
+ //TODO check connection to the leader
+
LOG.info("Begin buffering updates. core=[{}]", coreName);
ulog.bufferUpdates();
replayed = false;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93767bb4/solr/core/src/java/org/apache/solr/cloud/ZkCollectionTerms.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkCollectionTerms.java b/solr/core/src/java/org/apache/solr/cloud/ZkCollectionTerms.java
new file mode 100644
index 0000000..93126b0
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkCollectionTerms.java
@@ -0,0 +1,49 @@
+/*
+ * 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.cloud;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.util.ObjectReleaseTracker;
+
+public class ZkCollectionTerms implements AutoCloseable {
+ private String collection;
+ private Map<String, ZkShardTerms> terms;
+ private SolrZkClient zkClient;
+
+ public ZkCollectionTerms(String collection, SolrZkClient client) {
+ this.collection = collection;
+ this.terms = new HashMap<>();
+ this.zkClient = client;
+ ObjectReleaseTracker.track(this);
+ }
+
+
+ public synchronized ZkShardTerms getShard(String shardId) {
+ if (!terms.containsKey(shardId)) terms.put(shardId, new ZkShardTerms(collection, shardId, zkClient));
+ return terms.get(shardId);
+ }
+
+ public void close() {
+ terms.values().forEach(ZkShardTerms::close);
+ ObjectReleaseTracker.release(this);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93767bb4/solr/core/src/java/org/apache/solr/cloud/ZkController.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index 365da65..3817e04 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -107,7 +107,6 @@ import org.apache.solr.util.RTimer;
import org.apache.solr.util.RefCounted;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.ConnectionLossException;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.KeeperException.SessionExpiredException;
import org.apache.zookeeper.Op;
@@ -116,7 +115,6 @@ import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.slf4j.MDC;
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
@@ -210,6 +208,7 @@ public class ZkController {
private LeaderElector overseerElector;
private Map<String, ReplicateFromLeader> replicateFromLeaders = new ConcurrentHashMap<>();
+ private final Map<String, ZkCollectionTerms> collectionToTerms = new HashMap<>();
// for now, this can be null in tests, in which case recovery will be inactive, and other features
// may accept defaults or use mocks rather than pulling things from a CoreContainer
@@ -226,9 +225,6 @@ public class ZkController {
private volatile boolean isClosed;
- // keeps track of replicas that have been asked to recover by leaders running on this node
- private final Map<String, String> replicasInLeaderInitiatedRecovery = new HashMap<String, String>();
-
// This is an expert and unsupported development mode that does not create
// an Overseer or register a /live node. This let's you monitor the cluster
// and interact with zookeeper via the Solr admin UI on a node outside the cluster,
@@ -547,6 +543,7 @@ public class ZkController {
*/
public void close() {
this.isClosed = true;
+ collectionToTerms.values().forEach(ZkCollectionTerms::close);
try {
for (ElectionContext context : electionContexts.values()) {
try {
@@ -1034,7 +1031,8 @@ public class ZkController {
final String coreZkNodeName = desc.getCloudDescriptor().getCoreNodeName();
assert coreZkNodeName != null : "we should have a coreNodeName by now";
-
+
+ getShardTerms(collection, cloudDesc.getShardId()).registerTerm(coreZkNodeName);
String shardId = cloudDesc.getShardId();
Map<String,Object> props = new HashMap<>();
// we only put a subset of props into the leader node
@@ -1118,7 +1116,7 @@ public class ZkController {
}
}
boolean didRecovery
- = checkRecovery(recoverReloadedCores, isLeader, skipRecovery, collection, coreZkNodeName, core, cc, afterExpiration);
+ = checkRecovery(recoverReloadedCores, isLeader, skipRecovery, collection, coreZkNodeName, shardId, core, cc, afterExpiration);
if (!didRecovery) {
if (isTlogReplicaAndNotLeader) {
startReplicationFromLeader(coreName, true);
@@ -1126,7 +1124,7 @@ public class ZkController {
publish(desc, Replica.State.ACTIVE);
}
-
+ getShardTerms(collection, shardId).addListener(new RecoveringCoreTermWatcher(core));
core.getCoreDescriptor().getCloudDescriptor().setHasRegistered(true);
}
@@ -1295,7 +1293,7 @@ public class ZkController {
* Returns whether or not a recovery was started
*/
private boolean checkRecovery(boolean recoverReloadedCores, final boolean isLeader, boolean skipRecovery,
- final String collection, String shardId,
+ final String collection, String coreZkNodeName, String shardId,
SolrCore core, CoreContainer cc, boolean afterExpiration) {
if (SKIP_AUTO_RECOVERY) {
log.warn("Skipping recovery according to sys prop solrcloud.skip.autorecovery");
@@ -1314,11 +1312,8 @@ public class ZkController {
return true;
}
- // see if the leader told us to recover
- final Replica.State lirState = getLeaderInitiatedRecoveryState(collection, shardId,
- core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName());
- if (lirState == Replica.State.DOWN) {
- log.info("Leader marked core " + core.getName() + " down; starting recovery process");
+ if (!getShardTerms(collection, shardId).canBecomeLeader(coreZkNodeName)) {
+ log.info("Leader's term larger than core " + core.getName() + "; starting recovery process");
core.getUpdateHandler().getSolrCoreState().doRecovery(cc, core.getCoreDescriptor());
return true;
}
@@ -1374,25 +1369,9 @@ public class ZkController {
String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
// If the leader initiated recovery, then verify that this replica has performed
// recovery as requested before becoming active; don't even look at lirState if going down
- if (state != Replica.State.DOWN) {
- final Replica.State lirState = getLeaderInitiatedRecoveryState(collection, shardId, coreNodeName);
- if (lirState != null) {
- assert cd.getCloudDescriptor().getReplicaType() != Replica.Type.PULL: "LIR should not happen for pull replicas!";
- if (state == Replica.State.ACTIVE) {
- // trying to become active, so leader-initiated state must be recovering
- if (lirState == Replica.State.RECOVERING) {
- updateLeaderInitiatedRecoveryState(collection, shardId, coreNodeName, Replica.State.ACTIVE, cd, true);
- } else if (lirState == Replica.State.DOWN) {
- throw new SolrException(ErrorCode.INVALID_STATE,
- "Cannot publish state of core '" + cd.getName() + "' as active without recovering first!");
- }
- } else if (state == Replica.State.RECOVERING) {
- // if it is currently DOWN, then trying to enter into recovering state is good
- if (lirState == Replica.State.DOWN) {
- updateLeaderInitiatedRecoveryState(collection, shardId, coreNodeName, Replica.State.RECOVERING, cd, true);
- }
- }
- }
+ if (state == Replica.State.ACTIVE && !getShardTerms(collection, shardId).canBecomeLeader(coreNodeName)) {
+ throw new SolrException(ErrorCode.INVALID_STATE,
+ "Cannot publish state of core '" + cd.getName() + "' as active without recovering first!");
}
Map<String,Object> props = new HashMap<>();
@@ -1430,6 +1409,9 @@ public class ZkController {
log.info("The core '{}' had failed to initialize before.", cd.getName());
}
+ if (state == Replica.State.RECOVERING) {
+ getShardTerms(collection, shardId).setEqualsToMax(coreNodeName);
+ }
ZkNodeProps m = new ZkNodeProps(props);
if (updateLastState) {
@@ -1441,23 +1423,17 @@ public class ZkController {
}
}
- private boolean needsToBeAssignedShardId(final CoreDescriptor desc,
- final ClusterState state, final String coreNodeName) {
-
- final CloudDescriptor cloudDesc = desc.getCloudDescriptor();
-
- final String shardId = state.getShardId(getNodeName(), desc.getName());
-
- if (shardId != null) {
- cloudDesc.setShardId(shardId);
- return false;
+ public ZkShardTerms getShardTerms(String collection, String shardId) {
+ synchronized (collectionToTerms) {
+ if (!collectionToTerms.containsKey(collection)) collectionToTerms.put(collection, new ZkCollectionTerms(collection, zkClient));
}
- return true;
+ return collectionToTerms.get(collection).getShard(shardId);
}
public void unregister(String coreName, CoreDescriptor cd) throws Exception {
final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
final String collection = cd.getCloudDescriptor().getCollectionName();
+ getShardTerms(collection, cd.getCloudDescriptor().getShardId()).removeTerm(collection, cd);
if (Strings.isNullOrEmpty(collection)) {
log.error("No collection was specified.");
@@ -1733,17 +1709,7 @@ public class ZkController {
boolean isLeader = leaderProps.getCoreUrl().equals(ourUrl);
if (!isLeader && !SKIP_AUTO_RECOVERY) {
- // detect if this core is in leader-initiated recovery and if so,
- // then we don't need the leader to wait on seeing the down state
- Replica.State lirState = null;
- try {
- lirState = getLeaderInitiatedRecoveryState(collection, shard, myCoreNodeName);
- } catch (Exception exc) {
- log.error("Failed to determine if replica " + myCoreNodeName +
- " is in leader-initiated recovery due to: " + exc, exc);
- }
-
- if (lirState != null) {
+ if (!getShardTerms(collection, shard).canBecomeLeader(myCoreNodeName)) {
log.debug("Replica " + myCoreNodeName +
" is already in leader-initiated recovery, so not waiting for leader to see down state.");
} else {
@@ -2047,283 +2013,6 @@ public class ZkController {
return cc;
}
- /**
- * When a leader receives a communication error when trying to send a request to a replica,
- * it calls this method to ensure the replica enters recovery when connectivity is restored.
- * <p>
- * returns true if the node hosting the replica is still considered "live" by ZooKeeper;
- * false means the node is not live either, so no point in trying to send recovery commands
- * to it.
- */
- public boolean ensureReplicaInLeaderInitiatedRecovery(
- final CoreContainer container,
- final String collection, final String shardId, final ZkCoreNodeProps replicaCoreProps,
- CoreDescriptor leaderCd, boolean forcePublishState)
- throws KeeperException, InterruptedException {
- final String replicaUrl = replicaCoreProps.getCoreUrl();
-
- if (collection == null)
- throw new IllegalArgumentException("collection parameter cannot be null for starting leader-initiated recovery for replica: " + replicaUrl);
-
- if (shardId == null)
- throw new IllegalArgumentException("shard parameter cannot be null for starting leader-initiated recovery for replica: " + replicaUrl);
-
- if (replicaUrl == null)
- throw new IllegalArgumentException("replicaUrl parameter cannot be null for starting leader-initiated recovery");
-
- // First, determine if this replica is already in recovery handling
- // which is needed because there can be many concurrent errors flooding in
- // about the same replica having trouble and we only need to send the "needs"
- // recovery signal once
- boolean nodeIsLive = true;
- String replicaNodeName = replicaCoreProps.getNodeName();
- String replicaCoreNodeName = ((Replica) replicaCoreProps.getNodeProps()).getName();
- assert replicaCoreNodeName != null : "No core name for replica " + replicaNodeName;
- synchronized (replicasInLeaderInitiatedRecovery) {
- if (replicasInLeaderInitiatedRecovery.containsKey(replicaUrl)) {
- if (!forcePublishState) {
- log.debug("Replica {} already in leader-initiated recovery handling.", replicaUrl);
- return false; // already in this recovery process
- }
- }
-
- // we only really need to try to start the LIR process if the node itself is "live"
- if (getZkStateReader().getClusterState().liveNodesContain(replicaNodeName)) {
-
- LeaderInitiatedRecoveryThread lirThread =
- new LeaderInitiatedRecoveryThread(this,
- container,
- collection,
- shardId,
- replicaCoreProps,
- 120,
- leaderCd);
- ExecutorService executor = container.getUpdateShardHandler().getUpdateExecutor();
- try {
- MDC.put("DistributedUpdateProcessor.replicaUrlToRecover", replicaCoreProps.getCoreUrl());
- executor.execute(lirThread);
- } finally {
- MDC.remove("DistributedUpdateProcessor.replicaUrlToRecover");
- }
-
- // create a znode that requires the replica needs to "ack" to verify it knows it was out-of-sync
- replicasInLeaderInitiatedRecovery.put(replicaUrl,
- getLeaderInitiatedRecoveryZnodePath(collection, shardId, replicaCoreNodeName));
- log.info("Put replica core={} coreNodeName={} on " +
- replicaNodeName + " into leader-initiated recovery.", replicaCoreProps.getCoreName(), replicaCoreNodeName);
- } else {
- nodeIsLive = false; // we really don't need to send the recovery request if the node is NOT live
- log.info("Node " + replicaNodeName +
- " is not live, so skipping leader-initiated recovery for replica: core={} coreNodeName={}",
- replicaCoreProps.getCoreName(), replicaCoreNodeName);
- // publishDownState will be false to avoid publishing the "down" state too many times
- // as many errors can occur together and will each call into this method (SOLR-6189)
- }
- }
-
- return nodeIsLive;
- }
-
- public boolean isReplicaInRecoveryHandling(String replicaUrl) {
- boolean exists = false;
- synchronized (replicasInLeaderInitiatedRecovery) {
- exists = replicasInLeaderInitiatedRecovery.containsKey(replicaUrl);
- }
- return exists;
- }
-
- public void removeReplicaFromLeaderInitiatedRecoveryHandling(String replicaUrl) {
- synchronized (replicasInLeaderInitiatedRecovery) {
- replicasInLeaderInitiatedRecovery.remove(replicaUrl);
- }
- }
-
- public Replica.State getLeaderInitiatedRecoveryState(String collection, String shardId, String coreNodeName) {
- final Map<String, Object> stateObj = getLeaderInitiatedRecoveryStateObject(collection, shardId, coreNodeName);
- if (stateObj == null) {
- return null;
- }
- final String stateStr = (String) stateObj.get(ZkStateReader.STATE_PROP);
- return stateStr == null ? null : Replica.State.getState(stateStr);
- }
-
- public Map<String, Object> getLeaderInitiatedRecoveryStateObject(String collection, String shardId, String coreNodeName) {
-
- if (collection == null || shardId == null || coreNodeName == null)
- return null; // if we don't have complete data about a core in cloud mode, return null
-
- String znodePath = getLeaderInitiatedRecoveryZnodePath(collection, shardId, coreNodeName);
- byte[] stateData = null;
- try {
- stateData = zkClient.getData(znodePath, null, new Stat(), false);
- } catch (NoNodeException ignoreMe) {
- // safe to ignore as this znode will only exist if the leader initiated recovery
- } catch (ConnectionLossException | SessionExpiredException cle) {
- // sort of safe to ignore ??? Usually these are seen when the core is going down
- // or there are bigger issues to deal with than reading this znode
- log.warn("Unable to read " + znodePath + " due to: " + cle);
- } catch (Exception exc) {
- log.error("Failed to read data from znode " + znodePath + " due to: " + exc);
- if (exc instanceof SolrException) {
- throw (SolrException) exc;
- } else {
- throw new SolrException(ErrorCode.SERVER_ERROR,
- "Failed to read data from znodePath: " + znodePath, exc);
- }
- }
-
- Map<String, Object> stateObj = null;
- if (stateData != null && stateData.length > 0) {
- // TODO: Remove later ... this is for upgrading from 4.8.x to 4.10.3 (see: SOLR-6732)
- if (stateData[0] == (byte) '{') {
- Object parsedJson = Utils.fromJSON(stateData);
- if (parsedJson instanceof Map) {
- stateObj = (Map<String, Object>) parsedJson;
- } else {
- throw new SolrException(ErrorCode.SERVER_ERROR, "Leader-initiated recovery state data is invalid! " + parsedJson);
- }
- } else {
- // old format still in ZK
- stateObj = Utils.makeMap("state", new String(stateData, StandardCharsets.UTF_8));
- }
- }
-
- return stateObj;
- }
-
- public void updateLeaderInitiatedRecoveryState(String collection, String shardId, String coreNodeName,
- Replica.State state, CoreDescriptor leaderCd, boolean retryOnConnLoss) {
- if (collection == null || shardId == null || coreNodeName == null) {
- log.warn("Cannot set leader-initiated recovery state znode to "
- + state.toString() + " using: collection=" + collection
- + "; shardId=" + shardId + "; coreNodeName=" + coreNodeName);
- return; // if we don't have complete data about a core in cloud mode, do nothing
- }
-
- assert leaderCd != null;
- assert leaderCd.getCloudDescriptor() != null;
-
- String leaderCoreNodeName = leaderCd.getCloudDescriptor().getCoreNodeName();
-
- String znodePath = getLeaderInitiatedRecoveryZnodePath(collection, shardId, coreNodeName);
-
- if (state == Replica.State.ACTIVE) {
- // since we're marking it active, we don't need this znode anymore, so delete instead of update
- try {
- zkClient.delete(znodePath, -1, retryOnConnLoss);
- } catch (Exception justLogIt) {
- log.warn("Failed to delete znode " + znodePath, justLogIt);
- }
- return;
- }
-
- Map<String, Object> stateObj = null;
- try {
- stateObj = getLeaderInitiatedRecoveryStateObject(collection, shardId, coreNodeName);
- } catch (Exception exc) {
- log.warn(exc.getMessage(), exc);
- }
- if (stateObj == null) {
- stateObj = Utils.makeMap();
- }
-
- stateObj.put(ZkStateReader.STATE_PROP, state.toString());
- // only update the createdBy value if it's not set
- if (stateObj.get("createdByNodeName") == null) {
- stateObj.put("createdByNodeName", this.nodeName);
- }
- if (stateObj.get("createdByCoreNodeName") == null && leaderCoreNodeName != null) {
- stateObj.put("createdByCoreNodeName", leaderCoreNodeName);
- }
-
- byte[] znodeData = Utils.toJSON(stateObj);
-
- try {
- if (state == Replica.State.DOWN) {
- markShardAsDownIfLeader(collection, shardId, leaderCd, znodePath, znodeData, retryOnConnLoss);
- } else {
- // must retry on conn loss otherwise future election attempts may assume wrong LIR state
- if (zkClient.exists(znodePath, true)) {
- zkClient.setData(znodePath, znodeData, retryOnConnLoss);
- } else {
- zkClient.makePath(znodePath, znodeData, retryOnConnLoss);
- }
- }
- log.debug("Wrote {} to {}", state.toString(), znodePath);
- } catch (Exception exc) {
- if (exc instanceof SolrException) {
- throw (SolrException) exc;
- } else {
- throw new SolrException(ErrorCode.SERVER_ERROR,
- "Failed to update data to " + state.toString() + " for znode: " + znodePath, exc);
- }
- }
- }
-
- /**
- * we use ZK's multi-transactional semantics to ensure that we are able to
- * publish a replica as 'down' only if our leader election node still exists
- * in ZK. This ensures that a long running network partition caused by GC etc
- * doesn't let us mark a node as down *after* we've already lost our session
- */
- private void markShardAsDownIfLeader(String collection, String shardId, CoreDescriptor leaderCd,
- String znodePath, byte[] znodeData,
- boolean retryOnConnLoss) throws KeeperException, InterruptedException {
-
-
- if (!leaderCd.getCloudDescriptor().isLeader()) {
- log.info("No longer leader, aborting attempt to mark shard down as part of LIR");
- throw new NotLeaderException(ErrorCode.SERVER_ERROR, "Locally, we do not think we are the leader.");
- }
-
- ContextKey key = new ContextKey(collection, leaderCd.getCloudDescriptor().getCoreNodeName());
- ElectionContext context = electionContexts.get(key);
-
- // we make sure we locally think we are the leader before and after getting the context - then
- // we only try zk if we still think we are the leader and have our leader context
- if (context == null || !leaderCd.getCloudDescriptor().isLeader()) {
- log.info("No longer leader, aborting attempt to mark shard down as part of LIR");
- throw new NotLeaderException(ErrorCode.SERVER_ERROR, "Locally, we do not think we are the leader.");
- }
-
- // we think we are the leader - get the expected shard leader version
- // we use this version and multi to ensure *only* the current zk registered leader
- // for a shard can put a replica into LIR
-
- Integer leaderZkNodeParentVersion = ((ShardLeaderElectionContextBase)context).getLeaderZkNodeParentVersion();
-
- // TODO: should we do this optimistically to avoid races?
- if (zkClient.exists(znodePath, retryOnConnLoss)) {
- List<Op> ops = new ArrayList<>(2);
- ops.add(Op.check(new org.apache.hadoop.fs.Path(((ShardLeaderElectionContextBase)context).leaderPath).getParent().toString(), leaderZkNodeParentVersion));
- ops.add(Op.setData(znodePath, znodeData, -1));
- zkClient.multi(ops, retryOnConnLoss);
- } else {
- String parentZNodePath = getLeaderInitiatedRecoveryZnodePath(collection, shardId);
- try {
- // make sure we don't create /collections/{collection} if they do not exist with 2 param
- zkClient.makePath(parentZNodePath, (byte[]) null, CreateMode.PERSISTENT, (Watcher) null, true, retryOnConnLoss, 2);
- } catch (KeeperException.NodeExistsException nee) {
- // if it exists, that's great!
- }
-
- // we only create the entry if the context we are using is registered as the current leader in ZK
- List<Op> ops = new ArrayList<>(2);
- ops.add(Op.check(new org.apache.hadoop.fs.Path(((ShardLeaderElectionContextBase)context).leaderPath).getParent().toString(), leaderZkNodeParentVersion));
- ops.add(Op.create(znodePath, znodeData, zkClient.getZkACLProvider().getACLsToAdd(znodePath),
- CreateMode.PERSISTENT));
- zkClient.multi(ops, retryOnConnLoss);
- }
- }
-
- public String getLeaderInitiatedRecoveryZnodePath(String collection, String shardId) {
- return "/collections/" + collection + "/leader_initiated_recovery/" + shardId;
- }
-
- public String getLeaderInitiatedRecoveryZnodePath(String collection, String shardId, String coreNodeName) {
- return getLeaderInitiatedRecoveryZnodePath(collection, shardId) + "/" + coreNodeName;
- }
-
public void throwErrorIfReplicaReplaced(CoreDescriptor desc) {
ClusterState clusterState = getZkStateReader().getClusterState();
if (clusterState != null) {
@@ -2608,21 +2297,6 @@ public class ZkController {
};
}
- public String getLeaderSeqPath(String collection, String coreNodeName) {
- ContextKey key = new ContextKey(collection, coreNodeName);
- ElectionContext context = electionContexts.get(key);
- return context != null ? context.leaderSeqPath : null;
- }
-
- /**
- * Thrown during leader initiated recovery process if current node is not leader
- */
- public static class NotLeaderException extends SolrException {
- public NotLeaderException(ErrorCode code, String msg) {
- super(code, msg);
- }
- }
-
public boolean checkIfCoreNodeNameAlreadyExists(CoreDescriptor dcore) {
DocCollection collection = zkStateReader.getClusterState().getCollectionOrNull(dcore.getCollectionName());
if (collection != null) {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93767bb4/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java b/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
new file mode 100644
index 0000000..4b69370
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
@@ -0,0 +1,251 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.solr.client.solrj.cloud.autoscaling.AlreadyExistsException;
+import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
+import org.apache.solr.client.solrj.cloud.autoscaling.DistribStateManager;
+import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
+import org.apache.solr.client.solrj.impl.ZkDistribStateManager;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.util.ObjectReleaseTracker;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.CoreDescriptor;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ZkShardTerms implements AutoCloseable{
+
+ private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+ private final Object writingLock = new Object();
+ private final AtomicInteger numWatcher = new AtomicInteger(0);
+ private final String collection;
+ private final String shard;
+ private final String znodePath;
+ private final DistribStateManager stateManager;
+ private final Set<CoreTermWatcher> listeners = new HashSet<>();
+
+ private Map<String, Long> terms = new HashMap<>();
+ private int version = 0;
+
+ interface CoreTermWatcher {
+ // return true if the listener wanna to be triggered in the next time
+ boolean onTermChanged(ZkShardTerms zkShardTerms);
+ }
+
+ public ZkShardTerms(String collection, String shard, SolrZkClient client) {
+ this.znodePath = "/collections/" + collection + "/terms/" + shard;
+ this.collection = collection;
+ this.shard = shard;
+ this.stateManager = new ZkDistribStateManager(client);
+ ensureTermNodeExist();
+ updateTerms();
+ ObjectReleaseTracker.track(this);
+ }
+
+ public boolean ensureTermsIsHigher(String leader, Set<String> replicasInLowerTerms) {
+ while(!isLessThanLeaderTerm(leader, replicasInLowerTerms)) {
+ synchronized (writingLock) {
+ long leaderTerm = terms.get(leader);
+ for (String replica : terms.keySet()) {
+ if (Objects.equals(terms.get(replica), leaderTerm) && !replicasInLowerTerms.contains(replica)) {
+ terms.put(replica, leaderTerm+1);
+ }
+ }
+ if (forceSaveTerms()) return true;
+ }
+ }
+ return false;
+ }
+
+ public boolean canBecomeLeader(String coreNodeName) {
+ if (terms.isEmpty()) return true;
+ long maxTerm = Collections.max(terms.values());
+ return terms.getOrDefault(coreNodeName, 0L) == maxTerm;
+ }
+
+ public void close() {
+ // no watcher will be registered
+ numWatcher.addAndGet(1);
+ ObjectReleaseTracker.release(this);
+ }
+
+ // package private for testing, only used by tests
+ HashMap<String, Long> getTerms() {
+ return new HashMap<>(terms);
+ }
+
+ void addListener(CoreTermWatcher listener) {
+ synchronized (listeners) {
+ listeners.add(listener);
+ }
+ }
+
+ void removeTerm(String collection, CoreDescriptor cd) {
+ synchronized (listeners) {
+ // solrcore already closed
+ listeners.removeIf(coreTermWatcher -> !coreTermWatcher.onTermChanged(this));
+ }
+ while (true) {
+ synchronized (writingLock) {
+ terms.remove(cd.getCloudDescriptor().getCoreNodeName());
+ try {
+ if (saveTerms()) break;
+ } catch (NoSuchElementException e) {
+ return;
+ }
+ }
+ }
+ }
+
+ void registerTerm(String replica) {
+ while (!terms.containsKey(replica)) {
+ synchronized (writingLock) {
+ terms.put(replica, 0L);
+ forceSaveTerms();
+ }
+ }
+ }
+
+ void setEqualsToMax(String replica) {
+ while (true){
+ synchronized (writingLock) {
+ long maxTerm;
+ try {
+ maxTerm = Collections.max(terms.values());
+ } catch (NoSuchElementException e){
+ maxTerm = 0;
+ }
+ terms.put(replica, maxTerm);
+ if (forceSaveTerms()) break;
+ }
+ }
+ }
+
+ int getNumListeners() {
+ synchronized (listeners) {
+ return listeners.size();
+ }
+ }
+
+ private boolean forceSaveTerms() {
+ try {
+ return saveTerms();
+ } catch (NoSuchElementException e) {
+ ensureTermNodeExist();
+ return false;
+ }
+ }
+
+ private boolean saveTerms() throws NoSuchElementException {
+ byte[] znodeData = Utils.toJSON(terms);
+ // must retry on conn loss otherwise future election attempts may assume wrong LIR state
+ try {
+ stateManager.setData(znodePath, znodeData, version);
+ return true;
+ } catch (BadVersionException e) {
+ updateTerms();
+ } catch (NoSuchElementException e) {
+ throw e;
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ return false;
+ }
+
+
+ private void ensureTermNodeExist() {
+ String path = "/collections/"+collection+ "/terms";
+ try {
+ if (!stateManager.hasData(path)) {
+ try {
+ stateManager.makePath(path);
+ } catch (AlreadyExistsException e) {
+ // it's okay if another beats us creating the node
+ }
+ }
+ path += "/"+shard;
+ if (!stateManager.hasData(path)) {
+ try {
+ Map<String, Long> initialTerms = new HashMap<>();
+ stateManager.createData(path, Utils.toJSON(initialTerms), CreateMode.PERSISTENT);
+ } catch (AlreadyExistsException e) {
+ // it's okay if another beats us creating the node
+ }
+ }
+ } catch (InterruptedException e) {
+ Thread.interrupted();
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error creating shard term node in Zookeeper for collection:" + collection, e);
+ } catch (IOException | KeeperException e) {
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error creating shard term node in Zookeeper for collection:" + collection, e);
+ }
+ }
+
+ private void updateTerms() {
+ try {
+ Watcher watcher = null;
+ if (numWatcher.compareAndSet(0, 1)) {
+ watcher = event -> {
+ numWatcher.decrementAndGet();
+ updateTerms();
+ };
+ }
+
+ VersionedData data = stateManager.getData(znodePath, watcher);
+ version = data.getVersion();
+ terms = (Map<String, Long>) Utils.fromJSON(data.getData());
+ onTermUpdates();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+
+ private boolean isLessThanLeaderTerm(String leader, Set<String> replicasInLowerTerms) {
+ for (String replica : replicasInLowerTerms) {
+ if (!terms.containsKey(leader)) {
+ throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Can not find leader's term " + leader);
+ }
+ if (!terms.containsKey(replica)) return false;
+ if (terms.get(leader) <= terms.get(replica)) return false;
+ }
+ return true;
+ }
+
+ private void onTermUpdates() {
+ synchronized (listeners) {
+ listeners.removeIf(coreTermWatcher -> !coreTermWatcher.onTermChanged(this));
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93767bb4/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index de066d5..6943eaa 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -979,15 +979,6 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
"The shard already has an active leader. Force leader is not applicable. State: " + slice);
}
- // Clear out any LIR state
- String lirPath = handler.coreContainer.getZkController().getLeaderInitiatedRecoveryZnodePath(collectionName, sliceId);
- if (handler.coreContainer.getZkController().getZkClient().exists(lirPath, true)) {
- StringBuilder sb = new StringBuilder();
- handler.coreContainer.getZkController().getZkClient().printLayout(lirPath, 4, sb);
- log.info("Cleaning out LIR data, which was: {}", sb);
- handler.coreContainer.getZkController().getZkClient().clean(lirPath);
- }
-
// Call all live replicas to prepare themselves for leadership, e.g. set last published
// state to active.
for (Replica rep : slice.getReplicas()) {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93767bb4/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java b/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
index b418a19..739604f 100644
--- a/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
+++ b/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
@@ -308,19 +308,20 @@ public final class DefaultSolrCoreState extends SolrCoreState implements Recover
// after the current one, and if there is, bail
boolean locked = recoveryLock.tryLock();
try {
- if (!locked) {
- if (recoveryWaiting.get() > 0) {
- return;
- }
- recoveryWaiting.incrementAndGet();
- } else {
- recoveryWaiting.incrementAndGet();
- cancelRecovery();
+ if (!locked && recoveryWaiting.get() > 0) {
+ return;
}
+
+ recoveryWaiting.incrementAndGet();
+ cancelRecovery();
recoveryLock.lock();
try {
- recoveryWaiting.decrementAndGet();
+ // don't use recoveryLock.getQueueLength() for this
+ if (recoveryWaiting.decrementAndGet() > 0) {
+ // another recovery waiting behind us, let it run now instead of after we finish
+ return;
+ }
// to be air tight we must also check after lock
if (cc.isShutDown()) {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93767bb4/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
index 9f9b742..a9bc12a 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
+import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -343,12 +344,11 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
List<Node> nodes = new ArrayList<>(replicaProps.size());
for (ZkCoreNodeProps props : replicaProps) {
- if (skipList != null) {
- boolean skip = skipListSet.contains(props.getCoreUrl());
- log.info("check url:" + props.getCoreUrl() + " against:" + skipListSet + " result:" + skip);
- if (!skip) {
- nodes.add(new StdNode(props, collection, shardId));
- }
+ String coreNodeName = ((Replica) props.getNodeProps()).getName();
+ if (skipList != null && skipListSet.contains(props.getCoreUrl())) {
+ log.info("check url:" + props.getCoreUrl() + " against:" + skipListSet + " result:true");
+ } else if(!zkController.getShardTerms(collection, shardId).canBecomeLeader(coreNodeName)) {
+ log.info("skip url:{} cause its term is less than leader", props.getCoreUrl());
} else {
nodes.add(new StdNode(props, collection, shardId));
}
@@ -750,7 +750,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
// TODO - we may need to tell about more than one error...
List<Error> errorsForClient = new ArrayList<>(errors.size());
-
+ Map<ShardInfo, Set<String>> failedReplicas = new HashMap<>();
for (final SolrCmdDistributor.Error error : errors) {
if (error.req.node instanceof RetryNode) {
@@ -842,18 +842,14 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
&& foundErrorNodeInReplicaList // we found an error for one of replicas
&& !stdNode.getNodeProps().getCoreUrl().equals(leaderProps.getCoreUrl())) { // we do not want to put ourself into LIR
try {
+ String coreNodeName = ((Replica) stdNode.getNodeProps().getNodeProps()).getName();
// if false, then the node is probably not "live" anymore
// and we do not need to send a recovery message
Throwable rootCause = SolrException.getRootCause(error.e);
- log.error("Setting up to try to start recovery on replica {}", replicaUrl, rootCause);
- zkController.ensureReplicaInLeaderInitiatedRecovery(
- req.getCore().getCoreContainer(),
- collection,
- shardId,
- stdNode.getNodeProps(),
- req.getCore().getCoreDescriptor(),
- false /* forcePublishState */
- );
+ log.error("Setting up to try to start recovery on replica {} with url {}", coreNodeName, replicaUrl, rootCause);
+ ShardInfo shardInfo = new ShardInfo(collection, shardId, leaderCoreNodeName);
+ failedReplicas.putIfAbsent(shardInfo, new HashSet<>());
+ failedReplicas.get(shardInfo).add(coreNodeName);
} catch (Exception exc) {
Throwable setLirZnodeFailedCause = SolrException.getRootCause(exc);
log.error("Leader failed to set replica " +
@@ -872,6 +868,10 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
}
}
}
+ for (Map.Entry<ShardInfo, Set<String>> entry : failedReplicas.entrySet()) {
+ ShardInfo shardInfo = entry.getKey();
+ zkController.getShardTerms(shardInfo.collection, shardInfo.shard).ensureTermsIsHigher(shardInfo.leader, entry.getValue());
+ }
// in either case, we need to attach the achieved and min rf to the response.
if (leaderReplicationTracker != null || rollupReplicationTracker != null) {
int achievedRf = Integer.MAX_VALUE;
@@ -904,6 +904,38 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
}
}
+ private class ShardInfo {
+ private String collection;
+ private String shard;
+ private String leader;
+
+ public ShardInfo(String collection, String shard, String leader) {
+ this.collection = collection;
+ this.shard = shard;
+ this.leader = leader;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+
+ ShardInfo shardInfo = (ShardInfo) o;
+
+ if (!collection.equals(shardInfo.collection)) return false;
+ if (!shard.equals(shardInfo.shard)) return false;
+ return leader.equals(shardInfo.leader);
+ }
+
+ @Override
+ public int hashCode() {
+ int result = collection.hashCode();
+ result = 31 * result + shard.hashCode();
+ result = 31 * result + leader.hashCode();
+ return result;
+ }
+ }
+
// must be synchronized by bucket
private void doLocalAdd(AddUpdateCommand cmd) throws IOException {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/93767bb4/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java b/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java
index 89ff67a..d31e822 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java
@@ -18,7 +18,6 @@ package org.apache.solr.cloud;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
-import java.util.Arrays;
import java.util.Collections;
import java.util.List;
@@ -46,9 +45,6 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.Utils;
import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NoNodeException;
-import org.junit.Ignore;
-import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -63,100 +59,6 @@ public class ForceLeaderTest extends HttpPartitionTest {
return onlyLeaderIndexes;
}
- @Test
- @Override
- @Ignore
- public void test() throws Exception {
-
- }
-
- /***
- * Tests that FORCELEADER can get an active leader after leader puts all replicas in LIR and itself goes down,
- * hence resulting in a leaderless shard.
- */
- @Test
- @Slow
- public void testReplicasInLIRNoLeader() throws Exception {
- handle.put("maxScore", SKIPVAL);
- handle.put("timestamp", SKIPVAL);
-
- String testCollectionName = "forceleader_test_collection";
- createCollection(testCollectionName, "conf1", 1, 3, 1);
- cloudClient.setDefaultCollection(testCollectionName);
-
- try {
- List<Replica> notLeaders = ensureAllReplicasAreActive(testCollectionName, SHARD1, 1, 3, maxWaitSecsToSeeAllActive);
- assertEquals("Expected 2 replicas for collection " + testCollectionName
- + " but found " + notLeaders.size() + "; clusterState: "
- + printClusterStateInfo(testCollectionName), 2, notLeaders.size());
-
- Replica leader = cloudClient.getZkStateReader().getLeaderRetry(testCollectionName, SHARD1);
- JettySolrRunner notLeader0 = getJettyOnPort(getReplicaPort(notLeaders.get(0)));
- ZkController zkController = notLeader0.getCoreContainer().getZkController();
-
- putNonLeadersIntoLIR(testCollectionName, SHARD1, zkController, leader, notLeaders);
-
- cloudClient.getZkStateReader().forceUpdateCollection(testCollectionName);
- ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
- int numActiveReplicas = getNumberOfActiveReplicas(clusterState, testCollectionName, SHARD1);
- assertEquals("Expected only 0 active replica but found " + numActiveReplicas +
- "; clusterState: " + printClusterStateInfo(), 0, numActiveReplicas);
-
- int numReplicasOnLiveNodes = 0;
- for (Replica rep : clusterState.getCollection(testCollectionName).getSlice(SHARD1).getReplicas()) {
- if (clusterState.getLiveNodes().contains(rep.getNodeName())) {
- numReplicasOnLiveNodes++;
- }
- }
- assertEquals(2, numReplicasOnLiveNodes);
- log.info("Before forcing leader: " + printClusterStateInfo());
- // Assert there is no leader yet
- assertNull("Expected no leader right now. State: " + clusterState.getCollection(testCollectionName).getSlice(SHARD1),
- clusterState.getCollection(testCollectionName).getSlice(SHARD1).getLeader());
-
- assertSendDocFails(3);
-
- doForceLeader(cloudClient, testCollectionName, SHARD1);
-
- // By now we have an active leader. Wait for recoveries to begin
- waitForRecoveriesToFinish(testCollectionName, cloudClient.getZkStateReader(), true);
-
- cloudClient.getZkStateReader().forceUpdateCollection(testCollectionName);
- clusterState = cloudClient.getZkStateReader().getClusterState();
- log.info("After forcing leader: " + clusterState.getCollection(testCollectionName).getSlice(SHARD1));
- // we have a leader
- Replica newLeader = clusterState.getCollectionOrNull(testCollectionName).getSlice(SHARD1).getLeader();
- assertNotNull(newLeader);
- // leader is active
- assertEquals(State.ACTIVE, newLeader.getState());
-
- numActiveReplicas = getNumberOfActiveReplicas(clusterState, testCollectionName, SHARD1);
- assertEquals(2, numActiveReplicas);
-
- // Assert that indexing works again
- log.info("Sending doc 4...");
- sendDoc(4);
- log.info("Committing...");
- cloudClient.commit();
- log.info("Doc 4 sent and commit issued");
-
- assertDocsExistInAllReplicas(notLeaders, testCollectionName, 1, 1);
- assertDocsExistInAllReplicas(notLeaders, testCollectionName, 4, 4);
-
- // Docs 1 and 4 should be here. 2 was lost during the partition, 3 had failed to be indexed.
- log.info("Checking doc counts...");
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.add("q", "*:*");
- assertEquals("Expected only 2 documents in the index", 2, cloudClient.query(params).getResults().getNumFound());
-
- bringBackOldLeaderAndSendDoc(testCollectionName, leader, notLeaders, 5);
- } finally {
- log.info("Cleaning up after the test.");
- // try to clean up
- attemptCollectionDelete(cloudClient, testCollectionName);
- }
- }
-
/**
* Test that FORCELEADER can set last published state of all down (live) replicas to active (so
* that they become worthy candidates for leader election).
@@ -204,33 +106,6 @@ public class ForceLeaderTest extends HttpPartitionTest {
}
}
- protected void unsetLeader(String collection, String slice) throws Exception {
- ZkDistributedQueue inQueue = Overseer.getStateUpdateQueue(cloudClient.getZkStateReader().getZkClient());
- ZkStateReader zkStateReader = cloudClient.getZkStateReader();
-
- ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.LEADER.toLower(),
- ZkStateReader.SHARD_ID_PROP, slice,
- ZkStateReader.COLLECTION_PROP, collection);
- inQueue.offer(Utils.toJSON(m));
-
- ClusterState clusterState = null;
- boolean transition = false;
- for (int counter = 10; counter > 0; counter--) {
- clusterState = zkStateReader.getClusterState();
- Replica newLeader = clusterState.getCollection(collection).getSlice(slice).getLeader();
- if (newLeader == null) {
- transition = true;
- break;
- }
- Thread.sleep(1000);
- }
-
- if (!transition) {
- throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Could not unset replica leader" +
- ". Cluster state: " + printClusterStateInfo(collection));
- }
- }
-
protected void setReplicaState(String collection, String slice, Replica replica, Replica.State state) throws Exception {
DistributedQueue inQueue = Overseer.getStateUpdateQueue(cloudClient.getZkStateReader().getZkClient());
ZkStateReader zkStateReader = cloudClient.getZkStateReader();
@@ -263,23 +138,6 @@ public class ForceLeaderTest extends HttpPartitionTest {
". Last known state of the replica: " + replicaState);
}
}
-
- /*protected void setLastPublishedState(String collection, String slice, Replica replica, Replica.State state) throws SolrServerException, IOException,
- KeeperException, InterruptedException {
- ZkStateReader zkStateReader = cloudClient.getZkStateReader();
- String baseUrl = zkStateReader.getBaseUrlForNodeName(replica.getNodeName());
-
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.set(CoreAdminParams.ACTION, CoreAdminAction.FORCEPREPAREFORLEADERSHIP.toString());
- params.set(CoreAdminParams.CORE, replica.getStr("core"));
- params.set(ZkStateReader.STATE_PROP, state.toString());
-
- SolrRequest<SimpleSolrResponse> req = new GenericSolrRequest(METHOD.GET, "/admin/cores", params);
- NamedList resp = null;
- try (HttpSolrClient hsc = new HttpSolrClient(baseUrl)) {
- resp = hsc.request(req);
- }
- }*/
protected Replica.State getLastPublishedState(String collection, String slice, Replica replica) throws SolrServerException, IOException,
KeeperException, InterruptedException {
@@ -311,101 +169,6 @@ public class ForceLeaderTest extends HttpPartitionTest {
}
}
- void putNonLeadersIntoLIR(String collectionName, String shard, ZkController zkController, Replica leader, List<Replica> notLeaders) throws Exception {
- SocketProxy[] nonLeaderProxies = new SocketProxy[notLeaders.size()];
- for (int i = 0; i < notLeaders.size(); i++)
- nonLeaderProxies[i] = getProxyForReplica(notLeaders.get(i));
-
- sendDoc(1);
-
- // ok, now introduce a network partition between the leader and both replicas
- log.info("Closing proxies for the non-leader replicas...");
- for (SocketProxy proxy : nonLeaderProxies)
- proxy.close();
-
- // indexing during a partition
- log.info("Sending a doc during the network partition...");
- sendDoc(2);
-
- // Wait a little
- Thread.sleep(2000);
-
- // Kill the leader
- log.info("Killing leader for shard1 of " + collectionName + " on node " + leader.getNodeName() + "");
- JettySolrRunner leaderJetty = getJettyOnPort(getReplicaPort(leader));
- getProxyForReplica(leader).close();
- leaderJetty.stop();
-
- // Wait for a steady state, till LIR flags have been set and the shard is leaderless
- log.info("Sleep and periodically wake up to check for state...");
- for (int i = 0; i < 20; i++) {
- Thread.sleep(1000);
- State lirStates[] = new State[notLeaders.size()];
- for (int j = 0; j < notLeaders.size(); j++)
- lirStates[j] = zkController.getLeaderInitiatedRecoveryState(collectionName, shard, notLeaders.get(j).getName());
-
- ClusterState clusterState = zkController.getZkStateReader().getClusterState();
- boolean allDown = true;
- for (State lirState : lirStates)
- if (Replica.State.DOWN.equals(lirState) == false)
- allDown = false;
- if (allDown && clusterState.getCollection(collectionName).getSlice(shard).getLeader() == null) {
- break;
- }
- log.warn("Attempt " + i + ", waiting on for 1 sec to settle down in the steady state. State: " +
- printClusterStateInfo(collectionName));
- log.warn("LIR state: " + getLIRState(zkController, collectionName, shard));
- }
- log.info("Waking up...");
-
- // remove the network partition
- log.info("Reopening the proxies for the non-leader replicas...");
- for (SocketProxy proxy : nonLeaderProxies)
- proxy.reopen();
-
- log.info("LIR state: " + getLIRState(zkController, collectionName, shard));
-
- State lirStates[] = new State[notLeaders.size()];
- for (int j = 0; j < notLeaders.size(); j++)
- lirStates[j] = zkController.getLeaderInitiatedRecoveryState(collectionName, shard, notLeaders.get(j).getName());
- for (State lirState : lirStates)
- assertTrue("Expected that the replicas would be in LIR state by now. LIR states: "+Arrays.toString(lirStates),
- Replica.State.DOWN == lirState || Replica.State.RECOVERING == lirState);
- }
-
- protected void bringBackOldLeaderAndSendDoc(String collection, Replica leader, List<Replica> notLeaders, int docid) throws Exception {
- // Bring back the leader which was stopped
- log.info("Bringing back originally killed leader...");
- JettySolrRunner leaderJetty = getJettyOnPort(getReplicaPort(leader));
- leaderJetty.start();
- waitForRecoveriesToFinish(collection, cloudClient.getZkStateReader(), true);
- cloudClient.getZkStateReader().forceUpdateCollection(collection);
- ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
- log.info("After bringing back leader: " + clusterState.getCollection(collection).getSlice(SHARD1));
- int numActiveReplicas = getNumberOfActiveReplicas(clusterState, collection, SHARD1);
- assertEquals(1+notLeaders.size(), numActiveReplicas);
- log.info("Sending doc "+docid+"...");
- sendDoc(docid);
- log.info("Committing...");
- cloudClient.commit();
- log.info("Doc "+docid+" sent and commit issued");
- assertDocsExistInAllReplicas(notLeaders, collection, docid, docid);
- assertDocsExistInAllReplicas(Collections.singletonList(leader), collection, docid, docid);
- }
-
- protected String getLIRState(ZkController zkController, String collection, String shard) throws KeeperException, InterruptedException {
- StringBuilder sb = new StringBuilder();
- String path = zkController.getLeaderInitiatedRecoveryZnodePath(collection, shard);
- if (path == null)
- return null;
- try {
- zkController.getZkClient().printLayout(path, 4, sb);
- } catch (NoNodeException ex) {
- return null;
- }
- return sb.toString();
- }
-
@Override
protected int sendDoc(int docId) throws Exception {
SolrInputDocument doc = new SolrInputDocument();
@@ -420,15 +183,5 @@ public class ForceLeaderTest extends HttpPartitionTest {
client.request(forceLeader);
}
- protected int getNumberOfActiveReplicas(ClusterState clusterState, String collection, String sliceId) {
- int numActiveReplicas = 0;
- // Assert all replicas are active
- for (Replica rep : clusterState.getCollection(collection).getSlice(sliceId).getReplicas()) {
- if (rep.getState().equals(State.ACTIVE)) {
- numActiveReplicas++;
- }
- }
- return numActiveReplicas;
- }
}