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

svn commit: r1372423 [43/45] - in /lucene/dev/branches/LUCENE-2878: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/analysis/common/ dev-tools/maven/lucene/analysis/icu/ d...

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/UpdateLog.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/UpdateLog.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/UpdateLog.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/UpdateLog.java Mon Aug 13 13:52:46 2012
@@ -138,7 +138,7 @@ public class UpdateLog implements Plugin
 
   private SyncLevel defaultSyncLevel = SyncLevel.FLUSH;
 
-  private volatile UpdateHandler uhandler;    // a core reload can change this reference!
+  volatile UpdateHandler uhandler;    // a core reload can change this reference!
   private volatile boolean cancelApplyBufferUpdate;
   List<Long> startingVersions;
   int startingOperation;  // last operation in the logs on startup
@@ -177,6 +177,9 @@ public class UpdateLog implements Plugin
       if (debug) {
         log.debug("UpdateHandler init: tlogDir=" + tlogDir + ", next id=" + id, " this is a reopen... nothing else to do.");
       }
+
+      versionInfo.reload();
+
       // on a normal reopen, we currently shouldn't have to do anything
       return;
     }
@@ -209,7 +212,7 @@ public class UpdateLog implements Plugin
       if (newestLogsOnStartup.size() >= 2) break;
     }
     
-    versionInfo = new VersionInfo(uhandler, 256);
+    versionInfo = new VersionInfo(this, 256);
 
     // TODO: these startingVersions assume that we successfully recover from all non-complete tlogs.
     UpdateLog.RecentUpdates startingUpdates = getRecentUpdates();
@@ -1045,6 +1048,7 @@ public class UpdateLog implements Plugin
     try {
       cancelApplyBufferUpdate = false;
       if (state != State.BUFFERING) return null;
+      operationFlags &= ~FLAG_GAP;
 
       // handle case when no log was even created because no updates
       // were received.
@@ -1054,7 +1058,6 @@ public class UpdateLog implements Plugin
       }
       tlog.incref();
       state = State.APPLYING_BUFFERED;
-      operationFlags &= ~FLAG_GAP;
     } finally {
       versionInfo.unblockUpdates();
     }

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/VersionInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/VersionInfo.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/VersionInfo.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/VersionInfo.java Mon Aug 13 13:52:46 2012
@@ -19,7 +19,6 @@ package org.apache.solr.update;
 
 import java.io.IOException;
 import java.util.Map;
