You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ha...@apache.org on 2013/07/16 19:00:18 UTC

svn commit: r1503797 [14/19] - in /lucene/dev/branches/lucene3069: ./ dev-tools/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/suggest/ dev-tools/idea/solr/contrib/dataimporthandler/ dev-tools/idea/solr/core/src/test/ dev-tools/maven/ dev-tools...

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java Tue Jul 16 17:00:05 2013
@@ -17,35 +17,6 @@
 
 package org.apache.solr.servlet;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.OutputStreamWriter;
-import java.io.Writer;
-import java.net.HttpURLConnection;
-import java.net.URL;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Enumeration;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.WeakHashMap;
-
-import javax.servlet.Filter;
-import javax.servlet.FilterChain;
-import javax.servlet.FilterConfig;
-import javax.servlet.ServletException;
-import javax.servlet.ServletRequest;
-import javax.servlet.ServletResponse;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
 import org.apache.commons.io.IOUtils;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -81,6 +52,34 @@ import org.apache.solr.util.FastWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.servlet.Filter;
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.WeakHashMap;
+
 /**
  * This filter looks at the incoming URL maps them to handlers defined in solrconfig.xml
  *
@@ -116,12 +115,11 @@ public class SolrDispatchFilter implemen
   {
     log.info("SolrDispatchFilter.init()");
 
-    CoreContainer.Initializer init = createInitializer();
     try {
       // web.xml configuration
       this.pathPrefix = config.getInitParameter( "path-prefix" );
 
-      this.cores = init.initialize();
+      this.cores = createCoreContainer();
       log.info("user.dir=" + System.getProperty("user.dir"));
     }
     catch( Throwable t ) {
@@ -132,15 +130,20 @@ public class SolrDispatchFilter implemen
 
     log.info("SolrDispatchFilter.init() done");
   }
+
+  /**
+   * Override this to change CoreContainer initialization
+   * @return a CoreContainer to hold this server's cores
+   */
+  protected CoreContainer createCoreContainer() {
+    CoreContainer cores = new CoreContainer();
+    cores.load();
+    return cores;
+  }
   
   public CoreContainer getCores() {
     return cores;
   }
-
-  /** Method to override to change how CoreContainer initialization is performed. */
-  protected CoreContainer.Initializer createInitializer() {
-    return new CoreContainer.Initializer();
-  }
   
   @Override
   public void destroy() {
@@ -215,6 +218,13 @@ public class SolrDispatchFilter implemen
           handleAdminRequest(req, response, handler, solrReq);
           return;
         }
+        // Check for the core admin info url
+        if( path.startsWith( "/admin/info" ) ) {
+          handler = cores.getInfoHandler();
+          solrReq =  SolrRequestParsers.DEFAULT.parse(null,path, req);
+          handleAdminRequest(req, response, handler, solrReq);
+          return;
+        }
         else {
           //otherwise, we should find a core from the path
           idx = path.indexOf( "/", 1 );

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/spelling/SpellingQueryConverter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/spelling/SpellingQueryConverter.java?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/spelling/SpellingQueryConverter.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/spelling/SpellingQueryConverter.java Tue Jul 16 17:00:05 2013
@@ -18,8 +18,6 @@
 package org.apache.solr.spelling;
 
 import java.io.IOException;
-import java.io.Reader;
-import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -28,11 +26,10 @@ import java.util.regex.Pattern;
 
 import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
-import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
 
 
@@ -160,7 +157,7 @@ public class SpellingQueryConverter exte
         flagValue = TERM_PRECEDES_NEW_BOOLEAN_OPERATOR_FLAG;
       }
       try {
-        analyze(result, new StringReader(word), startIndex, flagValue);
+        analyze(result, word, startIndex, flagValue);
       } catch (IOException e) {
         // TODO: shouldn't we log something?
       }   
@@ -174,7 +171,7 @@ public class SpellingQueryConverter exte
     return result;
   }
   
-  protected void analyze(Collection<Token> result, Reader text, int offset, int flagsAttValue) throws IOException {
+  protected void analyze(Collection<Token> result, String text, int offset, int flagsAttValue) throws IOException {
     TokenStream stream = analyzer.tokenStream("", text);
     // TODO: support custom attributes
     CharTermAttribute termAtt = stream.addAttribute(CharTermAttribute.class);

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/spelling/SuggestQueryConverter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/spelling/SuggestQueryConverter.java?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/spelling/SuggestQueryConverter.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/spelling/SuggestQueryConverter.java Tue Jul 16 17:00:05 2013
@@ -18,7 +18,6 @@ package org.apache.solr.spelling;
  */
 
 import java.io.IOException;
-import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -38,7 +37,7 @@ public class SuggestQueryConverter exten
 
     Collection<Token> result = new ArrayList<Token>();
     try {
-      analyze(result, new StringReader(original), 0, 0);
+      analyze(result, original, 0, 0);
     } catch (IOException e) {
       throw new RuntimeException(e);
     }

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java Tue Jul 16 17:00:05 2013
@@ -187,6 +187,76 @@ public final class DefaultSolrCoreState 
       }
     }
   }
+  
+  @Override
+  public synchronized void closeIndexWriter(SolrCore core, boolean rollback)
+      throws IOException {
+    log.info("Closing IndexWriter...");
+    String coreName = core.getName();
+    synchronized (writerPauseLock) {
+      if (closed) {
+        throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "Already closed");
+      }
+      
+      // we need to wait for the Writer to fall out of use
+      // first lets stop it from being lent out
+      pauseWriter = true;
+      // then lets wait until its out of use
+      log.info("Waiting until IndexWriter is unused... core=" + coreName);
+      
+      while (!writerFree) {
+        try {
+          writerPauseLock.wait(100);
+        } catch (InterruptedException e) {}
+        
+        if (closed) {
+          throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE,
+              "SolrCoreState already closed");
+        }
+      }
+      
+      if (indexWriter != null) {
+        if (!rollback) {
+          try {
+            log.info("Closing old IndexWriter... core=" + coreName);
+            indexWriter.close();
+          } catch (Throwable t) {
+            SolrException.log(log, "Error closing old IndexWriter. core="
+                + coreName, t);
+          }
+        } else {
+          try {
+            log.info("Rollback old IndexWriter... core=" + coreName);
+            indexWriter.rollback();
+          } catch (Throwable t) {
+            SolrException.log(log, "Error rolling back old IndexWriter. core="
+                + coreName, t);
+          }
+        }
+      }
+      
+    }
+  }
+  
+  @Override
+  public synchronized void openIndexWriter(SolrCore core) throws IOException {
+    log.info("Creating new IndexWriter...");
+    synchronized (writerPauseLock) {
+      if (closed) {
+        throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "Already closed");
+      }
+      
+      try {
+        indexWriter = createMainIndexWriter(core, "DirectUpdateHandler2");
+        log.info("New IndexWriter is ready to be used.");
+        // we need to null this so it picks up the new writer next get call
+        refCntWriter = null;
+      } finally {
+        pauseWriter = false;
+        writerPauseLock.notifyAll();
+      }
+    }
+  }
 
   @Override
   public synchronized void rollbackIndexWriter(SolrCore core) throws IOException {

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java Tue Jul 16 17:00:05 2013
@@ -540,11 +540,17 @@ public class DirectUpdateHandler2 extend
           }
           
           // SolrCore.verbose("writer.commit() start writer=",writer);
