Return-Path: X-Original-To: apmail-lucene-commits-archive@www.apache.org Delivered-To: apmail-lucene-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 8E50C17CB5 for ; Sat, 4 Apr 2015 07:02:23 +0000 (UTC) Received: (qmail 46526 invoked by uid 500); 4 Apr 2015 07:02:23 -0000 Mailing-List: contact commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@lucene.apache.org Delivered-To: mailing list commits@lucene.apache.org Received: (qmail 46513 invoked by uid 99); 4 Apr 2015 07:02:23 -0000 Received: from eris.apache.org (HELO hades.apache.org) (140.211.11.105) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 04 Apr 2015 07:02:23 +0000 Received: from hades.apache.org (localhost [127.0.0.1]) by hades.apache.org (ASF Mail Server at hades.apache.org) with ESMTP id E8A02AC0044 for ; Sat, 4 Apr 2015 07:02:22 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1671240 [1/2] - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/cloud/ core/src/java/org/apache/solr/cloud/overseer/ core/src/java/org/apache/solr/core/ core/src/java/org/apache/solr/handler/ core/src/java/org/apache/solr/handl... Date: Sat, 04 Apr 2015 07:02:21 -0000 To: commits@lucene.apache.org From: shaie@apache.org X-Mailer: svnmailer-1.0.9 Message-Id: <20150404070222.E8A02AC0044@hades.apache.org> Author: shaie Date: Sat Apr 4 07:02:20 2015 New Revision: 1671240 URL: http://svn.apache.org/r1671240 Log: SOLR-7336: Add State enum to Replica Modified: lucene/dev/trunk/solr/CHANGES.txt lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/Overseer.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/JarRepository.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ZkContainer.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/ZookeeperInfoServlet.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrCLI.java lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/AssignTest.java lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/CustomCollectionTest.java lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DeleteInactiveReplicaTest.java lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/DeleteReplicaTest.java lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/LeaderInitiatedRecoveryOnCommitTest.java lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverUtilsTest.java lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/SyncSliceTest.java lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java Modified: lucene/dev/trunk/solr/CHANGES.txt URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/CHANGES.txt (original) +++ lucene/dev/trunk/solr/CHANGES.txt Sat Apr 4 07:02:20 2015 @@ -71,6 +71,9 @@ Upgrading from Solr 5.1 * SOLR-7325: Slice.getState() now returns a State enum instead of a String. This helps clarify the states a Slice can be in, as well comparing the state of a Slice. (Shai Erera) + +* SOLR-7336: Added Replica.getState() and removed ZkStateReader state-related constants. + You should use Replica.State to compare a replica's state. (Shai Erera) Detailed Change List ---------------------- Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java Sat Apr 4 07:02:20 2015 @@ -17,14 +17,15 @@ package org.apache.solr.cloud; * limitations under the License. */ -import com.google.common.base.Strings; +import java.util.Properties; + +import org.apache.solr.common.cloud.Replica; import org.apache.solr.common.cloud.Slice; -import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.params.SolrParams; import org.apache.solr.core.CoreDescriptor; import org.apache.solr.util.PropertiesUtil; -import java.util.Properties; +import com.google.common.base.Strings; public class CloudDescriptor { @@ -43,7 +44,7 @@ public class CloudDescriptor { volatile String shardParent = null; volatile boolean isLeader = false; - volatile String lastPublished = ZkStateReader.ACTIVE; + volatile Replica.State lastPublished = Replica.State.ACTIVE; public static final String NUM_SHARDS = "numShards"; @@ -61,7 +62,7 @@ public class CloudDescriptor { this.numShards = PropertiesUtil.toInteger(props.getProperty(CloudDescriptor.NUM_SHARDS), null); } - public String getLastPublished() { + public Replica.State getLastPublished() { return lastPublished; } Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java Sat Apr 4 07:02:20 2015 @@ -161,7 +161,7 @@ class ShardLeaderElectionContextBase ext leaderProps.getProperties().get(ZkStateReader.BASE_URL_PROP), ZkStateReader.CORE_NAME_PROP, leaderProps.getProperties().get(ZkStateReader.CORE_NAME_PROP), - ZkStateReader.STATE_PROP, ZkStateReader.ACTIVE); + ZkStateReader.STATE_PROP, Replica.State.ACTIVE.toString()); Overseer.getInQueue(zkClient).offer(ZkStateReader.toJSON(m)); } } @@ -384,11 +384,11 @@ final class ShardLeaderElectionContext e if (coreNodeName.equals(replicaCoreNodeName)) continue; // added safe-guard so we don't mark this core as down - String lirState = zkController.getLeaderInitiatedRecoveryState(coll, shardId, replicaCoreNodeName); - if (ZkStateReader.DOWN.equals(lirState) || ZkStateReader.RECOVERY_FAILED.equals(lirState)) { + 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 + " and needing recovery.", coreName, coreNodeName, replicaCoreNodeName); - List replicaProps = + + lirState.toString() + " and needing recovery.", coreName, coreNodeName, replicaCoreNodeName); + List replicaProps = zkController.getZkStateReader().getReplicaProps(collection, shardId, coreNodeName); if (replicaProps != null && replicaProps.size() > 0) { @@ -507,15 +507,15 @@ final class ShardLeaderElectionContext e return true; } - if (core.getCoreDescriptor().getCloudDescriptor().getLastPublished().equals(ZkStateReader.ACTIVE)) { + if (core.getCoreDescriptor().getCloudDescriptor().getLastPublished() == Replica.State.ACTIVE) { // maybe active but if the previous leader marked us as down and // we haven't recovered, then can't be leader - String lirState = zkController.getLeaderInitiatedRecoveryState(collection, shardId, + final Replica.State lirState = zkController.getLeaderInitiatedRecoveryState(collection, shardId, core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName()); - if (ZkStateReader.DOWN.equals(lirState) || ZkStateReader.RECOVERING.equals(lirState)) { + if (lirState == Replica.State.DOWN || lirState == Replica.State.RECOVERING) { log.warn("Although my last published state is Active, the previous leader marked me "+core.getName() - + " as " + lirState + + " as " + lirState.toString() + " and I haven't recovered yet, so I shouldn't be the leader."); return false; } Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/LeaderInitiatedRecoveryThread.java Sat Apr 4 07:02:20 2015 @@ -193,8 +193,7 @@ public class LeaderInitiatedRecoveryThre if (collection != null && shardId != null) { try { // call out to ZooKeeper to get the leader-initiated recovery state - String lirState = - zkController.getLeaderInitiatedRecoveryState(collection, shardId, replicaCoreNodeName); + 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+ @@ -203,7 +202,7 @@ public class LeaderInitiatedRecoveryThre break; } - if (ZkStateReader.RECOVERING.equals(lirState)) { + 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; @@ -216,12 +215,12 @@ public class LeaderInitiatedRecoveryThre zkStateReader.getReplicaProps(collection, shardId, leaderCoreNodeName); if (replicaProps != null && replicaProps.size() > 0) { for (ZkCoreNodeProps prop : replicaProps) { - if (replicaCoreNodeName.equals(((Replica) prop.getNodeProps()).getName())) { - String replicaState = prop.getState(); - if (ZkStateReader.ACTIVE.equals(replicaState)) { + 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 (ZkStateReader.DOWN.equals(lirState)) { + 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;" Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/Overseer.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/Overseer.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/Overseer.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/Overseer.java Sat Apr 4 07:02:20 2015 @@ -517,7 +517,7 @@ public class Overseer implements Closeab } private boolean isActive(Replica replica) { - return ZkStateReader.ACTIVE.equals(replica.getStr(ZkStateReader.STATE_PROP)); + return replica.getState() == Replica.State.ACTIVE; } // Collect a list of all the nodes that _can_ host the indicated property. Along the way, also collect any of Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java Sat Apr 4 07:02:20 2015 @@ -19,6 +19,7 @@ package org.apache.solr.cloud; import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; + import org.apache.solr.client.solrj.impl.HttpSolrClient; import org.apache.solr.client.solrj.request.CoreAdminRequest.Create; import org.apache.solr.common.SolrException; @@ -271,13 +272,13 @@ public class OverseerAutoReplicaFailover for (Replica replica : replicas) { // on a live node? boolean live = clusterState.liveNodesContain(replica.getNodeName()); - String state = replica.getStr(ZkStateReader.STATE_PROP); + final Replica.State state = replica.getState(); - boolean okayState = (state.equals(ZkStateReader.DOWN) - || state.equals(ZkStateReader.RECOVERING) || state - .equals(ZkStateReader.ACTIVE)); + final boolean okayState = state == Replica.State.DOWN + || state == Replica.State.RECOVERING + || state == Replica.State.ACTIVE; - log.debug("Process replica name={} live={} state={}", replica.getName(), live, state); + log.debug("Process replica name={} live={} state={}", replica.getName(), live, state.toString()); if (live && okayState) { goodReplicas++; @@ -395,13 +396,10 @@ public class OverseerAutoReplicaFailover if (replicas != null) { log.debug("check if replica already exists on node using replicas {}", getNames(replicas)); for (Replica replica : replicas) { + final Replica.State state = replica.getState(); if (!replica.getName().equals(badReplica.replica.getName()) && replica.getStr(ZkStateReader.BASE_URL_PROP).equals(baseUrl) && clusterState.liveNodesContain(replica.getNodeName()) - && (replica.getStr(ZkStateReader.STATE_PROP).equals( - ZkStateReader.ACTIVE) - || replica.getStr(ZkStateReader.STATE_PROP).equals( - ZkStateReader.DOWN) || replica.getStr( - ZkStateReader.STATE_PROP).equals(ZkStateReader.RECOVERING))) { + && (state == Replica.State.ACTIVE || state == Replica.State.DOWN || state == Replica.State.RECOVERING)) { log.debug("replica already exists on node, bad replica={}, existing replica={}, node name={}", badReplica.replica.getName(), replica.getName(), replica.getNodeName()); return true; } Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java Sat Apr 4 07:02:20 2015 @@ -17,6 +17,11 @@ package org.apache.solr.cloud; * limitations under the License. */ +import static org.apache.solr.cloud.Assign.*; +import static org.apache.solr.common.cloud.ZkStateReader.*; +import static org.apache.solr.common.params.CollectionParams.CollectionAction.*; +import static org.apache.solr.common.params.CommonParams.*; + import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; @@ -37,7 +42,6 @@ import java.util.concurrent.SynchronousQ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import com.google.common.collect.ImmutableSet; import org.apache.commons.lang.StringUtils; import org.apache.solr.client.solrj.SolrResponse; import org.apache.solr.client.solrj.SolrServerException; @@ -94,41 +98,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.MDC; -import static org.apache.solr.cloud.Assign.getNodesForNewShard; -import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.ELECTION_NODE_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_VALUE_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.REJOIN_AT_HEAD_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICAPROP; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDROLE; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.BALANCESHARDUNIQUE; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.CLUSTERSTATUS; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATE; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATESHARD; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETE; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEREPLICAPROP; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETESHARD; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.REMOVEROLE; -import static org.apache.solr.common.params.CommonParams.NAME; +import com.google.common.collect.ImmutableSet; public class OverseerCollectionProcessor implements Runnable, Closeable { public static final String NUM_SLICES = "numShards"; - // @Deprecated- see on ZkStateReader - public static final String REPLICATION_FACTOR = "replicationFactor"; - - // @Deprecated- see on ZkStateReader - public static final String MAX_SHARDS_PER_NODE = "maxShardsPerNode"; - static final boolean CREATE_NODE_SET_SHUFFLE_DEFAULT = true; public static final String CREATE_NODE_SET_SHUFFLE = "createNodeSet.shuffle"; public static final String CREATE_NODE_SET = "createNodeSet"; @@ -582,7 +558,7 @@ public class OverseerCollectionProcessor case RELOAD: ModifiableSolrParams params = new ModifiableSolrParams(); params.set(CoreAdminParams.ACTION, CoreAdminAction.RELOAD.toString()); - collectionCmd(zkStateReader.getClusterState(), message, params, results, ZkStateReader.ACTIVE); + collectionCmd(zkStateReader.getClusterState(), message, params, results, Replica.State.ACTIVE); break; case CREATEALIAS: createAlias(zkStateReader.getAliases(), message); @@ -932,12 +908,12 @@ public class OverseerCollectionProcessor Map replicas = (Map)shardMap.get("replicas"); for (Object nextReplica : replicas.values()) { Map replicaMap = (Map)nextReplica; - if (!ZkStateReader.DOWN.equals(replicaMap.get(ZkStateReader.STATE_PROP))) { + if (Replica.State.getState((String) replicaMap.get(ZkStateReader.STATE_PROP)) != Replica.State.DOWN) { // not down, so verify the node is live String node_name = (String)replicaMap.get(ZkStateReader.NODE_NAME_PROP); if (!liveNodes.contains(node_name)) { // node is not live, so this replica is actually down - replicaMap.put(ZkStateReader.STATE_PROP, ZkStateReader.DOWN); + replicaMap.put(ZkStateReader.STATE_PROP, Replica.State.DOWN.toString()); } } } @@ -1046,8 +1022,7 @@ public class OverseerCollectionProcessor // If users are being safe and only want to remove a shard if it is down, they can specify onlyIfDown=true // on the command. - if (Boolean.parseBoolean(message.getStr(ONLY_IF_DOWN)) && - ZkStateReader.DOWN.equals(replica.getStr(ZkStateReader.STATE_PROP)) == false) { + if (Boolean.parseBoolean(message.getStr(ONLY_IF_DOWN)) && replica.getState() != Replica.State.DOWN) { throw new SolrException(ErrorCode.BAD_REQUEST, "Attempted to remove replica : " + collectionName + "/" + shard + "/" + replicaName + " with onlyIfDown='true', but state is '" + replica.getStr(ZkStateReader.STATE_PROP) + "'"); @@ -1073,7 +1048,7 @@ public class OverseerCollectionProcessor log.warn("Exception trying to unload core " + sreq, e); } - collectShardResponses(!ZkStateReader.ACTIVE.equals(replica.getStr(ZkStateReader.STATE_PROP)) ? new NamedList() : results, + collectShardResponses(replica.getState() != Replica.State.ACTIVE ? new NamedList() : results, false, null, shardHandler); if (waitForCoreNodeGone(collectionName, shard, replicaName, 5000)) @@ -1586,7 +1561,7 @@ public class OverseerCollectionProcessor cmd.setCoreName(subShardName); cmd.setNodeName(nodeName); cmd.setCoreNodeName(coreNodeName); - cmd.setState(ZkStateReader.ACTIVE); + cmd.setState(Replica.State.ACTIVE); cmd.setCheckLive(true); cmd.setOnlyIfLeader(true); @@ -1709,7 +1684,7 @@ public class OverseerCollectionProcessor cmd.setCoreName(subShardNames.get(i - 1)); cmd.setNodeName(subShardNodeName); cmd.setCoreNodeName(coreNodeName); - cmd.setState(ZkStateReader.RECOVERING); + cmd.setState(Replica.State.RECOVERING); cmd.setCheckLive(true); cmd.setOnlyIfLeader(true); ModifiableSolrParams p = new ModifiableSolrParams(cmd.getParams()); @@ -2105,7 +2080,7 @@ public class OverseerCollectionProcessor cmd.setCoreName(tempCollectionReplica1); cmd.setNodeName(sourceLeader.getNodeName()); cmd.setCoreNodeName(coreNodeName); - cmd.setState(ZkStateReader.ACTIVE); + cmd.setState(Replica.State.ACTIVE); cmd.setCheckLive(true); cmd.setOnlyIfLeader(true); // we don't want this to happen asynchronously @@ -2164,7 +2139,7 @@ public class OverseerCollectionProcessor cmd.setCoreName(tempSourceLeader.getStr("core")); cmd.setNodeName(targetLeader.getNodeName()); cmd.setCoreNodeName(coreNodeName); - cmd.setState(ZkStateReader.ACTIVE); + cmd.setState(Replica.State.ACTIVE); cmd.setCheckLive(true); cmd.setOnlyIfLeader(true); params = new ModifiableSolrParams(cmd.getParams()); @@ -2406,7 +2381,7 @@ public class OverseerCollectionProcessor ZkStateReader.COLLECTION_PROP, collectionName, ZkStateReader.SHARD_ID_PROP, sliceName, ZkStateReader.CORE_NAME_PROP, coreName, - ZkStateReader.STATE_PROP, ZkStateReader.DOWN, + ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(), ZkStateReader.BASE_URL_PROP,baseUrl); Overseer.getInQueue(zkStateReader.getZkClient()).offer(ZkStateReader.toJSON(props)); } @@ -2551,7 +2526,7 @@ public class OverseerCollectionProcessor ZkStateReader.COLLECTION_PROP, collection, ZkStateReader.SHARD_ID_PROP, shard, ZkStateReader.CORE_NAME_PROP, coreName, - ZkStateReader.STATE_PROP, ZkStateReader.DOWN, + ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(), ZkStateReader.BASE_URL_PROP, zkStateReader.getBaseUrlForNodeName(node)); Overseer.getInQueue(zkStateReader.getZkClient()).offer(ZkStateReader.toJSON(props)); params.set(CoreAdminParams.CORE_NODE_NAME, waitToSeeReplicasInState(collection, Collections.singletonList(coreName)).get(coreName).getName()); @@ -2661,7 +2636,7 @@ public class OverseerCollectionProcessor } - private void collectionCmd(ClusterState clusterState, ZkNodeProps message, ModifiableSolrParams params, NamedList results, String stateMatcher) { + private void collectionCmd(ClusterState clusterState, ZkNodeProps message, ModifiableSolrParams params, NamedList results, Replica.State stateMatcher) { log.info("Executing Collection Cmd : " + params); String collectionName = message.getStr(NAME); ShardHandler shardHandler = shardHandlerFactory.getShardHandler(); @@ -2677,18 +2652,18 @@ public class OverseerCollectionProcessor } - private void sliceCmd(ClusterState clusterState, ModifiableSolrParams params, String stateMatcher, + private void sliceCmd(ClusterState clusterState, ModifiableSolrParams params, Replica.State stateMatcher, Slice slice, ShardHandler shardHandler) { Map shards = slice.getReplicasMap(); Set> shardEntries = shards.entrySet(); for (Map.Entry shardEntry : shardEntries) { final ZkNodeProps node = shardEntry.getValue(); - if (clusterState.liveNodesContain(node.getStr(ZkStateReader.NODE_NAME_PROP)) && (stateMatcher != null ? node.getStr(ZkStateReader.STATE_PROP).equals(stateMatcher) : true)) { + if (clusterState.liveNodesContain(node.getStr(ZkStateReader.NODE_NAME_PROP)) + && (stateMatcher == null || Replica.State.getState(node.getStr(ZkStateReader.STATE_PROP)) == stateMatcher)) { // For thread safety, only simple clone the ModifiableSolrParams ModifiableSolrParams cloneParams = new ModifiableSolrParams(); cloneParams.add(params); - cloneParams.set(CoreAdminParams.CORE, - node.getStr(ZkStateReader.CORE_NAME_PROP)); + cloneParams.set(CoreAdminParams.CORE, node.getStr(ZkStateReader.CORE_NAME_PROP)); String replica = node.getStr(ZkStateReader.BASE_URL_PROP); ShardRequest sreq = new ShardRequest(); Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java Sat Apr 4 07:02:20 2015 @@ -37,6 +37,7 @@ import org.apache.solr.client.solrj.requ import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException.ErrorCode; import org.apache.solr.common.cloud.ClosableThread; +import org.apache.solr.common.cloud.Replica; import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.ZkCoreNodeProps; import org.apache.solr.common.cloud.ZkNodeProps; @@ -126,7 +127,7 @@ public class RecoveryStrategy extends Th final String shardZkNodeName, final CoreDescriptor cd) throws KeeperException, InterruptedException { SolrException.log(log, "Recovery failed - I give up. core=" + coreName); try { - zkController.publish(cd, ZkStateReader.RECOVERY_FAILED); + zkController.publish(cd, Replica.State.RECOVERY_FAILED); } finally { close(); recoveryListener.failed(); @@ -338,12 +339,12 @@ public class RecoveryStrategy extends Th // we are now the leader - no one else must have been suitable log.warn("We have not yet recovered - but we are now the leader! core=" + coreName); log.info("Finished recovery process. core=" + coreName); - zkController.publish(core.getCoreDescriptor(), ZkStateReader.ACTIVE); + zkController.publish(core.getCoreDescriptor(), Replica.State.ACTIVE); return; } log.info("Publishing state of core "+core.getName()+" as recovering, leader is "+leaderUrl+" and I am "+ourUrl); - zkController.publish(core.getCoreDescriptor(), ZkStateReader.RECOVERING); + zkController.publish(core.getCoreDescriptor(), Replica.State.RECOVERING); final Slice slice = zkStateReader.getClusterState().getSlice(cloudDesc.getCollectionName(), cloudDesc.getShardId()); @@ -413,8 +414,7 @@ public class RecoveryStrategy extends Th } // sync success - register as active and return - zkController.publish(core.getCoreDescriptor(), - ZkStateReader.ACTIVE); + zkController.publish(core.getCoreDescriptor(), Replica.State.ACTIVE); successfulRecovery = true; close = true; return; @@ -453,7 +453,7 @@ public class RecoveryStrategy extends Th log.info("Replication Recovery was successful - registering as Active. core=" + coreName); // if there are pending recovery requests, don't advert as active - zkController.publish(core.getCoreDescriptor(), ZkStateReader.ACTIVE); + zkController.publish(core.getCoreDescriptor(), Replica.State.ACTIVE); close = true; successfulRecovery = true; recoveryListener.recovered(); @@ -577,7 +577,7 @@ public class RecoveryStrategy extends Th prepCmd.setCoreName(leaderCoreName); prepCmd.setNodeName(zkController.getNodeName()); prepCmd.setCoreNodeName(coreZkNodeName); - prepCmd.setState(ZkStateReader.RECOVERING); + prepCmd.setState(Replica.State.RECOVERING); prepCmd.setCheckLive(true); prepCmd.setOnlyIfLeader(true); final Slice.State state = slice.getState(); Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java Sat Apr 4 07:02:20 2015 @@ -106,7 +106,7 @@ import org.slf4j.MDC; */ public final class ZkController { - private static Logger log = LoggerFactory.getLogger(ZkController.class); + private static final Logger log = LoggerFactory.getLogger(ZkController.class); private final boolean SKIP_AUTO_RECOVERY = Boolean.getBoolean("solrcloud.skip.autorecovery"); @@ -379,7 +379,7 @@ public final class ZkController { for (CoreDescriptor descriptor : descriptors) { try { descriptor.getCloudDescriptor().setLeader(false); - publish(descriptor, ZkStateReader.DOWN, updateLastPublished); + publish(descriptor, Replica.State.DOWN, updateLastPublished); } catch (Exception e) { if (isClosed) { return; @@ -390,7 +390,7 @@ public final class ZkController { Thread.currentThread().interrupt(); } try { - publish(descriptor, ZkStateReader.DOWN); + publish(descriptor, Replica.State.DOWN); } catch (Exception e2) { SolrException.log(log, "", e2); continue; @@ -668,10 +668,9 @@ public final class ZkController { Collection replicas = slice.getReplicas(); for (Replica replica : replicas) { if (getNodeName().equals(replica.getNodeName()) - && !(replica.getStr(ZkStateReader.STATE_PROP) - .equals(ZkStateReader.DOWN))) { + && replica.getState() != Replica.State.DOWN) { ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, "state", - ZkStateReader.STATE_PROP, ZkStateReader.DOWN, + ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(), ZkStateReader.BASE_URL_PROP, getBaseUrl(), ZkStateReader.CORE_NAME_PROP, replica.getStr(ZkStateReader.CORE_NAME_PROP), @@ -702,8 +701,7 @@ public final class ZkController { for (Slice slice : slices) { Collection replicas = slice.getReplicas(); for (Replica replica : replicas) { - if (replica.getStr(ZkStateReader.STATE_PROP).equals( - ZkStateReader.DOWN)) { + if (replica.getState() == Replica.State.DOWN) { updatedNodes.remove(replica.getStr(ZkStateReader.CORE_NAME_PROP)); } @@ -903,7 +901,7 @@ public final class ZkController { boolean didRecovery = checkRecovery(coreName, desc, recoverReloadedCores, isLeader, cloudDesc, collection, coreZkNodeName, shardId, leaderProps, core, cc); if (!didRecovery) { - publish(desc, ZkStateReader.ACTIVE); + publish(desc, Replica.State.ACTIVE); } } } @@ -1072,9 +1070,9 @@ public final class ZkController { } // see if the leader told us to recover - String lirState = getLeaderInitiatedRecoveryState(collection, shardId, + final Replica.State lirState = getLeaderInitiatedRecoveryState(collection, shardId, core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName()); - if (ZkStateReader.DOWN.equals(lirState)) { + if (lirState == Replica.State.DOWN) { log.info("Leader marked core " + core.getName() + " down; starting recovery process"); core.getUpdateHandler().getSolrCoreState().doRecovery(cc, core.getCoreDescriptor()); return true; @@ -1091,18 +1089,18 @@ public final class ZkController { return baseURL; } - public void publish(final CoreDescriptor cd, final String state) throws KeeperException, InterruptedException { + public void publish(final CoreDescriptor cd, final Replica.State state) throws KeeperException, InterruptedException { publish(cd, state, true); } - public void publish(final CoreDescriptor cd, final String state, boolean updateLastState) throws KeeperException, InterruptedException { + public void publish(final CoreDescriptor cd, final Replica.State state, boolean updateLastState) throws KeeperException, InterruptedException { publish(cd, state, updateLastState, false); } /** * Publish core state to overseer. */ - public void publish(final CoreDescriptor cd, final String state, boolean updateLastState, boolean forcePublish) throws KeeperException, InterruptedException { + public void publish(final CoreDescriptor cd, final Replica.State state, boolean updateLastState, boolean forcePublish) throws KeeperException, InterruptedException { if (!forcePublish) { try (SolrCore core = cc.getCore(cd.getName())) { if (core == null || core.isClosed()) { @@ -1118,7 +1116,7 @@ public final class ZkController { try { if (cd != null && cd.getName() != null) MDCUtils.setCore(cd.getName()); - log.info("publishing core={} state={} collection={}", cd.getName(), state, collection); + log.info("publishing core={} state={} collection={}", cd.getName(), state.toString(), collection); //System.out.println(Thread.currentThread().getStackTrace()[3]); Integer numShards = cd.getCloudDescriptor().getNumShards(); if (numShards == null) { //XXX sys prop hack @@ -1133,21 +1131,21 @@ public final 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 (!ZkStateReader.DOWN.equals(state)) { - String lirState = getLeaderInitiatedRecoveryState(collection, shardId, coreNodeName); + if (state != Replica.State.DOWN) { + final Replica.State lirState = getLeaderInitiatedRecoveryState(collection, shardId, coreNodeName); if (lirState != null) { - if (ZkStateReader.ACTIVE.equals(state)) { + if (state == Replica.State.ACTIVE) { // trying to become active, so leader-initiated state must be recovering - if (ZkStateReader.RECOVERING.equals(lirState)) { - updateLeaderInitiatedRecoveryState(collection, shardId, coreNodeName, ZkStateReader.ACTIVE, null, true); - } else if (ZkStateReader.DOWN.equals(lirState)) { + if (lirState == Replica.State.RECOVERING) { + updateLeaderInitiatedRecoveryState(collection, shardId, coreNodeName, Replica.State.ACTIVE, null); + } 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 (ZkStateReader.RECOVERING.equals(state)) { + } else if (state == Replica.State.RECOVERING) { // if it is currently DOWN, then trying to enter into recovering state is good - if (ZkStateReader.DOWN.equals(lirState)) { - updateLeaderInitiatedRecoveryState(collection, shardId, coreNodeName, ZkStateReader.RECOVERING, null, true); + if (lirState == Replica.State.DOWN) { + updateLeaderInitiatedRecoveryState(collection, shardId, coreNodeName, Replica.State.RECOVERING, null); } } } @@ -1155,7 +1153,7 @@ public final class ZkController { Map props = new HashMap<>(); props.put(Overseer.QUEUE_OPERATION, "state"); - props.put(ZkStateReader.STATE_PROP, state); + props.put(ZkStateReader.STATE_PROP, state.toString()); props.put(ZkStateReader.BASE_URL_PROP, getBaseUrl()); props.put(ZkStateReader.CORE_NAME_PROP, cd.getName()); props.put(ZkStateReader.ROLES_PROP, cd.getCloudDescriptor().getRoles()); @@ -1250,7 +1248,7 @@ public final class ZkController { if (configLocation != null) { synchronized (confDirectoryListeners) { - log.info("This conf directory is no more watched {0}", configLocation); + log.info("This conf directory is no more watched {}", configLocation); confDirectoryListeners.remove(configLocation); } } @@ -1498,7 +1496,7 @@ public final class ZkController { cloudDesc.setCoreNodeName(coreNodeName); } - publish(cd, ZkStateReader.DOWN, false, true); + publish(cd, Replica.State.DOWN, false, true); DocCollection collection = zkStateReader.getClusterState().getCollectionOrNull(cd.getCloudDescriptor().getCollectionName()); if (collection != null && collection.getStateFormat() > 1) { log.info("Registering watch for external collection {}", cd.getCloudDescriptor().getCollectionName()); @@ -1600,7 +1598,7 @@ public final class ZkController { // 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 - String lirState = null; + Replica.State lirState = null; try { lirState = getLeaderInitiatedRecoveryState(collection, shard, myCoreNodeName); } catch (Exception exc) { @@ -1623,7 +1621,7 @@ public final class ZkController { prepCmd.setCoreName(leaderCoreName); prepCmd.setNodeName(getNodeName()); prepCmd.setCoreNodeName(coreZkNodeName); - prepCmd.setState(ZkStateReader.DOWN); + prepCmd.setState(Replica.State.DOWN); // let's retry a couple times - perhaps the leader just went down, // or perhaps he is just not quite ready for us yet @@ -1941,7 +1939,7 @@ public final class ZkController { // we only really need to try to send the recovery command if the node itself is "live" if (getZkStateReader().getClusterState().liveNodesContain(replicaNodeName)) { // create a znode that requires the replica needs to "ack" to verify it knows it was out-of-sync - updateLeaderInitiatedRecoveryState(collection, shardId, replicaCoreNodeName, ZkStateReader.DOWN, leaderCoreNodeName, retryOnConnLoss); + updateLeaderInitiatedRecoveryState(collection, shardId, replicaCoreNodeName, Replica.State.DOWN, leaderCoreNodeName); replicasInLeaderInitiatedRecovery.put(replicaUrl, getLeaderInitiatedRecoveryZnodePath(collection, shardId, replicaCoreNodeName)); log.info("Put replica core={} coreNodeName={} on " + @@ -1960,14 +1958,14 @@ public final class ZkController { if (publishDownState || forcePublishState) { String replicaCoreName = replicaCoreProps.getCoreName(); ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, "state", - ZkStateReader.STATE_PROP, ZkStateReader.DOWN, + ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(), ZkStateReader.BASE_URL_PROP, replicaCoreProps.getBaseUrl(), ZkStateReader.CORE_NAME_PROP, replicaCoreProps.getCoreName(), ZkStateReader.NODE_NAME_PROP, replicaCoreProps.getNodeName(), ZkStateReader.SHARD_ID_PROP, shardId, ZkStateReader.COLLECTION_PROP, collection); log.warn("Leader is publishing core={} coreNodeName ={} state={} on behalf of un-reachable replica {}; forcePublishState? " + forcePublishState, - replicaCoreName, replicaCoreNodeName, ZkStateReader.DOWN, replicaUrl); + replicaCoreName, replicaCoreNodeName, Replica.State.DOWN.toString(), replicaUrl); overseerJobQueue.offer(ZkStateReader.toJSON(m)); } @@ -1988,9 +1986,13 @@ public final class ZkController { } } - public String getLeaderInitiatedRecoveryState(String collection, String shardId, String coreNodeName) { - Map stateObj = getLeaderInitiatedRecoveryStateObject(collection, shardId, coreNodeName); - return (stateObj != null) ? (String) stateObj.get("state") : null; + public Replica.State getLeaderInitiatedRecoveryState(String collection, String shardId, String coreNodeName) { + final Map 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 getLeaderInitiatedRecoveryStateObject(String collection, String shardId, String coreNodeName) { @@ -2037,17 +2039,18 @@ public final class ZkController { return stateObj; } - private void updateLeaderInitiatedRecoveryState(String collection, String shardId, String coreNodeName, String state, - String leaderCoreNodeName, boolean retryOnConnLoss) { + private void updateLeaderInitiatedRecoveryState(String collection, String shardId, String coreNodeName, + Replica.State state, String leaderCoreNodeName) { if (collection == null || shardId == null || coreNodeName == null) { - log.warn("Cannot set leader-initiated recovery state znode to " + state + " using: collection=" + collection + - "; shardId=" + shardId + "; coreNodeName=" + coreNodeName); + 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 } String znodePath = getLeaderInitiatedRecoveryZnodePath(collection, shardId, coreNodeName); - if (ZkStateReader.ACTIVE.equals(state)) { + 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, false); @@ -2066,7 +2069,7 @@ public final class ZkController { if (stateObj == null) stateObj = ZkNodeProps.makeMap(); - stateObj.put("state", state); + 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", String.valueOf(this.nodeName)); @@ -2074,7 +2077,7 @@ public final class ZkController { byte[] znodeData = ZkStateReader.toJSON(stateObj); try { - if (ZkStateReader.DOWN.equals(state)) { + if (state == Replica.State.DOWN) { markShardAsDownIfLeader(collection, shardId, leaderCoreNodeName, znodePath, znodeData); } else { if (zkClient.exists(znodePath, true)) { @@ -2083,13 +2086,13 @@ public final class ZkController { zkClient.makePath(znodePath, znodeData, true); } } - log.info("Wrote " + state + " to " + znodePath); + log.info("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 + " for znode: " + znodePath, exc); + "Failed to update data to " + state.toString() + " for znode: " + znodePath, exc); } } } Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java Sat Apr 4 07:02:20 2015 @@ -314,7 +314,7 @@ public class ReplicaMutator { DocCollection collection = prevState.getCollectionOrNull(collectionName); if (slice != null) { collection = prevState.getCollection(collectionName); - collection = checkAndCompleteShardSplit(prevState, collection, coreNodeName, sliceName, replicaProps); + collection = checkAndCompleteShardSplit(prevState, collection, coreNodeName, sliceName, replica); // get the current slice again because it may have been updated due to checkAndCompleteShardSplit method slice = collection.getSlice(sliceName); sliceProps = slice.getProperties(); @@ -355,19 +355,19 @@ public class ReplicaMutator { return updateState(clusterState, message); } - private DocCollection checkAndCompleteShardSplit(ClusterState prevState, DocCollection collection, String coreNodeName, String sliceName, Map replicaProps) { + private DocCollection checkAndCompleteShardSplit(ClusterState prevState, DocCollection collection, String coreNodeName, String sliceName, Replica replica) { Slice slice = collection.getSlice(sliceName); Map sliceProps = slice.getProperties(); if (slice.getState() == Slice.State.RECOVERY) { log.info("Shard: {} is in recovery state", sliceName); // is this replica active? - if (ZkStateReader.ACTIVE.equals(replicaProps.get(ZkStateReader.STATE_PROP))) { + if (replica.getState() == Replica.State.ACTIVE) { log.info("Shard: {} is in recovery state and coreNodeName: {} is active", sliceName, coreNodeName); // are all other replicas also active? boolean allActive = true; for (Map.Entry entry : slice.getReplicasMap().entrySet()) { if (coreNodeName.equals(entry.getKey())) continue; - if (!ZkStateReader.ACTIVE.equals(entry.getValue().getStr(ZkStateReader.STATE_PROP))) { + if (entry.getValue().getState() != Replica.State.ACTIVE) { allActive = false; break; } @@ -387,7 +387,7 @@ public class ReplicaMutator { log.info("Shard: {} - Fellow sub-shard: {} found", sliceName, otherSlice.getName()); // this is a fellow sub shard so check if all replicas are active for (Map.Entry sliceEntry : otherSlice.getReplicasMap().entrySet()) { - if (!ZkStateReader.ACTIVE.equals(sliceEntry.getValue().getStr(ZkStateReader.STATE_PROP))) { + if (sliceEntry.getValue().getState() != Replica.State.ACTIVE) { allActive = false; break outer; } Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/JarRepository.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/JarRepository.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/JarRepository.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/JarRepository.java Sat Apr 4 07:02:20 2015 @@ -136,7 +136,7 @@ public class JarRepository { List replicas = new ArrayList<>(slice.getReplicasMap().values()); Collections.shuffle(replicas, RANDOM); for (Replica r : replicas) { - if (ZkStateReader.ACTIVE.equals(r.getStr(ZkStateReader.STATE_PROP))) { + if (r.getState() == Replica.State.ACTIVE) { if(zkStateReader.getClusterState().getLiveNodes().contains(r.get(ZkStateReader.NODE_NAME_PROP))){ replica = r; break; Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ZkContainer.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ZkContainer.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ZkContainer.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/ZkContainer.java Sat Apr 4 07:02:20 2015 @@ -17,12 +17,23 @@ package org.apache.solr.core; * limitations under the License. */ +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeoutException; + import org.apache.solr.cloud.CurrentCoreDescriptorProvider; import org.apache.solr.cloud.SolrZkServer; import org.apache.solr.cloud.ZkController; import org.apache.solr.common.SolrException; +import org.apache.solr.common.cloud.Replica; import org.apache.solr.common.cloud.ZkConfigManager; -import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZooKeeperException; import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.util.DefaultSolrThreadFactory; @@ -30,17 +41,6 @@ import org.apache.zookeeper.KeeperExcept import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeoutException; - public class ZkContainer { protected static Logger log = LoggerFactory.getLogger(ZkContainer.class); @@ -185,7 +185,7 @@ public class ZkContainer { SolrException.log(log, "", e); } catch (Exception e) { try { - zkController.publish(core.getCoreDescriptor(), ZkStateReader.DOWN); + zkController.publish(core.getCoreDescriptor(), Replica.State.DOWN); } catch (InterruptedException e1) { Thread.currentThread().interrupt(); log.error("", e1); @@ -215,7 +215,7 @@ public class ZkContainer { for (SolrCore core : cores) { try { - zkController.publish(core.getCoreDescriptor(), ZkStateReader.DOWN); + zkController.publish(core.getCoreDescriptor(), Replica.State.DOWN); } catch (KeeperException e) { CoreContainer.log.error("", e); } catch (InterruptedException e) { Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java Sat Apr 4 07:02:20 2015 @@ -722,8 +722,7 @@ public class SolrConfigHandler extends R if (replicasMap != null) { for (Map.Entry entry : replicasMap.entrySet()) { Replica replica = entry.getValue(); - if (ZkStateReader.ACTIVE.equals(replica.getStr(ZkStateReader.STATE_PROP)) && - liveNodes.contains(replica.getNodeName())) { + if (replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName())) { activeReplicaCoreUrls.add(replica.getCoreUrl()); } } Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java Sat Apr 4 07:02:20 2015 @@ -17,6 +17,14 @@ package org.apache.solr.handler.admin; * limitations under the License. */ +import static org.apache.solr.cloud.Overseer.*; +import static org.apache.solr.cloud.OverseerCollectionProcessor.*; +import static org.apache.solr.common.cloud.DocCollection.*; +import static org.apache.solr.common.cloud.ZkNodeProps.*; +import static org.apache.solr.common.cloud.ZkStateReader.*; +import static org.apache.solr.common.params.CollectionParams.CollectionAction.*; +import static org.apache.solr.common.params.CommonParams.*; + import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -28,7 +36,6 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; -import com.google.common.collect.ImmutableSet; import org.apache.commons.lang.StringUtils; import org.apache.solr.client.solrj.SolrResponse; import org.apache.solr.client.solrj.SolrServerException; @@ -70,55 +77,7 @@ import org.apache.zookeeper.KeeperExcept import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION; -import static org.apache.solr.cloud.OverseerCollectionProcessor.ASYNC; -import static org.apache.solr.cloud.OverseerCollectionProcessor.COLL_CONF; -import static org.apache.solr.cloud.OverseerCollectionProcessor.CREATE_NODE_SET; -import static org.apache.solr.cloud.OverseerCollectionProcessor.CREATE_NODE_SET_SHUFFLE; -import static org.apache.solr.cloud.OverseerCollectionProcessor.NUM_SLICES; -import static org.apache.solr.cloud.OverseerCollectionProcessor.ONLY_ACTIVE_NODES; -import static org.apache.solr.cloud.OverseerCollectionProcessor.ONLY_IF_DOWN; -import static org.apache.solr.cloud.OverseerCollectionProcessor.REPLICATION_FACTOR; -import static org.apache.solr.cloud.OverseerCollectionProcessor.REQUESTID; -import static org.apache.solr.cloud.OverseerCollectionProcessor.SHARDS_PROP; -import static org.apache.solr.cloud.OverseerCollectionProcessor.SHARD_UNIQUE; -import static org.apache.solr.common.cloud.DocCollection.DOC_ROUTER; -import static org.apache.solr.common.cloud.ZkNodeProps.makeMap; -import static org.apache.solr.common.cloud.ZkStateReader.ACTIVE; -import static org.apache.solr.common.cloud.ZkStateReader.AUTO_ADD_REPLICAS; -import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.ELECTION_NODE_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.LEADER_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.MAX_AT_ONCE_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE; -import static org.apache.solr.common.cloud.ZkStateReader.MAX_WAIT_SECONDS_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_VALUE_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.REJOIN_AT_HEAD_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP; -import static org.apache.solr.common.cloud.ZkStateReader.STATE_PROP; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICAPROP; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDROLE; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.BALANCESHARDUNIQUE; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.CLUSTERPROP; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATE; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATEALIAS; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATESHARD; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETE; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEALIAS; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEREPLICA; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEREPLICAPROP; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETESHARD; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.MIGRATE; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.OVERSEERSTATUS; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.REBALANCELEADERS; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.RELOAD; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.REMOVEROLE; -import static org.apache.solr.common.params.CollectionParams.CollectionAction.SPLITSHARD; -import static org.apache.solr.common.params.CommonParams.NAME; +import com.google.common.collect.ImmutableSet; public class CollectionsHandler extends RequestHandlerBase { protected static Logger log = LoggerFactory.getLogger(CollectionsHandler.class); @@ -357,7 +316,7 @@ public class CollectionsHandler extends } // We're the preferred leader, but someone else is leader. Only become leader if we're active. - if (StringUtils.equalsIgnoreCase(replica.getStr(STATE_PROP), ACTIVE) == false) { + if (replica.getState() != Replica.State.ACTIVE) { NamedList inactives = (NamedList) results.get(inactivePreferreds); if (inactives == null) { inactives = new NamedList<>(); Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java Sat Apr 4 07:02:20 2015 @@ -867,7 +867,7 @@ public class CoreAdminHandler extends Re String nodeName = params.get("nodeName"); String coreNodeName = params.get("coreNodeName"); - String waitForState = params.get("state"); + Replica.State waitForState = Replica.State.getState(params.get(ZkStateReader.STATE_PROP)); Boolean checkLive = params.getBool("checkLive"); Boolean onlyIfLeader = params.getBool("onlyIfLeader"); Boolean onlyIfLeaderActive = params.getBool("onlyIfLeaderActive"); @@ -877,7 +877,7 @@ public class CoreAdminHandler extends Re + ", onlyIfLeaderActive: "+onlyIfLeaderActive); int maxTries = 0; - String state = null; + Replica.State state = null; boolean live = false; int retry = 0; while (true) { @@ -918,41 +918,39 @@ public class CoreAdminHandler extends Re cloudDescriptor.getCollectionName() + ") have state: " + waitForState); } - ClusterState clusterState = coreContainer.getZkController() - .getClusterState(); + ClusterState clusterState = coreContainer.getZkController().getClusterState(); String collection = cloudDescriptor.getCollectionName(); - Slice slice = clusterState.getSlice(collection, - cloudDescriptor.getShardId()); + Slice slice = clusterState.getSlice(collection, cloudDescriptor.getShardId()); if (slice != null) { - ZkNodeProps nodeProps = slice.getReplicasMap().get(coreNodeName); - if (nodeProps != null) { - state = nodeProps.getStr(ZkStateReader.STATE_PROP); + final Replica replica = slice.getReplicasMap().get(coreNodeName); + if (replica != null) { + state = replica.getState(); live = clusterState.liveNodesContain(nodeName); - String localState = cloudDescriptor.getLastPublished(); + final Replica.State localState = cloudDescriptor.getLastPublished(); // TODO: This is funky but I've seen this in testing where the replica asks the // leader to be in recovery? Need to track down how that happens ... in the meantime, // this is a safeguard boolean leaderDoesNotNeedRecovery = (onlyIfLeader != null && onlyIfLeader && - core.getName().equals(nodeProps.getStr("core")) && - ZkStateReader.RECOVERING.equals(waitForState) && - ZkStateReader.ACTIVE.equals(localState) && - ZkStateReader.ACTIVE.equals(state)); + core.getName().equals(replica.getStr("core")) && + waitForState == Replica.State.RECOVERING && + localState == Replica.State.ACTIVE && + state == Replica.State.ACTIVE); if (leaderDoesNotNeedRecovery) { log.warn("Leader "+core.getName()+" ignoring request to be in the recovering state because it is live and active."); } - boolean onlyIfActiveCheckResult = onlyIfLeaderActive != null && onlyIfLeaderActive && (localState == null || !localState.equals(ZkStateReader.ACTIVE)); + boolean onlyIfActiveCheckResult = onlyIfLeaderActive != null && onlyIfLeaderActive && localState != Replica.State.ACTIVE; log.info("In WaitForState("+waitForState+"): collection="+collection+", shard="+slice.getName()+ ", thisCore="+core.getName()+", leaderDoesNotNeedRecovery="+leaderDoesNotNeedRecovery+ ", isLeader? "+core.getCoreDescriptor().getCloudDescriptor().isLeader()+ - ", live="+live+", checkLive="+checkLive+", currentState="+state+", localState="+localState+", nodeName="+nodeName+ - ", coreNodeName="+coreNodeName+", onlyIfActiveCheckResult="+onlyIfActiveCheckResult+", nodeProps: "+nodeProps); + ", live="+live+", checkLive="+checkLive+", currentState="+state.toString()+", localState="+localState+", nodeName="+nodeName+ + ", coreNodeName="+coreNodeName+", onlyIfActiveCheckResult="+onlyIfActiveCheckResult+", nodeProps: "+replica); - if (!onlyIfActiveCheckResult && nodeProps != null && (state.equals(waitForState) || leaderDoesNotNeedRecovery)) { + if (!onlyIfActiveCheckResult && replica != null && (state == waitForState || leaderDoesNotNeedRecovery)) { if (checkLive == null) { break; } else if (checkLive && live) { @@ -984,7 +982,7 @@ public class CoreAdminHandler extends Re "I was asked to wait on state " + waitForState + " for " + shardId + " in " + collection + " on " + nodeName + " but I still do not see the requested state. I see state: " - + state + " live:" + live + " leader from ZK: " + leaderInfo + + state.toString() + " live:" + live + " leader from ZK: " + leaderInfo ); } @@ -1050,7 +1048,7 @@ public class CoreAdminHandler extends Re SolrException.log(log, "Replay failed"); throw new SolrException(ErrorCode.SERVER_ERROR, "Replay failed"); } - coreContainer.getZkController().publish(core.getCoreDescriptor(), ZkStateReader.ACTIVE); + coreContainer.getZkController().publish(core.getCoreDescriptor(), Replica.State.ACTIVE); rsp.add("core", cname); rsp.add("status", "BUFFER_APPLIED"); } catch (InterruptedException e) { Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java Sat Apr 4 07:02:20 2015 @@ -16,11 +16,25 @@ package org.apache.solr.handler.componen * limitations under the License. */ -import com.google.common.base.Strings; +import java.net.ConnectException; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.ListIterator; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + import org.apache.http.client.HttpClient; +import org.apache.solr.client.solrj.SolrClient; import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.SolrResponse; -import org.apache.solr.client.solrj.SolrClient; import org.apache.solr.client.solrj.impl.HttpSolrClient; import org.apache.solr.client.solrj.impl.LBHttpSolrClient; import org.apache.solr.client.solrj.request.QueryRequest; @@ -33,7 +47,6 @@ import org.apache.solr.common.cloud.DocC import org.apache.solr.common.cloud.Replica; import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.ZkCoreNodeProps; -import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.ShardParams; @@ -46,20 +59,7 @@ import org.apache.solr.request.SolrQuery import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.net.ConnectException; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.ListIterator; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.CompletionService; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; +import com.google.common.base.Strings; public class HttpShardHandler extends ShardHandler { @@ -407,8 +407,7 @@ public class HttpShardHandler extends Sh String ourCollection = cloudDescriptor.getCollectionName(); if (rb.slices.length == 1 && rb.slices[0] != null && ( rb.slices[0].equals(ourSlice) || rb.slices[0].equals(ourCollection + "_" + ourSlice) ) // handle the _ format - && ZkStateReader.ACTIVE.equals(cloudDescriptor.getLastPublished()) ) - { + && cloudDescriptor.getLastPublished() == Replica.State.ACTIVE) { boolean shortCircuit = params.getBool("shortCircuit", true); // currently just a debugging parameter to check distrib search on a single node String targetHandler = params.get(ShardParams.SHARDS_QT); @@ -448,8 +447,9 @@ public class HttpShardHandler extends Sh boolean first = true; for (Replica replica : sliceShards.values()) { if (!clusterState.liveNodesContain(replica.getNodeName()) - || !replica.getStr(ZkStateReader.STATE_PROP).equals( - ZkStateReader.ACTIVE)) continue; + || replica.getState() != Replica.State.ACTIVE) { + continue; + } if (first) { first = false; } else { Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java Sat Apr 4 07:02:20 2015 @@ -41,8 +41,8 @@ import org.apache.solr.common.SolrExcept import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.DocCollection; +import org.apache.solr.common.cloud.Replica; import org.apache.solr.common.cloud.Slice; -import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.ShardParams; import org.apache.solr.common.params.SolrParams; @@ -560,7 +560,7 @@ public class RealTimeGetComponent extend boolean onlyIfActive = rb.req.getParams().getBool("onlyIfActive", false); if (onlyIfActive) { - if (!rb.req.getCore().getCoreDescriptor().getCloudDescriptor().getLastPublished().equals(ZkStateReader.ACTIVE)) { + if (rb.req.getCore().getCoreDescriptor().getCloudDescriptor().getLastPublished() != Replica.State.ACTIVE) { log.info("Last published state was not ACTIVE, cannot sync."); rb.rsp.add("sync", "false"); return; Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java Sat Apr 4 07:02:20 2015 @@ -294,7 +294,7 @@ public final class ManagedIndexSchema ex for (Map.Entry entry : replicasMap.entrySet()) { Replica replica = entry.getValue(); if (!localCoreNodeName.equals(replica.getName()) && - ZkStateReader.ACTIVE.equals(replica.getStr(ZkStateReader.STATE_PROP)) && + replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName())) { ZkCoreNodeProps replicaCoreProps = new ZkCoreNodeProps(replica); activeReplicaCoreUrls.add(replicaCoreProps.getCoreUrl()); Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java Sat Apr 4 07:02:20 2015 @@ -50,7 +50,6 @@ import org.apache.solr.common.SolrExcept import org.apache.solr.common.cloud.Aliases; import org.apache.solr.common.cloud.Replica; import org.apache.solr.common.cloud.Slice; -import org.apache.solr.common.cloud.ZkCoreNodeProps; import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.util.NamedList; @@ -162,11 +161,10 @@ public class JoinQParserPlugin extends Q fromReplica = replica.getStr(ZkStateReader.CORE_NAME_PROP); // found local replica, but is it Active? - ZkCoreNodeProps replicaCoreProps = new ZkCoreNodeProps(replica); - if (!ZkStateReader.ACTIVE.equals(replicaCoreProps.getState())) + if (replica.getState() != Replica.State.ACTIVE) throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "SolrCloud join: "+fromIndex+" has a local replica ("+fromReplica+ - ") on "+nodeName+", but it is "+replicaCoreProps.getState()); + ") on "+nodeName+", but it is "+replica.getState()); break; } Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java Sat Apr 4 07:02:20 2015 @@ -17,6 +17,32 @@ package org.apache.solr.servlet; +import java.io.ByteArrayInputStream; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; +import java.util.Set; + +import javax.servlet.FilterChain; +import javax.servlet.FilterConfig; +import javax.servlet.ServletException; +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + import org.apache.commons.io.IOUtils; import org.apache.commons.lang.StringUtils; import org.apache.http.Header; @@ -44,7 +70,6 @@ import org.apache.solr.common.cloud.Clus import org.apache.solr.common.cloud.Replica; import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.SolrZkClient; -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.CommonParams; @@ -75,31 +100,6 @@ import org.apache.solr.update.processor. import org.apache.solr.util.RTimer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.servlet.FilterChain; -import javax.servlet.FilterConfig; -import javax.servlet.ServletException; -import javax.servlet.ServletRequest; -import javax.servlet.ServletResponse; -import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; - -import java.io.ByteArrayInputStream; -import java.io.EOFException; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Enumeration; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Properties; -import java.util.Set; /** * This filter looks at the incoming URL maps them to handlers defined in solrconfig.xml @@ -687,24 +687,23 @@ public class SolrDispatchFilter extends Set liveNodes = clusterState.getLiveNodes(); for (Slice slice : slices) { Map sliceShards = slice.getReplicasMap(); - for (ZkNodeProps nodeProps : sliceShards.values()) { - ZkCoreNodeProps coreNodeProps = new ZkCoreNodeProps(nodeProps); - if (!activeReplicas || (liveNodes.contains(coreNodeProps.getNodeName()) - && coreNodeProps.getState().equals(ZkStateReader.ACTIVE))) { + for (Replica replica : sliceShards.values()) { + if (!activeReplicas || (liveNodes.contains(replica.getNodeName()) + && replica.getState() == Replica.State.ACTIVE)) { - if (byCoreName && !collectionName.equals(coreNodeProps.getCoreName())) { + if (byCoreName && !collectionName.equals(replica.getStr(ZkStateReader.CORE_NAME_PROP))) { // if it's by core name, make sure they match continue; } - if (coreNodeProps.getBaseUrl().equals(cores.getZkController().getBaseUrl())) { + if (replica.getStr(ZkStateReader.BASE_URL_PROP).equals(cores.getZkController().getBaseUrl())) { // don't count a local core continue; } if (origCorename != null) { - coreUrl = coreNodeProps.getBaseUrl() + "/" + origCorename; + coreUrl = replica.getStr(ZkStateReader.BASE_URL_PROP) + "/" + origCorename; } else { - coreUrl = coreNodeProps.getCoreUrl(); + coreUrl = replica.getCoreUrl(); if (coreUrl.endsWith("/")) { coreUrl = coreUrl.substring(0, coreUrl.length() - 1); } Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/ZookeeperInfoServlet.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/ZookeeperInfoServlet.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/ZookeeperInfoServlet.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/ZookeeperInfoServlet.java Sat Apr 4 07:02:20 2015 @@ -43,6 +43,7 @@ import org.apache.solr.cloud.ZkControlle import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException.ErrorCode; import org.apache.solr.common.cloud.OnReconnect; +import org.apache.solr.common.cloud.Replica; import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.ZkStateReader; @@ -162,17 +163,17 @@ public final class ZookeeperInfoServlet Map replicas = (Map)shard.get("replicas"); for (String replicaId : replicas.keySet()) { Map replicaState = (Map)replicas.get(replicaId); - String coreState = (String)replicaState.get("state"); + Replica.State coreState = Replica.State.getState((String)replicaState.get(ZkStateReader.STATE_PROP)); String nodeName = (String)replicaState.get("node_name"); // state can lie to you if the node is offline, so need to reconcile with live_nodes too if (!liveNodes.contains(nodeName)) - coreState = ZkStateReader.DOWN; // not on a live node, so must be down + coreState = Replica.State.DOWN; // not on a live node, so must be down - if (ZkStateReader.ACTIVE.equals(coreState)) { + if (coreState == Replica.State.ACTIVE) { hasActive = true; // assumed no replicas active and found one that is for this shard } else { - if (ZkStateReader.RECOVERING.equals(coreState)) { + if (coreState == Replica.State.RECOVERING) { replicaInRecovery = true; } isHealthy = false; // assumed healthy and found one replica that is not @@ -189,7 +190,7 @@ public final class ZookeeperInfoServlet return !hasDownedShard && !isHealthy; // means no shards offline but not 100% healthy either } else if ("downed_shard".equals(filter)) { return hasDownedShard; - } else if (ZkStateReader.RECOVERING.equals(filter)) { + } else if (Replica.State.getState(filter) == Replica.State.RECOVERING) { return !isHealthy && replicaInRecovery; } Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java Sat Apr 4 07:02:20 2015 @@ -372,7 +372,8 @@ public class DistributedUpdateProcessor slice = coll.getSlice(myShardId); shardId = myShardId; leaderReplica = zkController.getZkStateReader().getLeaderRetry(collection, myShardId); - List myReplicas = zkController.getZkStateReader().getReplicaProps(collection, shardId, leaderReplica.getName(), null, ZkStateReader.DOWN); + List myReplicas = zkController.getZkStateReader() + .getReplicaProps(collection, shardId, leaderReplica.getName(), null, Replica.State.DOWN); } } @@ -390,7 +391,7 @@ public class DistributedUpdateProcessor // so get the replicas... forwardToLeader = false; List replicaProps = zkController.getZkStateReader() - .getReplicaProps(collection, shardId, leaderReplica.getName(), null, ZkStateReader.DOWN); + .getReplicaProps(collection, shardId, leaderReplica.getName(), null, Replica.State.DOWN); if (replicaProps != null) { if (nodes == null) { @@ -1391,7 +1392,7 @@ public class DistributedUpdateProcessor Replica leaderReplica = zkController.getZkStateReader().getLeaderRetry( collection, myShardId); List replicaProps = zkController.getZkStateReader() - .getReplicaProps(collection, myShardId, leaderReplica.getName(), null, ZkStateReader.DOWN); + .getReplicaProps(collection, myShardId, leaderReplica.getName(), null, Replica.State.DOWN); if (replicaProps != null) { List myReplicas = new ArrayList<>(); for (ZkCoreNodeProps replicaProp : replicaProps) { Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrCLI.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrCLI.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrCLI.java (original) +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrCLI.java Sat Apr 4 07:02:20 2015 @@ -867,7 +867,7 @@ public class SolrCLI { if (replicaHealth.isLeader) hasLeader = true; - if (!ZkStateReader.ACTIVE.equals(replicaHealth.status)) { + if (!Replica.State.ACTIVE.toString().equals(replicaHealth.status)) { healthy = false; } else { atLeastOneActive = true; @@ -964,7 +964,7 @@ public class SolrCLI { // if replica's node is not live, its status is DOWN String nodeName = replicaCoreProps.getNodeName(); if (nodeName == null || !liveNodes.contains(nodeName)) { - replicaStatus = ZkStateReader.DOWN; + replicaStatus = Replica.State.DOWN.toString(); } else { // query this replica directly to get doc count and assess health q = new SolrQuery("*:*"); @@ -991,7 +991,7 @@ public class SolrCLI { log.error("ERROR: " + exc + " when trying to reach: " + coreUrl); if (checkCommunicationError(exc)) { - replicaStatus = "down"; + replicaStatus = Replica.State.DOWN.toString(); } else { replicaStatus = "error: "+exc; } Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/AssignTest.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/AssignTest.java?rev=1671240&r1=1671239&r2=1671240&view=diff ============================================================================== --- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/AssignTest.java (original) +++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/AssignTest.java Sat Apr 4 07:02:20 2015 @@ -65,7 +65,7 @@ public class AssignTest extends SolrTest Map replicas = new HashMap<>(); ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, "state", - ZkStateReader.STATE_PROP, "ACTIVE", + ZkStateReader.STATE_PROP, Replica.State.ACTIVE.toString(), ZkStateReader.BASE_URL_PROP, "0.0.0.0", ZkStateReader.CORE_NAME_PROP, "core1", ZkStateReader.ROLES_PROP, null,