-import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
@@ -36,16 +35,15 @@ import org.apache.solr.util.RefCounted;
 public class VersionInfo {
   public static final String VERSION_FIELD="_version_";
 
-  private SolrCore core;
-  private UpdateHandler updateHandler;
+  private final UpdateLog ulog;
   private final VersionBucket[] buckets;
   private SchemaField versionField;
   private SchemaField idField;
   final ReadWriteLock lock = new ReentrantReadWriteLock(true);
 
-  public VersionInfo(UpdateHandler updateHandler, int nBuckets) {
-    this.updateHandler = updateHandler;
-    this.core = updateHandler.core;
+  public VersionInfo(UpdateLog ulog, int nBuckets) {
+    this.ulog = ulog;
+    SolrCore core = ulog.uhandler.core;
     versionField = core.getSchema().getFieldOrNull(VERSION_FIELD);
     idField = core.getSchema().getUniqueKeyField();
     buckets = new VersionBucket[ BitUtil.nextHighestPowerOfTwo(nBuckets) ];
@@ -54,6 +52,10 @@ public class VersionInfo {
     }
   }
 
+  public void reload() {
+
+  }
+
   public SchemaField getVersionField() {
     return versionField;
   }
@@ -143,14 +145,14 @@ public class VersionInfo {
   }
 
   public Long lookupVersion(BytesRef idBytes) {
-    return updateHandler.ulog.lookupVersion(idBytes);
+    return ulog.lookupVersion(idBytes);
   }
 
   public Long getVersionFromIndex(BytesRef idBytes) {
     // TODO: we could cache much of this and invalidate during a commit.
     // TODO: most DocValues classes are threadsafe - expose which.
 
-    RefCounted<SolrIndexSearcher> newestSearcher = core.getRealtimeSearcher();
+    RefCounted<SolrIndexSearcher> newestSearcher = ulog.uhandler.core.getRealtimeSearcher();
     try {
       SolrIndexSearcher searcher = newestSearcher.get();
       long lookup = searcher.lookupId(idBytes);

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java Mon Aug 13 13:52:46 2012
@@ -19,9 +19,12 @@ package org.apache.solr.update.processor
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
 
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
@@ -33,7 +36,7 @@ import org.apache.solr.common.SolrExcept
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.SolrInputField;
-import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -130,6 +133,8 @@ public class DistributedUpdateProcessor 
   private boolean forwardToLeader = false;
   private List<Node> nodes;
 
+  private int numNodes;
+
   
   public DistributedUpdateProcessor(SolrQueryRequest req,
       SolrQueryResponse rsp, UpdateRequestProcessor next) {
@@ -153,18 +158,21 @@ public class DistributedUpdateProcessor 
     CoreDescriptor coreDesc = req.getCore().getCoreDescriptor();
     
     this.zkEnabled  = coreDesc.getCoreContainer().isZooKeeperAware();
+    zkController = req.getCore().getCoreDescriptor().getCoreContainer().getZkController();
+    if (zkEnabled) {
+      numNodes =  zkController.getZkStateReader().getClusterState().getLiveNodes().size();
+    }
     //this.rsp = reqInfo != null ? reqInfo.getRsp() : null;
 
-    
-    zkController = req.getCore().getCoreDescriptor().getCoreContainer().getZkController();
+   
     
     cloudDesc = coreDesc.getCloudDescriptor();
     
     if (cloudDesc != null) {
       collection = cloudDesc.getCollectionName();
     }
-    
-    cmdDistrib = new SolrCmdDistributor();
+
+    cmdDistrib = new SolrCmdDistributor(numNodes);
   }
 
   private List<Node> setupRequest(int hash) {
@@ -172,11 +180,14 @@ public class DistributedUpdateProcessor 
 
     // if we are in zk mode...
     if (zkEnabled) {
+      // set num nodes
+      numNodes = zkController.getClusterState().getLiveNodes().size();
+      
       // the leader is...
       // TODO: if there is no leader, wait and look again
       // TODO: we are reading the leader from zk every time - we should cache
       // this and watch for changes?? Just pull it from ZkController cluster state probably?
-      String shardId = getShard(hash, collection, zkController.getCloudState()); // get the right shard based on the hash...
+      String shardId = getShard(hash, collection, zkController.getClusterState()); // get the right shard based on the hash...
 
       try {
         // TODO: if we find out we cannot talk to zk anymore, we should probably realize we are not
@@ -204,8 +215,22 @@ public class DistributedUpdateProcessor 
                   coreName, null, ZkStateReader.DOWN);
           if (replicaProps != null) {
             nodes = new ArrayList<Node>(replicaProps.size());
+            // check for test param that lets us miss replicas
+            String[] skipList = req.getParams().getParams("test.distrib.skip.servers");
+            Set<String> skipListSet = null;
+            if (skipList != null) {
+              skipListSet = new HashSet<String>(skipList.length);
+              skipListSet.addAll(Arrays.asList(skipList));
+            }
+            
             for (ZkCoreNodeProps props : replicaProps) {
-              nodes.add(new StdNode(props));
+              if (skipList != null) {
+                if (!skipListSet.contains(props.getCoreUrl())) {
+                  nodes.add(new StdNode(props));
+                }
+              } else {
+                nodes.add(new StdNode(props));
+              }
             }
           }
           
@@ -227,11 +252,11 @@ public class DistributedUpdateProcessor 
   }
 
 
-  private String getShard(int hash, String collection, CloudState cloudState) {
+  private String getShard(int hash, String collection, ClusterState clusterState) {
     // ranges should be part of the cloud state and eventually gotten from zk
 
     // get the shard names
-    return cloudState.getShard(hash, collection);
+    return clusterState.getShard(hash, collection);
   }
 
   // used for deleteByQuery to get the list of nodes this leader should forward to
@@ -673,11 +698,11 @@ public class DistributedUpdateProcessor 
     if (zkEnabled && DistribPhase.NONE == phase) {
       boolean leaderForAnyShard = false;  // start off by assuming we are not a leader for any shard
 
-      Map<String,Slice> slices = zkController.getCloudState().getSlices(collection);
+      Map<String,Slice> slices = zkController.getClusterState().getSlices(collection);
       if (slices == null) {
         throw new SolrException(ErrorCode.BAD_REQUEST,
             "Cannot find collection:" + collection + " in "
-                + zkController.getCloudState().getCollections());
+                + zkController.getClusterState().getCollections());
       }
 
       ModifiableSolrParams params = new ModifiableSolrParams(req.getParams());
@@ -726,7 +751,10 @@ public class DistributedUpdateProcessor 
 
     if (zkEnabled && DistribPhase.TOLEADER == phase) {
       // This core should be a leader
+      isLeader = true;
       replicas = setupRequest();
+    } else if (DistribPhase.FROMLEADER == phase) {
+      isLeader = false;
     }
 
     if (vinfo == null) {
@@ -969,13 +997,13 @@ public class DistributedUpdateProcessor 
 
   
   private List<Node> getCollectionUrls(SolrQueryRequest req, String collection, String shardZkNodeName) {
-    CloudState cloudState = req.getCore().getCoreDescriptor()
-        .getCoreContainer().getZkController().getCloudState();
+    ClusterState clusterState = req.getCore().getCoreDescriptor()
+        .getCoreContainer().getZkController().getClusterState();
     List<Node> urls = new ArrayList<Node>();
-    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: " + cloudState);
+          "Could not find collection in zk: " + clusterState);
     }
     for (Map.Entry<String,Slice> sliceEntry : slices.entrySet()) {
       Slice replicas = slices.get(sliceEntry.getKey());
@@ -984,7 +1012,7 @@ public class DistributedUpdateProcessor 
       
       for (Entry<String,ZkNodeProps> entry : shardMap.entrySet()) {
         ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(entry.getValue());
-        if (cloudState.liveNodesContain(nodeProps.getNodeName()) && !entry.getKey().equals(shardZkNodeName)) {
+        if (clusterState.liveNodesContain(nodeProps.getNodeName()) && !entry.getKey().equals(shardZkNodeName)) {
           urls.add(new StdNode(nodeProps));
         }
       }

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessorFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessorFactory.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessorFactory.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessorFactory.java Mon Aug 13 13:52:46 2012
@@ -21,6 +21,11 @@ import org.apache.solr.common.util.Named
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 
+/**
+ * Factory for {@link DistributedUpdateProcessor}.
+ *
+ * @see DistributedUpdateProcessor
+ */
 public class DistributedUpdateProcessorFactory 
   extends UpdateRequestProcessorFactory 
   implements DistributingUpdateProcessorFactory {

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/DistributingUpdateProcessorFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/DistributingUpdateProcessorFactory.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/DistributingUpdateProcessorFactory.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/DistributingUpdateProcessorFactory.java Mon Aug 13 13:52:46 2012
@@ -23,7 +23,8 @@ import org.apache.solr.common.SolrExcept
  * A marker interface for denoting that a factory is responsible for handling
  * distributed communication of updates across a SolrCloud cluster.
  * 
- * @see UpdateRequestProcessorChain
+ * @see UpdateRequestProcessorChain#init
+ * @see UpdateRequestProcessorChain#createProcessor
  */
 public interface DistributingUpdateProcessorFactory {
 

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/LogUpdateProcessorFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/LogUpdateProcessorFactory.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/LogUpdateProcessorFactory.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/LogUpdateProcessorFactory.java Mon Aug 13 13:52:46 2012
@@ -35,12 +35,15 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
+ * <p>
  * A logging processor.  This keeps track of all commands that have passed through
  * the chain and prints them on finish().  At the Debug (FINE) level, a message
  * will be logged for each command prior to the next stage in the chain.
- * 
- * If the Log level is not >= INFO the processor will not be created or added to the chain.
- * 
+ * </p>
+ * <p>
+ * If the Log level is not &gt;= INFO the processor will not be created or added to the chain.
+ * </p>
+ *
  * @since solr 1.3
  */
 public class LogUpdateProcessorFactory extends UpdateRequestProcessorFactory {

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/RunUpdateProcessorFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/RunUpdateProcessorFactory.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/RunUpdateProcessorFactory.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/RunUpdateProcessorFactory.java Mon Aug 13 13:52:46 2012
@@ -25,9 +25,14 @@ import org.apache.solr.update.*;
 
 
 /**
- * Pass the command to the UpdateHandler without any modifications
+ * Executes the update commands using the underlying UpdateHandler.
+ * Allmost all processor chains should end with an instance of 
+ * <code>RunUpdateProcessorFactory</code> unless the user is explicitly 
+ * executing the update commands in an alternative custom 
+ * <code>UpdateRequestProcessorFactory</code>
  * 
  * @since solr 1.3
+ * @see DistributingUpdateProcessorFactory
  */
 public class RunUpdateProcessorFactory extends UpdateRequestProcessorFactory 
 {

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java Mon Aug 13 13:52:46 2012
@@ -305,16 +305,22 @@ public class StatelessScriptUpdateProces
       }
 
       scriptEngines.add(new EngineInfo((Invocable)engine, scriptFile));
-      Reader scriptSrc = scriptFile.openReader(resourceLoader);
-
       try {
-        engine.eval(scriptSrc);
-      } catch (ScriptException e) {
+        Reader scriptSrc = scriptFile.openReader(resourceLoader);
+  
+        try {
+          engine.eval(scriptSrc);
+        } catch (ScriptException e) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
+                                  "Unable to evaluate script: " + 
+                                  scriptFile.getFileName(), e);
+        } finally {
+          IOUtils.closeQuietly(scriptSrc);
+        }
+      } catch (IOException ioe) {
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
-                                "Unable to evaluate script: " + 
-                                scriptFile.getFileName(), e);
-      } finally {
-        IOUtils.closeQuietly(scriptSrc);
+            "Unable to evaluate script: " + 
+            scriptFile.getFileName(), ioe);        
       }
     }
     return scriptEngines;
@@ -485,7 +491,7 @@ public class StatelessScriptUpdateProces
       return extension;
     }
 
-    public Reader openReader(SolrResourceLoader resourceLoader) {
+    public Reader openReader(SolrResourceLoader resourceLoader) throws IOException {
       InputStream input = resourceLoader.openResource(fileName);
       return org.apache.lucene.util.IOUtils.getDecodingReader
         (input, org.apache.lucene.util.IOUtils.CHARSET_UTF_8);

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/UpdateRequestProcessorChain.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/UpdateRequestProcessorChain.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/UpdateRequestProcessorChain.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/processor/UpdateRequestProcessorChain.java Mon Aug 13 13:52:46 2012
@@ -36,7 +36,7 @@ import java.util.ArrayList;
  * <p>
  * Chain can be configured via solrconfig.xml:
  * </p>
- * <pre>
+ * <pre class="prettyprint">
  * &lt;updateRequestProcessors name="key" default="true"&gt;
  *   &lt;processor class="PathToClass1" /&gt;
  *   &lt;processor class="PathToClass2" /&gt;
@@ -46,9 +46,16 @@ import java.util.ArrayList;
  *   &lt;processor class="solr.RunUpdateProcessorFactory" /&gt;
  * &lt;/updateRequestProcessors&gt;
  * </pre>
+ * <p>
+ * Allmost all processor chains should end with an instance of 
+ * {@link RunUpdateProcessorFactory} unless the user is explicitly 
+ * executing the update commands in an alternative custom 
+ * <code>UpdateRequestProcessorFactory</code>.
+ * </p>
  *
  * @see UpdateRequestProcessorFactory
  * @see #init
+ * @see #createProcessor
  * @since solr 1.3
  */
 public final class UpdateRequestProcessorChain implements PluginInfoInitialized

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/util/FastWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/util/FastWriter.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/util/FastWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/util/FastWriter.java Mon Aug 13 13:52:46 2012
@@ -52,7 +52,7 @@ public class FastWriter extends Writer {
 
   public void write(char c) throws IOException {
     if (pos >= buf.length) {
-      sink.write(buf,0,pos);
+      flush(buf,0,pos);
       pos=0;
     }
     buf[pos++] = c;
@@ -61,7 +61,7 @@ public class FastWriter extends Writer {
   @Override
   public FastWriter append(char c) throws IOException {
     if (pos >= buf.length) {
-      sink.write(buf,0,pos);
+      flush(buf,0,pos);
       pos=0;
     }
     buf[pos++] = c;
@@ -77,14 +77,14 @@ public class FastWriter extends Writer {
     } else if (len<BUFSIZE) {
       // if the data to write is small enough, buffer it.
       System.arraycopy(cbuf, off, buf, pos, space);
-      sink.write(buf, 0, buf.length);
+      flush(buf, 0, buf.length);
       pos = len-space;
       System.arraycopy(cbuf, off+space, buf, 0, pos);
     } else {
-      sink.write(buf,0,pos);  // flush
+      flush(buf,0,pos);  // flush
       pos=0;
       // don't buffer, just write to sink
-      sink.write(cbuf, off, len);
+      flush(cbuf, off, len);
     }
   }
 
@@ -97,32 +97,40 @@ public class FastWriter extends Writer {
     } else if (len<BUFSIZE) {
       // if the data to write is small enough, buffer it.
       str.getChars(off, off+space, buf, pos);
-      sink.write(buf, 0, buf.length);
+      flush(buf, 0, buf.length);
       str.getChars(off+space, off+len, buf, 0);
       pos = len-space;
     } else {
-      sink.write(buf,0,pos);  // flush
+      flush(buf,0,pos);  // flush
       pos=0;
       // don't buffer, just write to sink
-      sink.write(str, off, len);
+      flush(str, off, len);
     }
   }
 
   @Override
   public void flush() throws IOException {
-    sink.write(buf,0,pos);
+    flush(buf, 0, pos);
     pos=0;
-    sink.flush();
+    if (sink != null) sink.flush();
+  }
+
+  public void flush(char[] buf, int offset, int len) throws IOException {
+    sink.write(buf, offset, len);
+  }
+
+  public void flush(String str, int offset, int len) throws IOException {
+    sink.write(str, offset, len);
   }
 
   @Override
   public void close() throws IOException {
     flush();
-    sink.close();
+    if (sink != null) sink.close();
   }
 
   public void flushBuffer() throws IOException {
-    sink.write(buf, 0, pos);
+    flush(buf, 0, pos);
     pos=0;
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/util/FileUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/util/FileUtils.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/util/FileUtils.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/util/FileUtils.java Mon Aug 13 13:52:46 2012
@@ -54,7 +54,7 @@ public class FileUtils {
   }
 
   /**
-   * Copied from Lucene's FSDirectory.fsync(String) <!-- protected -->
+   * Copied from Lucene's FSDirectory.fsync(String)
    *
    * @param fullFile the File to be synced to disk
    * @throws IOException if the file could not be synced

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/util/SimplePostTool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/util/SimplePostTool.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/util/SimplePostTool.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/util/SimplePostTool.java Mon Aug 13 13:52:46 2012
@@ -18,123 +18,170 @@ package org.apache.solr.util;
  */
 
 import java.io.File;
+import java.io.FileFilter;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.ByteArrayInputStream;
 import java.io.OutputStream;
 import java.io.UnsupportedEncodingException;
+import java.util.Locale;
+import java.util.HashMap;
 import java.util.Set;
 import java.util.HashSet;
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
 import java.net.HttpURLConnection;
 import java.net.MalformedURLException;
 import java.net.ProtocolException;
 import java.net.URL;
+import java.net.URLEncoder;
 
 /**
  * A simple utility class for posting raw updates to a Solr server, 
  * has a main method so it can be run on the command line.
- * 
  */
 public class SimplePostTool {
   public static final String DEFAULT_POST_URL = "http://localhost:8983/solr/update";
-  public static final String VERSION_OF_THIS_TOOL = "1.4";
+  public static final String VERSION_OF_THIS_TOOL = "1.5";
 
   private static final String DEFAULT_COMMIT = "yes";
   private static final String DEFAULT_OPTIMIZE = "no";
   private static final String DEFAULT_OUT = "no";
+  private static final String DEFAULT_AUTO = "no";
+  private static final String DEFAULT_RECURSIVE = "no";
 
-  public static final String DEFAULT_DATA_TYPE = "application/xml";
+  private static final String DEFAULT_CONTENT_TYPE = "application/xml";
+  private static final String DEFAULT_FILE_TYPES = "xml,json,csv,pdf,doc,docx,ppt,pptx,xls,xlsx,odt,odp,ods,ott,otp,ots,rtf,htm,html,txt,log"; 
 
   private static final String DATA_MODE_FILES = "files";
   private static final String DATA_MODE_ARGS = "args";
   private static final String DATA_MODE_STDIN = "stdin";
   private static final String DEFAULT_DATA_MODE = DATA_MODE_FILES;
 
+  private static final String TRUE_STRINGS = "true,on,yes,1"; 
+
+  private boolean auto = false;
+  private boolean recursive = false;
+  private String fileTypes;
+  
+  private static HashMap<String,String> mimeMap;
+  private GlobFileFilter globFileFilter;
+  
   private static final Set<String> DATA_MODES = new HashSet<String>();
+  private static final String USAGE_STRING_SHORT =
+      "Usage: java [SystemProperties] -jar post.jar [-h|-] [<file|folder|arg> [<file|folder|arg>...]]";
+
   static {
     DATA_MODES.add(DATA_MODE_FILES);
     DATA_MODES.add(DATA_MODE_ARGS);
     DATA_MODES.add(DATA_MODE_STDIN);
+    
+    mimeMap = new HashMap<String,String>();
+    mimeMap.put("xml", "text/xml");
+    mimeMap.put("csv", "text/csv");
+    mimeMap.put("json", "application/json");
+    mimeMap.put("pdf", "application/pdf");
+    mimeMap.put("rtf", "text/rtf");
+    mimeMap.put("html", "text/html");
+    mimeMap.put("htm", "text/html");
+    mimeMap.put("doc", "application/msword");
+    mimeMap.put("docx", "application/vnd.openxmlformats-officedocument.wordprocessingml.document");
+    mimeMap.put("ppt", "application/vnd.ms-powerpoint");
+    mimeMap.put("pptx", "application/vnd.openxmlformats-officedocument.presentationml.presentation");
+    mimeMap.put("xls", "application/vnd.ms-excel");
+    mimeMap.put("xlsx", "application/vnd.openxmlformats-officedocument.spreadsheetml.sheet");
+    mimeMap.put("odt", "application/vnd.oasis.opendocument.text");
+    mimeMap.put("ott", "application/vnd.oasis.opendocument.text");
+    mimeMap.put("odp", "application/vnd.oasis.opendocument.presentation");
+    mimeMap.put("otp", "application/vnd.oasis.opendocument.presentation");
+    mimeMap.put("ods", "application/vnd.oasis.opendocument.spreadsheet");
+    mimeMap.put("ots", "application/vnd.oasis.opendocument.spreadsheet");
+    mimeMap.put("txt", "text/plain");
+    mimeMap.put("log", "text/plain");
   }
 
   protected URL solrUrl;
-
+  
   public static void main(String[] args) {
-    info("version " + VERSION_OF_THIS_TOOL);
+    info("SimplePostTool version " + VERSION_OF_THIS_TOOL);
 
     if (0 < args.length && ("-help".equals(args[0]) || "--help".equals(args[0]) || "-h".equals(args[0]))) {
-      System.out.println
-        ("This is a simple command line tool for POSTing raw data to a Solr\n"+
-         "port.  Data can be read from files specified as commandline args,\n"+
-         "as raw commandline arg strings, or via STDIN.\n"+
-         "Examples:\n"+
-         "  java -jar post.jar *.xml\n"+
-         "  java -Ddata=args  -jar post.jar '<delete><id>42</id></delete>'\n"+
-         "  java -Ddata=stdin -jar post.jar < hd.xml\n"+
-         "  java -Dtype=text/csv -jar post.jar *.csv\n"+
-         "  java -Dtype=application/json -jar post.jar *.json\n"+
-         "  java -Durl=http://localhost:8983/solr/update/extract?literal.id=a -Dtype=application/pdf -jar post.jar a.pdf\n"+
-         "Other options controlled by System Properties include the Solr\n"+
-         "URL to POST to, the Content-Type of the data, whether a commit\n"+
-         "or optimize should be executed, and whether the response should\n"+
-         "be written to STDOUT. These are the defaults for all System Properties:\n"+
-         "  -Ddata=" + DEFAULT_DATA_MODE + "\n"+
-         "  -Dtype=" + DEFAULT_DATA_TYPE + "\n"+
-         "  -Durl=" + DEFAULT_POST_URL + "\n"+
-         "  -Dcommit=" + DEFAULT_COMMIT + "\n"+
-         "  -Doptimize=" + DEFAULT_OPTIMIZE + "\n"+
-         "  -Dout=" + DEFAULT_OUT + "\n");
+      usage();
       return;
     }
-
+    
     OutputStream out = null;
-    final String type = System.getProperty("type", DEFAULT_DATA_TYPE);
+    final String type = System.getProperty("type");
+
+    final String params = System.getProperty("params", "");
 
     URL u = null;
     try {
-      u = new URL(System.getProperty("url", DEFAULT_POST_URL));
+      u = new URL(System.getProperty("url", SimplePostTool.appendParam(DEFAULT_POST_URL, params)));
     } catch (MalformedURLException e) {
       fatal("System Property 'url' is not a valid URL: " + u);
     }
     final SimplePostTool t = new SimplePostTool(u);
 
+    if (isOn(System.getProperty("auto", DEFAULT_AUTO))) {
+      t.setAuto(true);
+    }
+    
+    if (isOn(System.getProperty("recursive", DEFAULT_RECURSIVE))) {
+      t.setRecursive(true);
+    }
+
     final String mode = System.getProperty("data", DEFAULT_DATA_MODE);
     if (! DATA_MODES.contains(mode)) {
       fatal("System Property 'data' is not valid for this tool: " + mode);
     }
 
-    if ("yes".equals(System.getProperty("out", DEFAULT_OUT))) {
+    if (isOn(System.getProperty("out", DEFAULT_OUT))) {
       out = System.out;
     }
+    
+    t.setFileTypes(System.getProperty("filetypes", DEFAULT_FILE_TYPES));
 
+    int numFilesPosted = 0;
+    
     try {
       if (DATA_MODE_FILES.equals(mode)) {
         if (0 < args.length) {
-          info("POSTing files to " + u + "..");
-          t.postFiles(args, 0, out, type);
+          // Skip posting files if special param "-" given  
+          if (!args[0].equals("-")) {
+            info("Posting files to base url " + u + (!t.auto?" using content-type "+(type==null?DEFAULT_CONTENT_TYPE:type):"")+"..");
+            if(t.auto)
+              info("Entering auto mode. File endings considered are "+t.getFileTypes());
+            if(t.recursive)
+              info("Entering recursive mode"); 
+            numFilesPosted = t.postFiles(args, 0, out, type);
+            info(numFilesPosted + " files indexed.");
+          }
         } else {
-          info("No files specified. (Use -h for help)");
+            usageShort();
+            return;
         }
-        
       } else if (DATA_MODE_ARGS.equals(mode)) {
         if (0 < args.length) {
           info("POSTing args to " + u + "..");
           for (String a : args) {
             t.postData(SimplePostTool.stringToStream(a), null, out, type);
           }
+        } else {
+          usageShort();
+          return;
         }
-        
       } else if (DATA_MODE_STDIN.equals(mode)) {
         info("POSTing stdin to " + u + "..");
         t.postData(System.in, null, out, type);
       }
-      if ("yes".equals(System.getProperty("commit",DEFAULT_COMMIT))) {
-        info("COMMITting Solr index changes..");
+      if (isOn(System.getProperty("commit",DEFAULT_COMMIT))) {
+        info("COMMITting Solr index changes to " + u + "..");
         t.commit();
       }
-      if ("yes".equals(System.getProperty("optimize",DEFAULT_OPTIMIZE))) {
-        info("Performing an OPTIMIZE..");
+      if (isOn(System.getProperty("optimize",DEFAULT_OPTIMIZE))) {
+        info("Performing an OPTIMIZE to " + u + "..");
         t.optimize();
       }
     
@@ -144,37 +191,115 @@ public class SimplePostTool {
     }
   }
 
-  /**
-   * @deprecated use {@link #postData(InputStream, Integer, OutputStream, String)} instead
-   */
-  @Deprecated
-  int postFiles(String [] args,int startIndexInArgs, OutputStream out) {
-    final String type = System.getProperty("type", DEFAULT_DATA_TYPE);
-    return postFiles(args, startIndexInArgs, out, type);
-  }
-  
-  /** Post all filenames provided in args, return the number of files posted*/
+  private static void usageShort() {
+    System.out.println(USAGE_STRING_SHORT+"\n"+
+        "       Please invoke with -h option for extended usage help.");
+  }
+
+  private static void usage() {
+    System.out.println
+    (USAGE_STRING_SHORT+"\n\n" +
+     "Supported System Properties and their defaults:\n"+
+     "  -Ddata=files|args|stdin (default=" + DEFAULT_DATA_MODE + ")\n"+
+     "  -Dtype=<content-type> (default=" + DEFAULT_CONTENT_TYPE + ")\n"+
+     "  -Durl=<solr-update-url> (default=" + DEFAULT_POST_URL + ")\n"+
+     "  -Dauto=yes|no (default=" + DEFAULT_AUTO + ")\n"+
+     "  -Drecursive=yes|no (default=" + DEFAULT_RECURSIVE + ")\n"+
+     "  -Dfiletypes=<type>[,<type>,...] (default=" + DEFAULT_FILE_TYPES + ")\n"+
+     "  -Dparams=\"<key>=<value>[&<key>=<value>...]\" (values must be URL-encoded)\n"+
+     "  -Dcommit=yes|no (default=" + DEFAULT_COMMIT + ")\n"+
+     "  -Doptimize=yes|no (default=" + DEFAULT_OPTIMIZE + ")\n"+
+     "  -Dout=yes|no (default=" + DEFAULT_OUT + ")\n\n"+
+     "This is a simple command line tool for POSTing raw data to a Solr\n"+
+     "port.  Data can be read from files specified as commandline args,\n"+
+     "as raw commandline arg strings, or via STDIN.\n"+
+     "Examples:\n"+
+     "  java -jar post.jar *.xml\n"+
+     "  java -Ddata=args  -jar post.jar '<delete><id>42</id></delete>'\n"+
+     "  java -Ddata=stdin -jar post.jar < hd.xml\n"+
+     "  java -Dtype=text/csv -jar post.jar *.csv\n"+
+     "  java -Dtype=application/json -jar post.jar *.json\n"+
+     "  java -Durl=http://localhost:8983/solr/update/extract -Dparams=literal.id=a -Dtype=application/pdf -jar post.jar a.pdf\n"+
+     "  java -Dauto -jar post.jar *\n"+
+     "  java -Dauto -Drecursive -jar post.jar afolder\n"+
+     "  java -Dauto -Dfiletypes=ppt,html -jar post.jar afolder\n"+
+     "The options controlled by System Properties include the Solr\n"+
+     "URL to POST to, the Content-Type of the data, whether a commit\n"+
+     "or optimize should be executed, and whether the response should\n"+
+     "be written to STDOUT. If auto=yes the tool will try to set type\n"+
+     "and url automatically from file name. When posting rich documents\n"+
+     "the file name will be propagated as \"resource.name\" and also used as \"literal.id\".\n" +
+     "You may override these or any other request parameter through the -Dparams property.\n"+
+     "If you want to do a commit only, use \"-\" as argument.");
+  }
+
+  private static boolean isOn(String property) {
+    return(TRUE_STRINGS.indexOf(property) >= 0);
+  }
+
+  /** Post all filenames provided in args
+   * @param args array of file names
+   * @param startIndexInArgs offset to start
+   * @param out output stream to post data to
+   * @param type default content-type to use when posting (may be overridden in auto mode)
+   * @return number of files posted
+   * */
   int postFiles(String [] args,int startIndexInArgs, OutputStream out, String type) {
     int filesPosted = 0;
     for (int j = startIndexInArgs; j < args.length; j++) {
       File srcFile = new File(args[j]);
-      if (srcFile.canRead()) {
-        info("POSTing file " + srcFile.getName());
-        postFile(srcFile, out, type);
-        filesPosted++;
+      if(srcFile.isDirectory() && srcFile.canRead()) {
+        filesPosted += postDirectory(srcFile, out, type);
+      } else if (srcFile.isFile() && srcFile.canRead()) {
+        filesPosted += postFiles(new File[] {srcFile}, out, type);
       } else {
-        warn("Cannot read input file: " + srcFile);
+        File parent = srcFile.getParentFile();
+        if(parent == null) parent = new File(".");
+        String fileGlob = srcFile.getName();
+        GlobFileFilter ff = new GlobFileFilter(fileGlob, false);
+        File[] files = parent.listFiles(ff);
+        if(files.length == 0) {
+          warn("No files or directories matching "+srcFile);
+          continue;          
+        }
+        filesPosted += postFiles(parent.listFiles(ff), out, type);
       }
     }
     return filesPosted;
   }
   
+  private int postDirectory(File dir, OutputStream out, String type) {
+    if(dir.isHidden() && !dir.getName().equals("."))
+      return(0);
+    info("Indexing directory "+dir.getPath());
+    int posted = 0;
+    posted += postFiles(dir.listFiles(globFileFilter), out, type);
+    if(recursive) {
+      for(File d : dir.listFiles()) {
+        if(d.isDirectory())
+          posted += postDirectory(d, out, type);
+      }
+    }
+    return posted;
+  }
+
+  int postFiles(File[] files, OutputStream out, String type) {
+    int filesPosted = 0;
+    for(File srcFile : files) {
+      if(!srcFile.isFile() || srcFile.isHidden())
+        continue;
+      postFile(srcFile, out, type);
+      filesPosted++;
+    }
+    return filesPosted;
+  }
+
   static void warn(String msg) {
     System.err.println("SimplePostTool: WARNING: " + msg);
   }
 
   static void info(String msg) {
-    System.out.println("SimplePostTool: " + msg);
+    System.out.println(msg);
   }
 
   static void fatal(String msg) {
@@ -204,30 +329,58 @@ public class SimplePostTool {
     doGet(appendParam(solrUrl.toString(), "optimize=true"));
   }
 
-  private String appendParam(String url, String param) {
-    return url + (url.indexOf('?')>0 ? "&" : "?") + param;
+  public static String appendParam(String url, String param) {
+    String[] pa = param.split("&");
+    for(String p : pa) {
+      if(p.trim().length() == 0) continue;
+      String[] kv = p.split("=");
+      if(kv.length == 2) {
+        url = url + (url.indexOf('?')>0 ? "&" : "?") + kv[0] +"="+ kv[1];
+      } else {
+        warn("Skipping param "+p+" which is not on form key=value");
+      }
+    }
+    return url;
   }
 
   /**
-   * @deprecated use {@link #postFile(File, OutputStream, String)} instead
-   */
-  public void postFile(File file, OutputStream output) {
-    final String type = System.getProperty("type", DEFAULT_DATA_TYPE);
-    postFile(file, output, type);
-  }
-  
-  /**
    * Opens the file and posts it's contents to the solrUrl,
    * writes to response to output. 
    */
   public void postFile(File file, OutputStream output, String type) {
-
     InputStream is = null;
     try {
+      URL url = solrUrl;
+      if(auto) {
+        if(type == null) {
+          type = guessType(file);
+        }
+        if(type != null) {
+          if(type.equals("text/xml") || type.equals("text/csv") || type.equals("application/json")) {
+            // Default handler
+          } else {
+            // SolrCell
+            String urlStr = url.getProtocol() + "://" + url.getAuthority() + url.getPath() + "/extract" + (url.getQuery() != null ? "?"+url.getQuery() : "");
+            if(urlStr.indexOf("resource.name")==-1)
+              urlStr = appendParam(urlStr, "resource.name=" + URLEncoder.encode(file.getAbsolutePath(), "UTF-8"));
+            if(urlStr.indexOf("literal.id")==-1)
+              urlStr = appendParam(urlStr, "literal.id=" + URLEncoder.encode(file.getAbsolutePath(), "UTF-8"));
+            url = new URL(urlStr);
+//            info("Indexing to ExtractingRequestHandler with URL "+url);
+          }
+        } else {
+          warn("Skipping "+file.getName()+". Unsupported file type for auto mode.");
+          return;
+        }
+      } else {
+        if(type == null) type = DEFAULT_CONTENT_TYPE;
+      }
+      info("POSTing file " + file.getName() + (auto?" ("+type+")":""));
       is = new FileInputStream(file);
-      postData(is, (int)file.length(), output, type);
+      postData(is, (int)file.length(), output, type, url);
     } catch (IOException e) {
-      fatal("Can't open/read file: " + file);
+      e.printStackTrace();
+      warn("Can't open/read file: " + file);
     } finally {
       try {
         if(is!=null) is.close();
@@ -237,53 +390,53 @@ public class SimplePostTool {
     }
   }
 
+  private String guessType(File file) {
+    String name = file.getName();
+    String suffix = name.substring(name.lastIndexOf(".")+1);
+    return mimeMap.get(suffix.toLowerCase(Locale.ROOT));
+  }
+
   /**
    * Performs a simple get on the given URL
-   * @param url
    */
   public static void doGet(String url) {
     try {
       doGet(new URL(url));
     } catch (MalformedURLException e) {
-      fatal("The specified URL "+url+" is not a valid URL. Please check");
+      warn("The specified URL "+url+" is not a valid URL. Please check");
     }
   }
   
   /**
    * Performs a simple get on the given URL
-   * @param url
    */
   public static void doGet(URL url) {
     try {
       HttpURLConnection urlc = (HttpURLConnection) url.openConnection();
       if (HttpURLConnection.HTTP_OK != urlc.getResponseCode()) {
-        fatal("Solr returned an error #" + urlc.getResponseCode() + 
-            " " + urlc.getResponseMessage());
+        warn("Solr returned an error #" + urlc.getResponseCode() + 
+            " " + urlc.getResponseMessage() + " for url "+url);
       }
     } catch (IOException e) {
-      fatal("An error occured posting data to "+url+". Please check that Solr is running.");
+      warn("An error occured posting data to "+url+". Please check that Solr is running.");
     }
   }
 
-  /**
-   * @deprecated use {@link #postData(InputStream, Integer, OutputStream, String)} instead
-   */
-  @Deprecated
-  public void postData(InputStream data, Integer length, OutputStream output) {
-    final String type = System.getProperty("type", DEFAULT_DATA_TYPE);
-    postData(data, length, output, type);
+  public void postData(InputStream data, Integer length, OutputStream output, String type) {
+    postData(data, length, output, type, solrUrl);
   }
-  
+
   /**
    * Reads data from the data stream and posts it to solr,
    * writes to the response to output
    */
-  public void postData(InputStream data, Integer length, OutputStream output, String type) {
-
+  public void postData(InputStream data, Integer length, OutputStream output, String type, URL url) {
+    if(type == null)
+      type = DEFAULT_CONTENT_TYPE;
     HttpURLConnection urlc = null;
     try {
       try {
-        urlc = (HttpURLConnection) solrUrl.openConnection();
+        urlc = (HttpURLConnection) url.openConnection();
         try {
           urlc.setRequestMethod("POST");
         } catch (ProtocolException e) {
@@ -315,14 +468,14 @@ public class SimplePostTool {
       InputStream in = null;
       try {
         if (HttpURLConnection.HTTP_OK != urlc.getResponseCode()) {
-          fatal("Solr returned an error #" + urlc.getResponseCode() + 
+          warn("Solr returned an error #" + urlc.getResponseCode() + 
                 " " + urlc.getResponseMessage());
         }
 
         in = urlc.getInputStream();
         pipe(in, output);
       } catch (IOException e) {
-        fatal("IOException while reading response: " + e);
+        warn("IOException while reading response: " + e);
       } finally {
         try { if(in!=null) in.close(); } catch (IOException x) { /*NOOP*/ }
       }
@@ -354,4 +507,68 @@ public class SimplePostTool {
     }
     if (null != dest) dest.flush();
   }
+
+  public boolean isAuto() {
+    return auto;
+  }
+
+  public void setAuto(boolean auto) {
+    this.auto = auto;
+  }
+
+  public boolean isRecursive() {
+    return recursive;
+  }
+
+  public void setRecursive(boolean recursive) {
+    this.recursive = recursive;
+  }
+
+  public String getFileTypes() {
+    return fileTypes;
+  }
+
+  public void setFileTypes(String fileTypes) {
+    this.fileTypes = fileTypes;
+    String glob;
+    if(fileTypes.equals("*"))
+      glob = ".*";
+    else
+      glob = "^.*\\.(" + fileTypes.replace(",", "|") + ")$";
+    this.globFileFilter = new GlobFileFilter(glob, true);
+  }
+
+  class GlobFileFilter implements FileFilter
+  {
+    private String _pattern;
+    private Pattern p;
+    
+    public GlobFileFilter(String pattern, boolean isRegex)
+    {
+      _pattern = pattern;
+      if(!isRegex) {
+        _pattern = _pattern
+            .replace("^", "\\^")
+            .replace("$", "\\$")
+            .replace(".", "\\.")
+            .replace("(", "\\(")
+            .replace(")", "\\)")
+            .replace("+", "\\+")
+            .replace("*", ".*")
+            .replace("?", ".");
+        _pattern = "^" + _pattern + "$";
+      }
+      
+      try {
+        p = Pattern.compile(_pattern,Pattern.CASE_INSENSITIVE);
+      } catch(PatternSyntaxException e) {
+        fatal("Invalid type list "+pattern+". "+e.getDescription());
+      }
+    }
+    
+    public boolean accept(File file)
+    {
+      return p.matcher(file.getName()).find();
+    }
+  }
 }

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java Mon Aug 13 13:52:46 2012
@@ -22,11 +22,11 @@ import java.util.List;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.lucene.analysis.util.ResourceLoader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.util.DOMUtil;
 import org.apache.solr.core.SolrConfig;
+import org.apache.solr.core.SolrResourceLoader;
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 
@@ -81,7 +81,7 @@ public abstract class AbstractPluginLoad
    * @param node - the XML node defining this plugin
    */
   @SuppressWarnings("unchecked")
-  protected T create( ResourceLoader loader, String name, String className, Node node ) throws Exception
+  protected T create( SolrResourceLoader loader, String name, String className, Node node ) throws Exception
   {
     return loader.newInstance(className, pluginClassType, getDefaultPackages());
   }
@@ -124,7 +124,7 @@ public abstract class AbstractPluginLoad
    * If a default element is defined, it will be returned from this function.
    * 
    */
-  public T load( ResourceLoader loader, NodeList nodes )
+  public T load( SolrResourceLoader loader, NodeList nodes )
   {
     List<PluginInitInfo> info = new ArrayList<PluginInitInfo>();
     T defaultPlugin = null;
@@ -204,7 +204,7 @@ public abstract class AbstractPluginLoad
    * The created class for the plugin will be returned from this function.
    * 
    */
-  public T loadSingle(ResourceLoader loader, Node node) {
+  public T loadSingle(SolrResourceLoader loader, Node node) {
     List<PluginInitInfo> info = new ArrayList<PluginInitInfo>();
     T plugin = null;
 

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test-files/solr/collection1/conf/schema.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test-files/solr/collection1/conf/schema.xml?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test-files/solr/collection1/conf/schema.xml (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test-files/solr/collection1/conf/schema.xml Mon Aug 13 13:52:46 2012
@@ -414,7 +414,32 @@
   
   <fieldType name="location" class="solr.LatLonType" subFieldSuffix="_coordinate"/>
   
- </types>
+  <!-- 
+    Example of using PathHierarchyTokenizerFactory at index time, so
+    queries for paths match documents at that path, or in descendent paths
+  -->
+  <fieldType name="path" class="solr.TextField">
+    <analyzer type="index">
+      <tokenizer class="solr.PathHierarchyTokenizerFactory" delimiter="/" />
+    </analyzer>
+    <analyzer type="query">
+      <tokenizer class="solr.KeywordTokenizerFactory" />
+    </analyzer>
+  </fieldType>
+  <!-- 
+    Example of using PathHierarchyTokenizerFactory at query time, so
+    queries for paths match documents at that path, or in ancestor paths
+  -->
+  <fieldType name="ancestor_path" class="solr.TextField">
+    <analyzer type="index">
+      <tokenizer class="solr.KeywordTokenizerFactory" />
+    </analyzer>
+    <analyzer type="query">
+      <tokenizer class="solr.PathHierarchyTokenizerFactory" delimiter="/" />
+    </analyzer>
+  </fieldType>
+
+</types>
 
 
  <fields>
@@ -534,7 +559,7 @@
    <field name="inStock" type="boolean" indexed="true" stored="true" />
 
    <field name="subword" type="subword" indexed="true" stored="true"/>
-   <field name="subword_offsets" type="subword" indexed="true" stored="true" termOffsets="true"/>
+   <field name="subword_offsets" type="subword" indexed="true" stored="true" termVectors="true" termOffsets="true"/>
    <field name="numericsubword" type="numericsubword" indexed="true" stored="true"/>
    <field name="protectedsubword" type="protectedsubword" indexed="true" stored="true"/>
 
@@ -640,6 +665,10 @@
 
    <!-- Type used to index the lat and lon components for the "location" FieldType -->
    <dynamicField name="*_coordinate"  type="tdouble" indexed="true"  stored="false" omitNorms="true" />
+
+   <dynamicField name="*_path"  type="path" indexed="true" stored="true" omitNorms="true" multiValued="true" />
+   <dynamicField name="*_ancestor"  type="ancestor_path" indexed="true" stored="true" omitNorms="true" multiValued="true" />
+
  </fields>
 
  <defaultSearchField>text</defaultSearchField>
@@ -673,6 +702,7 @@
 
    <!-- dynamic destination -->
    <copyField source="*_dynamic" dest="dynamic_*"/>
+   <copyField source="*_path" dest="*_ancestor"/>
 
  <!-- example of a custom similarity -->
  <similarity class="solr.CustomSimilarityFactory">

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml Mon Aug 13 13:52:46 2012
@@ -39,4 +39,6 @@
     </updateLog>
   </updateHandler>
 
+  <requestHandler name="/admin/" class="org.apache.solr.handler.admin.AdminHandlers" />
+
 </config>

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test-files/solr/solr.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test-files/solr/solr.xml?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test-files/solr/solr.xml (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test-files/solr/solr.xml Mon Aug 13 13:52:46 2012
@@ -28,7 +28,7 @@
   adminPath: RequestHandler path to manage cores.  
     If 'null' (or absent), cores will not be manageable via request handler
   -->
-  <cores adminPath="/admin/cores" defaultCoreName="collection1" host="127.0.0.1" hostPort="${hostPort:8983}" hostContext="solr" zkClientTimeout="12000" numShards="${numShards:3}">
+  <cores adminPath="/admin/cores" defaultCoreName="collection1" host="127.0.0.1" hostPort="${hostPort:8983}" hostContext="solr" zkClientTimeout="8000" numShards="${numShards:3}">
     <core name="collection1" instanceDir="collection1" shard="${shard:}" collection="${collection:collection1}" config="${solrconfig:solrconfig.xml}" schema="${schema:schema.xml}"/>
   </cores>
 </solr>

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/SolrInfoMBeanTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/SolrInfoMBeanTest.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/SolrInfoMBeanTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/SolrInfoMBeanTest.java Mon Aug 13 13:52:46 2012
@@ -25,6 +25,7 @@ import org.apache.solr.highlight.Default
 import org.apache.solr.search.LRUCache;
 import org.junit.BeforeClass;
 import java.io.File;
+import java.net.URI;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.Enumeration;
@@ -94,7 +95,10 @@ public class SolrInfoMBeanTest extends S
     String path = pckgname.replace('.', '/');
     Enumeration<URL> resources = cld.getResources(path);
     while (resources.hasMoreElements()) {
-      final File f = new File(resources.nextElement().toURI());
+      final URI uri = resources.nextElement().toURI();
+      if (!"file".equalsIgnoreCase(uri.getScheme()))
+        continue;
+      final File f = new File(uri);
       directories.add(f);
     }
       
@@ -114,6 +118,7 @@ public class SolrInfoMBeanTest extends S
         }
       }
     }
+    assertFalse("No classes found in package '"+pckgname+"'; maybe your test classes are packaged as JAR file?", classes.isEmpty());
     return classes;
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/TestDistributedSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/TestDistributedSearch.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/TestDistributedSearch.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/TestDistributedSearch.java Mon Aug 13 13:52:46 2012
@@ -310,7 +310,8 @@ public class TestDistributedSearch exten
     unIgnoreException("isShard is only acceptable");
 
     // test debugging
-    handle.put("explain", UNORDERED);
+    // handle.put("explain", UNORDERED);
+    handle.put("explain", SKIPVAL);  // internal docids differ, idf differs w/o global idf
     handle.put("debug", UNORDERED);
     handle.put("time", SKIPVAL);
     query("q","now their fox sat had put","fl","*,score",CommonParams.DEBUG_QUERY, "true");

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/analysis/TestWordDelimiterFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/analysis/TestWordDelimiterFilterFactory.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/analysis/TestWordDelimiterFilterFactory.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/analysis/TestWordDelimiterFilterFactory.java Mon Aug 13 13:52:46 2012
@@ -1,3 +1,5 @@
+package org.apache.solr.analysis;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -15,8 +17,6 @@
  * limitations under the License.
  */
 
-package org.apache.solr.analysis;
-
 import java.io.StringReader;
 import java.util.HashMap;
 import java.util.Map;
@@ -25,6 +25,7 @@ import org.apache.lucene.analysis.BaseTo
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.lucene.analysis.miscellaneous.WordDelimiterFilterFactory;
 import org.apache.lucene.analysis.util.ResourceLoader;
 import org.apache.solr.core.SolrResourceLoader;
 import org.junit.BeforeClass;
@@ -33,6 +34,7 @@ import org.junit.Test;
 /**
  * New WordDelimiterFilter tests... most of the tests are in ConvertedLegacyTest
  */
+// TODO: add a low-level test for this factory
 public class TestWordDelimiterFilterFactory extends SolrTestCaseJ4 {
 
   @BeforeClass

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java Mon Aug 13 13:52:46 2012
@@ -57,7 +57,7 @@ import org.apache.solr.common.SolrDocume
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -71,10 +71,11 @@ import org.apache.solr.update.SolrCmdDis
 import org.apache.solr.util.DefaultSolrThreadFactory;
 
 /**
- *
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
  */
 @Slow
-public class BasicDistributedZkTest extends AbstractDistributedZkTestCase {
+public class BasicDistributedZkTest extends AbstractDistribZkTestBase {
   
   private static final String DEFAULT_COLLECTION = "collection1";
   private static final boolean DEBUG = false;
@@ -281,7 +282,7 @@ public class BasicDistributedZkTest exte
     }
 
     // test debugging
-    handle.put("explain", UNORDERED);
+    handle.put("explain", SKIPVAL);
     handle.put("debug", UNORDERED);
     handle.put("time", SKIPVAL);
     query("q","now their fox sat had put","fl","*,score",CommonParams.DEBUG_QUERY, "true");
@@ -355,7 +356,7 @@ public class BasicDistributedZkTest exte
       HttpSolrServer collectionClient = new HttpSolrServer(url);
       
       // poll for a second - it can take a moment before we are ready to serve
-      waitForNon404(collectionClient);
+      waitForNon404or503(collectionClient);
     }
     
     List<String> collectionNameList = new ArrayList<String>();
@@ -451,7 +452,7 @@ public class BasicDistributedZkTest exte
   private void collectStartTimes(String collectionName,
       Map<String,Long> urlToTime) throws SolrServerException, IOException {
     Map<String,Map<String,Slice>> collections = solrj.getZkStateReader()
-        .getCloudState().getCollectionStates();
+        .getClusterState().getCollectionStates();
     if (collections.containsKey(collectionName)) {
       Map<String,Slice> slices = collections.get(collectionName);
 
@@ -478,8 +479,8 @@ public class BasicDistributedZkTest exte
   }
 
   private String getUrlFromZk(String collection) {
-    CloudState cloudState = solrj.getZkStateReader().getCloudState();
-    Map<String,Slice> slices = cloudState.getCollectionStates().get(collection);
+    ClusterState clusterState = solrj.getZkStateReader().getClusterState();
+    Map<String,Slice> slices = clusterState.getCollectionStates().get(collection);
     
     if (slices == null) {
       throw new SolrException(ErrorCode.BAD_REQUEST, "Could not find collection:" + collection);
@@ -491,7 +492,7 @@ public class BasicDistributedZkTest exte
       Set<Map.Entry<String,ZkNodeProps>> shardEntries = shards.entrySet();
       for (Map.Entry<String,ZkNodeProps> shardEntry : shardEntries) {
         final ZkNodeProps node = shardEntry.getValue();
-        if (cloudState.liveNodesContain(node.get(ZkStateReader.NODE_NAME_PROP))) {
+        if (clusterState.liveNodesContain(node.get(ZkStateReader.NODE_NAME_PROP))) {
           return new ZkCoreNodeProps(node).getCoreUrl();
         }
       }
@@ -500,20 +501,21 @@ public class BasicDistributedZkTest exte
     throw new RuntimeException("Could not find a live node for collection:" + collection);
   }
 
-  private void waitForNon404(HttpSolrServer collectionClient)
+  private void waitForNon404or503(HttpSolrServer collectionClient)
       throws Exception {
-    
+    SolrException exp = null;
     long timeoutAt = System.currentTimeMillis() + 30000;
     
     while (System.currentTimeMillis() < timeoutAt) {
       boolean missing = false;
+
       try {
         collectionClient.query(new SolrQuery("*:*"));
       } catch (SolrException e) {
-        // How do I get the response code!?
-        if (!e.getMessage().contains("(404)")) {
+        if (!(e.code() == 403 || e.code() == 503)) {
           throw e;
         }
+        exp = e;
         missing = true;
       }
       if (!missing) {
@@ -522,7 +524,7 @@ public class BasicDistributedZkTest exte
       Thread.sleep(50);
     }
     printLayout();
-    fail("Could not find the new collection - 404 : " + collectionClient.getBaseURL());
+    fail("Could not find the new collection - " + exp.code() + " : " + collectionClient.getBaseURL());
   }
 
   private void checkForCollection(String collectionName, int expectedSlices)
@@ -532,9 +534,9 @@ public class BasicDistributedZkTest exte
     boolean found = false;
     boolean sliceMatch = false;
     while (System.currentTimeMillis() < timeoutAt) {
-      solrj.getZkStateReader().updateCloudState(true);
-      CloudState cloudState = solrj.getZkStateReader().getCloudState();
-      Map<String,Map<String,Slice>> collections = cloudState
+      solrj.getZkStateReader().updateClusterState(true);
+      ClusterState clusterState = solrj.getZkStateReader().getClusterState();
+      Map<String,Map<String,Slice>> collections = clusterState
           .getCollectionStates();
       if (collections.containsKey(collectionName)) {
         Map<String,Slice> slices = collections.get(collectionName);
@@ -580,9 +582,9 @@ public class BasicDistributedZkTest exte
     long timeoutAt = System.currentTimeMillis() + 15000;
     boolean found = true;
     while (System.currentTimeMillis() < timeoutAt) {
-      solrj.getZkStateReader().updateCloudState(true);
-      CloudState cloudState = solrj.getZkStateReader().getCloudState();
-      Map<String,Map<String,Slice>> collections = cloudState
+      solrj.getZkStateReader().updateClusterState(true);
+      ClusterState clusterState = solrj.getZkStateReader().getClusterState();
+      Map<String,Map<String,Slice>> collections = clusterState
           .getCollectionStates();
       if (!collections.containsKey(collectionName)) {
         found = false;
@@ -772,8 +774,8 @@ public class BasicDistributedZkTest exte
     
     // we added a role of none on these creates - check for it
     ZkStateReader zkStateReader = solrj.getZkStateReader();
-    zkStateReader.updateCloudState(true);
-    Map<String,Slice> slices = zkStateReader.getCloudState().getSlices(oneInstanceCollection2);
+    zkStateReader.updateClusterState(true);
+    Map<String,Slice> slices = zkStateReader.getClusterState().getSlices(oneInstanceCollection2);
     assertNotNull(slices);
     String roles = slices.get("slice1").getShards().values().iterator().next().get(ZkStateReader.ROLES_PROP);
     assertEquals("none", roles);

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/BasicZkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/BasicZkTest.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/BasicZkTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/BasicZkTest.java Mon Aug 13 13:52:46 2012
@@ -126,8 +126,8 @@ public class BasicZkTest extends Abstrac
     
     // ensure zk still thinks node is up
     assertTrue(
-        zkController.getCloudState().getLiveNodes().toString(),
-        zkController.getCloudState().liveNodesContain(
+        zkController.getClusterState().getLiveNodes().toString(),
+        zkController.getClusterState().liveNodesContain(
             zkController.getNodeName()));
 
     // test maxint

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java Mon Aug 13 13:52:46 2012
@@ -20,7 +20,6 @@ package org.apache.solr.cloud;
 import java.net.ConnectException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.http.client.HttpClient;
 import org.apache.lucene.util.LuceneTestCase.Slow;
@@ -35,10 +34,15 @@ import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @Slow
 @Ignore("ignore while investigating jenkins fails")
-public class ChaosMonkeyNothingIsSafeTest extends FullSolrCloudTest {
+public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase {
+  public static Logger log = LoggerFactory.getLogger(ChaosMonkeyNothingIsSafeTest.class);
+  
+  private static final int BASE_RUN_LENGTH = 180000;
 
   @BeforeClass
   public static void beforeSuperClass() {
@@ -53,7 +57,7 @@ public class ChaosMonkeyNothingIsSafeTes
   public void setUp() throws Exception {
     super.setUp();
     // TODO use @Noisy annotation as we expect lots of exceptions
-    ignoreException(".*");
+    //ignoreException(".*");
     System.setProperty("numShards", Integer.toString(sliceCount));
   }
   
@@ -67,8 +71,8 @@ public class ChaosMonkeyNothingIsSafeTes
   
   public ChaosMonkeyNothingIsSafeTest() {
     super();
-    sliceCount = atLeast(2);
-    shardCount = atLeast(sliceCount * 2);
+    sliceCount = 2;
+    shardCount = 6;
   }
   
   @Override
@@ -83,7 +87,7 @@ public class ChaosMonkeyNothingIsSafeTes
       // as it's not supported for recovery
       // del("*:*");
       
-      List<StopableIndexingThread> threads = new ArrayList<StopableIndexingThread>();
+      List<StopableThread> threads = new ArrayList<StopableThread>();
       int threadCount = 1;
       int i = 0;
       for (i = 0; i < threadCount; i++) {
@@ -93,24 +97,33 @@ public class ChaosMonkeyNothingIsSafeTes
         indexThread.start();
       }
       
+      threadCount = 1;
+      i = 0;
+      for (i = 0; i < threadCount; i++) {
+        StopableSearchThread searchThread = new StopableSearchThread();
+        threads.add(searchThread);
+        searchThread.start();
+      }
+      
       FullThrottleStopableIndexingThread ftIndexThread = new FullThrottleStopableIndexingThread(
           clients, i * 50000, true);
       threads.add(ftIndexThread);
       ftIndexThread.start();
       
       chaosMonkey.startTheMonkey(true, 1500);
+      int runLength = atLeast(BASE_RUN_LENGTH);
       try {
-        Thread.sleep(180000);
+        Thread.sleep(runLength);
       } finally {
         chaosMonkey.stopTheMonkey();
       }
       
-      for (StopableIndexingThread indexThread : threads) {
+      for (StopableThread indexThread : threads) {
         indexThread.safeStop();
       }
       
       // wait for stop...
-      for (StopableIndexingThread indexThread : threads) {
+      for (StopableThread indexThread : threads) {
         indexThread.join();
       }
       
@@ -124,7 +137,7 @@ public class ChaosMonkeyNothingIsSafeTes
       Thread.sleep(2000);
       
       // wait until there are no recoveries...
-      waitForThingsToLevelOut();
+      waitForThingsToLevelOut(Math.round((runLength / 1000.0f / 5.0f)));
       
       // make sure we again have leaders for each shard
       for (int j = 1; j < sliceCount; j++) {
@@ -135,8 +148,8 @@ public class ChaosMonkeyNothingIsSafeTes
       
       // TODO: assert we didnt kill everyone
       
-      zkStateReader.updateCloudState(true);
-      assertTrue(zkStateReader.getCloudState().getLiveNodes().size() > 0);
+      zkStateReader.updateClusterState(true);
+      assertTrue(zkStateReader.getClusterState().getLiveNodes().size() > 0);
       
       checkShardConsistency(false, true);
       
@@ -156,35 +169,6 @@ public class ChaosMonkeyNothingIsSafeTes
       }
     }
   }
-
-  private void waitForThingsToLevelOut() throws Exception {
-    int cnt = 0;
-    boolean retry = false;
-    do {
-      waitForRecoveriesToFinish(VERBOSE);
-      
-      try {
-        commit();
-      } catch (Exception e) {
-        // we don't care if this commit fails on some nodes
-      }
-      
-      updateMappingsFromZk(jettys, clients);
-      
-      Set<String> theShards = shardToClient.keySet();
-      String failMessage = null;
-      for (String shard : theShards) {
-        failMessage = checkShardConsistency(shard, false);
-      }
-      
-      if (failMessage != null) {
-        retry  = true;
-      }
-      cnt++;
-      if (cnt > 10) break;
-      Thread.sleep(4000);
-    } while (retry);
-  }
   
   // skip the randoms - they can deadlock...
   protected void indexr(Object... fields) throws Exception {

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderTest.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderTest.java Mon Aug 13 13:52:46 2012
@@ -19,10 +19,12 @@ package org.apache.solr.cloud;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.servlet.SolrDispatchFilter;
+import org.apache.solr.update.DirectUpdateHandler2;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -30,11 +32,13 @@ import org.junit.BeforeClass;
 import org.junit.Ignore;
 
 @Ignore("SOLR-3126")
-public class ChaosMonkeySafeLeaderTest extends FullSolrCloudTest {
+public class ChaosMonkeySafeLeaderTest extends AbstractFullDistribZkTestBase {
   
+  private static final int BASE_RUN_LENGTH = 120000;
+
   @BeforeClass
   public static void beforeSuperClass() {
-    
+
   }
   
   @AfterClass
@@ -66,7 +70,7 @@ public class ChaosMonkeySafeLeaderTest e
   public ChaosMonkeySafeLeaderTest() {
     super();
     sliceCount = atLeast(2);
-    shardCount = atLeast(sliceCount);
+    shardCount = atLeast(sliceCount*2);
   }
   
   @Override
@@ -76,9 +80,10 @@ public class ChaosMonkeySafeLeaderTest e
     handle.put("QTime", SKIPVAL);
     handle.put("timestamp", SKIPVAL);
     
-    // we cannot do delete by query
-    // as it's not supported for recovery
-    //del("*:*");
+    // randomly turn on 5 seconds 'soft' commit
+    randomlyEnableAutoSoftCommit();
+
+    del("*:*");
     
     List<StopableIndexingThread> threads = new ArrayList<StopableIndexingThread>();
     int threadCount = 2;
@@ -89,8 +94,8 @@ public class ChaosMonkeySafeLeaderTest e
     }
     
     chaosMonkey.startTheMonkey(false, 500);
-    
-    Thread.sleep(atLeast(8000));
+    int runLength = atLeast(BASE_RUN_LENGTH);
+    Thread.sleep(runLength);
     
     chaosMonkey.stopTheMonkey();
     
@@ -109,39 +114,29 @@ public class ChaosMonkeySafeLeaderTest e
     
     // try and wait for any replications and what not to finish...
     
-    waitForThingsToLevelOut();
+    waitForThingsToLevelOut(Math.round((runLength / 1000.0f / 5.0f)));
 
     checkShardConsistency(true, true);
     
     if (VERBOSE) System.out.println("control docs:" + controlClient.query(new SolrQuery("*:*")).getResults().getNumFound() + "\n\n");
   }
 
-  private void waitForThingsToLevelOut() throws Exception {
-    int cnt = 0;
-    boolean retry = false;
-    do {
-      waitForRecoveriesToFinish(false);
-      
-      commit();
-      
-      updateMappingsFromZk(jettys, clients);
-      
-      Set<String> theShards = shardToClient.keySet();
-      String failMessage = null;
-      for (String shard : theShards) {
-        failMessage = checkShardConsistency(shard, false);
-      }
-      
-      if (failMessage != null) {
-        retry = true;
-      } else {
-        retry = false;
+  private void randomlyEnableAutoSoftCommit() {
+    if (r.nextBoolean()) {
+      log.info("Turning on auto soft commit");
+      for (CloudJettyRunner jetty : shardToJetty.get("shard1")) {
+        SolrCore core = ((SolrDispatchFilter) jetty.jetty.getDispatchFilter()
+            .getFilter()).getCores().getCore("collection1");
+        try {
+          ((DirectUpdateHandler2) core.getUpdateHandler()).getCommitTracker()
+              .setTimeUpperBound(5000);
+        } finally {
+          core.close();
+        }
       }
-      
-      cnt++;
-      if (cnt > 10) break;
-      Thread.sleep(2000);
-    } while (retry);
+    } else {
+      log.info("Not turning on auto soft commit");
+    }
   }
   
   // skip the randoms - they can deadlock...

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java Mon Aug 13 13:52:46 2012
@@ -23,8 +23,8 @@ import org.apache.lucene.util.LuceneTest
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrServer;
 import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.HttpSolrServer;
 import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrServer;
+import org.apache.solr.client.solrj.impl.HttpSolrServer;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.SolrDocument;
@@ -37,14 +37,13 @@ import org.apache.solr.common.params.Mod
 import org.apache.solr.update.VersionInfo;
 import org.apache.solr.update.processor.DistributedUpdateProcessor;
 import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
 import org.junit.BeforeClass;
 
 /**
  * Super basic testing, no shard restarting or anything.
  */
 @Slow
-public class FullSolrCloudDistribCmdsTest extends FullSolrCloudTest {
+public class FullSolrCloudDistribCmdsTest extends AbstractFullDistribZkTestBase {
   
   
   @BeforeClass

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/LeaderElectionIntegrationTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/LeaderElectionIntegrationTest.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/LeaderElectionIntegrationTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/LeaderElectionIntegrationTest.java Mon Aug 13 13:52:46 2012
@@ -191,7 +191,7 @@ public class LeaderElectionIntegrationTe
       int newLeaderPort = getLeaderPort(leader);
       int retry = 0;
       while (leaderPort == newLeaderPort) {
-        if (retry++ == 20) {
+        if (retry++ == 60) {
           break;
         }
         Thread.sleep(1000);

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java Mon Aug 13 13:52:46 2012
@@ -35,10 +35,12 @@ 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.util.DefaultSolrThreadFactory;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 
 @Slow
@@ -315,7 +317,7 @@ public class LeaderElectionTest extends 
   @Test
   public void testStressElection() throws Exception {
     final ScheduledExecutorService scheduler = Executors
-        .newScheduledThreadPool(15);
+        .newScheduledThreadPool(15, new DefaultSolrThreadFactory("stressElection"));
     final List<ClientThread> threads = Collections
         .synchronizedList(new ArrayList<ClientThread>());
     
@@ -369,9 +371,7 @@ public class LeaderElectionTest extends 
             }
 
             Thread.sleep(10);
-            
           } catch (Exception e) {
-
           }
         }
       }
@@ -382,7 +382,6 @@ public class LeaderElectionTest extends 
       public void run() {
         
         while (!stopStress) {
-
           try {
             Thread.sleep(50);
             int j;
@@ -426,6 +425,7 @@ public class LeaderElectionTest extends 
     
     // cleanup any threads still running
     for (ClientThread thread : threads) {
+      thread.zkClient.getSolrZooKeeper().close();
       thread.close();
     }