You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2013/05/07 13:21:14 UTC

svn commit: r1479862 [27/38] - in /lucene/dev/branches/lucene4258: ./ dev-tools/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/maven/ dev-tools/maven/solr/ dev-tools/maven/solr/core/src/java/ dev-tools/maven/solr/solrj/src/java/ dev-t...

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java Tue May  7 11:20:55 2013
@@ -25,6 +25,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.client.solrj.request.CoreAdminRequest;
 import org.apache.solr.cloud.DistributedQueue.QueueEvent;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -32,6 +33,8 @@ import org.apache.solr.common.cloud.Alia
 import org.apache.solr.common.cloud.ClosableThread;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.DocRouter;
+import org.apache.solr.common.cloud.PlainIdRouter;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -70,6 +73,8 @@ public class OverseerCollectionProcessor
   
   public static final String DELETEALIAS = "deletealias";
   
+  public static final String SPLITSHARD = "splitshard";
+
   // TODO: use from Overseer?
   private static final String QUEUE_OPERATION = "operation";
   
@@ -162,7 +167,28 @@ public class OverseerCollectionProcessor
         ModifiableSolrParams params = new ModifiableSolrParams();
         params.set(CoreAdminParams.ACTION, CoreAdminAction.UNLOAD.toString());
         params.set(CoreAdminParams.DELETE_INSTANCE_DIR, true);
+        params.set(CoreAdminParams.DELETE_DATA_DIR, true);
         collectionCmd(zkStateReader.getClusterState(), message, params, results, null);
+        
+        ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION,
+            Overseer.REMOVECOLLECTION, "name", message.getStr("name"));
+          Overseer.getInQueue(zkStateReader.getZkClient()).offer(ZkStateReader.toJSON(m));
+          
+        // wait for a while until we don't see the collection
+        long now = System.currentTimeMillis();
+        long timeout = now + 30000;
+        boolean removed = false;
+        while (System.currentTimeMillis() < timeout) {
+          Thread.sleep(100);
+          removed = !zkStateReader.getClusterState().getCollections().contains(message.getStr("name"));
+          if (removed) {
+            Thread.sleep(100); // just a bit of time so it's more likely other readers see on return
+            break;
+          }
+        }
+        if (!removed) {
+          throw new SolrException(ErrorCode.SERVER_ERROR, "Could not fully remove collection: " + message.getStr("name"));
+        }
       } else if (RELOADCOLLECTION.equals(operation)) {
         ModifiableSolrParams params = new ModifiableSolrParams();
         params.set(CoreAdminParams.ACTION, CoreAdminAction.RELOAD.toString());
@@ -171,37 +197,24 @@ public class OverseerCollectionProcessor
         createAlias(zkStateReader.getAliases(), message);
       } else if (DELETEALIAS.equals(operation)) {
         deleteAlias(zkStateReader.getAliases(), message);
+      } else if (SPLITSHARD.equals(operation))  {
+        splitShard(zkStateReader.getClusterState(), message, results);
       } else {
         throw new SolrException(ErrorCode.BAD_REQUEST, "Unknown operation:"
             + operation);
       }
-      int failed = 0;
-      ShardResponse srsp;
-      
-      do {
-        srsp = shardHandler.takeCompletedIncludingErrors();
-        if (srsp != null) {
-          Throwable e = srsp.getException();
-          if (e != null) {
-            failed++;
-            log.error("Error talking to shard: " + srsp.getShard(), e);
-            results.add(srsp.getShard(), e);
-          } else {
-            results.add(srsp.getShard(), srsp.getSolrResponse().getResponse());
-          }
-        }
-      } while (srsp != null);
-    } catch (Exception ex) {
+
+    } catch (Throwable t) {
       SolrException.log(log, "Collection " + operation + " of " + operation
-          + " failed", ex);
-      results.add("Operation " + operation + " caused exception:", ex);
+          + " failed", t);
+      results.add("Operation " + operation + " caused exception:", t);
       SimpleOrderedMap nl = new SimpleOrderedMap();
-      nl.add("msg", ex.getMessage());
-      nl.add("rspCode", ex instanceof SolrException ? ((SolrException)ex).code() : -1);
+      nl.add("msg", t.getMessage());
+      nl.add("rspCode", t instanceof SolrException ? ((SolrException)t).code() : -1);
       results.add("exception", nl);
-    } finally {
-      return new OverseerSolrResponse(results);
-    }
+    } 
+    
+    return new OverseerSolrResponse(results);
   }
 
   private void createAlias(Aliases aliases, ZkNodeProps message) {
@@ -305,6 +318,266 @@ public class OverseerCollectionProcessor
     
   }
   