-          final Map<String,String> commitData = new HashMap<String,String>();
-          commitData.put(SolrIndexWriter.COMMIT_TIME_MSEC_KEY,
-              String.valueOf(System.currentTimeMillis()));
-          writer.setCommitData(commitData);
-          writer.commit();
+
+          if (writer.hasUncommittedChanges()) {
+            final Map<String,String> commitData = new HashMap<String,String>();
+            commitData.put(SolrIndexWriter.COMMIT_TIME_MSEC_KEY,
+                String.valueOf(System.currentTimeMillis()));
+            writer.setCommitData(commitData);
+            writer.commit();
+          } else {
+            log.info("No uncommitted changes. Skipping IW.commit.");
+          }
+
           // SolrCore.verbose("writer.commit() end");
           numDocsPending.set(0);
           callPostCommitCallbacks();

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java Tue Jul 16 17:00:05 2013
@@ -204,7 +204,7 @@ public class SolrCmdDistributor {
     try {
       sreq.ursp = server.request(ureq);
     } catch (Exception e) {
-      throw new SolrException(ErrorCode.SERVER_ERROR, "Failed synchronous update on shard " + sreq.node, sreq.exception);
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Failed synchronous update on shard " + sreq.node + " update: " + ureq , e);
     }
   }
 

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrCoreState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrCoreState.java?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrCoreState.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrCoreState.java Tue Jul 16 17:00:05 2013
@@ -88,6 +88,26 @@ public abstract class SolrCoreState {
    */
   public abstract void newIndexWriter(SolrCore core, boolean rollback) throws IOException;
   
+  
+  /**
+   * Expert method that closes the IndexWriter - you must call {@link #openIndexWriter(SolrCore)}
+   * in a finally block after calling this method.
+   * 
+   * @param core that the IW belongs to
+   * @param rollback true if IW should rollback rather than close
+   * @throws IOException If there is a low-level I/O error.
+   */
+  public abstract void closeIndexWriter(SolrCore core, boolean rollback) throws IOException;
+  
+  /**
+   * Expert method that opens the IndexWriter - you must call {@link #closeIndexWriter(SolrCore, boolean)}
+   * first, and then call this method in a finally block.
+   * 
+   * @param core that the IW belongs to
+   * @throws IOException If there is a low-level I/O error.
+   */
+  public abstract void openIndexWriter(SolrCore core) throws IOException;
+  
   /**
    * Get the current IndexWriter. If a new IndexWriter must be created, use the
    * settings from the given {@link SolrCore}.

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java Tue Jul 16 17:00:05 2013
@@ -17,12 +17,15 @@
 
 package org.apache.solr.update;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.lucene.index.*;
 import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
 import org.apache.lucene.util.InfoStream;
+import org.apache.lucene.util.PrintStreamInfoStream;
 import org.apache.lucene.util.Version;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.schema.IndexSchema;
@@ -30,6 +33,10 @@ import org.apache.solr.util.SolrPluginUt
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.PrintStream;
 import java.util.List;
 
 /**
@@ -43,7 +50,14 @@ public class SolrIndexConfig {
   public static final String DEFAULT_MERGE_SCHEDULER_CLASSNAME = ConcurrentMergeScheduler.class.getName();
   public final Version luceneVersion;
   
+  /**
+   * The explicit value of &lt;useCompoundFile&gt; specified on this index config
+   * @deprecated use {@link #getUseCompoundFile}
+   */
+  @Deprecated
   public final boolean useCompoundFile;
+  private boolean effectiveUseCompountFileSetting;
+
   public final int maxBufferedDocs;
   public final int maxMergeDocs;
   public final int maxIndexingThreads;
@@ -59,7 +73,7 @@ public class SolrIndexConfig {
   
   public final PluginInfo mergedSegmentWarmerInfo;
   
-  public String infoStreamFile = null;
+  public InfoStream infoStream = InfoStream.NO_OUTPUT;
 
   // Available lock types
   public final static String LOCK_TYPE_SIMPLE = "simple";
@@ -73,7 +87,7 @@ public class SolrIndexConfig {
   @SuppressWarnings("deprecation")
   private SolrIndexConfig(SolrConfig solrConfig) {
     luceneVersion = solrConfig.luceneMatchVersion;
-    useCompoundFile = false;
+    useCompoundFile = effectiveUseCompountFileSetting = false;
     maxBufferedDocs = -1;
     maxMergeDocs = -1;
     maxIndexingThreads = IndexWriterConfig.DEFAULT_MAX_THREAD_STATES;
@@ -121,6 +135,7 @@ public class SolrIndexConfig {
 
     defaultMergePolicyClassName = def.defaultMergePolicyClassName;
     useCompoundFile=solrConfig.getBool(prefix+"/useCompoundFile", def.useCompoundFile);
+    effectiveUseCompountFileSetting = useCompoundFile;
     maxBufferedDocs=solrConfig.getInt(prefix+"/maxBufferedDocs",def.maxBufferedDocs);
     maxMergeDocs=solrConfig.getInt(prefix+"/maxMergeDocs",def.maxMergeDocs);
     maxIndexingThreads=solrConfig.getInt(prefix+"/maxIndexingThreads",def.maxIndexingThreads);
@@ -134,13 +149,17 @@ public class SolrIndexConfig {
     mergePolicyInfo = getPluginInfo(prefix + "/mergePolicy", solrConfig, def.mergePolicyInfo);
     
     termIndexInterval = solrConfig.getInt(prefix + "/termIndexInterval", def.termIndexInterval);
-    
+
     boolean infoStreamEnabled = solrConfig.getBool(prefix + "/infoStream", false);
     if(infoStreamEnabled) {
-      infoStreamFile= solrConfig.get(prefix + "/infoStream/@file", null);
-      log.info("IndexWriter infoStream debug log is enabled: " + infoStreamFile);
+      String infoStreamFile = solrConfig.get(prefix + "/infoStream/@file", null);
+      if (infoStreamFile == null) {
+        log.info("IndexWriter infoStream solr logging is enabled");
+        infoStream = new LoggingInfoStream();
+      } else {
+        throw new IllegalArgumentException("Remove @file from <infoStream> to output messages to solr's logfile");
+      }
     }
-    
     mergedSegmentWarmerInfo = getPluginInfo(prefix + "/mergedSegmentWarmer", solrConfig, def.mergedSegmentWarmerInfo);
     if (mergedSegmentWarmerInfo != null && solrConfig.reopenReaders == false) {
       throw new IllegalArgumentException("Supplying a mergedSegmentWarmer will do nothing since reopenReaders is false");
@@ -188,6 +207,11 @@ public class SolrIndexConfig {
     iwc.setSimilarity(schema.getSimilarity());
     iwc.setMergePolicy(buildMergePolicy(schema));
     iwc.setMergeScheduler(buildMergeScheduler(schema));
+    iwc.setInfoStream(infoStream);
+
+    // do this after buildMergePolicy since the backcompat logic 
+    // there may modify the effective useCompoundFile
+    iwc.setUseCompoundFile(getUseCompoundFile());
 
     if (maxIndexingThreads != -1) {
       iwc.setMaxThreadStates(maxIndexingThreads);
@@ -199,13 +223,22 @@ public class SolrIndexConfig {
                                                                         IndexReaderWarmer.class,
                                                                         null,
                                                                         new Class[] { InfoStream.class },
-                                                                        new Object[] { InfoStream.NO_OUTPUT });
+                                                                        new Object[] { iwc.getInfoStream() });
       iwc.setMergedSegmentWarmer(warmer);
     }
 
     return iwc;
   }
 
+  /**
+   * Builds a MergePolicy, may also modify the value returned by
+   * getUseCompoundFile() for use by the IndexWriterConfig if 
+   * "useCompoundFile" is specified as an init arg for 
+   * an out of the box MergePolicy that no longer supports it
+   *
+   * @see #fixUseCFMergePolicyInitArg
+   * @see #getUseCompoundFile
+   */
   private MergePolicy buildMergePolicy(IndexSchema schema) {
     String mpClassName = mergePolicyInfo == null ? defaultMergePolicyClassName : mergePolicyInfo.className;
 
@@ -213,25 +246,31 @@ public class SolrIndexConfig {
 
     if (policy instanceof LogMergePolicy) {
       LogMergePolicy logMergePolicy = (LogMergePolicy) policy;
+      fixUseCFMergePolicyInitArg(LogMergePolicy.class);
 
       if (maxMergeDocs != -1)
         logMergePolicy.setMaxMergeDocs(maxMergeDocs);
 
-      logMergePolicy.setNoCFSRatio(useCompoundFile ? 1.0 : 0.0);
+      logMergePolicy.setNoCFSRatio(getUseCompoundFile() ? 1.0 : 0.0);
 
       if (mergeFactor != -1)
         logMergePolicy.setMergeFactor(mergeFactor);
+
+
     } else if (policy instanceof TieredMergePolicy) {
       TieredMergePolicy tieredMergePolicy = (TieredMergePolicy) policy;
+      fixUseCFMergePolicyInitArg(TieredMergePolicy.class);
       
-      tieredMergePolicy.setNoCFSRatio(useCompoundFile ? 1.0 : 0.0);
+      tieredMergePolicy.setNoCFSRatio(getUseCompoundFile() ? 1.0 : 0.0);
       
       if (mergeFactor != -1) {
         tieredMergePolicy.setMaxMergeAtOnce(mergeFactor);
         tieredMergePolicy.setSegmentsPerTier(mergeFactor);
       }
-    } else {
-      log.warn("Use of compound file format or mergefactor cannot be configured if merge policy is not an instance of LogMergePolicy or TieredMergePolicy. The configured policy's defaults will be used.");
+
+
+    } else if (mergeFactor != -1) {
+      log.warn("Use of <mergeFactor> cannot be configured if merge policy is not an instance of LogMergePolicy or TieredMergePolicy. The configured policy's defaults will be used.");
     }
 
     if (mergePolicyInfo != null)
@@ -244,9 +283,58 @@ public class SolrIndexConfig {
     String msClassName = mergeSchedulerInfo == null ? SolrIndexConfig.DEFAULT_MERGE_SCHEDULER_CLASSNAME : mergeSchedulerInfo.className;
     MergeScheduler scheduler = schema.getResourceLoader().newInstance(msClassName, MergeScheduler.class);
 
-    if (mergeSchedulerInfo != null)
-      SolrPluginUtils.invokeSetters(scheduler, mergeSchedulerInfo.initArgs);
+    if (mergeSchedulerInfo != null) {
+      // LUCENE-5080: these two setters are removed, so we have to invoke setMaxMergesAndThreads
+      // if someone has them configured.
+      if (scheduler instanceof ConcurrentMergeScheduler) {
+        NamedList args = mergeSchedulerInfo.initArgs.clone();
+        Integer maxMergeCount = (Integer) args.remove("maxMergeCount");
+        if (maxMergeCount == null) {
+          maxMergeCount = ((ConcurrentMergeScheduler) scheduler).getMaxMergeCount();
+        }
+        Integer maxThreadCount = (Integer) args.remove("maxThreadCount");
+        if (maxThreadCount == null) {
+          maxThreadCount = ((ConcurrentMergeScheduler) scheduler).getMaxThreadCount();
+        }
+        ((ConcurrentMergeScheduler)scheduler).setMaxMergesAndThreads(maxMergeCount, maxThreadCount);
+        SolrPluginUtils.invokeSetters(scheduler, args);
+      } else {
+        SolrPluginUtils.invokeSetters(scheduler, mergeSchedulerInfo.initArgs);
+      }
+    }
 
     return scheduler;
   }
+
+  public boolean getUseCompoundFile() {
+    return effectiveUseCompountFileSetting;
+  }
+
+  /**
+   * Lucene 4.4 removed the setUseCompoundFile(boolean) method from the two 
+   * conrete MergePolicies provided with Lucene/Solr and added it to the 
+   * IndexWRiterConfig.  
+   * In the event that users have a value explicitly configured for this 
+   * setter in their MergePolicy init args, we remove it from the MergePolicy 
+   * init args, update the 'effective' useCompoundFile setting used by the 
+   * IndexWriterConfig, and warn about discontinuing to use this init arg.
+   * 
+   * @see #getUseCompoundFile
+   */
+  private void fixUseCFMergePolicyInitArg(Class c) {
+
+    if (null == mergePolicyInfo || null == mergePolicyInfo.initArgs) return;
+
+    Object useCFSArg = mergePolicyInfo.initArgs.remove("useCompoundFile");
+    if (null != useCFSArg) {
+      log.warn("Ignoring 'useCompoundFile' specified as an init arg for the <mergePolicy> since it is no directly longer supported by " + c.getSimpleName());
+      if (useCFSArg instanceof Boolean) {
+        boolean cfs = ((Boolean)useCFSArg).booleanValue();
+        log.warn("Please update your config to specify <useCompoundFile>"+cfs+"</useCompoundFile> directly in your <indexConfig> settings.");
+        effectiveUseCompountFileSetting = cfs;
+      } else {
+        log.error("MergePolicy's 'useCompoundFile' init arg is not a boolean, can not apply back compat logic to apply to the IndexWriterConfig: " + useCFSArg.toString());
+      }
+    }
+  }
 }

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java Tue Jul 16 17:00:05 2013
@@ -77,7 +77,7 @@ public class SolrIndexWriter extends Ind
     super(directory,
           config.toIndexWriterConfig(schema).
           setOpenMode(create ? IndexWriterConfig.OpenMode.CREATE : IndexWriterConfig.OpenMode.APPEND).
-          setIndexDeletionPolicy(delPolicy).setCodec(codec).setInfoStream(toInfoStream(config))
+          setIndexDeletionPolicy(delPolicy).setCodec(codec)
           );
     log.debug("Opened Writer " + name);
     this.name = name;
@@ -88,20 +88,6 @@ public class SolrIndexWriter extends Ind
     this.directoryFactory = factory;
   }
 
-  private static InfoStream toInfoStream(SolrIndexConfig config) throws IOException {
-    String infoStreamFile = config.infoStreamFile;
-    if (infoStreamFile != null) {
-      File f = new File(infoStreamFile);
-      File parent = f.getParentFile();
-      if (parent != null) parent.mkdirs();
-      FileOutputStream fos = new FileOutputStream(f, true);
-      return new PrintStreamInfoStream(new PrintStream(fos, true, "UTF-8"));
-    } else {
-      return InfoStream.NO_OUTPUT;
-    }
-  }
-
-
   /**
    * use DocumentBuilder now...
    * private final void addField(Document doc, String name, String val) {
@@ -164,11 +150,8 @@ public class SolrIndexWriter extends Ind
       if (infoStream != null) {
         infoStream.close();
       }
-      
       isClosed = true;
-      
       directoryFactory.release(directory);
-      
       numCloses.incrementAndGet();
     }
   }

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/TransactionLog.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/TransactionLog.java?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/TransactionLog.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/TransactionLog.java Tue Jul 16 17:00:05 2013
@@ -74,7 +74,7 @@ public class TransactionLog {
   FastOutputStream fos;    // all accesses to this stream should be synchronized on "this" (The TransactionLog)
   int numRecords;
   
-  volatile boolean deleteOnClose = true;  // we can delete old tlogs since they are currently only used for real-time-get (and in the future, recovery)
+  protected volatile boolean deleteOnClose = true;  // we can delete old tlogs since they are currently only used for real-time-get (and in the future, recovery)
 
   AtomicInteger refcount = new AtomicInteger(1);
   Map<String,Integer> globalStringMap = new HashMap<String, Integer>();
@@ -97,7 +97,7 @@ public class TransactionLog {
   };
 
   public class LogCodec extends JavaBinCodec {
-    public LogCodec() {
+    public LogCodec(JavaBinCodec.ObjectResolver resolver) {
       super(resolver);
     }
 
@@ -190,6 +190,9 @@ public class TransactionLog {
     }
   }
 
+  // for subclasses
+  protected TransactionLog() {}
+
   /** Returns the number of records in the log (currently includes the header and an optional commit).
    * Note: currently returns 0 for reopened existing log files.
    */
@@ -244,7 +247,7 @@ public class TransactionLog {
 
 
   public long writeData(Object o) {
-    LogCodec codec = new LogCodec();
+    LogCodec codec = new LogCodec(resolver);
     try {
       long pos = fos.size();   // if we had flushed, this should be equal to channel.position()
       codec.init(fos);
@@ -259,7 +262,7 @@ public class TransactionLog {
   private void readHeader(FastInputStream fis) throws IOException {
     // read existing header
     fis = fis != null ? fis : new ChannelFastInputStream(channel, 0);
-    LogCodec codec = new LogCodec();
+    LogCodec codec = new LogCodec(resolver);
     Map header = (Map)codec.unmarshal(fis);
 
     fis.readInt(); // skip size
@@ -275,7 +278,7 @@ public class TransactionLog {
     }
   }
 
-  private void addGlobalStrings(Collection<String> strings) {
+  protected void addGlobalStrings(Collection<String> strings) {
     if (strings == null) return;
     int origSize = globalStringMap.size();
     for (String s : strings) {
@@ -296,7 +299,7 @@ public class TransactionLog {
     }
   }
 
-  private void writeLogHeader(LogCodec codec) throws IOException {
+  protected void writeLogHeader(LogCodec codec) throws IOException {
     long pos = fos.size();
     assert pos == 0;
 
@@ -308,7 +311,7 @@ public class TransactionLog {
     endRecord(pos);
   }
 
-  private void endRecord(long startRecordPosition) throws IOException {
+  protected void endRecord(long startRecordPosition) throws IOException {
     fos.writeInt((int)(fos.size() - startRecordPosition));
     numRecords++;
   }
@@ -332,7 +335,7 @@ public class TransactionLog {
   int lastAddSize;
 
   public long write(AddUpdateCommand cmd, int flags) {
-    LogCodec codec = new LogCodec();
+    LogCodec codec = new LogCodec(resolver);
     SolrInputDocument sdoc = cmd.getSolrInputDocument();
 
     try {
@@ -374,7 +377,7 @@ public class TransactionLog {
   }
 
   public long writeDelete(DeleteUpdateCommand cmd, int flags) {
-    LogCodec codec = new LogCodec();
+    LogCodec codec = new LogCodec(resolver);
 
     try {
       checkWriteHeader(codec, null);
@@ -404,7 +407,7 @@ public class TransactionLog {
   }
 
   public long writeDeleteByQuery(DeleteUpdateCommand cmd, int flags) {
-    LogCodec codec = new LogCodec();
+    LogCodec codec = new LogCodec(resolver);
     try {
       checkWriteHeader(codec, null);
 
@@ -430,7 +433,7 @@ public class TransactionLog {
 
 
   public long writeCommit(CommitUpdateCommand cmd, int flags) {
-    LogCodec codec = new LogCodec();
+    LogCodec codec = new LogCodec(resolver);
     synchronized (this) {
       try {
         long pos = fos.size();   // if we had flushed, this should be equal to channel.position()
@@ -478,7 +481,7 @@ public class TransactionLog {
       }
 
       ChannelFastInputStream fis = new ChannelFastInputStream(channel, pos);
-      LogCodec codec = new LogCodec();
+      LogCodec codec = new LogCodec(resolver);
       return codec.readVal(fis);
     } catch (IOException e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
@@ -528,7 +531,7 @@ public class TransactionLog {
     }
   }
 
-  private void close() {
+  protected void close() {
     try {
       if (debug) {
         log.debug("Closing tlog" + this);
@@ -569,19 +572,22 @@ public class TransactionLog {
 
   /** Returns a single threaded reverse reader */
   public ReverseReader getReverseReader() throws IOException {
-    return new ReverseReader();
+    return new FSReverseReader();
   }
 
 
   public class LogReader {
-    ChannelFastInputStream fis;
-    private LogCodec codec = new LogCodec();
+    private ChannelFastInputStream fis;
+    private LogCodec codec = new LogCodec(resolver);
 
     public LogReader(long startingPos) {
       incref();
       fis = new ChannelFastInputStream(channel, startingPos);
     }
 
+    // for classes that extend
+    protected LogReader() {}
+
     /** Returns the next object from the log, or null if none available.
      *
      * @return The log record, or null if EOF
@@ -637,9 +643,30 @@ public class TransactionLog {
 
   }
 
-  public class ReverseReader {
+  public abstract class ReverseReader {
+
+
+
+    /** Returns the next object from the log, or null if none available.
+     *
+     * @return The log record, or null if EOF
+     * @throws IOException If there is a low-level I/O error.
+     */
+    public abstract Object next() throws IOException;
+
+    /* returns the position in the log file of the last record returned by next() */
+    public abstract long position();
+    public abstract void close();
+
+    @Override
+    public abstract String toString() ;
+
+
+  }
+  
+  public class FSReverseReader extends ReverseReader {
     ChannelFastInputStream fis;
-    private LogCodec codec = new LogCodec() {
+    private LogCodec codec = new LogCodec(resolver) {
       @Override
       public SolrInputDocument readSolrInputDocument(DataInputInputStream dis) {
         // Given that the SolrInputDocument is last in an add record, it's OK to just skip
@@ -651,7 +678,7 @@ public class TransactionLog {
     int nextLength;  // length of the next record (the next one closer to the start of the log file)
     long prevPos;    // where we started reading from last time (so prevPos - nextLength == start of next record)
 
-    public ReverseReader() throws IOException {
+    public FSReverseReader() throws IOException {
       incref();
 
       long sz;

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/UpdateHandler.java?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/UpdateHandler.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/UpdateHandler.java Tue Jul 16 17:00:05 2013
@@ -18,10 +18,11 @@
 package org.apache.solr.update;
 
 
-import java.io.File;
 import java.io.IOException;
 import java.util.Vector;
 
+import org.apache.solr.core.DirectoryFactory;
+import org.apache.solr.core.HdfsDirectoryFactory;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrEventListener;
@@ -52,7 +53,7 @@ public abstract class UpdateHandler impl
   protected Vector<SolrEventListener> softCommitCallbacks = new Vector<SolrEventListener>();
   protected Vector<SolrEventListener> optimizeCallbacks = new Vector<SolrEventListener>();
 
-  protected UpdateLog ulog;
+  protected final UpdateLog ulog;
 
   private void parseEventListeners() {
     final Class<SolrEventListener> clazz = SolrEventListener.class;
@@ -71,36 +72,6 @@ public abstract class UpdateHandler impl
     }
   }
 
-
-  private void initLog(PluginInfo ulogPluginInfo, UpdateLog existingUpdateLog) {
-    ulog = existingUpdateLog;
-    if (ulog == null && ulogPluginInfo != null && ulogPluginInfo.isEnabled()) {
-      ulog = new UpdateLog();
-      ulog.init(ulogPluginInfo);
-      // ulog = core.createInitInstance(ulogPluginInfo, UpdateLog.class, "update log", "solr.NullUpdateLog");
-      ulog.init(this, core);
-    }
-    // ulog.init() when reusing an existing log is deferred (currently at the end of the DUH2 constructor
-  }
-
-  // not thread safe - for startup
-  private void clearLog(PluginInfo ulogPluginInfo) {
-    if (ulogPluginInfo == null) return;
-    File tlogDir = UpdateLog.getTlogDir(core, ulogPluginInfo);
-    log.info("Clearing tlog files, tlogDir=" + tlogDir);
-    if (tlogDir.exists()) {
-      String[] files = UpdateLog.getLogList(tlogDir);
-      for (String file : files) {
-        File f = new File(tlogDir, file);
-        boolean s = f.delete();
-        if (!s) {
-          log.error("Could not remove tlog file:" + f.getAbsolutePath());
-          //throw new SolrException(ErrorCode.SERVER_ERROR, "Could not remove tlog file:" + f.getAbsolutePath());
-        }
-      }
-    }
-  }
-
   protected void callPostCommitCallbacks() {
     for (SolrEventListener listener : commitCallbacks) {
       listener.postCommit();
@@ -129,10 +100,43 @@ public abstract class UpdateHandler impl
     idFieldType = idField!=null ? idField.getType() : null;
     parseEventListeners();
     PluginInfo ulogPluginInfo = core.getSolrConfig().getPluginInfo(UpdateLog.class.getName());
-    if (!core.isReloaded() && !core.getDirectoryFactory().isPersistent()) {
-      clearLog(ulogPluginInfo);
+    
+
+    if (updateLog == null && ulogPluginInfo != null && ulogPluginInfo.isEnabled()) {
+      String dataDir = (String)ulogPluginInfo.initArgs.get("dir");
+      
+      String ulogDir = core.getCoreDescriptor().getUlogDir();
+      if (ulogDir != null) {
+        dataDir = ulogDir;
+      }
+      if (dataDir == null || dataDir.length()==0) {
+        dataDir = core.getDataDir();
+      }
+           
+      if (dataDir != null && dataDir.startsWith("hdfs:/")) {
+        DirectoryFactory dirFactory = core.getDirectoryFactory();
+        if (dirFactory instanceof HdfsDirectoryFactory) {
+          ulog = new HdfsUpdateLog(((HdfsDirectoryFactory)dirFactory).getConfDir());
+        } else {
+          ulog = new HdfsUpdateLog();
+        }
+        
+      } else {
+        ulog = new UpdateLog();
+      }
+      
+      if (!core.isReloaded() && !core.getDirectoryFactory().isPersistent()) {
+        ulog.clearLog(core, ulogPluginInfo);
+      }
+      
+      ulog.init(ulogPluginInfo);
+
+      ulog.init(this, core);
+    } else {
+      ulog = updateLog;
     }
-    initLog(ulogPluginInfo, updateLog);
+    // ulog.init() when reusing an existing log is deferred (currently at the end of the DUH2 constructor
+
   }
 
   /**

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/UpdateLog.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/UpdateLog.java?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/UpdateLog.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/UpdateLog.java Tue Jul 16 17:00:05 2013
@@ -17,15 +17,38 @@
 
 package org.apache.solr.update;
 
+import static org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase.FROMLEADER;
+import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.lucene.util.BytesRef;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.params.UpdateParams;
 import org.apache.solr.common.util.ExecutorUtil;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.LocalSolrQueryRequest;
@@ -34,9 +57,6 @@ import org.apache.solr.request.SolrReque
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.update.processor.DistributedUpdateProcessor;
-import org.apache.solr.update.processor.DistributedUpdateProcessorFactory;
-import org.apache.solr.update.processor.DistributingUpdateProcessorFactory;
-import org.apache.solr.update.processor.RunUpdateProcessorFactory;
 import org.apache.solr.update.processor.UpdateRequestProcessor;
 import org.apache.solr.update.processor.UpdateRequestProcessorChain;
 import org.apache.solr.util.DefaultSolrThreadFactory;
@@ -45,15 +65,6 @@ import org.apache.solr.util.plugin.Plugi
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.util.*;
-import java.util.concurrent.*;
-
-import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
-import static org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase.FROMLEADER;
-
 
 /** @lucene.experimental */
 public class UpdateLog implements PluginInfoInitialized {
@@ -64,6 +75,10 @@ public class UpdateLog implements Plugin
   public boolean debug = log.isDebugEnabled();
   public boolean trace = log.isTraceEnabled();
 
+  // TODO: hack
+  public FileSystem getFs() {
+    return null;
+  }
 
   public enum SyncLevel { NONE, FLUSH, FSYNC;
     public static SyncLevel getSyncLevel(String level){
@@ -108,27 +123,27 @@ public class UpdateLog implements Plugin
   }
 
   long id = -1;
-  private State state = State.ACTIVE;
-  private int operationFlags;  // flags to write in the transaction log with operations (i.e. FLAG_GAP)
+  protected State state = State.ACTIVE;
+  protected int operationFlags;  // flags to write in the transaction log with operations (i.e. FLAG_GAP)
 
-  private TransactionLog tlog;
-  private TransactionLog prevTlog;
-  private Deque<TransactionLog> logs = new LinkedList<TransactionLog>();  // list of recent logs, newest first
-  private LinkedList<TransactionLog> newestLogsOnStartup = new LinkedList<TransactionLog>();
-  private int numOldRecords;  // number of records in the recent logs
-
-  private Map<BytesRef,LogPtr> map = new HashMap<BytesRef, LogPtr>();
-  private Map<BytesRef,LogPtr> prevMap;  // used while committing/reopening is happening
-  private Map<BytesRef,LogPtr> prevMap2;  // used while committing/reopening is happening
-  private TransactionLog prevMapLog;  // the transaction log used to look up entries found in prevMap
-  private TransactionLog prevMapLog2;  // the transaction log used to look up entries found in prevMap
+  protected TransactionLog tlog;
+  protected TransactionLog prevTlog;
+  protected Deque<TransactionLog> logs = new LinkedList<TransactionLog>();  // list of recent logs, newest first
+  protected LinkedList<TransactionLog> newestLogsOnStartup = new LinkedList<TransactionLog>();
+  protected int numOldRecords;  // number of records in the recent logs
+
+  protected Map<BytesRef,LogPtr> map = new HashMap<BytesRef, LogPtr>();
+  protected Map<BytesRef,LogPtr> prevMap;  // used while committing/reopening is happening
+  protected Map<BytesRef,LogPtr> prevMap2;  // used while committing/reopening is happening
+  protected TransactionLog prevMapLog;  // the transaction log used to look up entries found in prevMap
+  protected TransactionLog prevMapLog2;  // the transaction log used to look up entries found in prevMap
 
-  private final int numDeletesToKeep = 1000;
-  private final int numDeletesByQueryToKeep = 100;
+  protected final int numDeletesToKeep = 1000;
+  protected final int numDeletesByQueryToKeep = 100;
   public final int numRecordsToKeep = 100;
 
   // keep track of deletes only... this is not updated on an add
-  private LinkedHashMap<BytesRef, LogPtr> oldDeletes = new LinkedHashMap<BytesRef, LogPtr>(numDeletesToKeep) {
+  protected LinkedHashMap<BytesRef, LogPtr> oldDeletes = new LinkedHashMap<BytesRef, LogPtr>(numDeletesToKeep) {
     @Override
     protected boolean removeEldestEntry(Map.Entry eldest) {
       return size() > numDeletesToKeep;
@@ -145,21 +160,21 @@ public class UpdateLog implements Plugin
     }
   }
 
-  private LinkedList<DBQ> deleteByQueries = new LinkedList<DBQ>();
+  protected LinkedList<DBQ> deleteByQueries = new LinkedList<DBQ>();
 
-  private String[] tlogFiles;
-  private File tlogDir;
-  private Collection<String> globalStrings;
+  protected String[] tlogFiles;
+  protected File tlogDir;
+  protected Collection<String> globalStrings;
 
-  private String dataDir;
-  private String lastDataDir;
+  protected String dataDir;
+  protected String lastDataDir;
 
-  private VersionInfo versionInfo;
+  protected VersionInfo versionInfo;
 
-  private SyncLevel defaultSyncLevel = SyncLevel.FLUSH;
+  protected SyncLevel defaultSyncLevel = SyncLevel.FLUSH;
 
   volatile UpdateHandler uhandler;    // a core reload can change this reference!
-  private volatile boolean cancelApplyBufferUpdate;
+  protected volatile boolean cancelApplyBufferUpdate;
   List<Long> startingVersions;
   int startingOperation;  // last operation in the logs on startup
 
@@ -199,7 +214,7 @@ public class UpdateLog implements Plugin
     if (ulogDir != null) {
       dataDir = ulogDir;
     }
-    
+
     if (dataDir == null || dataDir.length()==0) {
       dataDir = core.getDataDir();
     }
@@ -280,8 +295,8 @@ public class UpdateLog implements Plugin
 
   }
   
-  public File getLogDir() {
-    return tlogDir;
+  public String getLogDir() {
+    return tlogDir.getAbsolutePath();
   }
   
   public List<Long> getStartingVersions() {
@@ -295,7 +310,7 @@ public class UpdateLog implements Plugin
   /* Takes over ownership of the log, keeping it until no longer needed
      and then decrementing it's reference and dropping it.
    */
-  private void addOldLog(TransactionLog oldLog, boolean removeOld) {
+  protected void addOldLog(TransactionLog oldLog, boolean removeOld) {
     if (oldLog == null) return;
 
     numOldRecords += oldLog.numRecords();
@@ -326,7 +341,7 @@ public class UpdateLog implements Plugin
   }
 
 
-  public static String[] getLogList(File directory) {
+  public String[] getLogList(File directory) {
     final String prefix = TLOG_NAME+'.';
     String[] names = directory.list(new FilenameFilter() {
       @Override
@@ -334,6 +349,9 @@ public class UpdateLog implements Plugin
         return name.startsWith(prefix);
       }
     });
+    if (names == null) {
+      throw new RuntimeException(new FileNotFoundException(directory.getAbsolutePath()));
+    }
     Arrays.sort(names);
     return names;
   }
@@ -544,7 +562,7 @@ public class UpdateLog implements Plugin
     }
   }
 
-  private void newMap() {
+  protected void newMap() {
     prevMap2 = prevMap;
     prevMapLog2 = prevMapLog;
 
@@ -797,7 +815,7 @@ public class UpdateLog implements Plugin
   }
 
 
-  private void ensureLog() {
+  protected void ensureLog() {
     if (tlog == null) {
       String newLogName = String.format(Locale.ROOT, LOG_FILENAME_PATTERN, TLOG_NAME, id);
       tlog = new TransactionLog(new File(tlogDir, newLogName), globalStrings);
@@ -1145,7 +1163,7 @@ public class UpdateLog implements Plugin
 
 
 
-  private RecoveryInfo recoveryInfo;
+  protected RecoveryInfo recoveryInfo;
 
   class LogReplayer implements Runnable {
     private Logger loglog = log;  // set to something different?
@@ -1422,7 +1440,7 @@ public class UpdateLog implements Plugin
     }
   }
   
-  public static File getTlogDir(SolrCore core, PluginInfo info) {
+  protected String getTlogDir(SolrCore core, PluginInfo info) {
     String dataDir = (String) info.initArgs.get("dir");
     
     String ulogDir = core.getCoreDescriptor().getUlogDir();
@@ -1433,11 +1451,30 @@ public class UpdateLog implements Plugin
     if (dataDir == null || dataDir.length() == 0) {
       dataDir = core.getDataDir();
     }
-    
-    return new File(dataDir, TLOG_NAME);
+
+    return dataDir + "/" + TLOG_NAME;
+  }
+  
+  /**
+   * Clears the logs on the file system. Only call before init.
+   * 
+   * @param core the SolrCore
+   * @param ulogPluginInfo the init info for the UpdateHandler
+   */
+  public void clearLog(SolrCore core, PluginInfo ulogPluginInfo) {
+    if (ulogPluginInfo == null) return;
+    File tlogDir = new File(getTlogDir(core, ulogPluginInfo));
+    if (tlogDir.exists()) {
+      String[] files = getLogList(tlogDir);
+      for (String file : files) {
+        File f = new File(tlogDir, file);
+        boolean s = f.delete();
+        if (!s) {
+          log.error("Could not remove tlog file:" + f);
+        }
+      }
+    }
   }
   
 }
 
-
-

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java Tue Jul 16 17:00:05 2013
@@ -139,7 +139,7 @@ public class DistributedUpdateProcessor 
   // method in this update processor
   private boolean isLeader = true;
   private boolean forwardToLeader = false;
-  private boolean forwardToSubShard = false;
+  private boolean isSubShardLeader = false;
   private List<Node> nodes;
 
   private int numNodes;
@@ -222,9 +222,20 @@ public class DistributedUpdateProcessor 
         // Replica leader = slice.getLeader();
         Replica leaderReplica = zkController.getZkStateReader().getLeaderRetry(
             collection, shardId);
-        ZkCoreNodeProps leaderProps = new ZkCoreNodeProps(leaderReplica);
-        String coreNodeName = zkController.getCoreNodeName(req.getCore().getCoreDescriptor());
-        isLeader = coreNodeName.equals(leaderReplica.getName());
+        isLeader = leaderReplica.getName().equals(
+            req.getCore().getCoreDescriptor().getCloudDescriptor()
+                .getCoreNodeName());
+
+        if (!isLeader) {
+          isSubShardLeader = amISubShardLeader(coll, slice, id, doc);
+          if (isSubShardLeader) {
+            String myShardId = req.getCore().getCoreDescriptor().getCloudDescriptor().getShardId();
+            slice = coll.getSlice(myShardId);
+            shardId = myShardId;
+            leaderReplica = zkController.getZkStateReader().getLeaderRetry(collection, myShardId);
+            List<ZkCoreNodeProps> myReplicas = zkController.getZkStateReader().getReplicaProps(collection, shardId, leaderReplica.getName(), coreName, null, ZkStateReader.DOWN);
+          }
+        }
 
         DistribPhase phase =
             DistribPhase.parseParam(req.getParams().get(DISTRIB_UPDATE_PARAM));
@@ -232,15 +243,15 @@ public class DistributedUpdateProcessor 
         doDefensiveChecks(phase);
 
 
-        if (DistribPhase.FROMLEADER == phase) {
+        if (DistribPhase.FROMLEADER == phase && !isSubShardLeader) {
           // we are coming from the leader, just go local - add no urls
           forwardToLeader = false;
-        } else if (isLeader) {
+        } else if (isLeader || isSubShardLeader) {
           // that means I want to forward onto my replicas...
           // so get the replicas...
           forwardToLeader = false;
           List<ZkCoreNodeProps> replicaProps = zkController.getZkStateReader()
-              .getReplicaProps(collection, shardId, coreNodeName,
+              .getReplicaProps(collection, shardId, leaderReplica.getName(),
                   coreName, null, ZkStateReader.DOWN);
 
           if (replicaProps != null) {
@@ -272,7 +283,7 @@ public class DistributedUpdateProcessor 
         } else {
           // I need to forward onto the leader...
           nodes = new ArrayList<Node>(1);
-          nodes.add(new RetryNode(leaderProps, zkController.getZkStateReader(), collection, shardId));
+          nodes.add(new RetryNode(new ZkCoreNodeProps(leaderReplica), zkController.getZkStateReader(), collection, shardId));
           forwardToLeader = true;
         }
 
@@ -286,6 +297,31 @@ public class DistributedUpdateProcessor 
     return nodes;
   }
 
+  private boolean amISubShardLeader(DocCollection coll, Slice parentSlice, String id, SolrInputDocument doc) throws InterruptedException {
+    // Am I the leader of a shard in "construction" state?
+    String myShardId = req.getCore().getCoreDescriptor().getCloudDescriptor().getShardId();
+    Slice mySlice = coll.getSlice(myShardId);
+    if (Slice.CONSTRUCTION.equals(mySlice.getState())) {
+      Replica myLeader = zkController.getZkStateReader().getLeaderRetry(collection, myShardId);
+      boolean amILeader = myLeader.getName().equals(
+          req.getCore().getCoreDescriptor().getCloudDescriptor()
+              .getCoreNodeName());
+      if (amILeader) {
+        // Does the document belong to my hash range as well?
+        DocRouter.Range myRange = mySlice.getRange();
+        if (myRange == null) myRange = new DocRouter.Range(Integer.MIN_VALUE, Integer.MAX_VALUE);
+        if (parentSlice != null)  {
+          boolean isSubset = parentSlice.getRange() != null && myRange.isSubsetOf(parentSlice.getRange());
+          return isSubset && coll.getRouter().isTargetSlice(id, doc, req.getParams(), myShardId, coll);
+        } else  {
+          // delete by query case -- as long as I am a sub shard leader we're fine
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
   private List<Node> getSubShardLeaders(DocCollection coll, String shardId, String docId, SolrInputDocument doc) {
     Collection<Slice> allSlices = coll.getSlices();
     List<Node> nodes = null;
@@ -303,7 +339,6 @@ public class DistributedUpdateProcessor 
             if (nodes == null) nodes = new ArrayList<Node>();
             ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(sliceLeader);
             nodes.add(new StdNode(nodeProps));
-            forwardToSubShard = true;
           }
         }
       }
@@ -341,9 +376,11 @@ public class DistributedUpdateProcessor 
       }
     }
 
-    if (isLeader && !localIsLeader) {
+    if ((isLeader && !localIsLeader) || (isSubShardLeader && !localIsLeader)) {
       log.error("ClusterState says we are the leader, but locally we don't think so");
-      throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "ClusterState says we are the leader, but locally we don't think so");
+      throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE,
+          "ClusterState says we are the leader (" + zkController.getBaseUrl()
+              + "/" + req.getCore().getName() + "), but locally we don't think so. Request came from " + from);
     }
   }
 
@@ -356,16 +393,15 @@ public class DistributedUpdateProcessor 
     try {
       Replica leaderReplica = zkController.getZkStateReader().getLeaderRetry(
           collection, shardId);
-      String leaderCoreNodeName = leaderReplica.getName();
-
-      String coreNodeName = zkController.getCoreNodeName(req.getCore().getCoreDescriptor());
-      isLeader = coreNodeName.equals(leaderCoreNodeName);
+      isLeader = leaderReplica.getName().equals(
+          req.getCore().getCoreDescriptor().getCloudDescriptor()
+              .getCoreNodeName());
 
       // TODO: what if we are no longer the leader?
 
       forwardToLeader = false;
       List<ZkCoreNodeProps> replicaProps = zkController.getZkStateReader()
-          .getReplicaProps(collection, shardId, coreNodeName,
+          .getReplicaProps(collection, shardId, leaderReplica.getName(),
               req.getCore().getName());
       if (replicaProps != null) {
         nodes = new ArrayList<Node>(replicaProps.size());
@@ -404,7 +440,7 @@ public class DistributedUpdateProcessor 
       return;
     }
 
-    if (zkEnabled && isLeader)  {
+    if (zkEnabled && isLeader && !isSubShardLeader)  {
       DocCollection coll = zkController.getClusterState().getCollection(collection);
       List<Node> subShardLeaders = getSubShardLeaders(coll, cloudDesc.getShardId(), cmd.getHashableId(), cmd.getSolrInputDocument());
       // the list<node> will actually have only one element for an add request
@@ -425,10 +461,10 @@ public class DistributedUpdateProcessor 
 
       params = new ModifiableSolrParams(filterParams(req.getParams()));
       params.set(DISTRIB_UPDATE_PARAM,
-                 (isLeader ?
+                 (isLeader || isSubShardLeader ?
                   DistribPhase.FROMLEADER.toString() :
                   DistribPhase.TOLEADER.toString()));
-      if (isLeader) {
+      if (isLeader || isSubShardLeader) {
         params.set("distrib.from", ZkCoreNodeProps.getCoreUrl(
             zkController.getBaseUrl(), req.getCore().getName()));
       }
@@ -792,7 +828,7 @@ public class DistributedUpdateProcessor 
       return;
     }
 
-    if (zkEnabled && isLeader)  {
+    if (zkEnabled && isLeader && !isSubShardLeader)  {
       DocCollection coll = zkController.getClusterState().getCollection(collection);
       List<Node> subShardLeaders = getSubShardLeaders(coll, cloudDesc.getShardId(), null, null);
       // the list<node> will actually have only one element for an add request
@@ -812,10 +848,10 @@ public class DistributedUpdateProcessor 
 
       params = new ModifiableSolrParams(filterParams(req.getParams()));
       params.set(DISTRIB_UPDATE_PARAM,
-                 (isLeader ?
+                 (isLeader || isSubShardLeader ?
                   DistribPhase.FROMLEADER.toString() :
                   DistribPhase.TOLEADER.toString()));
-      if (isLeader) {
+      if (isLeader || isSubShardLeader) {
         params.set("distrib.from", ZkCoreNodeProps.getCoreUrl(
             zkController.getBaseUrl(), req.getCore().getName()));
       }
@@ -843,9 +879,11 @@ public class DistributedUpdateProcessor 
   }
 
   private void passParam(SolrParams params, ModifiableSolrParams fparams, String param) {
-    String value = params.get(param);
-    if (value != null) {
-      fparams.add(param, value);
+    String[] values = params.getParams(param);
+    if (values != null) {
+      for (String value : values) {
+        fparams.add(param, value);
+      }
     }
   }
 
@@ -897,7 +935,7 @@ public class DistributedUpdateProcessor 
         // Am I the leader for this slice?
         ZkCoreNodeProps coreLeaderProps = new ZkCoreNodeProps(leader);
         String leaderCoreNodeName = leader.getName();
-        String coreNodeName = zkController.getCoreNodeName(req.getCore().getCoreDescriptor());
+        String coreNodeName = req.getCore().getCoreDescriptor().getCloudDescriptor().getCoreNodeName();
         isLeader = coreNodeName.equals(leaderCoreNodeName);
 
         if (isLeader) {
@@ -985,20 +1023,51 @@ public class DistributedUpdateProcessor 
       vinfo.unblockUpdates();
     }
 
-    // forward to all replicas
-    if (leaderLogic && zkEnabled) {
-      List<Node> subShardLeaders = getSubShardLeaders(coll, cloudDesc.getShardId(), null, null);
-
+    if (zkEnabled)  {
+      // forward to all replicas
       ModifiableSolrParams params = new ModifiableSolrParams(filterParams(req.getParams()));
       params.set(VERSION_FIELD, Long.toString(cmd.getVersion()));
       params.set(DISTRIB_UPDATE_PARAM, DistribPhase.FROMLEADER.toString());
       params.set("update.from", ZkCoreNodeProps.getCoreUrl(
           zkController.getBaseUrl(), req.getCore().getName()));
-      if (subShardLeaders != null)  {
-        cmdDistrib.syncDelete(cmd, subShardLeaders, params);
+
+      boolean someReplicas = false;
+      boolean subShardLeader = false;
+      try {
+        subShardLeader = amISubShardLeader(coll, null, null, null);
+        if (subShardLeader)  {
+          String myShardId = req.getCore().getCoreDescriptor().getCloudDescriptor().getShardId();
+          Replica leaderReplica = zkController.getZkStateReader().getLeaderRetry(
+              collection, myShardId);
+          List<ZkCoreNodeProps> replicaProps = zkController.getZkStateReader()
+              .getReplicaProps(collection, myShardId, leaderReplica.getName(),
+                  req.getCore().getName(), null, ZkStateReader.DOWN);
+          if (replicaProps != null) {
+            List<Node> myReplicas = new ArrayList<Node>();
+            for (ZkCoreNodeProps replicaProp : replicaProps) {
+              myReplicas.add(new StdNode(replicaProp));
+            }
+            cmdDistrib.distribDelete(cmd, myReplicas, params);
+            someReplicas = true;
+          }
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new ZooKeeperException(ErrorCode.SERVER_ERROR, "", e);
       }
-      if (replicas != null) {
-        cmdDistrib.distribDelete(cmd, replicas, params);
+
+      if (leaderLogic) {
+        List<Node> subShardLeaders = getSubShardLeaders(coll, cloudDesc.getShardId(), null, null);
+        if (subShardLeaders != null)  {
+          cmdDistrib.syncDelete(cmd, subShardLeaders, params);
+        }
+        if (replicas != null) {
+          cmdDistrib.distribDelete(cmd, replicas, params);
+          someReplicas = true;
+        }
+      }
+
+      if (someReplicas)  {
         cmdDistrib.finish();
       }
     }
@@ -1131,15 +1200,37 @@ public class DistributedUpdateProcessor 
     }
   }
 
-
   @Override
   public void processCommit(CommitUpdateCommand cmd) throws IOException {
     updateCommand = cmd;
-
+    List<Node> nodes = null;
+    boolean singleLeader = false;
     if (zkEnabled) {
       zkCheck();
+      
+      nodes = getCollectionUrls(req, req.getCore().getCoreDescriptor()
+          .getCloudDescriptor().getCollectionName());
+      if (isLeader && nodes.size() == 1) {
+        singleLeader = true;
+      }
     }
     
+    if (!zkEnabled || req.getParams().getBool(COMMIT_END_POINT, false) || singleLeader) {
+      doLocalCommit(cmd);
+    } else if (zkEnabled) {
+      ModifiableSolrParams params = new ModifiableSolrParams(filterParams(req.getParams()));
+      if (!req.getParams().getBool(COMMIT_END_POINT, false)) {
+        params.set(COMMIT_END_POINT, true);
+
+        if (nodes != null) {
+          cmdDistrib.distribCommit(cmd, nodes, params);
+          finish();
+        }
+      }
+    }
+  }
+
+  private void doLocalCommit(CommitUpdateCommand cmd) throws IOException {
     if (vinfo != null) {
       vinfo.lockForUpdate();
     }
@@ -1156,23 +1247,6 @@ public class DistributedUpdateProcessor 
         vinfo.unlockForUpdate();
       }
     }
-    // TODO: we should consider this? commit everyone in the current collection
-
-    if (zkEnabled) {
-      ModifiableSolrParams params = new ModifiableSolrParams(filterParams(req.getParams()));
-      if (!req.getParams().getBool(COMMIT_END_POINT, false)) {
-        params.set(COMMIT_END_POINT, true);
-
-        String coreNodeName = zkController.getCoreNodeName(req.getCore().getCoreDescriptor());
-        List<Node> nodes = getCollectionUrls(req, req.getCore().getCoreDescriptor()
-            .getCloudDescriptor().getCollectionName(), coreNodeName);
-
-        if (nodes != null) {
-          cmdDistrib.distribCommit(cmd, nodes, params);
-          finish();
-        }
-      }
-    }
   }
   
   @Override
@@ -1184,7 +1258,7 @@ public class DistributedUpdateProcessor 
  
 
   
-  private List<Node> getCollectionUrls(SolrQueryRequest req, String collection, String coreNodeName) {
+  private List<Node> getCollectionUrls(SolrQueryRequest req, String collection) {
     ClusterState clusterState = req.getCore().getCoreDescriptor()
         .getCoreContainer().getZkController().getClusterState();
     List<Node> urls = new ArrayList<Node>();
@@ -1200,7 +1274,7 @@ public class DistributedUpdateProcessor 
       
       for (Entry<String,Replica> entry : shardMap.entrySet()) {
         ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(entry.getValue());
-        if (clusterState.liveNodesContain(nodeProps.getNodeName()) && !entry.getKey().equals(coreNodeName)) {
+        if (clusterState.liveNodesContain(nodeProps.getNodeName())) {
           urls.add(new StdNode(nodeProps));
         }
       }

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java Tue Jul 16 17:00:05 2013
@@ -108,7 +108,7 @@ public abstract class FieldMutatingUpdat
         // for now, don't allow it.
         if (! fname.equals(dest.getName()) ) {
           throw new SolrException(SERVER_ERROR,
-                                  "mutute returned field with different name: " 
+                                  "mutate returned field with different name: " 
                                   + fname + " => " + dest.getName());
         }
         doc.put(dest.getName(), dest);
@@ -118,7 +118,7 @@ public abstract class FieldMutatingUpdat
   }
   
   /**
-   * Interface for idenfifying which fileds should be mutated
+   * Interface for identifying which fields should be mutated
    */
   public static interface FieldNameSelector {
     public boolean shouldMutate(final String fieldName);

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessorFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessorFactory.java?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessorFactory.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessorFactory.java Tue Jul 16 17:00:05 2013
@@ -58,6 +58,15 @@ import org.apache.solr.util.plugin.SolrC
  * <b>at least one of each</b> to be selected.
  * </p>
  * <p>
+ * The following additional selector may be specified as a &lt;bool&gt; - when specified
+ * as false, only fields that <b>do not</b> match a schema field/dynamic field are selected;
+ * when specified as true, only fields that <b>do</b> match a schema field/dynamic field are
+ * selected:
+ * </p>
+ * <ul>
+ *   <li><code>fieldNameMatchesSchemaField</code> - selecting specific fields based on whether or not they match a schema field</li>
+ * </ul>
+ * <p>
  * One or more <code>excludes</code> &lt;lst&gt; params may also be specified, 
  * containing any of the above criteria, identifying fields to be excluded 
  * from seelction even if they match the selection criteria.  As with the main 
@@ -71,7 +80,7 @@ import org.apache.solr.util.plugin.SolrC
  * fields will be mutated if the name starts with "foo" <i>or</i> "bar"; 
  * <b>unless</b> the field name contains the substring "SKIP" <i>or</i> 
  * the fieldType is (or subclasses) DateField.  Meaning a field named 
- * "foo_SKIP" is gaurunteed not to be selected, but a field named "bar_smith" 
+ * "foo_SKIP" is guaranteed not to be selected, but a field named "bar_smith" 
  * that uses StrField will be selected.
  * </p>
  * <pre class="prettyprint">
@@ -156,41 +165,47 @@ public abstract class FieldMutatingUpdat
     
     return params;
   }
-                                                            
-
-  /**
-   * Handles common initialization related to source fields for 
-   * constructing the FieldNameSelector to be used.
-   *
-   * Will error if any unexpected init args are found, so subclasses should
-   * remove any subclass-specific init args before calling this method.
-   */
-  @SuppressWarnings("unchecked")
-  @Override
-  public void init(NamedList args) {
-
-    inclusions = parseSelectorParams(args);
-
+                               
+  public static Collection<SelectorParams> parseSelectorExclusionParams(NamedList args) {
+    Collection<SelectorParams> exclusions = new ArrayList<SelectorParams>();
     List<Object> excList = args.getAll("exclude");
     for (Object excObj : excList) {
       if (null == excObj) {
         throw new SolrException
-          (SERVER_ERROR, "'exclude' init param can not be null"); 
+            (SERVER_ERROR, "'exclude' init param can not be null");
       }
       if (! (excObj instanceof NamedList) ) {
         throw new SolrException
-          (SERVER_ERROR, "'exclude' init param must be <lst/>"); 
+            (SERVER_ERROR, "'exclude' init param must be <lst/>");
       }
       NamedList exc = (NamedList) excObj;
       exclusions.add(parseSelectorParams(exc));
       if (0 < exc.size()) {
-        throw new SolrException(SERVER_ERROR, 
-                                "Unexpected 'exclude' init sub-param(s): '" + 
-                                args.getName(0) + "'");
+        throw new SolrException(SERVER_ERROR,
+            "Unexpected 'exclude' init sub-param(s): '" +
+                args.getName(0) + "'");
       }
       // call once per instance
       args.remove("exclude");
     }
+    return exclusions;
+  }
+  
+
+  /**
+   * Handles common initialization related to source fields for 
+   * constructing the FieldNameSelector to be used.
+   *
+   * Will error if any unexpected init args are found, so subclasses should
+   * remove any subclass-specific init args before calling this method.
+   */
+  @SuppressWarnings("unchecked")
+  @Override
+  public void init(NamedList args) {
+
+    inclusions = parseSelectorParams(args);
+    exclusions = parseSelectorExclusionParams(args);
+
     if (0 < args.size()) {
       throw new SolrException(SERVER_ERROR, 
                               "Unexpected init param(s): '" + 

Modified: lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java Tue Jul 16 17:00:05 2013
@@ -129,4 +129,24 @@ public class PropertiesUtil {
     }
   }
 
+  /**
+   * Parse the given String value as an integer.  If the string cannot
+   * be parsed, returns the default
+   * @param value    the value to parse
+   * @param defValue the default to return if the value cannot be parsed
+   * @return an integer version of the passed in value
+   */
+  public static Integer toInteger(String value, Integer defValue) {
+    try {
+      return Integer.parseInt(value);
+    }
+    catch (NumberFormatException e) {
+      return defValue;
+    }
+  }
+
+  public static boolean toBoolean(String value) {
+    return "true".equalsIgnoreCase(value) || "on".equalsIgnoreCase(value);
+  }
+
 }

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/log4j.properties
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/log4j.properties?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/log4j.properties (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/log4j.properties Tue Jul 16 17:00:05 2013
@@ -7,3 +7,4 @@ log4j.appender.CONSOLE.layout=org.apache
 log4j.appender.CONSOLE.layout.ConversionPattern=%-5p - %d{yyyy-MM-dd HH:mm:ss.SSS}; %C; %m\n
 
 log4j.logger.org.apache.zookeeper=WARN
+log4j.logger.org.apache.hadoop=WARN

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema-folding.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema-folding.xml?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema-folding.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema-folding.xml Tue Jul 16 17:00:05 2013
@@ -221,8 +221,6 @@
     <fieldType name="float" class="solr.TrieFloatField" precisionStep="4" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="long" class="solr.TrieLongField" precisionStep="4" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="double" class="solr.TrieDoubleField" precisionStep="4" omitNorms="true" positionIncrementGap="0"/>
-    <fieldType name="byte" class="solr.ByteField" omitNorms="true" positionIncrementGap="0"/>
-    <fieldType name="short" class="solr.ShortField" omitNorms="true" positionIncrementGap="0"/>
     <fieldtype name="boolean" class="solr.BoolField" sortMissingLast="true"/>
     <fieldtype name="date" class="solr.TrieDateField" precisionStep="0"/>
   </types>
@@ -233,8 +231,6 @@
     <field name="float_f" type="float"/>
     <field name="long_f" type="long"/>
     <field name="double_f" type="double"/>
-    <field name="byte_f" type="byte"/>
-    <field name="short_f" type="short"/>
     <field name="bool_f" type="boolean"/>
     <field name="date_f" type="date"/>
 

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema.xml?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/schema.xml Tue Jul 16 17:00:05 2013
@@ -48,8 +48,6 @@
     <fieldType name="float" class="solr.TrieFloatField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="long" class="solr.TrieLongField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="double" class="solr.TrieDoubleField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
-    <fieldType name="byte" class="solr.ByteField" omitNorms="true" positionIncrementGap="0"/>
-    <fieldType name="short" class="solr.ShortField" omitNorms="true" positionIncrementGap="0"/>
 
 
     <fieldType name="tint" class="solr.TrieIntField" precisionStep="8" omitNorms="true" positionIncrementGap="0"/>
@@ -609,10 +607,6 @@
    <dynamicField name="*_dt" type="date"    indexed="true"  stored="true"/>
    <dynamicField name="*_dt1" type="date"    indexed="true"  stored="true" multiValued="false"/>
    <dynamicField name="*_bcd" type="bcdstr" indexed="true"  stored="true"/>
-   <dynamicField name="*_by"  type="byte"  indexed="true" stored="true"/>
-   <dynamicField name="*_by1" type="byte"  indexed="true" stored="true" multiValued="false"/>
-   <dynamicField name="*_sh" type="short"  indexed="true" stored="true"/>
-   <dynamicField name="*_sh1" type="short"  indexed="true" stored="true" multiValued="false"/>
 
 
       <!-- some trie-coded dynamic fields for faster range queries -->

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-SOLR-749.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-SOLR-749.xml?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-SOLR-749.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-SOLR-749.xml Tue Jul 16 17:00:05 2013
@@ -19,6 +19,7 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <queryParser name="foo" class="FooQParserPlugin"/>
   <!-- override the default "lucene" qparser -->

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-altdirectory.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-altdirectory.xml?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-altdirectory.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-altdirectory.xml Tue Jul 16 17:00:05 2013
@@ -19,6 +19,7 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
   <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
   <directoryFactory name="DirectoryFactory" class="org.apache.solr.core.AlternateDirectoryTest$TestFSDirectoryFactory"></directoryFactory>
   <indexReaderFactory name="IndexReaderFactory" class="org.apache.solr.core.AlternateDirectoryTest$TestIndexReaderFactory"></indexReaderFactory >

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml Tue Jul 16 17:00:05 2013
@@ -22,6 +22,7 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
   <dataDir>${solr.data.dir:}</dataDir>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
 </config>

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-caching.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-caching.xml?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-caching.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-caching.xml Tue Jul 16 17:00:05 2013
@@ -16,6 +16,7 @@
   -->
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <query>
     <cache name="lfuCacheDecayFalse"
@@ -35,4 +36,4 @@
            size="10"
            initialSize="9" />
   </query>
-</config>
\ No newline at end of file
+</config>

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-components-name.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-components-name.xml?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-components-name.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-components-name.xml Tue Jul 16 17:00:05 2013
@@ -17,21 +17,12 @@
  limitations under the License.
 -->
 
-<!-- This is a "kitchen sink" config file that tests can use.
-     When writting a new test, feel free to add *new* items (plugins,
-     config options, etc...) as long as they don't break any existing
-     tests.  if you need to test something esoteric please add a new
-     "solrconfig-your-esoteric-purpose.xml" config file.
-
-     Note in particular that this test is used by MinimalSchemaTest so
-     Anything added to this file needs to work correctly even if there
-     is now uniqueKey or defaultSearch Field.
-  -->
-
 <config>
 
   <jmx />
 
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+
   <!-- Used to specify an alternate directory to hold all index data.
        It defaults to "index" if not present, and should probably
        not be changed if replication is in use. -->

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml Tue Jul 16 17:00:05 2013
@@ -17,7 +17,6 @@
  limitations under the License.
 -->
 
-
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
@@ -26,6 +25,7 @@
   <requestHandler name="standard" class="solr.StandardRequestHandler"/>
 
   <indexConfig>
+   <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
    <deletionPolicy class="solr.SolrDeletionPolicy">
     <str name="keepOptimizedOnly">true</str>
     <str name="maxCommitsToKeep">3</str>

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml Tue Jul 16 17:00:05 2013
@@ -21,6 +21,7 @@
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <indexConfig>
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
     <deletionPolicy class="org.apache.solr.core.FakeDeletionPolicy">
       <str name="var1">value1</str>
 		  <str name="var2">value2</str>

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-elevate.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-elevate.xml?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-elevate.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-elevate.xml Tue Jul 16 17:00:05 2013
@@ -32,6 +32,7 @@
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
 
   <indexConfig>
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
     <lockType>single</lockType>
   </indexConfig>
   
@@ -66,10 +67,8 @@
     </listener>
     -->
 
-
   </updateHandler>
 
-
   <query>
     <!-- Maximum number of clauses in a boolean query... can affect
         range or wildcard queries that expand to big boolean
@@ -77,7 +76,6 @@
     -->
     <maxBooleanClauses>1024</maxBooleanClauses>
 
-
     <!-- Cache specification for Filters or DocSets - unordered set of *all* documents
          that match a particular query.
       -->
@@ -108,19 +106,16 @@
     <!-- set maxSize artificially low to exercise both types of sets -->
     <HashDocSet maxSize="3" loadFactor="0.75"/>
 
-
     <!-- boolToFilterOptimizer converts boolean clauses with zero boost
          into cached filters if the number of docs selected by the clause exceeds
          the threshold (represented as a fraction of the total index)
     -->
     <boolTofilterOptimizer enabled="false" cacheSize="32" threshold=".05"/>
 
-
   </query>
 
   <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
 
-
   <!-- test elevation -->
   <searchComponent name="elevate" class="org.apache.solr.handler.component.QueryElevationComponent" >
     <str name="queryFieldType">string</str>

Modified: lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-enableplugin.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-enableplugin.xml?rev=1503797&r1=1503796&r2=1503797&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-enableplugin.xml (original)
+++ lucene/dev/branches/lucene3069/solr/core/src/test-files/solr/collection1/conf/solrconfig-enableplugin.xml Tue Jul 16 17:00:05 2013
@@ -28,6 +28,7 @@
 
   <indexConfig>
     <lockType>single</lockType>
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
   </indexConfig>
 
   <updateHandler class="solr.DirectUpdateHandler2">
@@ -45,7 +46,6 @@
   <requestHandler name="disabled" class="solr.StandardRequestHandler" enable="false"/>
   <requestHandler name="enabled" class="solr.StandardRequestHandler" enable="true"/>
 
-
   <!-- test query parameter defaults -->
   <requestHandler name="lazy" class="solr.StandardRequestHandler" startup="lazy">
   </requestHandler>