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 B75781033C for ; Wed, 5 Mar 2014 17:29:03 +0000 (UTC) Received: (qmail 60820 invoked by uid 500); 5 Mar 2014 17:28:58 -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 60807 invoked by uid 99); 5 Mar 2014 17:28:57 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 05 Mar 2014 17:28:57 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 05 Mar 2014 17:28:53 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 9AB612388860; Wed, 5 Mar 2014 17:28:31 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1574580 - in /lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud: Overseer.java ZkController.java Date: Wed, 05 Mar 2014 17:28:31 -0000 To: commits@lucene.apache.org From: markrmiller@apache.org X-Mailer: svnmailer-1.0.9 Message-Id: <20140305172831.9AB612388860@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: markrmiller Date: Wed Mar 5 17:28:31 2014 New Revision: 1574580 URL: http://svn.apache.org/r1574580 Log: SOLR-5811: Additional cleanup. Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/Overseer.java lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java 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=1574580&r1=1574579&r2=1574580&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 Wed Mar 5 17:28:31 2014 @@ -311,7 +311,7 @@ public class Overseer { private ClusterState createReplica(ClusterState clusterState, ZkNodeProps message) { log.info("createReplica() {} ", message); String coll = message.getStr(ZkStateReader.COLLECTION_PROP); - checkCollection(message, coll); + if (!checkCollectionKeyExistence(message)) return clusterState; String slice = message.getStr(ZkStateReader.SHARD_ID_PROP); Slice sl = clusterState.getSlice(coll, slice); if(sl == null){ @@ -352,7 +352,7 @@ public class Overseer { private ClusterState updateShardState(ClusterState clusterState, ZkNodeProps message) { String collection = message.getStr(ZkStateReader.COLLECTION_PROP); - checkCollection(message, collection); + if (!checkCollectionKeyExistence(message)) return clusterState; log.info("Update shard state invoked for collection: " + collection + " with message: " + message); for (String key : message.keySet()) { if (ZkStateReader.COLLECTION_PROP.equals(key)) continue; @@ -377,7 +377,7 @@ public class Overseer { private ClusterState addRoutingRule(ClusterState clusterState, ZkNodeProps message) { String collection = message.getStr(ZkStateReader.COLLECTION_PROP); - checkCollection(message, collection); + if (!checkCollectionKeyExistence(message)) return clusterState; String shard = message.getStr(ZkStateReader.SHARD_ID_PROP); String routeKey = message.getStr("routeKey"); String range = message.getStr("range"); @@ -417,15 +417,22 @@ public class Overseer { return clusterState; } - private void checkCollection(ZkNodeProps message, String collection) { - if (collection == null || collection.trim().length() == 0) { - log.error("Skipping invalid Overseer message because it has no collection specified: " + message); + private boolean checkCollectionKeyExistence(ZkNodeProps message) { + return checkKeyExistence(message, ZkStateReader.COLLECTION_PROP); + } + + private boolean checkKeyExistence(ZkNodeProps message, String key) { + String value = message.getStr(key); + if (value == null || value.trim().length() == 0) { + log.error("Skipping invalid Overseer message because it has no " + key + " specified: " + message); + return false; } + return true; } private ClusterState removeRoutingRule(ClusterState clusterState, ZkNodeProps message) { String collection = message.getStr(ZkStateReader.COLLECTION_PROP); - checkCollection(message, collection); + if (!checkCollectionKeyExistence(message)) return clusterState; String shard = message.getStr(ZkStateReader.SHARD_ID_PROP); String routeKeyStr = message.getStr("routeKey"); @@ -451,7 +458,7 @@ public class Overseer { private ClusterState createShard(ClusterState clusterState, ZkNodeProps message) { String collection = message.getStr(ZkStateReader.COLLECTION_PROP); - checkCollection(message, collection); + if (!checkCollectionKeyExistence(message)) return clusterState; String shardId = message.getStr(ZkStateReader.SHARD_ID_PROP); Slice slice = clusterState.getSlice(collection, shardId); if (slice == null) { @@ -498,7 +505,7 @@ public class Overseer { private ClusterState updateStateNew(ClusterState clusterState, ZkNodeProps message) { String collection = message.getStr(ZkStateReader.COLLECTION_PROP); - checkCollection(message, collection); + if (!checkCollectionKeyExistence(message)) return clusterState; String sliceName = message.getStr(ZkStateReader.SHARD_ID_PROP); if(collection==null || sliceName == null){ @@ -517,30 +524,30 @@ public class Overseer { /** * Try to assign core to the cluster. */ - private ClusterState updateState(ClusterState state, final ZkNodeProps message) { + private ClusterState updateState(ClusterState clusterState, final ZkNodeProps message) { final String collection = message.getStr(ZkStateReader.COLLECTION_PROP); - checkCollection(message, collection); + if (!checkCollectionKeyExistence(message)) return clusterState; Integer numShards = message.getInt(ZkStateReader.NUM_SHARDS_PROP, null); log.info("Update state numShards={} message={}", numShards, message); List shardNames = new ArrayList(); //collection does not yet exist, create placeholders if num shards is specified - boolean collectionExists = state.hasCollection(collection); + boolean collectionExists = clusterState.hasCollection(collection); if (!collectionExists && numShards!=null) { getShardNames(numShards, shardNames); - state = createCollection(state, collection, shardNames, message); + clusterState = createCollection(clusterState, collection, shardNames, message); } String sliceName = message.getStr(ZkStateReader.SHARD_ID_PROP); String coreNodeName = message.getStr(ZkStateReader.CORE_NODE_NAME_PROP); if (coreNodeName == null) { - coreNodeName = getAssignedCoreNodeName(state, message); + coreNodeName = getAssignedCoreNodeName(clusterState, message); if (coreNodeName != null) { log.info("node=" + coreNodeName + " is already registered"); } else { // if coreNodeName is null, auto assign one - coreNodeName = Assign.assignNode(collection, state); + coreNodeName = Assign.assignNode(collection, clusterState); } message.getProperties().put(ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName); @@ -549,7 +556,7 @@ public class Overseer { // use the provided non null shardId if (sliceName == null) { //get shardId from ClusterState - sliceName = getAssignedId(state, coreNodeName, message); + sliceName = getAssignedId(clusterState, coreNodeName, message); if (sliceName != null) { log.info("shard=" + sliceName + " is already registered"); } @@ -558,14 +565,14 @@ public class Overseer { //request new shardId if (collectionExists) { // use existing numShards - numShards = state.getCollection(collection).getSlices().size(); + numShards = clusterState.getCollection(collection).getSlices().size(); log.info("Collection already exists with " + ZkStateReader.NUM_SHARDS_PROP + "=" + numShards); } - sliceName = Assign.assignShard(collection, state, numShards); + sliceName = Assign.assignShard(collection, clusterState, numShards); log.info("Assigning new node to shard shard=" + sliceName); } - Slice slice = state.getSlice(collection, sliceName); + Slice slice = clusterState.getSlice(collection, sliceName); Map replicaProps = new LinkedHashMap(); @@ -611,9 +618,9 @@ public class Overseer { Map replicas; if (slice != null) { - state = checkAndCompleteShardSplit(state, collection, coreNodeName, sliceName, replicaProps); + clusterState = checkAndCompleteShardSplit(clusterState, collection, coreNodeName, sliceName, replicaProps); // get the current slice again because it may have been updated due to checkAndCompleteShardSplit method - slice = state.getSlice(collection, sliceName); + slice = clusterState.getSlice(collection, sliceName); sliceProps = slice.getProperties(); replicas = slice.getReplicasCopy(); } else { @@ -627,7 +634,7 @@ public class Overseer { replicas.put(replica.getName(), replica); slice = new Slice(sliceName, replicas, sliceProps); - ClusterState newClusterState = updateSlice(state, collection, slice); + ClusterState newClusterState = updateSlice(clusterState, collection, slice); return newClusterState; } @@ -876,11 +883,9 @@ public class Overseer { * Remove collection from cloudstate */ private ClusterState removeCollection(final ClusterState clusterState, ZkNodeProps message) { - final String collection = message.getStr("name"); - checkCollection(message, collection); + if (!checkKeyExistence(message, "name")) return clusterState; -// ClusterState newState = new ClusterState(clusterState.getLiveNodes(), newCollections); return clusterState.copyWith(singletonMap(collection, (DocCollection)null)); } @@ -888,34 +893,28 @@ public class Overseer { * Remove collection slice from cloudstate */ private ClusterState removeShard(final ClusterState clusterState, ZkNodeProps message) { - final String collection = message.getStr(ZkStateReader.COLLECTION_PROP); - checkCollection(message, collection); final String sliceId = message.getStr(ZkStateReader.SHARD_ID_PROP); + final String collection = message.getStr(ZkStateReader.COLLECTION_PROP); + if (!checkCollectionKeyExistence(message)) return clusterState; log.info("Removing collection: " + collection + " shard: " + sliceId + " from clusterstate"); -// final Map newCollections = new LinkedHashMap(clusterState.getCollectionStates()); // shallow copy DocCollection coll = clusterState.getCollection(collection); Map newSlices = new LinkedHashMap(coll.getSlicesMap()); newSlices.remove(sliceId); DocCollection newCollection = new DocCollection(coll.getName(), newSlices, coll.getProperties(), coll.getRouter()); -// newCollections.put(newCollection.getName(), newCollection); return newState(clusterState, singletonMap(collection,newCollection)); - -// return new ClusterState(clusterState.getLiveNodes(), newCollections); } /* * Remove core from cloudstate */ private ClusterState removeCore(final ClusterState clusterState, ZkNodeProps message) { - - String cnn = message.getStr(ZkStateReader.CORE_NODE_NAME_PROP); - + final String cnn = message.getStr(ZkStateReader.CORE_NODE_NAME_PROP); final String collection = message.getStr(ZkStateReader.COLLECTION_PROP); - checkCollection(message, collection); + if (!checkCollectionKeyExistence(message)) return clusterState; // final Map newCollections = new LinkedHashMap(clusterState.getCollectionStates()); // shallow copy // DocCollection coll = newCollections.get(collection); 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=1574580&r1=1574579&r2=1574580&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 Wed Mar 5 17:28:31 2014 @@ -1013,7 +1013,8 @@ public final class ZkController { core.close(); } } - log.info("publishing core={} state={}", cd.getName(), state); + String collection = cd.getCloudDescriptor().getCollectionName(); + log.info("publishing core={} state={} collection={}", cd.getName(), state, collection); //System.out.println(Thread.currentThread().getStackTrace()[3]); Integer numShards = cd.getCloudDescriptor().getNumShards(); if (numShards == null) { //XXX sys prop hack @@ -1021,8 +1022,7 @@ public final class ZkController { numShards = Integer.getInteger(ZkStateReader.NUM_SHARDS_PROP); } - assert cd.getCloudDescriptor().getCollectionName() != null && cd.getCloudDescriptor() - .getCollectionName().length() > 0; + assert collection != null && collection.length() > 0; String coreNodeName = cd.getCloudDescriptor().getCoreNodeName(); //assert cd.getCloudDescriptor().getShardId() != null; @@ -1033,12 +1033,9 @@ public final class ZkController { ZkStateReader.ROLES_PROP, cd.getCloudDescriptor().getRoles(), ZkStateReader.NODE_NAME_PROP, getNodeName(), ZkStateReader.SHARD_ID_PROP, cd.getCloudDescriptor().getShardId(), - ZkStateReader.COLLECTION_PROP, cd.getCloudDescriptor() - .getCollectionName(), - ZkStateReader.NUM_SHARDS_PROP, numShards != null ? numShards.toString() - : null, - ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName != null ? coreNodeName - : null); + ZkStateReader.COLLECTION_PROP, collection, + ZkStateReader.NUM_SHARDS_PROP, numShards != null ? numShards.toString() : null, + ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName != null ? coreNodeName : null); if (updateLastState) { cd.getCloudDescriptor().lastPublished = state; } @@ -1368,7 +1365,6 @@ public final class ZkController { CloudDescriptor cloudDesc = cd.getCloudDescriptor(); - // make sure the node name is set on the descriptor if (cloudDesc.getCoreNodeName() == null) { cloudDesc.setCoreNodeName(coreNodeName);