+  private boolean splitShard(ClusterState clusterState, ZkNodeProps message, NamedList results) {
+    log.info("Split shard invoked");
+    String collection = message.getStr("collection");
+    String slice = message.getStr(ZkStateReader.SHARD_ID_PROP);
+    Slice parentSlice = clusterState.getSlice(collection, slice);
+    
+    if (parentSlice == null) {
+      if(clusterState.getCollections().contains(collection)) {
+        throw new SolrException(ErrorCode.BAD_REQUEST, "No shard with the specified name exists: " + slice);
+      } else {
+        throw new SolrException(ErrorCode.BAD_REQUEST, "No collection with the specified name exists: " + collection);
+      }      
+    }
+    
+    // find the leader for the shard
+    Replica parentShardLeader = clusterState.getLeader(collection, slice);
+    
+    DocRouter.Range range = parentSlice.getRange();
+    if (range == null) {
+      range = new PlainIdRouter().fullRange();
+    }
+
+    // todo: fixed to two partitions?
+    // todo: accept the range as a param to api?
+    // todo: handle randomizing subshard name in case a shard with the same name already exists.
+    List<DocRouter.Range> subRanges = new PlainIdRouter().partitionRange(2, range);
+    try {
+      List<String> subSlices = new ArrayList<String>(subRanges.size());
+      List<String> subShardNames = new ArrayList<String>(subRanges.size());
+      String nodeName = parentShardLeader.getNodeName();
+      for (int i = 0; i < subRanges.size(); i++) {
+        String subSlice = slice + "_" + i;
+        subSlices.add(subSlice);
+        String subShardName = collection + "_" + subSlice + "_replica1";
+        subShardNames.add(subShardName);
+
+        Slice oSlice = clusterState.getSlice(collection, subSlice);
+        if (oSlice != null) {
+          if (Slice.ACTIVE.equals(oSlice.getState())) {
+            throw new SolrException(ErrorCode.BAD_REQUEST, "Sub-shard: " + subSlice + " exists in active state. Aborting split shard.");
+          } else if (Slice.CONSTRUCTION.equals(oSlice.getState()))  {
+            for (Replica replica : oSlice.getReplicas()) {
+              String core = replica.getStr("core");
+              log.info("Unloading core: " + core + " from node: " + replica.getNodeName());
+              ModifiableSolrParams params = new ModifiableSolrParams();
+              params.set(CoreAdminParams.ACTION, CoreAdminAction.UNLOAD.toString());
+              params.set(CoreAdminParams.CORE, core);
+              params.set(CoreAdminParams.DELETE_INDEX, "true");
+              sendShardRequest(replica.getNodeName(), params);
+            }
+          }
+        }
+      }
+
+      ShardResponse srsp;
+      do {
+        srsp = shardHandler.takeCompletedOrError();
+        if (srsp != null) {
+          processResponse(results, srsp);
+        }
+      } while (srsp != null);
+
+      for (int i=0; i<subRanges.size(); i++)  {
+        String subSlice = subSlices.get(i);
+        String subShardName = subShardNames.get(i);
+        DocRouter.Range subRange = subRanges.get(i);
+
+        log.info("Creating shard " + subShardName + " as part of slice "
+            + subSlice + " of collection " + collection + " on "
+            + nodeName);
+
+        ModifiableSolrParams params = new ModifiableSolrParams();
+        params.set(CoreAdminParams.ACTION, CoreAdminAction.CREATE.toString());
+
+        params.set(CoreAdminParams.NAME, subShardName);
+        params.set(CoreAdminParams.COLLECTION, collection);
+        params.set(CoreAdminParams.SHARD, subSlice);
+        params.set(CoreAdminParams.SHARD_RANGE, subRange.toString());
+        params.set(CoreAdminParams.SHARD_STATE, Slice.CONSTRUCTION);
+        //params.set(ZkStateReader.NUM_SHARDS_PROP, numSlices); todo: is it necessary, we're not creating collections?
+
+        sendShardRequest(nodeName, params);
+
+        // wait for parent leader to acknowledge the sub-shard core
+        log.info("Asking parent leader to wait for: " + subShardName + " to be alive on: " + nodeName);
+        CoreAdminRequest.WaitForState cmd = new CoreAdminRequest.WaitForState();
+        cmd.setCoreName(subShardName);
+        cmd.setNodeName(nodeName);
+        cmd.setCoreNodeName(nodeName + "_" + subShardName);
+        cmd.setState(ZkStateReader.ACTIVE);
+        cmd.setCheckLive(true);
+        cmd.setOnlyIfLeader(true);
+        sendShardRequest(nodeName, new ModifiableSolrParams(cmd.getParams()));
+      }
+
+      do {
+        srsp = shardHandler.takeCompletedOrError();
+        if (srsp != null) {
+          processResponse(results, srsp);
+        }
+      } while (srsp != null);
+      
+      log.info("Successfully created all sub-shards for collection "
+          + collection + " parent shard: " + slice + " on: " + parentShardLeader);
+
+      log.info("Splitting shard " + parentShardLeader.getName() + " as part of slice "
+          + slice + " of collection " + collection + " on "
+          + parentShardLeader);
+
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      params.set(CoreAdminParams.ACTION, CoreAdminAction.SPLIT.toString());
+      params.set(CoreAdminParams.CORE, parentShardLeader.getStr("core"));
+      for (int i = 0; i < subShardNames.size(); i++) {
+        String subShardName = subShardNames.get(i);
+        params.add(CoreAdminParams.TARGET_CORE, subShardName);
+      }
+
+      sendShardRequest(parentShardLeader.getNodeName(), params);
+      do {
+        srsp = shardHandler.takeCompletedOrError();
+        if (srsp != null) {
+          processResponse(results, srsp);
+        }
+      } while (srsp != null);
+
+      log.info("Index on shard: " + nodeName + " split into two successfully");
+
+      // apply buffered updates on sub-shards
+      for (int i = 0; i < subShardNames.size(); i++) {
+        String subShardName = subShardNames.get(i);
+
+        log.info("Applying buffered updates on : " + subShardName);
+
+        params = new ModifiableSolrParams();
+        params.set(CoreAdminParams.ACTION, CoreAdminAction.REQUESTAPPLYUPDATES.toString());
+        params.set(CoreAdminParams.NAME, subShardName);
+
+        sendShardRequest(nodeName, params);
+      }
+
+      do {
+        srsp = shardHandler.takeCompletedOrError();
+        if (srsp != null) {
+          processResponse(results, srsp);
+        }
+      } while (srsp != null);
+
+      log.info("Successfully applied buffered updates on : " + subShardNames);
+
+      // Replica creation for the new Slices
+
+      // look at the replication factor and see if it matches reality
+      // if it does not, find best nodes to create more cores
+
+      // TODO: Have replication factor decided in some other way instead of numShards for the parent
+
+      int repFactor = clusterState.getSlice(collection, slice).getReplicas().size();
+
+      // we need to look at every node and see how many cores it serves
+      // add our new cores to existing nodes serving the least number of cores
+      // but (for now) require that each core goes on a distinct node.
+
+      // TODO: add smarter options that look at the current number of cores per
+      // node?
+      // for now we just go random
+      Set<String> nodes = clusterState.getLiveNodes();
+      List<String> nodeList = new ArrayList<String>(nodes.size());
+      nodeList.addAll(nodes);
+      
+      Collections.shuffle(nodeList);
+
+      // TODO: Have maxShardsPerNode param for this operation?
+
+      // Remove the node that hosts the parent shard for replica creation.
+      nodeList.remove(nodeName);
+      
+      // TODO: change this to handle sharding a slice into > 2 sub-shards.
+
+      for (int i = 1; i <= subSlices.size(); i++) {
+        Collections.shuffle(nodeList);
+        String sliceName = subSlices.get(i - 1);
+        for (int j = 2; j <= repFactor; j++) {
+          String subShardNodeName = nodeList.get((repFactor * (i - 1) + (j - 2)) % nodeList.size());
+          String shardName = collection + "_" + sliceName + "_replica" + (j);
+
+          log.info("Creating replica shard " + shardName + " as part of slice "
+              + sliceName + " of collection " + collection + " on "
+              + subShardNodeName);
+
+          // Need to create new params for each request
+          params = new ModifiableSolrParams();
+          params.set(CoreAdminParams.ACTION, CoreAdminAction.CREATE.toString());
+
+          params.set(CoreAdminParams.NAME, shardName);
+          params.set(CoreAdminParams.COLLECTION, collection);
+          params.set(CoreAdminParams.SHARD, sliceName);
+          // TODO:  Figure the config used by the parent shard and use it.
+          //params.set("collection.configName", configName);
+          
+          //Not using this property. Do we really need to use it?
+          //params.set(ZkStateReader.NUM_SHARDS_PROP, numSlices);
+
+          sendShardRequest(subShardNodeName, params);
+
+          // wait for the replicas to be seen as active on sub shard leader
+          log.info("Asking sub shard leader to wait for: " + shardName + " to be alive on: " + subShardNodeName);
+          CoreAdminRequest.WaitForState cmd = new CoreAdminRequest.WaitForState();
+          cmd.setCoreName(subShardNames.get(i-1));
+          cmd.setNodeName(subShardNodeName);
+          cmd.setCoreNodeName(subShardNodeName + "_" + shardName);
+          cmd.setState(ZkStateReader.ACTIVE);
+          cmd.setCheckLive(true);
+          cmd.setOnlyIfLeader(true);
+          sendShardRequest(nodeName, new ModifiableSolrParams(cmd.getParams()));
+        }
+      }
+
+      do {
+        srsp = shardHandler.takeCompletedOrError();
+        if (srsp != null) {
+          processResponse(results, srsp);
+        }
+      } while (srsp != null);
+      log.info("Successfully created all replica shards for all sub-slices "
+          + subSlices);
+
+      log.info("Requesting update shard state");
+      DistributedQueue inQueue = Overseer.getInQueue(zkStateReader.getZkClient());
+      Map<String, Object> propMap = new HashMap<String, Object>();
+      propMap.put(Overseer.QUEUE_OPERATION, "updateshardstate");
+      propMap.put(slice, Slice.INACTIVE);
+      for (String subSlice : subSlices) {
+        propMap.put(subSlice, Slice.ACTIVE);
+      }
+      propMap.put(ZkStateReader.COLLECTION_PROP, collection);
+      ZkNodeProps m = new ZkNodeProps(propMap);
+      inQueue.offer(ZkStateReader.toJSON(m));
+
+      return true;
+    } catch (SolrException e) {
+      throw e;
+    } catch (Exception e) {
+      log.error("Error executing split operation for collection: " + collection + " parent shard: " + slice, e);
+      throw new SolrException(ErrorCode.SERVER_ERROR, null, e);
+    }
+  }
+
+  private void sendShardRequest(String nodeName, ModifiableSolrParams params) {
+    ShardRequest sreq = new ShardRequest();
+    params.set("qt", adminPath);
+    sreq.purpose = 1;
+    String replica = zkStateReader.getZkClient().getBaseUrlForNodeName(nodeName);
+    if (replica.startsWith("http://")) replica = replica.substring(7);
+    sreq.shards = new String[]{replica};
+    sreq.actualShards = sreq.shards;
+    sreq.params = params;
+
+    shardHandler.submit(sreq, replica, sreq.params);
+  }
+  
   private void createCollection(ClusterState clusterState, ZkNodeProps message, NamedList results) {
     String collectionName = message.getStr("name");
     if (clusterState.getCollections().contains(collectionName)) {
@@ -319,7 +592,7 @@ public class OverseerCollectionProcessor
       Integer numSlices = msgStrToInt(message, NUM_SLICES, null);
       
       if (numSlices == null) {
-        throw new SolrException(ErrorCode.BAD_REQUEST, "collection already exists: " + collectionName);
+        throw new SolrException(ErrorCode.BAD_REQUEST, NUM_SLICES + " is a required param");
       }
       
       int maxShardsPerNode = msgStrToInt(message, MAX_SHARDS_PER_NODE, 1);
@@ -327,7 +600,7 @@ public class OverseerCollectionProcessor
       List<String> createNodeList = ((createNodeSetStr = message.getStr(CREATE_NODE_SET)) == null)?null:StrUtils.splitSmart(createNodeSetStr, ",", true);
       
       if (repFactor <= 0) {
-        throw new SolrException(ErrorCode.BAD_REQUEST, NUM_SLICES + " is a required paramater");
+        throw new SolrException(ErrorCode.BAD_REQUEST, REPLICATION_FACTOR + " must be greater than or equal to 0");
       }
       
       if (numSlices <= 0) {
@@ -399,7 +672,6 @@ public class OverseerCollectionProcessor
           params.set(ZkStateReader.NUM_SHARDS_PROP, numSlices);
           
           ShardRequest sreq = new ShardRequest();
-          sreq.nodeName = nodeName;
           params.set("qt", adminPath);
           sreq.purpose = 1;
           String replica = zkStateReader.getZkClient()
@@ -509,7 +781,7 @@ public class OverseerCollectionProcessor
     }
   }
   
-  private int msgStrToInt(ZkNodeProps message, String key, Integer def)
+  private Integer msgStrToInt(ZkNodeProps message, String key, Integer def)
       throws Exception {
     String str = message.getStr(key);
     try {

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/cloud/ZkCLI.java Tue May  7 11:20:55 2013
@@ -20,11 +20,13 @@ import org.apache.commons.cli.PosixParse
 import org.apache.commons.io.IOUtils;
 import org.apache.solr.common.cloud.OnReconnect;
 import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.core.Config;
 import org.apache.solr.core.ConfigSolr;
-import org.apache.solr.core.ConfigSolrXmlBackCompat;
-import org.apache.solr.core.SolrProperties;
+import org.apache.solr.core.ConfigSolrXml;
+import org.apache.solr.core.ConfigSolrXmlOld;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.zookeeper.KeeperException;
+import org.xml.sax.InputSource;
 import org.xml.sax.SAXException;
 
 /*
@@ -177,21 +179,20 @@ public class ZkCLI {
           solrHome = loader.getInstanceDir();
 
           File configFile = new File(solrHome, SOLR_XML);
-          boolean isXml = true;
-          if (! configFile.exists()) {
-            configFile = new File(solrHome, SolrProperties.SOLR_PROPERTIES_FILE);
-            isXml = false;
-          }
           InputStream is = new FileInputStream(configFile);
 
           ConfigSolr cfg;
 
           try {
-            if (isXml) {
-              cfg = new ConfigSolrXmlBackCompat(loader, null, is, null, false);
-            } else {
-              cfg = new SolrProperties(null, loader, is, null);
-            }
+            Config config = new Config(loader, null, new InputSource(is), null, false);
+            
+            boolean oldStyle = (config.getNode("solr/cores", false) != null);
+
+             if (oldStyle) {
+               cfg = new ConfigSolrXmlOld(config, null);
+             } else {
+               cfg = new ConfigSolrXml(config, null);
+             }
           } finally {
             IOUtils.closeQuietly(is);
           }

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/cloud/ZkController.java Tue May  7 11:20:55 2013
@@ -405,7 +405,7 @@ public final class ZkController {
   // input can be null, host, or url_prefix://host
   private String getHostAddress(String host) throws IOException {
 
-    if (host == null) {
+    if (host == null || host.length() == 0) {
       String hostaddress;
       try {
         hostaddress = InetAddress.getLocalHost().getHostAddress();
@@ -792,21 +792,27 @@ public final class ZkController {
 
       UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
       if (!core.isReloaded() && ulog != null) {
-        Future<UpdateLog.RecoveryInfo> recoveryFuture = core.getUpdateHandler()
-            .getUpdateLog().recoverFromLog();
-        if (recoveryFuture != null) {
-          recoveryFuture.get(); // NOTE: this could potentially block for
-          // minutes or more!
-          // TODO: public as recovering in the mean time?
-          // TODO: in the future we could do peersync in parallel with recoverFromLog
+        // disable recovery in case shard is in construction state (for shard splits)
+        Slice slice = getClusterState().getSlice(collection, shardId);
+        if (Slice.CONSTRUCTION.equals(slice.getState())) {
+          publish(desc, ZkStateReader.ACTIVE);
         } else {
-          log.info("No LogReplay needed for core="+core.getName() + " baseURL=" + baseUrl);
+          Future<UpdateLog.RecoveryInfo> recoveryFuture = core.getUpdateHandler()
+              .getUpdateLog().recoverFromLog();
+          if (recoveryFuture != null) {
+            recoveryFuture.get(); // NOTE: this could potentially block for
+            // minutes or more!
+            // TODO: public as recovering in the mean time?
+            // TODO: in the future we could do peersync in parallel with recoverFromLog
+          } else {
+            log.info("No LogReplay needed for core=" + core.getName() + " baseURL=" + baseUrl);
+          }
+          boolean didRecovery = checkRecovery(coreName, desc, recoverReloadedCores, isLeader, cloudDesc,
+              collection, coreZkNodeName, shardId, leaderProps, core, cc);
+          if (!didRecovery) {
+            publish(desc, ZkStateReader.ACTIVE);
+          }
         }
-      }      
-      boolean didRecovery = checkRecovery(coreName, desc, recoverReloadedCores, isLeader, cloudDesc,
-          collection, coreZkNodeName, shardId, leaderProps, core, cc);
-      if (!didRecovery) {
-        publish(desc, ZkStateReader.ACTIVE);
       }
     } finally {
       if (core != null) {
@@ -817,7 +823,6 @@ public final class ZkController {
     
     // make sure we have an update cluster state right away
     zkStateReader.updateClusterState(true);
-
     return shardId;
   }
 
@@ -993,6 +998,8 @@ public final class ZkController {
         ZkStateReader.ROLES_PROP, cd.getCloudDescriptor().getRoles(),
         ZkStateReader.NODE_NAME_PROP, getNodeName(),
         ZkStateReader.SHARD_ID_PROP, cd.getCloudDescriptor().getShardId(),
+        ZkStateReader.SHARD_RANGE_PROP, cd.getCloudDescriptor().getShardRange(),
+        ZkStateReader.SHARD_STATE_PROP, cd.getCloudDescriptor().getShardState(),
         ZkStateReader.COLLECTION_PROP, cd.getCloudDescriptor()
             .getCollectionName(),
         ZkStateReader.NUM_SHARDS_PROP, numShards != null ? numShards.toString()
@@ -1270,10 +1277,20 @@ public final class ZkController {
     downloadFromZK(zkClient, ZkController.CONFIGS_ZKNODE + "/" + configName, dir);
   }
 
-  public void preRegister(CoreDescriptor cd) throws KeeperException, InterruptedException {
+  public void preRegister(SolrCore core) throws KeeperException, InterruptedException {
+    CoreDescriptor cd = core.getCoreDescriptor();
+    if (Slice.CONSTRUCTION.equals(cd.getCloudDescriptor().getShardState())) {
+      // set update log to buffer before publishing the core
+      core.getUpdateHandler().getUpdateLog().bufferUpdates();
+    }
     // before becoming available, make sure we are not live and active
     // this also gets us our assigned shard id if it was not specified
     publish(cd, ZkStateReader.DOWN, false);
+    // shardState and shardRange are for one-time use only, thereafter the actual values in the Slice should be used
+    if (Slice.CONSTRUCTION.equals(cd.getCloudDescriptor().getShardState())) {
+      cd.getCloudDescriptor().setShardState(null);
+      cd.getCloudDescriptor().setShardRange(null);
+    }
     String coreNodeName = getCoreNodeName(cd);
     
     // make sure the node name is set on the descriptor
@@ -1424,12 +1441,15 @@ public final class ZkController {
   public static void bootstrapConf(SolrZkClient zkClient, ConfigSolr cfg, String solrHome) throws IOException,
       KeeperException, InterruptedException {
 
-    log.info("bootstraping config into ZooKeeper using solr.xml");
     List<String> allCoreNames = cfg.getAllCoreNames();
+    
+    log.info("bootstraping config for " + allCoreNames.size() + " cores into ZooKeeper using solr.xml from " + solrHome);
+
     for (String coreName : allCoreNames) {
       String rawName = PropertiesUtil.substituteProperty(cfg.getProperty(coreName, "name", null), new Properties());
       String instanceDir = cfg.getProperty(coreName, "instanceDir", null);
       File idir = new File(instanceDir);
+      System.out.println("idir:" + idir);
       if (!idir.isAbsolute()) {
         idir = new File(solrHome, instanceDir);
       }

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java Tue May  7 11:20:55 2013
@@ -125,4 +125,7 @@ public class ZkSolrResourceLoader extend
     return collectionZkPath;
   }
   
+  public ZkController getZkController() {
+    return zkController;
+  }
 }

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/AbstractSolrEventListener.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/AbstractSolrEventListener.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/AbstractSolrEventListener.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/AbstractSolrEventListener.java Tue May  7 11:20:55 2013
@@ -24,15 +24,18 @@ import org.apache.solr.search.SolrIndexS
 /**
  */
 public class AbstractSolrEventListener implements SolrEventListener {
-  protected final SolrCore core;
+  private final SolrCore core;
+  public SolrCore getCore() { return core; }
+
   public AbstractSolrEventListener(SolrCore core) {
     this.core = core;
   }
-  protected NamedList args;
+  private NamedList args;
+  public NamedList getArgs() { return args; }
 
   @Override
   public void init(NamedList args) {
-    this.args = args;
+    this.args = args.clone();
   }
 
   @Override

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java Tue May  7 11:20:55 2013
@@ -29,6 +29,7 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext.Context;
 import org.apache.lucene.store.NativeFSLockFactory;
@@ -37,6 +38,7 @@ import org.apache.lucene.store.RateLimit
 import org.apache.lucene.store.SimpleFSLockFactory;
 import org.apache.lucene.store.SingleInstanceLockFactory;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.util.NamedList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -53,7 +55,8 @@ public abstract class CachingDirectoryFa
   protected class CacheValue {
     final public String path;
     final public Directory directory;
-    
+    // for debug
+    //final Exception originTrace;
     // use the setter!
     private boolean deleteOnClose = false;
     
@@ -61,14 +64,12 @@ public abstract class CachingDirectoryFa
       this.path = path;
       this.directory = directory;
       this.closeEntries.add(this);
+      // for debug
+      // this.originTrace = new RuntimeException("Originated from:");
     }
     public int refCnt = 1;
-    // if we are latestForPath, I'm currently using my path
-    // otherwise a new Directory instance is using my path
-    // and I must be manipulated by Directory
-    public boolean latestForPath = false;
-    // has close(Directory) been called on this?
-    public boolean closeDirectoryCalled = false;
+    // has doneWithDirectory(Directory) been called on this?
+    public boolean closeCacheValueCalled = false;
     public boolean doneWithDir = false;
     private boolean deleteAfterCoreClose = false;
     public Set<CacheValue> removeEntries = new HashSet<CacheValue>();
@@ -84,7 +85,7 @@ public abstract class CachingDirectoryFa
     
     @Override
     public String toString() {
-      return "CachedDir<<" + directory.toString() + ";refCount=" + refCnt + ";path=" + path + ";done=" + doneWithDir + ">>";
+      return "CachedDir<<" + "refCount=" + refCnt + ";path=" + path + ";done=" + doneWithDir + ">>";
     }
   }
   
@@ -142,9 +143,12 @@ public abstract class CachingDirectoryFa
             + " " + byDirectoryCache);
       }
       cacheValue.doneWithDir = true;
-      if (cacheValue.refCnt == 0) {
-        cacheValue.refCnt++; // this will go back to 0 in close
-        close(directory);
+      log.debug("Done with dir: {}", cacheValue);
+      if (cacheValue.refCnt == 0 && !closed) {
+        boolean cl = closeCacheValue(cacheValue);
+        if (cl) {
+          removeFromCache(cacheValue);
+        }
       }
     }
   }
