You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2012/08/09 12:21:07 UTC

svn commit: r1371142 [32/32] - in /lucene/dev/branches/lucene3312: ./ dev-tools/ dev-tools/eclipse/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/analysis/common/ dev-tools/maven/lucene/analysis/icu/ dev-tools/maven/lucene/analysis/ku...

Modified: lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java (original)
+++ lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java Thu Aug  9 10:20:53 2012
@@ -42,6 +42,7 @@ import org.apache.zookeeper.KeeperExcept
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -68,7 +69,7 @@ public class ZkStateReader {
   public static final String DOWN = "down";
   public static final String SYNC = "sync";
   
-  private volatile CloudState cloudState;
+  private volatile ClusterState clusterState;
 
   private static final long SOLRCLOUD_UPDATE_DELAY = Long.parseLong(System.getProperty("solrcloud.update.delay", "5000"));
 
@@ -120,7 +121,7 @@ public class ZkStateReader {
   }
   private ScheduledExecutorService updateCloudExecutor = Executors.newScheduledThreadPool(1, new ZKTF());
 
-  private boolean cloudStateUpdateScheduled;
+  private boolean clusterStateUpdateScheduled;
 
   private SolrZkClient zkClient;
   
@@ -158,13 +159,13 @@ public class ZkStateReader {
   }
   
   // load and publish a new CollectionInfo
-  public void updateCloudState(boolean immediate) throws KeeperException, InterruptedException {
-    updateCloudState(immediate, false);
+  public void updateClusterState(boolean immediate) throws KeeperException, InterruptedException {
+    updateClusterState(immediate, false);
   }
   
   // load and publish a new CollectionInfo
   public void updateLiveNodes() throws KeeperException, InterruptedException {
-    updateCloudState(true, true);
+    updateClusterState(true, true);
   }
   
   public synchronized void createClusterStateWatchersAndUpdate() throws KeeperException,
@@ -185,21 +186,22 @@ public class ZkStateReader {
           if (EventType.None.equals(event.getType())) {
             return;
           }
-          log.info("A cluster state change has occurred");
+          log.info("A cluster state change has occurred - updating...");
           try {
             
             // delayed approach
-            // ZkStateReader.this.updateCloudState(false, false);
+            // ZkStateReader.this.updateClusterState(false, false);
             synchronized (ZkStateReader.this.getUpdateLock()) {
               // remake watch
               final Watcher thisWatch = this;
-              byte[] data = zkClient.getData(CLUSTER_STATE, thisWatch, null,
+              Stat stat = new Stat();
+              byte[] data = zkClient.getData(CLUSTER_STATE, thisWatch, stat ,
                   true);
               
-              CloudState clusterState = CloudState.load(data,
-                  ZkStateReader.this.cloudState.getLiveNodes());
+              ClusterState clusterState = ClusterState.load(stat.getVersion(), data,
+                  ZkStateReader.this.clusterState.getLiveNodes());
               // update volatile
-              cloudState = clusterState;
+              ZkStateReader.this.clusterState = clusterState;
             }
           } catch (KeeperException e) {
             if (e.code() == KeeperException.Code.SESSIONEXPIRED
@@ -236,15 +238,17 @@ public class ZkStateReader {
               log.info("Updating live nodes");
               try {
                 // delayed approach
-                // ZkStateReader.this.updateCloudState(false, true);
+                // ZkStateReader.this.updateClusterState(false, true);
                 synchronized (ZkStateReader.this.getUpdateLock()) {
                   List<String> liveNodes = zkClient.getChildren(
                       LIVE_NODES_ZKNODE, this, true);
                   Set<String> liveNodesSet = new HashSet<String>();
                   liveNodesSet.addAll(liveNodes);
-                  CloudState clusterState = new CloudState(liveNodesSet,
-                      ZkStateReader.this.cloudState.getCollectionStates());
-                  ZkStateReader.this.cloudState = clusterState;
+                  ClusterState clusterState = new ClusterState(
+                      ZkStateReader.this.clusterState.getZkClusterStateVersion(),
+                      liveNodesSet, ZkStateReader.this.clusterState
+                          .getCollectionStates());
+                  ZkStateReader.this.clusterState = clusterState;
                 }
               } catch (KeeperException e) {
                 if (e.code() == KeeperException.Code.SESSIONEXPIRED
@@ -267,51 +271,53 @@ public class ZkStateReader {
     
       Set<String> liveNodeSet = new HashSet<String>();
       liveNodeSet.addAll(liveNodes);
-      CloudState clusterState = CloudState.load(zkClient, liveNodeSet);
-      this.cloudState = clusterState;
+      ClusterState clusterState = ClusterState.load(zkClient, liveNodeSet);
+      this.clusterState = clusterState;
     }
   }
   
   
   // load and publish a new CollectionInfo
-  private synchronized void updateCloudState(boolean immediate,
+  private synchronized void updateClusterState(boolean immediate,
       final boolean onlyLiveNodes) throws KeeperException,
       InterruptedException {
     // build immutable CloudInfo
     
     if (immediate) {
-      CloudState clusterState;
+      ClusterState clusterState;
       synchronized (getUpdateLock()) {
-      List<String> liveNodes = zkClient.getChildren(LIVE_NODES_ZKNODE, null, true);
-      Set<String> liveNodesSet = new HashSet<String>();
-      liveNodesSet.addAll(liveNodes);
-      
+        List<String> liveNodes = zkClient.getChildren(LIVE_NODES_ZKNODE, null,
+            true);
+        Set<String> liveNodesSet = new HashSet<String>();
+        liveNodesSet.addAll(liveNodes);
+        
         if (!onlyLiveNodes) {
           log.info("Updating cloud state from ZooKeeper... ");
           
-          clusterState = CloudState.load(zkClient, liveNodesSet);
+          clusterState = ClusterState.load(zkClient, liveNodesSet);
         } else {
           log.info("Updating live nodes from ZooKeeper... ");
-          clusterState = new CloudState(liveNodesSet,
-              ZkStateReader.this.cloudState.getCollectionStates());
+          clusterState = new ClusterState(
+              ZkStateReader.this.clusterState.getZkClusterStateVersion(), liveNodesSet,
+              ZkStateReader.this.clusterState.getCollectionStates());
         }
       }
 
-      this.cloudState = clusterState;
+      this.clusterState = clusterState;
     } else {
-      if (cloudStateUpdateScheduled) {
+      if (clusterStateUpdateScheduled) {
         log.info("Cloud state update for ZooKeeper already scheduled");
         return;
       }
       log.info("Scheduling cloud state update from ZooKeeper...");
-      cloudStateUpdateScheduled = true;
+      clusterStateUpdateScheduled = true;
       updateCloudExecutor.schedule(new Runnable() {
         
         public void run() {
           log.info("Updating cluster state from ZooKeeper...");
           synchronized (getUpdateLock()) {
-            cloudStateUpdateScheduled = false;
-            CloudState clusterState;
+            clusterStateUpdateScheduled = false;
+            ClusterState clusterState;
             try {
               List<String> liveNodes = zkClient.getChildren(LIVE_NODES_ZKNODE,
                   null, true);
@@ -321,13 +327,13 @@ public class ZkStateReader {
               if (!onlyLiveNodes) {
                 log.info("Updating cloud state from ZooKeeper... ");
                 
-                clusterState = CloudState.load(zkClient, liveNodesSet);
+                clusterState = ClusterState.load(zkClient, liveNodesSet);
               } else {
                 log.info("Updating live nodes from ZooKeeper... ");
-                clusterState = new CloudState(liveNodesSet, ZkStateReader.this.cloudState.getCollectionStates());
+                clusterState = new ClusterState(ZkStateReader.this.clusterState .getZkClusterStateVersion(), liveNodesSet, ZkStateReader.this.clusterState.getCollectionStates());
               }
               
-              ZkStateReader.this.cloudState = clusterState;
+              ZkStateReader.this.clusterState = clusterState;
               
             } catch (KeeperException e) {
               if (e.code() == KeeperException.Code.SESSIONEXPIRED
@@ -346,7 +352,7 @@ public class ZkStateReader {
                   SolrException.ErrorCode.SERVER_ERROR, "", e);
             } 
             // update volatile
-            ZkStateReader.this.cloudState = cloudState;
+            ZkStateReader.this.clusterState = clusterState;
           }
         }
       }, SOLRCLOUD_UPDATE_DELAY, TimeUnit.MILLISECONDS);
@@ -357,8 +363,8 @@ public class ZkStateReader {
   /**
    * @return information about the cluster from ZooKeeper
    */
-  public CloudState getCloudState() {
-    return cloudState;
+  public ClusterState getClusterState() {
+    return clusterState;
   }
   
   public Object getUpdateLock() {
@@ -411,9 +417,8 @@ public class ZkStateReader {
   public ZkNodeProps getLeaderProps(String collection, String shard, int timeout) throws InterruptedException {
     long timeoutAt = System.currentTimeMillis() + timeout;
     while (System.currentTimeMillis() < timeoutAt) {
-      if (cloudState != null) {
-        final CloudState currentState = cloudState;      
-        final ZkNodeProps nodeProps = currentState.getLeader(collection, shard);
+      if (clusterState != null) {    
+        final ZkNodeProps nodeProps = clusterState.getLeader(collection, shard);
         if (nodeProps != null) {
           return nodeProps;
         }
@@ -452,15 +457,15 @@ public class ZkStateReader {
   
   public List<ZkCoreNodeProps> getReplicaProps(String collection,
       String shardId, String thisNodeName, String coreName, String mustMatchStateFilter, String mustNotMatchStateFilter) {
-    CloudState cloudState = this.cloudState;
-    if (cloudState == null) {
+    ClusterState clusterState = this.clusterState;
+    if (clusterState == null) {
       return null;
     }
-    Map<String,Slice> slices = cloudState.getSlices(collection);
+    Map<String,Slice> slices = clusterState.getSlices(collection);
     if (slices == null) {
       throw new ZooKeeperException(ErrorCode.BAD_REQUEST,
           "Could not find collection in zk: " + collection + " "
-              + cloudState.getCollections());
+              + clusterState.getCollections());
     }
     
     Slice replicas = slices.get(shardId);
@@ -474,7 +479,7 @@ public class ZkStateReader {
     for (Entry<String,ZkNodeProps> entry : shardMap.entrySet()) {
       ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(entry.getValue());
       String coreNodeName = nodeProps.getNodeName() + "_" + nodeProps.getCoreName();
-      if (cloudState.liveNodesContain(nodeProps.getNodeName()) && !coreNodeName.equals(filterNodeName)) {
+      if (clusterState.liveNodesContain(nodeProps.getNodeName()) && !coreNodeName.equals(filterNodeName)) {
         if (mustMatchStateFilter == null || mustMatchStateFilter.equals(nodeProps.getState())) {
           if (mustNotMatchStateFilter == null || !mustNotMatchStateFilter.equals(nodeProps.getState())) {
             nodes.add(nodeProps);

Modified: lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/util/Hash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/util/Hash.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/util/Hash.java (original)
+++ lucene/dev/branches/lucene3312/solr/solrj/src/java/org/apache/solr/common/util/Hash.java Thu Aug  9 10:20:53 2012
@@ -291,4 +291,132 @@ public class Hash {
     return h1;
   }
 
+
+
+  /** Returns the MurmurHash3_x86_32 hash of the UTF-8 bytes of the String without actually encoding
+   * the string to a temporary buffer.  This is more than 2x faster than hashing the result
+   * of String.getBytes().
+   */
+  public static int murmurhash3_x86_32(CharSequence data, int offset, int len, int seed) {
+
+    final int c1 = 0xcc9e2d51;
+    final int c2 = 0x1b873593;
+
+    int h1 = seed;
+
+    int pos = offset;
+    int end = offset + len;
+    int k1 = 0;
+    int k2 = 0;
+    int shift = 0;
+    int bits = 0;
+    int nBytes = 0;   // length in UTF8 bytes
+
+
+    while (pos < end) {
+      int code = data.charAt(pos++);
+      if (code < 0x80) {
+        k2 = code;
+        bits = 8;
+
+        /***
+         // optimized ascii implementation (currently slower!!! code size?)
+         if (shift == 24) {
+         k1 = k1 | (code << 24);
+
+         k1 *= c1;
+         k1 = (k1 << 15) | (k1 >>> 17);  // ROTL32(k1,15);
+         k1 *= c2;
+
+         h1 ^= k1;
+         h1 = (h1 << 13) | (h1 >>> 19);  // ROTL32(h1,13);
+         h1 = h1*5+0xe6546b64;
+
+         shift = 0;
+         nBytes += 4;
+         k1 = 0;
+         } else {
+         k1 |= code << shift;
+         shift += 8;
+         }
+         continue;
+         ***/
+
+      }
+      else if (code < 0x800) {
+        k2 = (0xC0 | (code >> 6))
+            | ((0x80 | (code & 0x3F)) << 8);
+        bits = 16;
+      }
+      else if (code < 0xD800 || code > 0xDFFF || pos>=end) {
+        // we check for pos>=end to encode an unpaired surrogate as 3 bytes.
+        k2 = (0xE0 | (code >> 12))
+            | ((0x80 | ((code >> 6) & 0x3F)) << 8)
+            | ((0x80 | (code & 0x3F)) << 16);
+        bits = 24;
+      } else {
+        // surrogate pair
+        // int utf32 = pos < end ? (int) data.charAt(pos++) : 0;
+        int utf32 = (int) data.charAt(pos++);
+        utf32 = ((code - 0xD7C0) << 10) + (utf32 & 0x3FF);
+        k2 = (0xff & (0xF0 | (utf32 >> 18)))
+            | ((0x80 | ((utf32 >> 12) & 0x3F))) << 8
+            | ((0x80 | ((utf32 >> 6) & 0x3F))) << 16
+            |  (0x80 | (utf32 & 0x3F)) << 24;
+        bits = 32;
+      }
+
+
+      k1 |= k2 << shift;
+
+      // int used_bits = 32 - shift;  // how many bits of k2 were used in k1.
+      // int unused_bits = bits - used_bits; //  (bits-(32-shift)) == bits+shift-32  == bits-newshift
+
+      shift += bits;
+      if (shift >= 32) {
+        // mix after we have a complete word
+
+        k1 *= c1;
+        k1 = (k1 << 15) | (k1 >>> 17);  // ROTL32(k1,15);
+        k1 *= c2;
+
+        h1 ^= k1;
+        h1 = (h1 << 13) | (h1 >>> 19);  // ROTL32(h1,13);
+        h1 = h1*5+0xe6546b64;
+
+        shift -= 32;
+        // unfortunately, java won't let you shift 32 bits off, so we need to check for 0
+        if (shift != 0) {
+          k1 = k2 >>> (bits-shift);   // bits used == bits - newshift
+        } else {
+          k1 = 0;
+        }
+        nBytes += 4;
+      }
+
+    } // inner
+
+    // handle tail
+    if (shift > 0) {
+      nBytes += shift >> 3;
+      k1 *= c1;
+      k1 = (k1 << 15) | (k1 >>> 17);  // ROTL32(k1,15);
+      k1 *= c2;
+      h1 ^= k1;
+    }
+
+    // finalization
+    h1 ^= nBytes;
+
+    // fmix(h1);
+    h1 ^= h1 >>> 16;
+    h1 *= 0x85ebca6b;
+    h1 ^= h1 >>> 13;
+    h1 *= 0xc2b2ae35;
+    h1 ^= h1 >>> 16;
+
+    return h1;
+  }
+
+
 }

Modified: lucene/dev/branches/lucene3312/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrServer.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrServer.java (original)
+++ lucene/dev/branches/lucene3312/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrServer.java Thu Aug  9 10:20:53 2012
@@ -237,7 +237,7 @@ public class TestLBHttpSolrServer extend
     }
 
     public String getSchemaFile() {
-      return "solrj/solr/conf/schema-replication1.xml";
+      return "solrj/solr/collection1/conf/schema-replication1.xml";
     }
 
     public String getConfDir() {
@@ -249,7 +249,7 @@ public class TestLBHttpSolrServer extend
     }
 
     public String getSolrConfigFile() {
-      return "solrj/solr/conf/solrconfig-slave1.xml";
+      return "solrj/solr/collection1/conf/solrconfig-slave1.xml";
     }
 
     public void setUp() throws Exception {

Modified: lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java (original)
+++ lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java Thu Aug  9 10:20:53 2012
@@ -450,11 +450,25 @@ public abstract class BaseDistributedSea
 //    System.out.println("resp b:" + b);
     boolean ordered = (flags & UNORDERED) == 0;
 
+    if (!ordered) {
+      Map mapA = new HashMap(a.size());
+      for (int i=0; i<a.size(); i++) {
+        Object prev = mapA.put(a.getName(i), a.getVal(i));
+      }
+
+      Map mapB = new HashMap(b.size());
+      for (int i=0; i<b.size(); i++) {
+        Object prev = mapB.put(b.getName(i), b.getVal(i));
+      }
+
+      return compare(mapA, mapB, flags, handle);
+    }
+
     int posa = 0, posb = 0;
     int aSkipped = 0, bSkipped = 0;
 
     for (; ;) {
-      if (posa >= a.size() || posb >= b.size()) {
+      if (posa >= a.size() && posb >= b.size()) {
         break;
       }
 
@@ -468,29 +482,27 @@ public abstract class BaseDistributedSea
         posa++;
         flagsa = flags(handle, namea);
         if ((flagsa & SKIP) != 0) {
+          namea = null; vala = null;
           aSkipped++;
           continue;
         }
         break;
       }
 
-      if (!ordered) posb = 0;  // reset if not ordered
-
       while (posb < b.size()) {
         nameb = b.getName(posb);
         valb = b.getVal(posb);
         posb++;
         flagsb = flags(handle, nameb);
         if ((flagsb & SKIP) != 0) {
+          nameb = null; valb = null;
           bSkipped++;
           continue;
         }
         if (eq(namea, nameb)) {
           break;
         }
-        if (ordered) {
-          return "." + namea + "!=" + nameb + " (unordered or missing)";
-        }
+        return "." + namea + "!=" + nameb + " (unordered or missing)";
         // if unordered, continue until we find the right field.
       }
 
@@ -503,7 +515,7 @@ public abstract class BaseDistributedSea
 
 
     if (a.size() - aSkipped != b.size() - bSkipped) {
-      return ".size()==" + a.size() + "," + b.size() + "skipped=" + aSkipped + "," + bSkipped;
+      return ".size()==" + a.size() + "," + b.size() + " skipped=" + aSkipped + "," + bSkipped;
     }
 
     return null;

Modified: lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java (original)
+++ lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java Thu Aug  9 10:20:53 2012
@@ -1388,11 +1388,10 @@ public abstract class SolrTestCaseJ4 ext
       return file;
     } catch (Exception e) {
       /* more friendly than NPE */
-      throw new RuntimeException("Cannot find resource: " + name);
+      throw new RuntimeException("Cannot find resource: " + new File(name).getAbsolutePath());
     }
   }
   
-  // TODO: use solr rather than solr/collection1
   public static String TEST_HOME() {
     return getFile("solr/collection1").getParent();
   }

Modified: lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/util/ExternalPaths.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/util/ExternalPaths.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/util/ExternalPaths.java (original)
+++ lucene/dev/branches/lucene3312/solr/test-framework/src/java/org/apache/solr/util/ExternalPaths.java Thu Aug  9 10:20:53 2012
@@ -25,7 +25,7 @@ import java.io.File;
  * @lucene.internal
  */
 public class ExternalPaths {
-  private static final String SOURCE_HOME = determineSourceHome();
+  public static final String SOURCE_HOME = determineSourceHome();
   public static String WEBAPP_HOME = new File(SOURCE_HOME, "webapp/web").getAbsolutePath();
   public static String EXAMPLE_HOME = new File(SOURCE_HOME, "example/solr").getAbsolutePath();
   public static String EXAMPLE_MULTICORE_HOME = new File(SOURCE_HOME, "example/multicore").getAbsolutePath();

Modified: lucene/dev/branches/lucene3312/solr/webapp/web/admin.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/webapp/web/admin.html?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/webapp/web/admin.html (original)
+++ lucene/dev/branches/lucene3312/solr/webapp/web/admin.html Thu Aug  9 10:20:53 2012
@@ -63,7 +63,15 @@ limitations under the License.
     </div>
 
     <div id="main" class="clearfix">
-        
+    
+      <div id="init-failures">
+
+          <h2>SolrCore Initialization Failures</h2>
+          <ul></ul>
+          <p>Please check your logs for more information</p>
+                
+      </div>
+
       <div id="content-wrapper">
         <div id="content">
                   
@@ -87,8 +95,8 @@ limitations under the License.
 
             <li id="cloud" class="global optional"><p><a href="#/~cloud">Cloud</a></p>
               <ul>
-                <li class="tree"><a href="#/~cloud">Tree</a></li>
-                <li class="graph"><a href="#/~cloud?view=graph">Graph</a></li>
+                <li class="tree"><a href="#/~cloud?view=tree">Tree</a></li>
+                <li class="graph"><a href="#/~cloud">Graph</a></li>
                 <li class="rgraph"><a href="#/~cloud?view=rgraph">Graph (Radial)</a></li>
                 <li class="dump"><a href="#/~cloud">Dump</a></li>
               </ul>

Modified: lucene/dev/branches/lucene3312/solr/webapp/web/css/styles/common.css
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/webapp/web/css/styles/common.css?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/webapp/web/css/styles/common.css (original)
+++ lucene/dev/branches/lucene3312/solr/webapp/web/css/styles/common.css Thu Aug  9 10:20:53 2012
@@ -284,6 +284,38 @@ ul
   border-color: #66b866;
 }
 
+#init-failures
+{
+  border: 1px solid #f00;
+  display: none;
+  margin-left: 150px;
+  margin-bottom: 20px;
+}
+
+#main.error #init-failures
+{
+  margin-left: 0;
+}
+
+#init-failures h2,
+#init-failures ul,
+#init-failures p
+{
+  padding: 10px;
+}
+
+#init-failures h2
+{
+  background-color: #f00;
+  color: #fff;
+  font-weight: bold;
+}
+
+#init-failures p
+{
+  color: #c0c0c0;
+  padding-top: 0;
+}
 
 #content-wrapper
 {

Modified: lucene/dev/branches/lucene3312/solr/webapp/web/js/scripts/app.js
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/solr/webapp/web/js/scripts/app.js?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/solr/webapp/web/js/scripts/app.js (original)
+++ lucene/dev/branches/lucene3312/solr/webapp/web/js/scripts/app.js Thu Aug  9 10:20:53 2012
@@ -152,6 +152,24 @@ var solr_admin = function( app_config )
 
   this.timeout = null;
 
+  show_global_error = function( error )
+  {
+    var main = $( '#main' );
+
+    $( 'div[id$="-wrapper"]', main )
+      .remove();
+
+    main
+      .addClass( 'error' )
+      .append( error );
+
+    var pre_tags = $( 'pre', main );
+    if( 0 !== pre_tags.size() )
+    {
+      hljs.highlightBlock( pre_tags.get(0) ); 
+    }
+  };
+
   this.run = function()
   {
     $.ajax
@@ -167,9 +185,11 @@ var solr_admin = function( app_config )
         success : function( response )
         {
           self.cores_data = response.status;
+          var core_count = 0;
 
           for( var core_name in response.status )
           {
+            core_count++;
             var core_path = config.solr_path + '/' + core_name;
             var schema =  response['status'][core_name]['schema'];
             var solrconfig =  response['status'][core_name]['config'];
@@ -211,10 +231,44 @@ var solr_admin = function( app_config )
               .append( core_tpl );
           }
 
+          if( response.initFailures )
+          {
+            var failures = [];
+            for( var core_name in response.initFailures )
+            {
+              failures.push
+              (
+                '<li>' + 
+                  '<strong>' + core_name.esc() + ':</strong>' + "\n" +
+                  response.initFailures[core_name].esc() + "\n" +
+                '</li>'
+              );
+            }
+
+            if( 0 !== failures.length )
+            {
+              var init_failures = $( '#init-failures' );
+
+              init_failures.show();
+              $( 'ul', init_failures ).html( failures.join( "\n" ) );
+            }
+          }
+
+          if( 0 === core_count )
+          {
+            show_global_error
+            (
+              '<div class="message">There are no SolrCores running — for the current functionality ' +
+              'we require at least one SolrCore, sorry :)</div>'
+            );
+            return;
+          } // else: we have at least one core....
+
+          var system_url = environment_basepath + '/admin/system?wt=json';
           $.ajax
           (
             {
-              url : environment_basepath + '/admin/system?wt=json',
+              url : system_url,
               dataType : 'json',
               beforeSend : function( arr, form, options )
               {
@@ -280,23 +334,17 @@ var solr_admin = function( app_config )
               },
               error : function()
               {
-                var main = $( '#main' );
-
-                $( 'div[id$="-wrapper"]', main )
-                  .remove();
-
-                main
-                  .addClass( 'error' )
-                  .append
-                  (
-                    '<div class="message">This interface requires that you activate the admin request handlers, add the following configuration to your <code>solrconfig.xml:</code></div>' +
-                    '<div class="code"><pre class="syntax language-xml"><code>' +
-                    '<!-- Admin Handlers - This will register all the standard admin RequestHandlers. -->'.esc() + "\n" +
-                    '<requestHandler name="/admin/" class="solr.admin.AdminHandlers" />'.esc() +
-                    '</code></pre></div>'
-                  );
-
-                hljs.highlightBlock( $( 'pre', main ).get(0) );
+                show_global_error
+                (
+                  '<div class="message"><p>Unable to load environment info from <code>' + system_url.esc() + '</code>.</p>' +
+                  '<p>This interface requires that you activate the admin request handlers in all SolrCores by adding the ' +
+                  'following configuration to your <code>solrconfig.xml</code>:</p></div>' + "\n" +
+
+                  '<div class="code"><pre class="syntax language-xml"><code>' +
+                  '<!-- Admin Handlers - This will register all the standard admin RequestHandlers. -->'.esc() + "\n" +
+                  '<requestHandler name="/admin/" class="solr.admin.AdminHandlers" />'.esc() +
+                  '</code></pre></div>'
+                );
               },
               complete : function()
               {
@@ -317,4 +365,4 @@ var solr_admin = function( app_config )
 
 };
 
-var app = new solr_admin( app_config );
\ No newline at end of file
+var app = new solr_admin( app_config );