You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2012/01/27 19:15:55 UTC

svn commit: r1236796 [2/2] - in /lucene/dev/branches/lucene3661: ./ lucene/ lucene/src/java/org/apache/lucene/index/ lucene/src/java/org/apache/lucene/search/ lucene/src/java/org/apache/lucene/search/similarities/ lucene/src/test-framework/java/org/apa...

Modified: lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java Fri Jan 27 18:15:52 2012
@@ -199,7 +199,6 @@ public  class LeaderElector {
    * watch the next lowest numbered node.
    * 
    * @param context
-   * @param SolrCore - optional - sometimes null
    * @return sequential node number
    * @throws KeeperException
    * @throws InterruptedException
@@ -256,8 +255,7 @@ public  class LeaderElector {
   /**
    * Set up any ZooKeeper nodes needed for leader election.
    * 
-   * @param shardId
-   * @param collection
+   * @param context
    * @throws InterruptedException
    * @throws KeeperException
    */

Modified: lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java Fri Jan 27 18:15:52 2012
@@ -20,10 +20,8 @@ package org.apache.solr.cloud;
 import java.io.IOException;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
-import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
 import org.apache.solr.client.solrj.request.CoreAdminRequest.PrepRecovery;
@@ -38,10 +36,8 @@ import org.apache.solr.core.RequestHandl
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.ReplicationHandler;
 import org.apache.solr.request.SolrRequestHandler;
-import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.update.UpdateLog;
 import org.apache.solr.update.UpdateLog.RecoveryInfo;
-import org.apache.solr.util.RefCounted;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -67,7 +63,7 @@ public class RecoveryStrategy extends Th
   public RecoveryStrategy(SolrCore core) {
     this.core = core;
     this.coreName = core.getName();
-    
+    setName("RecoveryThread");
     zkController = core.getCoreDescriptor().getCoreContainer().getZkController();
     zkStateReader = zkController.getZkStateReader();
     baseUrl = zkController.getBaseUrl();

Modified: lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java Fri Jan 27 18:15:52 2012
@@ -46,7 +46,8 @@ public class SyncStrategy {
   
   public boolean sync(ZkController zkController, SolrCore core,
       ZkNodeProps leaderProps) {
-    zkController.publish(core, ZkStateReader.SYNC);
+    // TODO: look at our state usage of sync
+    // zkController.publish(core, ZkStateReader.SYNC);
     
     // solrcloud_debug
     // System.out.println("SYNC UP");

Modified: lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/cloud/ZkController.java Fri Jan 27 18:15:52 2012
@@ -130,16 +130,15 @@ public final class ZkController {
     zkServer.stop();
   }
 
-
   /**
-   * @param coreContainer if null, recovery will not be enabled
+   * @param cc if null, recovery will not be enabled
    * @param zkServerAddress
    * @param zkClientTimeout
    * @param zkClientConnectTimeout
    * @param localHost
    * @param locaHostPort
    * @param localHostContext
-   * @param numShards 
+   * @param registerOnReconnect
    * @throws InterruptedException
    * @throws TimeoutException
    * @throws IOException
@@ -437,13 +436,14 @@ public final class ZkController {
   }
 
 
+
   /**
    * Register shard with ZooKeeper.
    * 
    * @param coreName
-   * @param cloudDesc
-   * @return
-   * @throws Exception 
+   * @param desc
+   * @return the shardId for the SolrCore
+   * @throws Exception
    */
   public String register(String coreName, final CoreDescriptor desc) throws Exception {  
     return register(coreName, desc, false);
@@ -456,7 +456,7 @@ public final class ZkController {
    * @param coreName
    * @param desc
    * @param recoverReloadedCores
-   * @return
+   * @return the shardId for the SolrCore
    * @throws Exception
    */
   public String register(String coreName, final CoreDescriptor desc, boolean recoverReloadedCores) throws Exception {  
@@ -508,9 +508,33 @@ public final class ZkController {
       try {
         core = cc.getCore(desc.getName());
 
-        boolean startRecovery = checkRecovery(coreName, desc, recoverReloadedCores, isLeader, cloudDesc,
+        if (isLeader) {
+          // recover from local transaction log and wait for it to complete before
+          // going active
+          // TODO: should this be moved to another thread? To recoveryStrat?
+          // TODO: should this actually be done earlier, before (or as part of)
+          // leader election perhaps?
+          // TODO: ensure that a replica that is trying to recover waits until I'm
+          // active (or don't make me the
+          // leader until my local replay is done. But this replay is only needed
+          // on the leader - replicas
+          // will do recovery anyway
+          
+          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?
+            }
+          }
+        }
+        
+        boolean didRecovery = checkRecovery(coreName, desc, recoverReloadedCores, isLeader, cloudDesc,
             collection, coreZkNodeName, shardId, leaderProps, core, cc);
-        if (!startRecovery) {
+        if (!didRecovery) {
           publishAsActive(baseUrl, desc, coreZkNodeName, coreName);
         }
       } finally {
@@ -539,6 +563,24 @@ public final class ZkController {
   }
 
 
+  /**
+   * @param coreName
+   * @param desc
+   * @param recoverReloadedCores
+   * @param isLeader
+   * @param cloudDesc
+   * @param collection
+   * @param shardZkNodeName
+   * @param shardId
+   * @param leaderProps
+   * @param core
+   * @param cc
+   * @return whether or not a recovery was started
+   * @throws InterruptedException
+   * @throws KeeperException
+   * @throws IOException
+   * @throws ExecutionException
+   */
   private boolean checkRecovery(String coreName, final CoreDescriptor desc,
       boolean recoverReloadedCores, final boolean isLeader,
       final CloudDescriptor cloudDesc, final String collection,
@@ -546,46 +588,18 @@ public final class ZkController {
       SolrCore core, CoreContainer cc) throws InterruptedException,
       KeeperException, IOException, ExecutionException {
 
-    
     boolean doRecovery = true;
-
-
-    if (isLeader) {
-      doRecovery = false;
-      
-      // recover from local transaction log and wait for it to complete before
-      // going active
-      // TODO: should this be moved to another thread? To recoveryStrat?
-      // TODO: should this actually be done earlier, before (or as part of)
-      // leader election perhaps?
-      // TODO: ensure that a replica that is trying to recover waits until I'm
-      // active (or don't make me the
-      // leader until my local replay is done. But this replay is only needed
-      // on the leader - replicas
-      // will do recovery anyway
-      
-      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?
-        }
-      }
-      return false;
-    } else {
+    if (!isLeader) {
       
       if (core.isReloaded() && !recoverReloadedCores) {
         doRecovery = false;
       }
-    }
-    
-    if (doRecovery && !SKIP_AUTO_RECOVERY) {
-      log.info("Core needs to recover:" + core.getName());
-      core.getUpdateHandler().getSolrCoreState().doRecovery(core);
-      return true;
+      
+      if (doRecovery && !SKIP_AUTO_RECOVERY) {
+        log.info("Core needs to recover:" + core.getName());
+        core.getUpdateHandler().getSolrCoreState().doRecovery(core);
+        return true;
+      }
     }
     
     return false;

Modified: lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/core/SolrCore.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/core/SolrCore.java Fri Jan 27 18:15:52 2012
@@ -732,8 +732,17 @@ public final class SolrCore implements S
       if (!searcherExecutor.awaitTermination(60, TimeUnit.SECONDS)) {
         log.error("Timeout waiting for searchExecutor to terminate");
       }
+    } catch (InterruptedException e) {
+      searcherExecutor.shutdownNow();
+      try {
+        if (!searcherExecutor.awaitTermination(30, TimeUnit.SECONDS)) {
+          log.error("Timeout waiting for searchExecutor to terminate");
+        }
+      } catch (InterruptedException e2) {
+        SolrException.log(log, e2);
+      }
     } catch (Exception e) {
-      SolrException.log(log,e);
+      SolrException.log(log, e);
     }
     try {
       // Since we waited for the searcherExecutor to shut down,
@@ -744,7 +753,7 @@ public final class SolrCore implements S
       // then the searchExecutor will throw an exception when getSearcher()
       // tries to use it, and the exception handling code should close it.
       closeSearcher();
-    } catch (Exception e) {
+    } catch (Throwable e) {
       SolrException.log(log,e);
     }
 
@@ -1053,14 +1062,13 @@ public final class SolrCore implements S
 
     openSearcherLock.lock();
     try {
-      String newIndexDir = null;
+      String newIndexDir = getNewIndexDir();
       File indexDirFile = null;
       File newIndexDirFile = null;
 
       // if it's not a normal near-realtime update, check that paths haven't changed.
       if (!nrt) {
         indexDirFile = new File(getIndexDir()).getCanonicalFile();
-        newIndexDir = getNewIndexDir();
         newIndexDirFile = new File(newIndexDir).getCanonicalFile();
       }
 

Modified: lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/handler/SnapPuller.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/handler/SnapPuller.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/handler/SnapPuller.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/handler/SnapPuller.java Fri Jan 27 18:15:52 2012
@@ -288,7 +288,7 @@ public class SnapPuller {
         return true;
       }
       
-      if (commit.getVersion() == latestVersion && commit.getGeneration() == latestGeneration) {
+      if (!force && commit.getVersion() == latestVersion && commit.getGeneration() == latestGeneration) {
         //master and slave are already in sync just return
         LOG.info("Slave in sync with master.");
         successfulInstall = true;

Modified: lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/request/UnInvertedField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/request/UnInvertedField.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/request/UnInvertedField.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/request/UnInvertedField.java Fri Jan 27 18:15:52 2012
@@ -175,7 +175,7 @@ public class UnInvertedField extends Doc
     final String prefix = TrieField.getMainValuePrefix(searcher.getSchema().getFieldType(field));
     this.searcher = searcher;
     try {
-      uninvert(searcher.getIndexReader(), prefix == null ? null : new BytesRef(prefix));
+      uninvert(new SlowMultiReaderWrapper(searcher.getIndexReader()), prefix == null ? null : new BytesRef(prefix));
     } catch (IllegalStateException ise) {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, ise.getMessage());
     }
@@ -227,7 +227,7 @@ public class UnInvertedField extends Doc
       int startTerm = 0;
       int endTerm = numTermsInField;  // one past the end
 
-      TermsEnum te = getOrdTermsEnum(searcher.getIndexReader());
+      TermsEnum te = getOrdTermsEnum(new SlowMultiReaderWrapper(searcher.getIndexReader()));
       if (prefix != null && prefix.length() > 0) {
         final BytesRef prefixBr = new BytesRef(prefix);
         if (te.seekCeil(prefixBr, true) == TermsEnum.SeekStatus.END) {
@@ -497,7 +497,7 @@ public class UnInvertedField extends Doc
     final int[] index = this.index;
     final int[] counts = new int[numTermsInField];//keep track of the number of times we see each word in the field for all the documents in the docset
 
-    TermsEnum te = getOrdTermsEnum(searcher.getIndexReader());
+    TermsEnum te = getOrdTermsEnum(new SlowMultiReaderWrapper(searcher.getIndexReader()));
 
     boolean doNegative = false;
     if (finfo.length == 0) {

Modified: lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/update/TransactionLog.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/update/TransactionLog.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/update/TransactionLog.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/update/TransactionLog.java Fri Jan 27 18:15:52 2012
@@ -55,6 +55,8 @@ import java.util.concurrent.atomic.Atomi
  */
 public class TransactionLog {
   public static Logger log = LoggerFactory.getLogger(TransactionLog.class);
+  final boolean debug = log.isDebugEnabled();
+  final boolean trace = log.isTraceEnabled();
 
   public final static String END_MESSAGE="SOLR_TLOG_END";
 
@@ -71,7 +73,6 @@ public class TransactionLog {
   AtomicInteger refcount = new AtomicInteger(1);
   Map<String,Integer> globalStringMap = new HashMap<String, Integer>();
   List<String> globalStringList = new ArrayList<String>();
-  final boolean debug = log.isDebugEnabled();
 
   long snapshot_size;
   int snapshot_numRecords;
@@ -156,6 +157,9 @@ public class TransactionLog {
           addGlobalStrings(globalStrings);
         }
       } else {
+        if (start > 0) {
+          log.error("New transaction log already exists:" + tlogFile + " size=" + raf.length());
+        }
         assert start==0;
         if (start > 0) {
           raf.setLength(0);
@@ -543,8 +547,8 @@ public class TransactionLog {
 
 
       synchronized (TransactionLog.this) {
-        if (debug) {
-          log.debug("Reading log record.  pos="+pos+" currentSize="+fos.size());
+        if (trace) {
+          log.trace("Reading log record.  pos="+pos+" currentSize="+fos.size());
         }
 
         if (pos >= fos.size()) {

Modified: lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/update/UpdateLog.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/update/UpdateLog.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/update/UpdateLog.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/java/org/apache/solr/update/UpdateLog.java Fri Jan 27 18:15:52 2012
@@ -48,6 +48,7 @@ import java.util.concurrent.*;
 public class UpdateLog implements PluginInfoInitialized {
   public static Logger log = LoggerFactory.getLogger(UpdateLog.class);
   public boolean debug = log.isDebugEnabled();
+  public boolean trace = log.isTraceEnabled();
 
 
   public enum SyncLevel { NONE, FLUSH, FSYNC }
@@ -141,6 +142,9 @@ public class UpdateLog implements Plugin
     this.uhandler = uhandler;
 
     if (dataDir.equals(lastDataDir)) {
+      if (debug) {
+        log.debug("UpdateHandler init: tlogDir=" + tlogDir + ", next id=" + id, " this is a reopen... nothing else to do.");
+      }
       // on a normal reopen, we currently shouldn't have to do anything
       return;
     }
@@ -150,6 +154,10 @@ public class UpdateLog implements Plugin
     tlogFiles = getLogList(tlogDir);
     id = getLastLogId() + 1;   // add 1 since we will create a new log for the next update
 
+    if (debug) {
+      log.debug("UpdateHandler init: tlogDir=" + tlogDir + ", existing tlogs=" + Arrays.asList(tlogFiles) + ", next id=" + id);
+    }
+    
     TransactionLog oldLog = null;
     for (String oldLogName : tlogFiles) {
       File f = new File(tlogDir, oldLogName);
@@ -247,8 +255,8 @@ public class UpdateLog implements Plugin
         map.put(cmd.getIndexedId(), ptr);
       }
 
-      if (debug) {
-        log.debug("TLOG: added id " + cmd.getPrintableId() + " to " + tlog + " " + ptr + " map=" + System.identityHashCode(map));
+      if (trace) {
+        log.trace("TLOG: added id " + cmd.getPrintableId() + " to " + tlog + " " + ptr + " map=" + System.identityHashCode(map));
       }
     }
   }
@@ -274,8 +282,8 @@ public class UpdateLog implements Plugin
         oldDeletes.put(br, ptr);
       }
 
-      if (debug) {
-        log.debug("TLOG: added delete for id " + cmd.id + " to " + tlog + " " + ptr + " map=" + System.identityHashCode(map));
+      if (trace) {
+        log.trace("TLOG: added delete for id " + cmd.id + " to " + tlog + " " + ptr + " map=" + System.identityHashCode(map));
       }
     }
   }
@@ -312,8 +320,8 @@ public class UpdateLog implements Plugin
 
       LogPtr ptr = new LogPtr(pos, cmd.getVersion());
 
-      if (debug) {
-        log.debug("TLOG: added deleteByQuery " + cmd.query + " to " + tlog + " " + ptr + " map=" + System.identityHashCode(map));
+      if (trace) {
+        log.trace("TLOG: added deleteByQuery " + cmd.query + " to " + tlog + " " + ptr + " map=" + System.identityHashCode(map));
       }
     }
   }
@@ -385,6 +393,7 @@ public class UpdateLog implements Plugin
 
   public void preSoftCommit(CommitUpdateCommand cmd) {
     debug = log.isDebugEnabled(); // refresh our view of debugging occasionally
+    trace = log.isTraceEnabled();
 
     synchronized (this) {
 
@@ -562,7 +571,7 @@ public class UpdateLog implements Plugin
 
   private void ensureLog() {
     if (tlog == null) {
-      String newLogName = String.format("%s.%019d", TLOG_NAME, id);
+      String newLogName = String.format(Locale.ENGLISH, "%s.%019d", TLOG_NAME, id);
       try {
         tlog = new TransactionLog(new File(tlogDir, newLogName), globalStrings);
       } catch (IOException e) {

Modified: lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/ChaosMonkey.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/ChaosMonkey.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/ChaosMonkey.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/ChaosMonkey.java Fri Jan 27 18:15:52 2012
@@ -242,11 +242,7 @@ public class ChaosMonkey {
   
   public JettySolrRunner getRandomJetty(String slice, boolean aggressivelyKillLeaders) throws KeeperException, InterruptedException {
     
-    // get latest cloud state
-    zkStateReader.updateCloudState(true);
-    
-    Slice theShards = zkStateReader.getCloudState().getSlices(collection)
-        .get(slice);
+
     int numRunning = 0;
     int numRecovering = 0;
     int numActive = 0;
@@ -254,6 +250,12 @@ public class ChaosMonkey {
     for (CloudJettyRunner cloudJetty : shardToJetty.get(slice)) {
       boolean running = true;
       
+      // get latest cloud state
+      zkStateReader.updateCloudState(true);
+      
+      Slice theShards = zkStateReader.getCloudState().getSlices(collection)
+          .get(slice);
+      
       ZkNodeProps props = theShards.getShards().get(cloudJetty.coreNodeName);
       if (props == null) {
         throw new RuntimeException("shard name " + cloudJetty.coreNodeName + " not found in " + theShards.getShards().keySet());

Modified: lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java Fri Jan 27 18:15:52 2012
@@ -37,7 +37,9 @@ import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 
+@Ignore
 public class ChaosMonkeyNothingIsSafeTest extends FullSolrCloudTest {
 
   
@@ -75,78 +77,88 @@ public class ChaosMonkeyNothingIsSafeTes
   
   @Override
   public void doTest() throws Exception {
-    
-    handle.clear();
-    handle.put("QTime", SKIPVAL);
-    handle.put("timestamp", SKIPVAL);
-    
-    // we cannot do delete by query
-    // as it's not supported for recovery
-    //del("*:*");
-    
-    List<StopableIndexingThread> threads = new ArrayList<StopableIndexingThread>();
-    int threadCount = 1;
-    int i = 0;
-    for (i = 0; i < threadCount; i++) {
-      StopableIndexingThread indexThread = new StopableIndexingThread(i * 50000, true);
-      threads.add(indexThread);
-      indexThread.start();
-    }
-    
-    FullThrottleStopableIndexingThread ftIndexThread = new FullThrottleStopableIndexingThread(
-        clients, i * 50000, true);
-    threads.add(ftIndexThread);
-    ftIndexThread.start();
-    
-    chaosMonkey.startTheMonkey(true, 1500);
+    boolean testsSuccesful = false;
     try {
-      Thread.sleep(atLeast(6000));
+      handle.clear();
+      handle.put("QTime", SKIPVAL);
+      handle.put("timestamp", SKIPVAL);
+      
+      // we cannot do delete by query
+      // as it's not supported for recovery
+      // del("*:*");
+      
+      List<StopableIndexingThread> threads = new ArrayList<StopableIndexingThread>();
+      int threadCount = 1;
+      int i = 0;
+      for (i = 0; i < threadCount; i++) {
+        StopableIndexingThread indexThread = new StopableIndexingThread(
+            i * 50000, true);
+        threads.add(indexThread);
+        indexThread.start();
+      }
+      
+      FullThrottleStopableIndexingThread ftIndexThread = new FullThrottleStopableIndexingThread(
+          clients, i * 50000, true);
+      threads.add(ftIndexThread);
+      ftIndexThread.start();
+      
+      chaosMonkey.startTheMonkey(true, 1500);
+      try {
+        Thread.sleep(atLeast(6000));
+      } finally {
+        chaosMonkey.stopTheMonkey();
+      }
+      
+      for (StopableIndexingThread indexThread : threads) {
+        indexThread.safeStop();
+      }
+      
+      // wait for stop...
+      for (StopableIndexingThread indexThread : threads) {
+        indexThread.join();
+      }
+      
+      // fails will happen...
+      // for (StopableIndexingThread indexThread : threads) {
+      // assertEquals(0, indexThread.getFails());
+      // }
+      
+      // try and wait for any replications and what not to finish...
+      
+      Thread.sleep(2000);
+      
+      // wait until there are no recoveries...
+      waitForThingsToLevelOut();
+      
+      // make sure we again have leaders for each shard
+      for (int j = 1; j < sliceCount; j++) {
+        zkStateReader.getLeaderProps(DEFAULT_COLLECTION, "shard" + j, 10000);
+      }
+      
+      commit();
+      
+      // TODO: assert we didnt kill everyone
+      
+      zkStateReader.updateCloudState(true);
+      assertTrue(zkStateReader.getCloudState().getLiveNodes().size() > 0);
+      
+      checkShardConsistency(false, true);
+      
+      // ensure we have added more than 0 docs
+      long cloudClientDocs = cloudClient.query(new SolrQuery("*:*"))
+          .getResults().getNumFound();
+      
+      assertTrue(cloudClientDocs > 0);
+      
+      if (VERBOSE) System.out.println("control docs:"
+          + controlClient.query(new SolrQuery("*:*")).getResults()
+              .getNumFound() + "\n\n");
+      testsSuccesful = true;
     } finally {
-      chaosMonkey.stopTheMonkey();
-    }
-    
-    for (StopableIndexingThread indexThread : threads) {
-      indexThread.safeStop();
-    }
-    
-    // wait for stop...
-    for (StopableIndexingThread indexThread : threads) {
-      indexThread.join();
-    }
-    
-    
-    // fails will happen...
-//    for (StopableIndexingThread indexThread : threads) {
-//      assertEquals(0, indexThread.getFails());
-//    }
-    
-    // try and wait for any replications and what not to finish...
-    
-    Thread.sleep(2000);
-    
-    // wait until there are no recoveries...
-    waitForThingsToLevelOut();
-    
-    // make sure we again have leaders for each shard
-    for (int j = 1; j < sliceCount; j++) {
-      zkStateReader.getLeaderProps(DEFAULT_COLLECTION, "shard" + j, 10000);
+      if (!testsSuccesful) {
+        printLayout();
+      }
     }
-
-    commit();
-    
-    // TODO: assert we didnt kill everyone
-    
-    zkStateReader.updateCloudState(true);
-    assertTrue(zkStateReader.getCloudState().getLiveNodes().size() > 0);
-    
-    checkShardConsistency(false, false);
-    
-    // ensure we have added more than 0 docs
-    long cloudClientDocs = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
-
-    assertTrue(cloudClientDocs > 0);
-    
-    if (VERBOSE) System.out.println("control docs:" + controlClient.query(new SolrQuery("*:*")).getResults().getNumFound() + "\n\n");
   }
 
   private void waitForThingsToLevelOut() throws KeeperException,
@@ -156,7 +168,11 @@ public class ChaosMonkeyNothingIsSafeTes
     do {
       waitForRecoveriesToFinish(VERBOSE);
       
-      commit();
+      try {
+        commit();
+      } catch (Exception e) {
+        // we don't care if this commit fails on some nodes
+      }
       
       updateMappingsFromZk(jettys, clients);
       

Modified: lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java Fri Jan 27 18:15:52 2012
@@ -40,6 +40,7 @@ import org.apache.solr.update.processor.
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 
 /**
  * Super basic testing, no shard restarting or anything.

Modified: lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudTest.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudTest.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudTest.java Fri Jan 27 18:15:52 2012
@@ -51,6 +51,7 @@ import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 
 /**
  * 
@@ -58,12 +59,15 @@ import org.junit.BeforeClass;
  * what we test now - the default update chain
  * 
  */
+@Ignore
 public class FullSolrCloudTest extends AbstractDistributedZkTestCase {
   
   private static final String SHARD2 = "shard2";
   
   protected static final String DEFAULT_COLLECTION = "collection1";
   
+  private boolean printLayoutOnTearDown = false;
+  
   String t1 = "a_t";
   String i1 = "a_si";
   String nint = "n_i";
@@ -223,13 +227,25 @@ public class FullSolrCloudTest extends A
     System.clearProperty("collection");
     controlClient = createNewSolrServer(controlJetty.getLocalPort());
     
-    createJettys(numServers);
+    createJettys(numServers, true);
     
   }
   
-  private List<JettySolrRunner> createJettys(int numJettys) throws Exception,
-      InterruptedException, TimeoutException, IOException, KeeperException,
-      URISyntaxException {
+  private List<JettySolrRunner> createJettys(int numJettys) throws Exception {
+    return createJettys(numJettys, false);
+  }
+  
+
+  /**
+   * @param numJettys
+   * @param checkCreatedVsState
+   *          if true, make sure the number created (numJettys) matches the
+   *          number in the cluster state - if you add more jetties this may not
+   *          be the case
+   * @return
+   * @throws Exception
+   */
+  private List<JettySolrRunner> createJettys(int numJettys, boolean checkCreatedVsState) throws Exception {
     List<JettySolrRunner> jettys = new ArrayList<JettySolrRunner>();
     List<SolrServer> clients = new ArrayList<SolrServer>();
     StringBuilder sb = new StringBuilder();
@@ -247,6 +263,28 @@ public class FullSolrCloudTest extends A
     this.jettys.addAll(jettys);
     this.clients.addAll(clients);
     
+    if (checkCreatedVsState) {
+      // now wait until we see that the number of shards in the cluster state
+      // matches what we expect
+      int numShards = getNumShards(DEFAULT_COLLECTION);
+      int retries = 0;
+      while (numShards != shardCount) {
+        numShards = getNumShards(DEFAULT_COLLECTION);
+        if (numShards == shardCount) break;
+        if (retries++ == 20) {
+          printLayoutOnTearDown = true;
+          fail("Shards in the state does not match what we set:" + numShards
+              + " vs " + shardCount);
+        }
+        Thread.sleep(500);
+      }
+
+      // also make sure we have a leader for each shard
+      for (int i = 1; i <= sliceCount; i++) {
+        zkStateReader.getLeaderProps(DEFAULT_COLLECTION, "shard" + i, 10000);
+      }
+    }
+
     updateMappingsFromZk(this.jettys, this.clients);
     
     // build the shard string
@@ -261,6 +299,16 @@ public class FullSolrCloudTest extends A
     
     return jettys;
   }
+
+  private int getNumShards(String defaultCollection) {
+    Map<String,Slice> slices = this.zkStateReader.getCloudState().getSlices(defaultCollection);
+    int cnt = 0;
+    for (Map.Entry<String,Slice> entry : slices.entrySet()) {
+      cnt += entry.getValue().getShards().size();
+    }
+    
+    return cnt;
+  }
   
   public JettySolrRunner createJetty(String dataDir, String shardList,
       String solrConfigOverride) throws Exception {
@@ -467,71 +515,81 @@ public class FullSolrCloudTest extends A
    */
   @Override
   public void doTest() throws Exception {
-    handle.clear();
-    handle.put("QTime", SKIPVAL);
-    handle.put("timestamp", SKIPVAL);
-    
-    indexr(id, 1, i1, 100, tlong, 100, t1, "now is the time for all good men",
-        "foo_f", 1.414f, "foo_b", "true", "foo_d", 1.414d);
-    
-    // make sure we are in a steady state...
-    waitForRecoveriesToFinish(false);
-    
-    commit();
-    
-    assertDocCounts(false);
-    
-    indexAbunchOfDocs();
-    
-    commit();
-    
-    assertDocCounts(VERBOSE);
-    checkQueries();
-    
-    assertDocCounts(VERBOSE);
-    
-    query("q", "*:*", "sort", "n_tl1 desc");
-    
-    brindDownShardIndexSomeDocsAndRecover();
-    
-    query("q", "*:*", "sort", "n_tl1 desc");
-    
-    // test adding another replica to a shard - it should do a
-    // recovery/replication to pick up the index from the leader
-    addNewReplica();
-    
-    long docId = testUpdateAndDelete();
-    
-    // index a bad doc...
+    boolean testFinished = false;
     try {
-      indexr(t1, "a doc with no id");
-      fail("this should fail");
-    } catch (SolrException e) {
-      // expected
+      handle.clear();
+      handle.put("QTime", SKIPVAL);
+      handle.put("timestamp", SKIPVAL);
+      
+      indexr(id, 1, i1, 100, tlong, 100, t1,
+          "now is the time for all good men", "foo_f", 1.414f, "foo_b", "true",
+          "foo_d", 1.414d);
+      
+      // make sure we are in a steady state...
+      waitForRecoveriesToFinish(false);
+      
+      commit();
+      
+      assertDocCounts(false);
+      
+      indexAbunchOfDocs();
+      
+      commit();
+      
+      assertDocCounts(VERBOSE);
+      checkQueries();
+      
+      assertDocCounts(VERBOSE);
+      
+      query("q", "*:*", "sort", "n_tl1 desc");
+      
+      brindDownShardIndexSomeDocsAndRecover();
+      
+      query("q", "*:*", "sort", "n_tl1 desc");
+      
+      // test adding another replica to a shard - it should do a
+      // recovery/replication to pick up the index from the leader
+      addNewReplica();
+      
+      long docId = testUpdateAndDelete();
+      
+      // index a bad doc...
+      try {
+        indexr(t1, "a doc with no id");
+        fail("this should fail");
+      } catch (SolrException e) {
+        // expected
+      }
+      
+      // TODO: bring this to it's own method?
+      // try indexing to a leader that has no replicas up
+      ZkNodeProps leaderProps = zkStateReader.getLeaderProps(
+          DEFAULT_COLLECTION, SHARD2);
+      
+      String nodeName = leaderProps.get(ZkStateReader.NODE_NAME_PROP);
+      chaosMonkey.stopShardExcept(SHARD2, nodeName);
+      
+      SolrServer client = getClient(nodeName);
+      
+      index_specific(client, "id", docId + 1, t1, "what happens here?");
+      
+      // expire a session...
+      CloudJettyRunner cloudJetty = shardToJetty.get("shard1").get(0);
+      chaosMonkey.expireSession(cloudJetty.jetty);
+      
+      indexr("id", docId + 1, t1, "slip this doc in");
+      
+      waitForRecoveriesToFinish(false);
+      
+      checkShardConsistency("shard1");
+      
+      testFinished = true;
+    } finally {
+      if (!testFinished) {
+        printLayoutOnTearDown = true;
+      }
     }
     
-    // TODO: bring this to it's own method?
-    // try indexing to a leader that has no replicas up
-    ZkNodeProps leaderProps = zkStateReader.getLeaderProps(DEFAULT_COLLECTION,
-        SHARD2);
-    
-    String nodeName = leaderProps.get(ZkStateReader.NODE_NAME_PROP);
-    chaosMonkey.stopShardExcept(SHARD2, nodeName);
-    
-    SolrServer client = getClient(nodeName);
-    
-    index_specific(client, "id", docId + 1, t1, "what happens here?");
-    
-    // expire a session...
-    CloudJettyRunner cloudJetty = shardToJetty.get("shard1").get(0);
-    chaosMonkey.expireSession(cloudJetty.jetty);
-    
-    indexr("id", docId + 1, t1, "slip this doc in");
-    
-    waitForRecoveriesToFinish(false);
-    
-    checkShardConsistency("shard1");
-    
   }
   
   private long testUpdateAndDelete() throws Exception, SolrServerException,
@@ -1182,7 +1240,7 @@ public class FullSolrCloudTest extends A
   @Override
   @After
   public void tearDown() throws Exception {
-    if (VERBOSE) {
+    if (VERBOSE || printLayoutOnTearDown) {
       super.printLayout();
     }
     ((CommonsHttpSolrServer) controlClient).shutdown();
@@ -1222,7 +1280,7 @@ public class FullSolrCloudTest extends A
           + DEFAULT_COLLECTION;
       CommonsHttpSolrServer s = new CommonsHttpSolrServer(url);
       s.setConnectionTimeout(100); // 1/10th sec
-      s.setSoTimeout(30000);
+      s.setSoTimeout(45000);
       s.setDefaultMaxConnectionsPerHost(100);
       s.setMaxTotalConnections(100);
       return s;

Modified: lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/LeaderElectionIntegrationTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/LeaderElectionIntegrationTest.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/LeaderElectionIntegrationTest.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/LeaderElectionIntegrationTest.java Fri Jan 27 18:15:52 2012
@@ -34,7 +34,6 @@ import org.apache.solr.common.cloud.ZkNo
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.CoreContainer.Initializer;
-import org.apache.solr.core.SolrConfig;
 import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -150,12 +149,12 @@ public class LeaderElectionIntegrationTe
       //printLayout(zkServer.getZkAddress());
       
       // poll until leader change is visible
-      for (int j = 0; j < 30; j++) {
+      for (int j = 0; j < 90; j++) {
         String currentLeader = getLeader();
         if(!leader.equals(currentLeader)) {
           break;
         }
-        Thread.sleep(100);
+        Thread.sleep(500);
       }
       
       leader = getLeader();
@@ -216,23 +215,12 @@ public class LeaderElectionIntegrationTe
     //Thread.sleep(100000);
   }
   
-  private String getLeader() throws InterruptedException {
-    String leader = null;
-    int tries = 30;
-    while (tries-- > 0) {
-      ZkNodeProps props;
-      try {
-        reader.updateCloudState(true);
-        props = reader.getLeaderProps("collection1", "shard1", 500);
-        leader = props.get(ZkStateReader.NODE_NAME_PROP);
-        if (leader != null) {
-          break;
-        }
-      } catch (KeeperException e) {
-        // ignore
-      }
-      Thread.sleep(200);
-    }
+  private String getLeader() throws InterruptedException, KeeperException {
+    
+    reader.updateCloudState(true);
+    ZkNodeProps props = reader.getLeaderProps("collection1", "shard1", 15000);
+    String leader = props.get(ZkStateReader.NODE_NAME_PROP);
+    
     return leader;
   }
   

Modified: lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java Fri Jan 27 18:15:52 2012
@@ -32,7 +32,6 @@ import org.apache.solr.common.cloud.Solr
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.core.SolrConfig;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.junit.AfterClass;
@@ -154,7 +153,7 @@ public class LeaderElectionTest extends 
   
   private String getLeaderUrl(final String collection, final String slice)
       throws KeeperException, InterruptedException {
-    int iterCount = 30;
+    int iterCount = 60;
     while (iterCount-- > 0)
       try {
         byte[] data = zkClient.getData(
@@ -164,7 +163,7 @@ public class LeaderElectionTest extends 
             ZkNodeProps.load(data));
         return leaderProps.getCoreUrl();
       } catch (NoNodeException e) {
-        Thread.sleep(100);
+        Thread.sleep(500);
       }
     throw new RuntimeException("Could not get leader props");
   }

Modified: lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java Fri Jan 27 18:15:52 2012
@@ -146,7 +146,7 @@ public class OverseerTest extends SolrTe
     SolrZkClient zkClient = null;
     ZkStateReader reader = null;
     final ZkController[] controllers = new ZkController[nodeCount];
-
+    final ExecutorService[] nodeExecutors = new ExecutorService[nodeCount];
     try {
       server.run();
       AbstractZkTestCase.tryCleanSolrZkNode(server.getZkHost());
@@ -174,7 +174,6 @@ public class OverseerTest extends SolrTe
           .getAbsolutePath());
 
       
-      final ExecutorService[] nodeExecutors = new ExecutorService[nodeCount];
       for (int i = 0; i < nodeCount; i++) {
         nodeExecutors[i] = Executors.newFixedThreadPool(1);
       }
@@ -232,7 +231,7 @@ public class OverseerTest extends SolrTe
       }
 
       // make sure all cores have been returned a id
-      for (int i = 0; i < 150; i++) {
+      for (int i = 0; i < 90; i++) {
         int assignedCount = 0;
         for (int j = 0; j < coreCount; j++) {
           if (ids[j] != null) {
@@ -242,7 +241,7 @@ public class OverseerTest extends SolrTe
         if (coreCount == assignedCount) {
           break;
         }
-        Thread.sleep(200);
+        Thread.sleep(500);
       }
       
       final HashMap<String, AtomicInteger> counters = new HashMap<String,AtomicInteger>();
@@ -289,6 +288,9 @@ public class OverseerTest extends SolrTe
           controllers[i].close();
         }
       server.shutdown();
+      for (int i = 0; i < nodeCount; i++) {
+        nodeExecutors[i].shutdownNow();
+      }
     }
     
     System.clearProperty(ZkStateReader.NUM_SHARDS_PROP);

Modified: lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java Fri Jan 27 18:15:52 2012
@@ -102,6 +102,7 @@ public class ZkControllerTest extends So
 
     ZkTestServer server = new ZkTestServer(zkDir);
     ZkController zkController = null;
+    boolean testFinished = false;
     try {
       server.run();
 
@@ -127,8 +128,12 @@ public class ZkControllerTest extends So
       if (DEBUG) {
         zkController.printLayoutToStdOut();
       }
-
+      testFinished = true;
     } finally {
+      if (testFinished) {
+        zkController.getZkClient().printLayoutToStdOut();
+      }
+      
       if (zkController != null) {
         zkController.close();
       }

Modified: lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/request/TestFaceting.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/request/TestFaceting.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/request/TestFaceting.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/request/TestFaceting.java Fri Jan 27 18:15:52 2012
@@ -21,6 +21,7 @@ import java.util.Locale;
 import java.util.Random;
 
 import org.apache.lucene.index.DocTermOrds;
+import org.apache.lucene.index.SlowMultiReaderWrapper;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
@@ -80,7 +81,7 @@ public class TestFaceting extends SolrTe
 
     assertEquals(size, uif.getNumTerms());
 
-    TermsEnum te = uif.getOrdTermsEnum(req.getSearcher().getIndexReader());
+    TermsEnum te = uif.getOrdTermsEnum(new SlowMultiReaderWrapper(req.getSearcher().getIndexReader()));
     assertEquals(size == 0, te == null);
 
     Random r = new Random(size);

Modified: lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/update/SoftAutoCommitTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/update/SoftAutoCommitTest.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/update/SoftAutoCommitTest.java (original)
+++ lucene/dev/branches/lucene3661/solr/core/src/test/org/apache/solr/update/SoftAutoCommitTest.java Fri Jan 27 18:15:52 2012
@@ -191,7 +191,7 @@ public class SoftAutoCommitTest extends 
     long postAdd529 = System.currentTimeMillis();
 
     // wait for first hard/soft commit
-    Long soft529 = monitor.soft.poll(softCommitWaitMillis * 2, MILLISECONDS);
+    Long soft529 = monitor.soft.poll(softCommitWaitMillis * 3, MILLISECONDS);
     assertNotNull("soft529 wasn't fast enough", soft529);
     Long manCommit = monitor.hard.poll(0, MILLISECONDS);
 

Modified: lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CommonsHttpSolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CommonsHttpSolrServer.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CommonsHttpSolrServer.java (original)
+++ lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CommonsHttpSolrServer.java Fri Jan 27 18:15:52 2012
@@ -475,10 +475,10 @@ public class CommonsHttpSolrServer exten
       return processor.processResponse(respBody, charset);
     }
     catch (HttpException e) {
-      throw new SolrServerException( e );
+      throw new SolrServerException(getBaseURL(), e);
     }
     catch (IOException e) {
-      throw new SolrServerException( e );
+      throw new SolrServerException(getBaseURL(), e);
     }
     finally {
       method.releaseConnection();

Modified: lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/common/cloud/HashPartitioner.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/common/cloud/HashPartitioner.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/common/cloud/HashPartitioner.java (original)
+++ lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/common/cloud/HashPartitioner.java Fri Jan 27 18:15:52 2012
@@ -40,7 +40,7 @@ public class HashPartitioner {
    * works up to 65537 before requested num of ranges is one short
    * 
    * @param partitions
-   * @return
+   * @return Range for each partition
    */
   public List<Range> partitionRange(int partitions) {
     // some hokey code to partition the int space

Modified: lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java (original)
+++ lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java Fri Jan 27 18:15:52 2012
@@ -34,7 +34,6 @@ import javax.xml.transform.stream.Stream
 import javax.xml.transform.stream.StreamSource;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ZkClientConnectionStrategy.ZkUpdate;
 import org.apache.zookeeper.CreateMode;
@@ -644,9 +643,6 @@ public class SolrZkClient {
    * @throws InterruptedException
    */
   public void close() throws InterruptedException {
-    if (isClosed) {
-      throw new AlreadyClosedException("This client has already been closed");
-    }
     isClosed = true;
     keeper.close();
     numCloses.incrementAndGet();

Modified: lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java (original)
+++ lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java Fri Jan 27 18:15:52 2012
@@ -54,7 +54,6 @@ public class ZkCmdExecutor {
   /**
    * Perform the given operation, retrying if the connection fails
    * 
-   * @return
    * @throws IOException 
    */
   @SuppressWarnings("unchecked")

Modified: lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/common/cloud/ZkOperation.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/common/cloud/ZkOperation.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/common/cloud/ZkOperation.java (original)
+++ lucene/dev/branches/lucene3661/solr/solrj/src/java/org/apache/solr/common/cloud/ZkOperation.java Fri Jan 27 18:15:52 2012
@@ -22,8 +22,7 @@ import java.io.IOException;
 import org.apache.zookeeper.KeeperException;
 
 /**
- * A callback object which can be used for implementing retry-able operations in the 
- * {@link org.apache.solr.common.cloud.ZkCmdExecutor.lock.ProtocolSupport} class
+ * A callback object which can be used for implementing retry-able operations.
  *
  */
 public abstract class ZkOperation {

Modified: lucene/dev/branches/lucene3661/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java (original)
+++ lucene/dev/branches/lucene3661/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java Fri Jan 27 18:15:52 2012
@@ -110,8 +110,18 @@ public abstract class SolrTestCaseJ4 ext
   static long numOpens;
   static long numCloses;
   public static void startTrackingSearchers() {
-    numOpens = SolrIndexSearcher.numOpens.get();
-    numCloses = SolrIndexSearcher.numCloses.get();
+    numOpens = SolrIndexSearcher.numOpens.getAndSet(0);
+    numCloses = SolrIndexSearcher.numCloses.getAndSet(0);
+    if (numOpens != 0 || numCloses != 0) {
+      // NOTE: some other tests don't use this base class and hence won't reset the counts.
+      log.warn("startTrackingSearchers: numOpens="+numOpens+" numCloses="+numCloses);
+      try {
+        throw new RuntimeException();
+      } catch (Exception e) {
+        log.error("",e);
+      }
+      numOpens = numCloses = 0;
+    }
   }
   static long zkClientNumOpens;
   static long zkClientNumCloses;
@@ -124,13 +134,10 @@ public abstract class SolrTestCaseJ4 ext
      long endNumOpens = SolrIndexSearcher.numOpens.get();
      long endNumCloses = SolrIndexSearcher.numCloses.get();
 
-     SolrIndexSearcher.numOpens.getAndSet(0);
-     SolrIndexSearcher.numCloses.getAndSet(0);
-
      // wait a bit in case any ending threads have anything to release
      int retries = 0;
      while (endNumOpens - numOpens != endNumCloses - numCloses) {
-       if (retries++ > 15) {
+       if (retries++ > 30) {
          break;
        }
        try {
@@ -139,7 +146,10 @@ public abstract class SolrTestCaseJ4 ext
        endNumOpens = SolrIndexSearcher.numOpens.get();
        endNumCloses = SolrIndexSearcher.numCloses.get();
      }
-     
+
+     SolrIndexSearcher.numOpens.getAndSet(0);
+     SolrIndexSearcher.numCloses.getAndSet(0);
+
      if (endNumOpens-numOpens != endNumCloses-numCloses) {
        String msg = "ERROR: SolrIndexSearcher opens=" + (endNumOpens-numOpens) + " closes=" + (endNumCloses-numCloses);
        log.error(msg);

Modified: lucene/dev/branches/lucene3661/solr/testlogging.properties
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3661/solr/testlogging.properties?rev=1236796&r1=1236795&r2=1236796&view=diff
==============================================================================
--- lucene/dev/branches/lucene3661/solr/testlogging.properties (original)
+++ lucene/dev/branches/lucene3661/solr/testlogging.properties Fri Jan 27 18:15:52 2012
@@ -1,4 +1,7 @@
 handlers=java.util.logging.ConsoleHandler
-.level=SEVERE
+java.util.logging.ConsoleHandler.level=FINEST
 java.util.logging.ConsoleHandler.formatter=java.util.logging.SimpleFormatter
 
+.level=SEVERE
+#org.apache.solr.update.UpdateLog.level=FINEST
+#org.apache.solr.update.TransactionLog.level=FINEST