@@ -157,19 +161,24 @@ public abstract class CachingDirectoryFa
   @Override
   public void close() throws IOException {
     synchronized (this) {
+      log.info("Closing " + this.getClass().getSimpleName() + " - " + byDirectoryCache.size() + " directories currently being tracked");
       this.closed = true;
-      Collection<CacheValue> values = new ArrayList<CacheValue>();
-      values.addAll(byDirectoryCache.values());
+      Collection<CacheValue> values = byDirectoryCache.values();
       for (CacheValue val : values) {
+        log.debug("Closing {} - currently tracking: {}", 
+                  this.getClass().getSimpleName(), val);
         try {
           // if there are still refs out, we have to wait for them
           int cnt = 0;
           while(val.refCnt != 0) {
             wait(100);
             
-            if (cnt++ >= 1200) {
-              log.error("Timeout waiting for all directory ref counts to be released");
-              break;
+            if (cnt++ >= 120) {
+              String msg = "Timeout waiting for all directory ref counts to be released - gave up waiting on " + val;
+              log.error(msg);
+              // debug
+              // val.originTrace.printStackTrace();
+              throw new SolrException(ErrorCode.SERVER_ERROR, msg);
             }
           }
           assert val.refCnt == 0 : val.refCnt;
@@ -179,56 +188,47 @@ public abstract class CachingDirectoryFa
       }
       
       values = byDirectoryCache.values();
+      Set<CacheValue> closedDirs = new HashSet<CacheValue>();
       for (CacheValue val : values) {
         try {
-          assert val.refCnt == 0 : val.refCnt;
-          log.info("Closing directory when closing factory: " + val.path);
-          closeDirectory(val);
+          for (CacheValue v : val.closeEntries) {
+            assert v.refCnt == 0 : val.refCnt;
+            log.debug("Closing directory when closing factory: " + v.path);
+            boolean cl = closeCacheValue(v);
+            if (cl) {
+              closedDirs.add(v);
+            }
+          }
         } catch (Throwable t) {
           SolrException.log(log, "Error closing directory", t);
         }
       }
-      
-      byDirectoryCache.clear();
-      byPathCache.clear();
-      
+
       for (CacheValue val : removeEntries) {
-        log.info("Removing directory: " + val.path);
-        removeDirectory(val);
+        log.info("Removing directory after core close: " + val.path);
+        try {
+          removeDirectory(val);
+        } catch (Throwable t) {
+          SolrException.log(log, "Error removing directory", t);
+        }
       }
-    }
-  }
-  
-  private void close(Directory directory) throws IOException {
-    synchronized (this) {
-      // don't check if already closed here - we need to able to release
-      // while #close() waits.
       
-      CacheValue cacheValue = byDirectoryCache.get(directory);
-      if (cacheValue == null) {
-        throw new IllegalArgumentException("Unknown directory: " + directory
-            + " " + byDirectoryCache);
-      }
-      log.debug("Releasing directory: " + cacheValue.path);
-
-      cacheValue.refCnt--;
-
-      if (cacheValue.refCnt == 0 && cacheValue.doneWithDir) {
-        closeDirectory(cacheValue);
-        
-        byDirectoryCache.remove(directory);
-        
-        // if it's been closed, it's path is now
-        // owned by another Directory instance
-        if (!cacheValue.latestForPath) {
-          byPathCache.remove(cacheValue.path);
-          cacheValue.latestForPath = true;
-        }
+      for (CacheValue v : closedDirs) {
+        removeFromCache(v);
       }
     }
   }
 
-  private void closeDirectory(CacheValue cacheValue) {
+  private void removeFromCache(CacheValue v) {
+    log.debug("Removing from cache: {}", v);
+    byDirectoryCache.remove(v.directory);
+    byPathCache.remove(v.path);
+  }
+
+  // be sure this is called with the this sync lock
+  // returns true if we closed the cacheValue, false if it will be closed later
+  private boolean closeCacheValue(CacheValue cacheValue) {
+    log.info("looking to close " + cacheValue.path + " " + cacheValue.closeEntries.toString());
     List<CloseListener> listeners = closeListeners.remove(cacheValue.directory);
     if (listeners != null) {
       for (CloseListener listener : listeners) {
@@ -239,21 +239,16 @@ public abstract class CachingDirectoryFa
         }
       }
     }
-    
-    cacheValue.closeDirectoryCalled = true;
-    
+    cacheValue.closeCacheValueCalled = true;
     if (cacheValue.deleteOnClose) {
-      
       // see if we are a subpath
       Collection<CacheValue> values = byPathCache.values();
       
-      Collection<CacheValue> cacheValues = new ArrayList<CacheValue>();
-      cacheValues.addAll(values);
+      Collection<CacheValue> cacheValues = new ArrayList<CacheValue>(values);
       cacheValues.remove(cacheValue);
       for (CacheValue otherCacheValue : cacheValues) {
-        // if we are a parent path and all our sub children are not already closed,
-        // get a sub path to close us later
-        if (otherCacheValue.path.startsWith(cacheValue.path) && !otherCacheValue.closeDirectoryCalled) {
+        // if we are a parent path and a sub path is not already closed, get a sub path to close us later
+        if (isSubPath(cacheValue, otherCacheValue) && !otherCacheValue.closeCacheValueCalled) {
           // we let the sub dir remove and close us
           if (!otherCacheValue.deleteAfterCoreClose && cacheValue.deleteAfterCoreClose) {
             otherCacheValue.deleteAfterCoreClose = true;
@@ -261,15 +256,24 @@ public abstract class CachingDirectoryFa
           otherCacheValue.removeEntries.addAll(cacheValue.removeEntries);
           otherCacheValue.closeEntries.addAll(cacheValue.closeEntries);
           cacheValue.closeEntries.clear();
-          break;
+          cacheValue.removeEntries.clear();
+          return false;
         }
       }
     }
-    
+
+    boolean cl = false;
+    for (CacheValue val : cacheValue.closeEntries) {
+      close(val);
+      if (val == cacheValue) {
+        cl = true;
+      }
+    }
+
     for (CacheValue val : cacheValue.removeEntries) {
       if (!val.deleteAfterCoreClose) {
+        log.info("Removing directory before core close: " + val.path);
         try {
-          log.info("Removing directory: " + val.path);
           removeDirectory(val);
         } catch (Throwable t) {
           SolrException.log(log, "Error removing directory", t);
@@ -279,16 +283,6 @@ public abstract class CachingDirectoryFa
       }
     }
     
-    for (CacheValue val : cacheValue.closeEntries) {
-      try {
-        log.info("Closing directory: " + val.path);
-        val.directory.close();
-      } catch (Throwable t) {
-        SolrException.log(log, "Error closing directory", t);
-      }
-      
-    }
-
     if (listeners != null) {
       for (CloseListener listener : listeners) {
         try {
@@ -298,6 +292,23 @@ public abstract class CachingDirectoryFa
         }
       }
     }
+    return cl;
+  }
+
+  private void close(CacheValue val) {
+    try {
+      log.info("Closing directory: " + val.path);
+      val.directory.close();
+    } catch (Throwable t) {
+      SolrException.log(log, "Error closing directory", t);
+    }
+  }
+
+  private boolean isSubPath(CacheValue cacheValue, CacheValue otherCacheValue) {
+    int one = cacheValue.path.lastIndexOf('/');
+    int two = otherCacheValue.path.lastIndexOf('/');
+    
+    return otherCacheValue.path.startsWith(cacheValue.path + "/") && two > one;
   }
 
   @Override
@@ -314,51 +325,24 @@ public abstract class CachingDirectoryFa
    * (non-Javadoc)
    * 
    * @see org.apache.solr.core.DirectoryFactory#get(java.lang.String,
-   * java.lang.String)
-   */
-  @Override
-  public final Directory get(String path,  DirContext dirContext, String rawLockType)
-      throws IOException {
-    return get(path, dirContext, rawLockType, false);
-  }
-  
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.solr.core.DirectoryFactory#get(java.lang.String,
    * java.lang.String, boolean)
    */
   @Override
-  public final Directory get(String path,  DirContext dirContext, String rawLockType, boolean forceNew)
+  public final Directory get(String path,  DirContext dirContext, String rawLockType)
       throws IOException {
     String fullPath = normalize(path);
     synchronized (this) {
       if (closed) {
-        throw new RuntimeException("Already closed");
+        throw new AlreadyClosedException("Already closed");
       }
       
       final CacheValue cacheValue = byPathCache.get(fullPath);
       Directory directory = null;
       if (cacheValue != null) {
         directory = cacheValue.directory;
-        if (forceNew) {
-          cacheValue.doneWithDir = true;
-          
-          // we make a quick close attempt,
-          // otherwise this should be closed
-          // when whatever is using it, releases it
-          if (cacheValue.refCnt == 0) {
-            closeDirectory(cacheValue);
-          }
-          
-          // close the entry, it will be owned by the new dir
-          // we count on it being released by directory
-          cacheValue.latestForPath = true;
-          
-        }
       }
       
-      if (directory == null || forceNew) { 
+      if (directory == null) { 
         directory = create(fullPath, dirContext);
         
         directory = rateLimit(directory);
@@ -369,9 +353,10 @@ public abstract class CachingDirectoryFa
         
         byDirectoryCache.put(directory, newCacheValue);
         byPathCache.put(fullPath, newCacheValue);
-        log.info("return new directory for " + fullPath + " forceNew: " + forceNew);
+        log.info("return new directory for " + fullPath);
       } else {
         cacheValue.refCnt++;
+        log.debug("Reusing cached directory: {}", cacheValue);
       }
       
       return directory;
@@ -407,12 +392,16 @@ public abstract class CachingDirectoryFa
   @Override
   public void incRef(Directory directory) {
     synchronized (this) {
+      if (closed) {
+        throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "Already closed");
+      }
       CacheValue cacheValue = byDirectoryCache.get(directory);
       if (cacheValue == null) {
         throw new IllegalArgumentException("Unknown directory: " + directory);
       }
       
       cacheValue.refCnt++;
+      log.debug("incRef'ed: {}", cacheValue);
     }
   }
   
@@ -436,7 +425,28 @@ public abstract class CachingDirectoryFa
     if (directory == null) {
       throw new NullPointerException();
     }
-    close(directory);
+    synchronized (this) {
+      // don't check if already closed here - we need to able to release
+      // while #close() waits.
+      
+      CacheValue cacheValue = byDirectoryCache.get(directory);
+      if (cacheValue == null) {
+        throw new IllegalArgumentException("Unknown directory: " + directory
+            + " " + byDirectoryCache);
+      }
+      log.debug("Releasing directory: " + cacheValue.path + " " + (cacheValue.refCnt - 1) + " " + cacheValue.doneWithDir);
+
+      cacheValue.refCnt--;
+      
+      assert cacheValue.refCnt >= 0 : cacheValue.refCnt;
+
+      if (cacheValue.refCnt == 0 && cacheValue.doneWithDir && !closed) {
+        boolean cl = closeCacheValue(cacheValue);
+        if (cl) {
+          removeFromCache(cacheValue);
+        }
+      }
+    }
   }
   
   @Override
@@ -499,8 +509,8 @@ public abstract class CachingDirectoryFa
     return dir;
   }
   
-  protected void removeDirectory(CacheValue cacheValue) throws IOException {
-    empty(cacheValue.directory);
+  protected synchronized void removeDirectory(CacheValue cacheValue) throws IOException {
+     // this page intentionally left blank
   }
   
   @Override
@@ -515,4 +525,21 @@ public abstract class CachingDirectoryFa
     }
     return path;
   }
+  
+  /**
+   * Test only method for inspecting the cache
+   * @return paths in the cache which have not been marked "done"
+   *
+   * @see #doneWithDirectory
+   * @lucene.internal
+   */
+  public synchronized Set<String> getLivePaths() {
+    HashSet<String> livePaths = new HashSet<String>();
+    for (CacheValue val : byPathCache.values()) {
+      if (!val.doneWithDir) {
+        livePaths.add(val.path);
+      }
+    }
+    return livePaths;
+  }
 }

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/Config.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/Config.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/Config.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/Config.java Tue May  7 11:20:55 2013
@@ -25,7 +25,10 @@ import org.apache.solr.common.util.XMLEr
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.NamedNodeMap;
 import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
 import org.xml.sax.InputSource;
 import org.xml.sax.SAXException;
 import org.apache.commons.io.IOUtils;
@@ -34,13 +37,25 @@ import javax.xml.namespace.QName;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.Transformer;
 import javax.xml.transform.TransformerException;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMResult;
+import javax.xml.transform.dom.DOMSource;
 import javax.xml.xpath.XPath;
 import javax.xml.xpath.XPathConstants;
 import javax.xml.xpath.XPathExpressionException;
 import javax.xml.xpath.XPathFactory;
+
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
@@ -65,18 +80,6 @@ public class Config {
     this( loader, name, null, null );
   }
 
-  /**
-   * For the transition from using solr.xml to solr.properties, see SOLR-4196. Remove
-   * for 5.0, thus it's already deprecated
-   * @param loader - Solr resource loader
-   * @param cfg    - SolrConfig, for backwards compatability with solr.xml layer.
-   * @throws TransformerException if the XML file is mal-formed
-   */
-  @Deprecated
-  public Config(SolrResourceLoader loader, Config cfg) throws TransformerException {
-    this(loader, null, ConfigSolrXmlBackCompat.copyDoc(cfg.getDocument()));
-  }
-
   public Config(SolrResourceLoader loader, String name, InputSource is, String prefix) throws ParserConfigurationException, IOException, SAXException 
   {
     this(loader, name, is, prefix, true);
@@ -154,6 +157,16 @@ public class Config {
     this.loader = loader;
   }
 
+  
+  private static Document copyDoc(Document doc) throws TransformerException {
+    TransformerFactory tfactory = TransformerFactory.newInstance();
+    Transformer tx = tfactory.newTransformer();
+    DOMSource source = new DOMSource(doc);
+    DOMResult result = new DOMResult();
+    tx.transform(source, result);
+    return (Document) result.getNode();
+  }
+  
   /**
    * @since solr 1.3
    */
@@ -235,6 +248,98 @@ public class Config {
     }
   }
 
+  public NodeList getNodeList(String path, boolean errIfMissing) {
+    XPath xpath = xpathFactory.newXPath();
+    String xstr = normalize(path);
+
+    try {
+      NodeList nodeList = (NodeList)xpath.evaluate(xstr, doc, XPathConstants.NODESET);
+
+      if (null == nodeList) {
+        if (errIfMissing) {
+          throw new RuntimeException(name + " missing "+path);
+        } else {
+          log.debug(name + " missing optional " + path);
+          return null;
+        }
+      }
+
+      log.trace(name + ":" + path + "=" + nodeList);
+      return nodeList;
+
+    } catch (XPathExpressionException e) {
+      SolrException.log(log,"Error in xpath",e);
+      throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,"Error in xpath:" + xstr + " for " + name,e);
+    } catch (SolrException e) {
+      throw(e);
+    } catch (Throwable e) {
+      SolrException.log(log,"Error in xpath",e);
+      throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,"Error in xpath:" + xstr+ " for " + name,e);
+    }
+  }
+
+  /**
+   * Returns the set of attributes on the given element that are not among the given knownAttributes,
+   * or null if all attributes are known.
+   */
+  public Set<String> getUnknownAttributes(Element element, String... knownAttributes) {
+    Set<String> knownAttributeSet = new HashSet<String>(Arrays.asList(knownAttributes));
+    Set<String> unknownAttributeSet = null;
+    NamedNodeMap attributes = element.getAttributes();
+    for (int i = 0 ; i < attributes.getLength() ; ++i) {
+      final String attributeName = attributes.item(i).getNodeName();
+      if ( ! knownAttributeSet.contains(attributeName)) {
+        if (null == unknownAttributeSet) {
+          unknownAttributeSet = new HashSet<String>();
+        }
+        unknownAttributeSet.add(attributeName);
+      }
+    }
+    return unknownAttributeSet;
+  }
+
+  /**
+   * Logs an error and throws an exception if any of the element(s) at the given elementXpath
+   * contains an attribute name that is not among knownAttributes. 
+   */
+  public void complainAboutUnknownAttributes(String elementXpath, String... knownAttributes) {
+    SortedMap<String,SortedSet<String>> problems = new TreeMap<String,SortedSet<String>>(); 
+    NodeList nodeList = getNodeList(elementXpath, false);
+    for (int i = 0 ; i < nodeList.getLength() ; ++i) {
+      Element element = (Element)nodeList.item(i);
+      Set<String> unknownAttributes = getUnknownAttributes(element, knownAttributes);
+      if (null != unknownAttributes) {
+        String elementName = element.getNodeName();
+        SortedSet<String> allUnknownAttributes = problems.get(elementName);
+        if (null == allUnknownAttributes) {
+          allUnknownAttributes = new TreeSet<String>();
+          problems.put(elementName, allUnknownAttributes);
+        }
+        allUnknownAttributes.addAll(unknownAttributes);
+      }
+    }
+    if (problems.size() > 0) {
+      StringBuilder message = new StringBuilder();
+      for (Map.Entry<String,SortedSet<String>> entry : problems.entrySet()) {
+        if (message.length() > 0) {
+          message.append(", ");
+        }
+        message.append('<');
+        message.append(entry.getKey());
+        for (String attributeName : entry.getValue()) {
+          message.append(' ');
+          message.append(attributeName);
+          message.append("=\"...\"");
+        }
+        message.append('>');
+      }
+      message.insert(0, "Unknown attribute(s) on element(s): ");
+      String msg = message.toString();
+      SolrException.log(log, msg);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
+    }
+  }
+
   public String getVal(String path, boolean errIfMissing) {
     Node nd = getNode(path,errIfMissing);
     if (nd==null) return null;

Modified: lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/ConfigSolr.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/ConfigSolr.java?rev=1479862&r1=1479861&r2=1479862&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/ConfigSolr.java (original)
+++ lucene/dev/branches/lucene4258/solr/core/src/java/org/apache/solr/core/ConfigSolr.java Tue May  7 11:20:55 2013
@@ -17,67 +17,127 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
-import org.apache.solr.cloud.ZkController;
-import org.apache.solr.handler.component.ShardHandlerFactory;
-import org.apache.solr.schema.IndexSchema;
-import org.apache.zookeeper.KeeperException;
-
-import java.io.File;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
-/**
- * ConfigSolr is a new interface  to aid us in obsoleting solr.xml and replacing it with solr.properties. The problem here
- * is that the Config class is used for _all_ the xml file, e.g. solrconfig.xml and we can't mess with _that_ as part
- * of this issue. Primarily used in CoreContainer at present.
- * <p/>
- * This is already deprecated, it's only intended to exist for while transitioning to properties-based replacement for
- * solr.xml
- *
- * @since solr 4.2
- */
-@Deprecated
-public interface ConfigSolr {
-
-  public static enum ConfLevel {
-    SOLR, SOLR_CORES, SOLR_CORES_CORE, SOLR_LOGGING, SOLR_LOGGING_WATCHER
+import javax.xml.xpath.XPath;
+import javax.xml.xpath.XPathConstants;
+import javax.xml.xpath.XPathExpressionException;
+
+import org.apache.solr.common.SolrException;
+import org.apache.solr.util.DOMUtil;
+import org.apache.solr.util.PropertiesUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+
+public abstract class ConfigSolr {
+  protected static Logger log = LoggerFactory.getLogger(ConfigSolr.class);
+  
+  public final static String SOLR_XML_FILE = "solr.xml";
+  
+  // Ugly for now, but we'll at least be able to centralize all of the differences between 4x and 5x.
+  public static enum CfgProp {
+    SOLR_ADMINHANDLER,
+    SOLR_CORELOADTHREADS,
+    SOLR_COREROOTDIRECTORY,
+    SOLR_DISTRIBUPDATECONNTIMEOUT,
+    SOLR_DISTRIBUPDATESOTIMEOUT,
+    SOLR_HOST,
+    SOLR_HOSTCONTEXT,
+    SOLR_HOSTPORT,
+    SOLR_LEADERVOTEWAIT,
+    SOLR_LOGGING_CLASS,
+    SOLR_LOGGING_ENABLED,
+    SOLR_LOGGING_WATCHER_SIZE,
+    SOLR_LOGGING_WATCHER_THRESHOLD,
+    SOLR_MANAGEMENTPATH,
+    SOLR_SHAREDLIB,
+    SOLR_SHARDHANDLERFACTORY_CLASS,
+    SOLR_SHARDHANDLERFACTORY_CONNTIMEOUT,
+    SOLR_SHARDHANDLERFACTORY_NAME,
+    SOLR_SHARDHANDLERFACTORY_SOCKETTIMEOUT,
+    SOLR_SHARESCHEMA,
+    SOLR_TRANSIENTCACHESIZE,
+    SOLR_ZKCLIENTTIMEOUT,
+    SOLR_ZKHOST,
+
+    //TODO: Remove all of these elements for 5.0
+    SOLR_PERSISTENT,
+    SOLR_CORES_DEFAULT_CORE_NAME,
+    SOLR_ADMINPATH
   }
 
-  ;
-
-  public int getInt(ConfLevel level, String tag, int def);
+  protected Config config;
+  protected Map<CfgProp, String> propMap = new HashMap<CfgProp, String>();
 
-  public boolean getBool(ConfLevel level, String tag, boolean defValue);
-
-  public String get(ConfLevel level, String tag, String def);
-
-  public void substituteProperties();
+  public ConfigSolr(Config config) {
+    this.config = config;
+  }
+  
+  public Config getConfig() {
+    return config;
+  }
 
-  public ShardHandlerFactory initShardHandler();
+  public int getInt(CfgProp prop, int def) {
+    String val = propMap.get(prop);
+    if (val != null) val = PropertiesUtil.substituteProperty(val, null);
+    return (val == null) ? def : Integer.parseInt(val);
+  }
 
-  public Properties getSolrProperties(ConfigSolr cfg, String context);
+  public boolean getBool(CfgProp prop, boolean defValue) {
+    String val = propMap.get(prop);
+    if (val != null) val = PropertiesUtil.substituteProperty(val, null);
+    return (val == null) ? defValue : Boolean.parseBoolean(val);
+  }
 
-  public IndexSchema getSchemaFromZk(ZkController zkController, String zkConfigName, String schemaName,
-                                     SolrConfig config) throws KeeperException, InterruptedException;
+  public String get(CfgProp prop, String def) {
+    String val = propMap.get(prop);
+    if (val != null) val = PropertiesUtil.substituteProperty(val, null);
+    return (val == null) ? def : val;
+  }
 
-  public SolrConfig getSolrConfigFromZk(ZkController zkController, String zkConfigName, String solrConfigFileName,
-                                        SolrResourceLoader resourceLoader);
+  // For saving the original property, ${} syntax and all.
+  public String getOrigProp(CfgProp prop, String def) {
+    String val = propMap.get(prop);
+    return (val == null) ? def : val;
+  }
 
-  public void initPersist();
+  public Properties getSolrProperties(String path) {
+    try {
+      return readProperties(((NodeList) config.evaluate(
+          path, XPathConstants.NODESET)).item(0));
+    } catch (Throwable e) {
+      SolrException.log(log, null, e);
+    }
+    return null;
 
-  public void addPersistCore(String coreName, Properties attribs, Map<String, String> props);
+  }
+  
+  protected Properties readProperties(Node node) throws XPathExpressionException {
+    XPath xpath = config.getXPath();
+    NodeList props = (NodeList) xpath.evaluate("property", node, XPathConstants.NODESET);
+    Properties properties = new Properties();
+    for (int i = 0; i < props.getLength(); i++) {
+      Node prop = props.item(i);
+      properties.setProperty(DOMUtil.getAttr(prop, "name"), DOMUtil.getAttr(prop, "value"));
+    }
+    return properties;
+  }
 
-  public void addPersistAllCores(Properties containerProperties, Map<String, String> rootSolrAttribs, Map<String, String> coresAttribs,
-                                 File file);
+  public abstract void substituteProperties();
 
-  public String getCoreNameFromOrig(String origCoreName, SolrResourceLoader loader, String coreName);
+  public abstract List<String> getAllCoreNames();
 
-  public List<String> getAllCoreNames();
+  public abstract String getProperty(String coreName, String property, String defaultVal);
 
-  public String getProperty(String coreName, String property, String defaultVal);
+  public abstract Properties readCoreProperties(String coreName);
 
-  public Properties readCoreProperties(String coreName);
+  public abstract Map<String, String> readCoreAttributes(String coreName);
 
-  public Map<String, String> readCoreAttributes(String coreName);
 }
+