You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ja...@apache.org on 2013/05/30 09:53:46 UTC

svn commit: r1487777 [43/50] - in /lucene/dev/branches/security: ./ dev-tools/ dev-tools/eclipse/dot.settings/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/replicator/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/ma...

Modified: lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java (original)
+++ lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/SolrIndexConfig.java Thu May 30 07:53:18 2013
@@ -18,6 +18,8 @@
 package org.apache.solr.update;
 
 import org.apache.lucene.index.*;
+import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
+import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.Version;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -55,6 +57,8 @@ public class SolrIndexConfig {
   public final PluginInfo mergeSchedulerInfo;
   public final int termIndexInterval;
   
+  public final PluginInfo mergedSegmentWarmerInfo;
+  
   public String infoStreamFile = null;
 
   // Available lock types
@@ -81,6 +85,7 @@ public class SolrIndexConfig {
     mergePolicyInfo = null;
     mergeSchedulerInfo = null;
     defaultMergePolicyClassName = TieredMergePolicy.class.getName();
+    mergedSegmentWarmerInfo = null;
   }
   
   /**
@@ -135,6 +140,11 @@ public class SolrIndexConfig {
       infoStreamFile= solrConfig.get(prefix + "/infoStream/@file", null);
       log.info("IndexWriter infoStream debug log is enabled: " + infoStreamFile);
     }
+    
+    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");
+    }
   }
 
   /*
@@ -182,6 +192,16 @@ public class SolrIndexConfig {
     if (maxIndexingThreads != -1) {
       iwc.setMaxThreadStates(maxIndexingThreads);
     }
+    
+    if (mergedSegmentWarmerInfo != null) {
+      // TODO: add infostream -> normal logging system (there is an issue somewhere)
+      IndexReaderWarmer warmer = schema.getResourceLoader().newInstance(mergedSegmentWarmerInfo.className, 
+                                                                        IndexReaderWarmer.class,
+                                                                        null,
+                                                                        new Class[] { InfoStream.class },
+                                                                        new Object[] { InfoStream.NO_OUTPUT });
+      iwc.setMergedSegmentWarmer(warmer);
+    }
 
     return iwc;
   }

Modified: lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/SolrIndexSplitter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/SolrIndexSplitter.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/SolrIndexSplitter.java (original)
+++ lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/SolrIndexSplitter.java Thu May 30 07:53:18 2013
@@ -28,12 +28,15 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.OpenBitSet;
 import org.apache.solr.common.cloud.DocRouter;
+import org.apache.solr.common.cloud.HashBasedRouter;
 import org.apache.solr.common.util.Hash;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.SchemaField;
+import org.apache.solr.schema.StrField;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.util.RefCounted;
 import org.slf4j.Logger;
@@ -52,14 +55,26 @@ public class SolrIndexSplitter {
   DocRouter.Range[] rangesArr; // same as ranges list, but an array for extra speed in inner loops
   List<String> paths;
   List<SolrCore> cores;
+  DocRouter router;
+  HashBasedRouter hashRouter;
+  int numPieces;
+  int currPartition = 0;
 
   public SolrIndexSplitter(SplitIndexCommand cmd) {
-    field = cmd.getReq().getSchema().getUniqueKeyField();
     searcher = cmd.getReq().getSearcher();
+    field = searcher.getSchema().getUniqueKeyField();
     ranges = cmd.ranges;
-    rangesArr = ranges.toArray(new DocRouter.Range[ranges.size()]);
     paths = cmd.paths;
     cores = cmd.cores;
+    router = cmd.router;
+    hashRouter = router instanceof HashBasedRouter ? (HashBasedRouter)router : null;
+
+    if (ranges == null) {
+      numPieces =  paths != null ? paths.size() : cores.size();
+    } else  {
+      numPieces = ranges.size();
+      rangesArr = ranges.toArray(new DocRouter.Range[ranges.size()]);
+    }
   }
 
   public void split() throws IOException {
@@ -67,7 +82,7 @@ public class SolrIndexSplitter {
     List<AtomicReaderContext> leaves = searcher.getTopReaderContext().leaves();
     List<OpenBitSet[]> segmentDocSets = new ArrayList<OpenBitSet[]>(leaves.size());
 
-    log.info("SolrIndexSplitter: partitions=" + ranges.size() + " segments="+leaves.size());
+    log.info("SolrIndexSplitter: partitions=" + numPieces + " segments="+leaves.size());
 
     for (AtomicReaderContext readerContext : leaves) {
       assert readerContext.ordInParent == segmentDocSets.size();  // make sure we're going in order
@@ -82,8 +97,8 @@ public class SolrIndexSplitter {
     // - would be more efficient on the read side, but prob less efficient merging
 
     IndexReader[] subReaders = new IndexReader[leaves.size()];
-    for (int partitionNumber=0; partitionNumber<ranges.size(); partitionNumber++) {
-      log.info("SolrIndexSplitter: partition #" + partitionNumber + " range=" + ranges.get(partitionNumber));
+    for (int partitionNumber=0; partitionNumber<numPieces; partitionNumber++) {
+      log.info("SolrIndexSplitter: partition #" + partitionNumber + (ranges != null ? " range=" + ranges.get(partitionNumber) : ""));
 
       for (int segmentNumber = 0; segmentNumber<subReaders.length; segmentNumber++) {
         subReaders[segmentNumber] = new LiveDocsReader( leaves.get(segmentNumber), segmentDocSets.get(segmentNumber)[partitionNumber] );
@@ -100,9 +115,9 @@ public class SolrIndexSplitter {
       } else {
         SolrCore core = searcher.getCore();
         String path = paths.get(partitionNumber);
-        iw = SolrIndexWriter.create("SplittingIndexWriter"+partitionNumber + " " + ranges.get(partitionNumber), path,
-                                    core.getDirectoryFactory(), true, core.getSchema(),
-                                    core.getSolrConfig().indexConfig, core.getDeletionPolicy(), core.getCodec(), true);
+        iw = SolrIndexWriter.create("SplittingIndexWriter"+partitionNumber + (ranges != null ? " " + ranges.get(partitionNumber) : ""), path,
+                                    core.getDirectoryFactory(), true, core.getLatestSchema(),
+                                    core.getSolrConfig().indexConfig, core.getDeletionPolicy(), core.getCodec());
       }
 
       try {
@@ -129,7 +144,7 @@ public class SolrIndexSplitter {
 
   OpenBitSet[] split(AtomicReaderContext readerContext) throws IOException {
     AtomicReader reader = readerContext.reader();
-    OpenBitSet[] docSets = new OpenBitSet[ranges.size()];
+    OpenBitSet[] docSets = new OpenBitSet[numPieces];
     for (int i=0; i<docSets.length; i++) {
       docSets[i] = new OpenBitSet(reader.maxDoc());
     }
@@ -143,21 +158,36 @@ public class SolrIndexSplitter {
     BytesRef term = null;
     DocsEnum docsEnum = null;
 
+    CharsRef idRef = new CharsRef(100);
     for (;;) {
       term = termsEnum.next();
       if (term == null) break;
 
       // figure out the hash for the term
-      // TODO: hook in custom hashes (or store hashes)
-      int hash = Hash.murmurhash3_x86_32(term.bytes, term.offset, term.length, 0);
+
+      // FUTURE: if conversion to strings costs too much, we could
+      // specialize and use the hash function that can work over bytes.
+      idRef = field.getType().indexedToReadable(term, idRef);
+      String idString = idRef.toString();
+
+      int hash = 0;
+      if (hashRouter != null) {
+        hash = hashRouter.sliceHash(idString, null, null);
+      }
+      // int hash = Hash.murmurhash3_x86_32(ref, ref.offset, ref.length, 0);
 
       docsEnum = termsEnum.docs(liveDocs, docsEnum, DocsEnum.FLAG_NONE);
       for (;;) {
         int doc = docsEnum.nextDoc();
         if (doc == DocsEnum.NO_MORE_DOCS) break;
-        for (int i=0; i<rangesArr.length; i++) {      // inner-loop: use array here for extra speed.
-          if (rangesArr[i].includes(hash)) {
-            docSets[i].fastSet(doc);
+        if (ranges == null) {
+          docSets[currPartition].fastSet(doc);
+          currPartition = (currPartition + 1) % numPieces;
+        } else  {
+          for (int i=0; i<rangesArr.length; i++) {      // inner-loop: use array here for extra speed.
+            if (rangesArr[i].includes(hash)) {
+              docSets[i].fastSet(doc);
+            }
           }
         }
       }
@@ -184,11 +214,6 @@ public class SolrIndexSplitter {
     }
 
     @Override
-    public boolean hasDeletions() {
-      return (in.maxDoc() != numDocs);
-    }
-
-    @Override
     public Bits getLiveDocs() {
       return liveDocs;
     }

Modified: lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java (original)
+++ lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java Thu May 30 07:53:18 2013
@@ -56,10 +56,10 @@ public class SolrIndexWriter extends Ind
   String name;
   private DirectoryFactory directoryFactory;
 
-  public static SolrIndexWriter create(String name, String path, DirectoryFactory directoryFactory, boolean create, IndexSchema schema, SolrIndexConfig config, IndexDeletionPolicy delPolicy, Codec codec, boolean forceNewDirectory) throws IOException {
+  public static SolrIndexWriter create(String name, String path, DirectoryFactory directoryFactory, boolean create, IndexSchema schema, SolrIndexConfig config, IndexDeletionPolicy delPolicy, Codec codec) throws IOException {
 
     SolrIndexWriter w = null;
-    final Directory d = directoryFactory.get(path, DirContext.DEFAULT, config.lockType, forceNewDirectory);
+    final Directory d = directoryFactory.get(path, DirContext.DEFAULT, config.lockType);
     try {
       w = new SolrIndexWriter(name, path, d, create, schema, 
                               config, delPolicy, codec);
@@ -175,6 +175,7 @@ public class SolrIndexWriter extends Ind
 
   @Override
   public void rollback() throws IOException {
+    Directory dir = getDirectory();
     try {
       while (true) {
         try {
@@ -187,7 +188,7 @@ public class SolrIndexWriter extends Ind
       }
     } finally {
       isClosed = true;
-      directoryFactory.release(getDirectory());
+      directoryFactory.release(dir);
       numCloses.incrementAndGet();
     }
   }

Modified: lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/SplitIndexCommand.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/SplitIndexCommand.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/SplitIndexCommand.java (original)
+++ lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/SplitIndexCommand.java Thu May 30 07:53:18 2013
@@ -34,13 +34,14 @@ public class SplitIndexCommand extends U
   public List<String> paths;
   public List<SolrCore> cores;  // either paths or cores should be specified
   public List<DocRouter.Range> ranges;
-  // TODO: allow specification of custom hash function
+  public DocRouter router;
 
-  public SplitIndexCommand(SolrQueryRequest req, List<String> paths,  List<SolrCore> cores, List<DocRouter.Range> ranges) {
+  public SplitIndexCommand(SolrQueryRequest req, List<String> paths,  List<SolrCore> cores, List<DocRouter.Range> ranges, DocRouter router) {
     super(req);
     this.paths = paths;
     this.cores = cores;
     this.ranges = ranges;
+    this.router = router;
   }
 
   @Override
@@ -54,6 +55,7 @@ public class SplitIndexCommand extends U
     sb.append(",paths=" + paths);
     sb.append(",cores=" + cores);
     sb.append(",ranges=" + ranges);
+    sb.append(",router=" + router);
     sb.append('}');
     return sb.toString();
   }

Modified: lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/UpdateHandler.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/UpdateHandler.java (original)
+++ lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/UpdateHandler.java Thu May 30 07:53:18 2013
@@ -22,14 +22,11 @@ import java.io.File;
 import java.io.IOException;
 import java.util.Vector;
 
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrEventListener;
 import org.apache.solr.core.SolrInfoMBean;
 import org.apache.solr.schema.FieldType;
-import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.util.plugin.SolrCoreAware;
 import org.slf4j.Logger;
@@ -47,7 +44,6 @@ public abstract class UpdateHandler impl
   protected final static Logger log = LoggerFactory.getLogger(UpdateHandler.class);
 
   protected final SolrCore core;
-  protected final IndexSchema schema;
 
   protected final SchemaField idField;
   protected final FieldType idFieldType;
@@ -122,16 +118,23 @@ public abstract class UpdateHandler impl
   }
 
   public UpdateHandler(SolrCore core)  {
+    this(core, null);
+  }
+  
+  public UpdateHandler(SolrCore core, UpdateLog updateLog)  {
     this.core=core;
-    schema = core.getSchema();
-    idField = schema.getUniqueKeyField();
+    idField = core.getLatestSchema().getUniqueKeyField();
     idFieldType = idField!=null ? idField.getType() : null;
     parseEventListeners();
     PluginInfo ulogPluginInfo = core.getSolrConfig().getPluginInfo(UpdateLog.class.getName());
     if (!core.isReloaded() && !core.getDirectoryFactory().isPersistent()) {
       clearLog(ulogPluginInfo);
     }
-    initLog(ulogPluginInfo);
+    if (updateLog == null) {
+      initLog(ulogPluginInfo);
+    } else {
+      this.ulog = updateLog;
+    }
   }
 
   /**
@@ -139,11 +142,10 @@ public abstract class UpdateHandler impl
    * all of the index files.
    * 
    * @param rollback IndexWriter if true else close
-   * @param forceNewDir Force a new Directory instance
    * 
    * @throws IOException If there is a low-level I/O error.
    */
-  public abstract void newIndexWriter(boolean rollback, boolean forceNewDir) throws IOException;
+  public abstract void newIndexWriter(boolean rollback) throws IOException;
 
   public abstract SolrCoreState getSolrCoreState();
 

Modified: lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/UpdateLog.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/UpdateLog.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/UpdateLog.java (original)
+++ lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/UpdateLog.java Thu May 30 07:53:18 2013
@@ -23,6 +23,7 @@ import org.apache.solr.common.SolrExcept
 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;
@@ -34,8 +35,10 @@ import org.apache.solr.response.SolrQuer
 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;
 import org.apache.solr.util.RefCounted;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
@@ -986,7 +989,7 @@ public class UpdateLog implements Plugin
     }
   }
 
-
+  /** The RecentUpdates object returned must be closed after use */
   public RecentUpdates getRecentUpdates() {
     Deque<TransactionLog> logList;
     synchronized (this) {
@@ -1006,9 +1009,21 @@ public class UpdateLog implements Plugin
 
     // TODO: what if I hand out a list of updates, then do an update, then hand out another list (and
     // one of the updates I originally handed out fell off the list).  Over-request?
-    RecentUpdates recentUpdates = new RecentUpdates();
-    recentUpdates.logList = logList;
-    recentUpdates.update();
+
+    boolean success = false;
+    RecentUpdates recentUpdates = null;
+    try {
+      recentUpdates = new RecentUpdates();
+      recentUpdates.logList = logList;
+      recentUpdates.update();
+      success = true;
+    } finally {
+      // defensive: if some unknown exception is thrown,
+      // make sure we close so that the tlogs are decref'd
+      if (!success && recentUpdates != null) {
+        recentUpdates.close();
+      }
+    }
 
     return recentUpdates;
   }
@@ -1129,14 +1144,15 @@ public class UpdateLog implements Plugin
   class LogReplayer implements Runnable {
     private Logger loglog = log;  // set to something different?
 
-    List<TransactionLog> translogs;
+    Deque<TransactionLog> translogs;
     TransactionLog.LogReader tlogReader;
     boolean activeLog;
     boolean finishing = false;  // state where we lock out other updates and finish those updates that snuck in before we locked
     boolean debug = loglog.isDebugEnabled();
 
     public LogReplayer(List<TransactionLog> translogs, boolean activeLog) {
-      this.translogs = translogs;
+      this.translogs = new LinkedList<TransactionLog>();
+      this.translogs.addAll(translogs);
       this.activeLog = activeLog;
     }
 
@@ -1156,7 +1172,9 @@ public class UpdateLog implements Plugin
       SolrRequestInfo.setRequestInfo(new SolrRequestInfo(req, rsp));    // setting request info will help logging
 
       try {
-        for (TransactionLog translog : translogs) {
+        for(;;) {
+          TransactionLog translog = translogs.pollFirst();
+          if (translog == null) break;
           doReplay(translog);
         }
       } catch (SolrException e) {
@@ -1176,6 +1194,13 @@ public class UpdateLog implements Plugin
         if (finishing) {
           versionInfo.unblockUpdates();
         }
+
+        // clean up in case we hit some unexpected exception and didn't get
+        // to more transaction logs
+        for (TransactionLog translog : translogs) {
+          log.error("ERROR: didn't get to recover from tlog " + translog);
+          translog.decref();
+        }
       }
 
       loglog.warn("Log replay finished. recoveryInfo=" + recoveryInfo);
@@ -1195,13 +1220,8 @@ public class UpdateLog implements Plugin
         // NOTE: we don't currently handle a core reload during recovery.  This would cause the core
         // to change underneath us.
 
-        // TODO: use the standard request factory?  We won't get any custom configuration instantiating this way.
-        RunUpdateProcessorFactory runFac = new RunUpdateProcessorFactory();
-        DistributedUpdateProcessorFactory magicFac = new DistributedUpdateProcessorFactory();
-        runFac.init(new NamedList());
-        magicFac.init(new NamedList());
-
-        UpdateRequestProcessor proc = magicFac.getInstance(req, rsp, runFac.getInstance(req, rsp, null));
+        UpdateRequestProcessorChain processorChain = req.getCore().getUpdateProcessingChain(null);
+        UpdateRequestProcessor proc = processorChain.createProcessor(req, rsp);
 
         long commitVersion = 0;
         int operationAndFlags = 0;

Modified: lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/VersionInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/VersionInfo.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/VersionInfo.java (original)
+++ lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/VersionInfo.java Thu May 30 07:53:18 2013
@@ -27,7 +27,6 @@ import org.apache.lucene.queries.functio
 import org.apache.lucene.util.BitUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.SolrIndexSearcher;
@@ -79,9 +78,9 @@ public class VersionInfo {
 
   public VersionInfo(UpdateLog ulog, int nBuckets) {
     this.ulog = ulog;
-    SolrCore core = ulog.uhandler.core;
-    versionField = getAndCheckVersionField(core.getSchema());
-    idField = core.getSchema().getUniqueKeyField();
+    IndexSchema schema = ulog.uhandler.core.getLatestSchema(); 
+    versionField = getAndCheckVersionField(schema);
+    idField = schema.getUniqueKeyField();
     buckets = new VersionBucket[ BitUtil.nextHighestPowerOfTwo(nBuckets) ];
     for (int i=0; i<buckets.length; i++) {
       buckets[i] = new VersionBucket();

Modified: lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/CloneFieldUpdateProcessorFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/CloneFieldUpdateProcessorFactory.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/CloneFieldUpdateProcessorFactory.java (original)
+++ lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/CloneFieldUpdateProcessorFactory.java Thu May 30 07:53:18 2013
@@ -17,17 +17,12 @@
 package org.apache.solr.update.processor;
 
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.HashSet;
 
-import org.apache.solr.schema.IndexSchema;
-
 import org.apache.solr.core.SolrCore;
-import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.util.plugin.SolrCoreAware;
 
 import org.apache.solr.common.util.NamedList;
@@ -36,15 +31,13 @@ import org.apache.solr.common.SolrInputF
 import org.apache.solr.common.SolrInputDocument;
 
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
-import static org.apache.solr.common.SolrException.ErrorCode.*;
+import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
 
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 
 import org.apache.solr.update.AddUpdateCommand;
 
-import org.apache.solr.update.processor.FieldMutatingUpdateProcessorFactory;
 import org.apache.solr.update.processor.FieldMutatingUpdateProcessorFactory.SelectorParams;
 import org.apache.solr.update.processor.FieldMutatingUpdateProcessor.FieldNameSelector;
 
@@ -200,12 +193,10 @@ public class CloneFieldUpdateProcessorFa
   @Override
   public void inform(final SolrCore core) {
     
-    final IndexSchema schema = core.getSchema();
-
     srcSelector = 
       FieldMutatingUpdateProcessor.createFieldNameSelector
       (core.getResourceLoader(),
-       core.getSchema(),
+       core,
        srcInclusions.fieldName,
        srcInclusions.typeName,
        srcInclusions.typeClass,
@@ -217,7 +208,7 @@ public class CloneFieldUpdateProcessorFa
         (srcSelector,
          FieldMutatingUpdateProcessor.createFieldNameSelector
          (core.getResourceLoader(),
-          core.getSchema(),
+          core,
           exc.fieldName,
           exc.typeName,
           exc.typeClass,

Modified: lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/ConcatFieldUpdateProcessorFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/ConcatFieldUpdateProcessorFactory.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/ConcatFieldUpdateProcessorFactory.java (original)
+++ lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/ConcatFieldUpdateProcessorFactory.java Thu May 30 07:53:18 2013
@@ -95,10 +95,10 @@ public final class ConcatFieldUpdateProc
   public FieldMutatingUpdateProcessor.FieldNameSelector 
     getDefaultSelector(final SolrCore core) {
 
-    final IndexSchema schema = core.getSchema();
     return new FieldMutatingUpdateProcessor.FieldNameSelector() {
       @Override
       public boolean shouldMutate(final String fieldName) {
+        final IndexSchema schema = core.getLatestSchema();
 
         // first check type since it should be fastest
         FieldType type = schema.getFieldTypeNoEx(fieldName);

Modified: lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java (original)
+++ lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java Thu May 30 07:53:18 2013
@@ -39,6 +39,7 @@ import org.apache.solr.common.SolrInputD
 import org.apache.solr.common.SolrInputField;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.DocRouter;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
@@ -56,6 +57,7 @@ import org.apache.solr.handler.component
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.update.AddUpdateCommand;
 import org.apache.solr.update.CommitUpdateCommand;
@@ -77,6 +79,7 @@ import static org.apache.solr.update.pro
 // NOT mt-safe... create a new processor for each add thread
 // TODO: we really should not wait for distrib after local? unless a certain replication factor is asked for
 public class DistributedUpdateProcessor extends UpdateRequestProcessor {
+  private static final String TEST_DISTRIB_SKIP_SERVERS = "test.distrib.skip.servers";
   public final static Logger log = LoggerFactory.getLogger(DistributedUpdateProcessor.class);
 
   /**
@@ -136,6 +139,7 @@ public class DistributedUpdateProcessor 
   // method in this update processor
   private boolean isLeader = true;
   private boolean forwardToLeader = false;
+  private boolean forwardToSubShard = false;
   private List<Node> nodes;
 
   private int numNodes;
@@ -238,19 +242,26 @@ public class DistributedUpdateProcessor 
           List<ZkCoreNodeProps> replicaProps = zkController.getZkStateReader()
               .getReplicaProps(collection, shardId, coreNodeName,
                   coreName, null, ZkStateReader.DOWN);
+
+          nodes = addSubShardLeaders(coll, shardId, id, doc, nodes);
           if (replicaProps != null) {
+            if (nodes == 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");
+            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));
+              log.info("test.distrib.skip.servers was found and contains:" + skipListSet);
             }
 
             for (ZkCoreNodeProps props : replicaProps) {
               if (skipList != null) {
-                if (!skipListSet.contains(props.getCoreUrl())) {
+                boolean skip = skipListSet.contains(props.getCoreUrl());
+                log.info("check url:" + props.getCoreUrl() + " against:" + skipListSet + " result:" + skip);
+                if (!skip) {
                   nodes.add(new StdNode(props));
                 }
               } else {
@@ -276,18 +287,60 @@ public class DistributedUpdateProcessor 
     return nodes;
   }
 
+  private List<Node> addSubShardLeaders(DocCollection coll, String shardId, String docId, SolrInputDocument doc, List<Node> nodes) {
+    Collection<Slice> allSlices = coll.getSlices();
+    for (Slice aslice : allSlices) {
+      if (Slice.CONSTRUCTION.equals(aslice.getState()))  {
+        DocRouter.Range myRange = coll.getSlice(shardId).getRange();
+        if (myRange == null) myRange = new DocRouter.Range(Integer.MIN_VALUE, Integer.MAX_VALUE);
+        boolean isSubset = aslice.getRange() != null && aslice.getRange().isSubsetOf(myRange);
+        if (isSubset &&
+            (docId == null // in case of deletes
+            || (docId != null && coll.getRouter().isTargetSlice(docId, doc, req.getParams(), aslice.getName(), coll)))) {
+          Replica sliceLeader = aslice.getLeader();
+          // slice leader can be null because node/shard is created zk before leader election
+          if (sliceLeader != null && zkController.getClusterState().liveNodesContain(sliceLeader.getNodeName()))  {
+            if (nodes == null) nodes = new ArrayList<Node>();
+            ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(sliceLeader);
+            nodes.add(new StdNode(nodeProps));
+            forwardToSubShard = true;
+          }
+        }
+      }
+    }
+    return nodes;
+  }
 
   private void doDefensiveChecks(DistribPhase phase) {
-    boolean isReplayOrPeersync = (updateCommand.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.REPLAY)) != 0;
+    boolean isReplayOrPeersync = (updateCommand.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.PEER_SYNC)) != 0;
     if (isReplayOrPeersync) return;
 
     String from = req.getParams().get("distrib.from");
-    boolean localIsLeader = req.getCore().getCoreDescriptor().getCloudDescriptor().isLeader();
+    ClusterState clusterState = zkController.getClusterState();
+    CloudDescriptor cloudDescriptor = req.getCore().getCoreDescriptor().getCloudDescriptor();
+    Slice mySlice = clusterState.getSlice(collection, cloudDescriptor.getShardId());
+    boolean localIsLeader = cloudDescriptor.isLeader();
     if (DistribPhase.FROMLEADER == phase && localIsLeader && from != null) { // from will be null on log replay
-      log.error("Request says it is coming from leader, but we are the leader: " + req.getParamString());
-      throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "Request says it is coming from leader, but we are the leader");
+      String fromShard = req.getParams().get("distrib.from.parent");
+      if (fromShard != null) {
+        if (!Slice.CONSTRUCTION.equals(mySlice.getState()))  {
+          throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE,
+              "Request says it is coming from parent shard leader but we are not in construction state");
+        }
+        // shard splitting case -- check ranges to see if we are a sub-shard
+        Slice fromSlice = zkController.getClusterState().getCollection(collection).getSlice(fromShard);
+        DocRouter.Range parentRange = fromSlice.getRange();
+        if (parentRange == null) parentRange = new DocRouter.Range(Integer.MIN_VALUE, Integer.MAX_VALUE);
+        if (mySlice.getRange() != null && !mySlice.getRange().isSubsetOf(parentRange)) {
+          throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE,
+              "Request says it is coming from parent shard leader but parent hash range is not superset of my range");
+        }
+      } else {
+        log.error("Request says it is coming from leader, but we are the leader: " + req.getParamString());
+        throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "Request says it is coming from leader, but we are the leader");
+      }
     }
-    
+
     if (isLeader && !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");
@@ -320,6 +373,8 @@ public class DistributedUpdateProcessor 
           nodes.add(new StdNode(props));
         }
       }
+
+      nodes = addSubShardLeaders(zkController.getClusterState().getCollection(collection), shardId, null, null, nodes);
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
       throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "",
@@ -363,6 +418,9 @@ public class DistributedUpdateProcessor 
         params.set("distrib.from", ZkCoreNodeProps.getCoreUrl(
             zkController.getBaseUrl(), req.getCore().getName()));
       }
+      if (forwardToSubShard)  {
+        params.set("distrib.from.parent", req.getCore().getCoreDescriptor().getCloudDescriptor().getShardId());
+      }
 
       params.set("distrib.from", ZkCoreNodeProps.getCoreUrl(
           zkController.getBaseUrl(), req.getCore().getName()));
@@ -501,7 +559,7 @@ public class DistributedUpdateProcessor 
       }
     }
 
-    boolean isReplayOrPeersync = (cmd.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.REPLAY)) != 0;
+    boolean isReplayOrPeersync = (cmd.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.PEER_SYNC)) != 0;
     boolean leaderLogic = isLeader && !isReplayOrPeersync;
 
 
@@ -632,6 +690,7 @@ public class DistributedUpdateProcessor 
       oldDoc.remove(VERSION_FIELD);
     }
 
+    IndexSchema schema = cmd.getReq().getSchema();
     for (SolrInputField sif : sdoc.values()) {
       Object val = sif.getValue();
       if (val instanceof Map) {
@@ -653,7 +712,7 @@ public class DistributedUpdateProcessor 
             } else {
               // TODO: fieldtype needs externalToObject?
               String oldValS = numericField.getFirstValue().toString();
-              SchemaField sf = cmd.getReq().getSchema().getField(sif.getName());
+              SchemaField sf = schema.getField(sif.getName());
               BytesRef term = new BytesRef();
               sf.getType().readableToIndexed(oldValS, term);
               Object oldVal = sf.getType().toObject(sf, term);
@@ -753,6 +812,7 @@ public class DistributedUpdateProcessor 
   private ModifiableSolrParams filterParams(SolrParams params) {
     ModifiableSolrParams fparams = new ModifiableSolrParams();
     passParam(params, fparams, UpdateParams.UPDATE_CHAIN);
+    passParam(params, fparams, TEST_DISTRIB_SKIP_SERVERS);
     return fparams;
   }
 
@@ -857,7 +917,7 @@ public class DistributedUpdateProcessor 
     }
     versionOnUpdate = Math.abs(versionOnUpdate);  // normalize to positive version
 
-    boolean isReplayOrPeersync = (cmd.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.REPLAY)) != 0;
+    boolean isReplayOrPeersync = (cmd.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.PEER_SYNC)) != 0;
     boolean leaderLogic = isLeader && !isReplayOrPeersync;
 
     if (!leaderLogic && versionOnUpdate==0) {
@@ -969,7 +1029,7 @@ public class DistributedUpdateProcessor 
     long signedVersionOnUpdate = versionOnUpdate;
     versionOnUpdate = Math.abs(versionOnUpdate);  // normalize to positive version
 
-    boolean isReplayOrPeersync = (cmd.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.REPLAY)) != 0;
+    boolean isReplayOrPeersync = (cmd.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.PEER_SYNC)) != 0;
     boolean leaderLogic = isLeader && !isReplayOrPeersync;
 
     if (!leaderLogic && versionOnUpdate==0) {

Modified: lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java (original)
+++ lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java Thu May 30 07:53:18 2013
@@ -18,27 +18,21 @@
 package org.apache.solr.update.processor;
 
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
 import java.util.Set;
 import java.util.regex.Pattern;
-import java.util.regex.PatternSyntaxException;
 
-import static org.apache.solr.common.SolrException.ErrorCode.*;
+import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
+import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
 
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.SolrInputField;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.FieldType;
 
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.update.AddUpdateCommand;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -197,7 +191,7 @@ public abstract class FieldMutatingUpdat
    */
   public static FieldNameSelector createFieldNameSelector
     (final SolrResourceLoader loader,
-     final IndexSchema schema,
+     final SolrCore core,
      final Set<String> fields,
      final Set<String> typeNames,
      final Collection<String> typeClasses,
@@ -223,25 +217,24 @@ public abstract class FieldMutatingUpdat
       return defSelector;
     }
     
-    return new ConfigurableFieldNameSelector
-      (schema, fields, typeNames, classes, regexes); 
+    return new ConfigurableFieldNameSelector(core, fields, typeNames, classes, regexes); 
   }
   
   private static final class ConfigurableFieldNameSelector 
     implements FieldNameSelector {
 
-    final IndexSchema schema;
+    final SolrCore core;
     final Set<String> fields;
     final Set<String> typeNames;
     final Collection<Class> classes;
     final Collection<Pattern> regexes;
 
-    private ConfigurableFieldNameSelector(final IndexSchema schema,
+    private ConfigurableFieldNameSelector(final SolrCore core,
                                           final Set<String> fields,
                                           final Set<String> typeNames,
                                           final Collection<Class> classes,
                                           final Collection<Pattern> regexes) {
-      this.schema = schema;
+      this.core = core;
       this.fields = fields;
       this.typeNames = typeNames;
       this.classes = classes;
@@ -260,7 +253,7 @@ public abstract class FieldMutatingUpdat
       
       // do not consider it an error if the fieldName has no type
       // there might be another processor dealing with it later
-      FieldType t = schema.getFieldTypeNoEx(fieldName);
+      FieldType t =  core.getLatestSchema().getFieldTypeNoEx(fieldName);
       if (null != t) {
         if (! (typeNames.isEmpty() || typeNames.contains(t.getTypeName())) ) {
           return false;

Modified: lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessorFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessorFactory.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessorFactory.java (original)
+++ lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessorFactory.java Thu May 30 07:53:18 2013
@@ -17,7 +17,6 @@
 
 package org.apache.solr.update.processor;
 
-import java.io.IOException;
 import java.util.Arrays;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -32,11 +31,6 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.common.SolrException;
 import static org.apache.solr.common.SolrException.ErrorCode.*;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.update.AddUpdateCommand;
-import org.apache.solr.schema.IndexSchema;
-import org.apache.solr.schema.FieldType;
 import org.apache.solr.util.plugin.SolrCoreAware;
 
 
@@ -199,12 +193,10 @@ public abstract class FieldMutatingUpdat
   @Override
   public void inform(final SolrCore core) {
     
-    final IndexSchema schema = core.getSchema();
-
     selector = 
       FieldMutatingUpdateProcessor.createFieldNameSelector
       (core.getResourceLoader(),
-       core.getSchema(),
+       core,
        inclusions.fieldName,
        inclusions.typeName,
        inclusions.typeClass,
@@ -216,7 +208,7 @@ public abstract class FieldMutatingUpdat
         (selector,
          FieldMutatingUpdateProcessor.createFieldNameSelector
          (core.getResourceLoader(),
-          core.getSchema(),
+          core,
           exc.fieldName,
           exc.typeName,
           exc.typeClass,

Modified: lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/IgnoreFieldUpdateProcessorFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/IgnoreFieldUpdateProcessorFactory.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/IgnoreFieldUpdateProcessorFactory.java (original)
+++ lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/IgnoreFieldUpdateProcessorFactory.java Thu May 30 07:53:18 2013
@@ -22,7 +22,6 @@ import org.apache.solr.schema.IndexSchem
 import org.apache.solr.schema.FieldType;
 
 import org.apache.solr.common.SolrInputField;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 
@@ -71,11 +70,10 @@ public final class IgnoreFieldUpdateProc
   public FieldMutatingUpdateProcessor.FieldNameSelector 
     getDefaultSelector(final SolrCore core) {
 
-    final IndexSchema schema = core.getSchema();
     return new FieldMutatingUpdateProcessor.FieldNameSelector() {
       @Override
       public boolean shouldMutate(final String fieldName) {
-
+        final IndexSchema schema = core.getLatestSchema();
         FieldType type = schema.getFieldTypeNoEx(fieldName);
         return (null == type);
 

Modified: lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/SignatureUpdateProcessorFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/SignatureUpdateProcessorFactory.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/SignatureUpdateProcessorFactory.java (original)
+++ lucene/dev/branches/security/solr/core/src/java/org/apache/solr/update/processor/SignatureUpdateProcessorFactory.java Thu May 30 07:53:18 2013
@@ -75,7 +75,7 @@ public class SignatureUpdateProcessorFac
 
   @Override
   public void inform(SolrCore core) {
-    final SchemaField field = core.getSchema().getFieldOrNull(getSignatureField());
+    final SchemaField field = core.getLatestSchema().getFieldOrNull(getSignatureField());
     if (null == field) {
       throw new SolrException
         (ErrorCode.SERVER_ERROR,

Modified: lucene/dev/branches/security/solr/core/src/java/org/apache/solr/util/DOMUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/java/org/apache/solr/util/DOMUtil.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/java/org/apache/solr/util/DOMUtil.java (original)
+++ lucene/dev/branches/security/solr/core/src/java/org/apache/solr/util/DOMUtil.java Thu May 30 07:53:18 2013
@@ -301,6 +301,103 @@ public class DOMUtil {
       }
     }
   }
+  
+  public static String substituteProperty(String value, Properties coreProperties) {
+    if (value == null || value.indexOf('$') == -1) {
+      return value;
+    }
+
+    List<String> fragments = new ArrayList<String>();
+    List<String> propertyRefs = new ArrayList<String>();
+    parsePropertyString(value, fragments, propertyRefs);
+
+    StringBuilder sb = new StringBuilder();
+    Iterator<String> i = fragments.iterator();
+    Iterator<String> j = propertyRefs.iterator();
+
+    while (i.hasNext()) {
+      String fragment = i.next();
+      if (fragment == null) {
+        String propertyName = j.next();
+        String defaultValue = null;
+        int colon_index = propertyName.indexOf(':');
+        if (colon_index > -1) {
+          defaultValue = propertyName.substring(colon_index + 1);
+          propertyName = propertyName.substring(0,colon_index);
+        }
+        if (coreProperties != null) {
+          fragment = coreProperties.getProperty(propertyName);
+        }
+        if (fragment == null) {
+          fragment = System.getProperty(propertyName, defaultValue);
+        }
+        if (fragment == null) {
+          throw new SolrException( SolrException.ErrorCode.SERVER_ERROR, "No system property or default value specified for " + propertyName + " value:" + value);
+        }
+      }
+      sb.append(fragment);
+    }
+    return sb.toString();
+  }
+  
+  /*
+   * This method borrowed from Ant's PropertyHelper.parsePropertyStringDefault:
+   *   http://svn.apache.org/repos/asf/ant/core/trunk/src/main/org/apache/tools/ant/PropertyHelper.java
+   */
+  private static void parsePropertyString(String value, List<String> fragments, List<String> propertyRefs) {
+      int prev = 0;
+      int pos;
+      //search for the next instance of $ from the 'prev' position
+      while ((pos = value.indexOf("$", prev)) >= 0) {
+
+          //if there was any text before this, add it as a fragment
+          //TODO, this check could be modified to go if pos>prev;
+          //seems like this current version could stick empty strings
+          //into the list
+          if (pos > 0) {
+              fragments.add(value.substring(prev, pos));
+          }
+          //if we are at the end of the string, we tack on a $
+          //then move past it
+          if (pos == (value.length() - 1)) {
+              fragments.add("$");
+              prev = pos + 1;
+          } else if (value.charAt(pos + 1) != '{') {
+              //peek ahead to see if the next char is a property or not
+              //not a property: insert the char as a literal
+              /*
+              fragments.addElement(value.substring(pos + 1, pos + 2));
+              prev = pos + 2;
+              */
+              if (value.charAt(pos + 1) == '$') {
+                  //backwards compatibility two $ map to one mode
+                  fragments.add("$");
+                  prev = pos + 2;
+              } else {
+                  //new behaviour: $X maps to $X for all values of X!='$'
+                  fragments.add(value.substring(pos, pos + 2));
+                  prev = pos + 2;
+              }
+
+          } else {
+              //property found, extract its name or bail on a typo
+              int endName = value.indexOf('}', pos);
+              if (endName < 0) {
+                throw new RuntimeException("Syntax error in property: " + value);
+              }
+              String propertyName = value.substring(pos + 2, endName);
+              fragments.add(null);
+              propertyRefs.add(propertyName);
+              prev = endName + 1;
+          }
+      }
+      //no more $ signs found
+      //if there is any tail to the string, append it
+      if (prev < value.length()) {
+          fragments.add(value.substring(prev));
+      }
+  }
+
 
 
 }

Modified: lucene/dev/branches/security/solr/core/src/java/org/apache/solr/util/SolrPluginUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/java/org/apache/solr/util/SolrPluginUtils.java?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/java/org/apache/solr/util/SolrPluginUtils.java (original)
+++ lucene/dev/branches/security/solr/core/src/java/org/apache/solr/util/SolrPluginUtils.java Thu May 30 07:53:18 2013
@@ -19,8 +19,13 @@ package org.apache.solr.util;
 
 import org.apache.lucene.index.StorableField;
 import org.apache.lucene.index.StoredDocument;
-import org.apache.lucene.search.*;
+import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.DisjunctionMaxQuery;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Sort;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrException;
@@ -38,10 +43,27 @@ import org.apache.solr.request.SolrQuery
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
-import org.apache.solr.search.*;
+import org.apache.solr.search.CacheRegenerator;
+import org.apache.solr.search.DocIterator;
+import org.apache.solr.search.DocList;
+import org.apache.solr.search.DocSet;
+import org.apache.solr.search.FieldParams;
+import org.apache.solr.search.QParser;
+import org.apache.solr.search.QueryParsing;
+import org.apache.solr.search.ReturnFields;
+import org.apache.solr.search.SolrCache;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.search.SolrQueryParser;
+import org.apache.solr.search.SyntaxError;
 
 import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 import java.util.regex.Pattern;
 import java.lang.reflect.Method;
 import java.lang.reflect.InvocationTargetException;
@@ -152,7 +174,7 @@ public class SolrPluginUtils {
           fieldFilter.add(field);
 
         // fetch unique key if one exists.
-        SchemaField keyField = req.getSearcher().getSchema().getUniqueKeyField();
+        SchemaField keyField = searcher.getSchema().getUniqueKeyField();
         if(null != keyField)
           fieldFilter.add(keyField.getName());
       }
@@ -270,7 +292,7 @@ public class SolrPluginUtils {
   {
     if (dbgResults) {
       SolrIndexSearcher searcher = req.getSearcher();
-      IndexSchema schema = req.getSchema();
+      IndexSchema schema = searcher.getSchema();
       boolean explainStruct = req.getParams().getBool(CommonParams.EXPLAIN_STRUCT, false);
 
       NamedList<Explanation> explain = getExplanations(query, results, searcher, schema);

Modified: lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml (original)
+++ lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml Thu May 30 07:53:18 2013
@@ -277,7 +277,7 @@
     <fieldtype name="syn" class="solr.TextField">
       <analyzer>
           <tokenizer class="solr.MockTokenizerFactory"/>
-          <filter name="syn" class="solr.SynonymFilterFactory" synonyms="synonyms.txt"/>
+          <filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt"/>
       </analyzer>
     </fieldtype>
     
@@ -346,8 +346,8 @@
      termPositions="true" termOffsets="true"/>
 
    <!-- test highlit field settings -->
-   <field name="test_hlt" type="highlittext" indexed="true" compressed="true"/>
-   <field name="test_hlt_off" type="highlittext" indexed="true" compressed="false"/>
+   <field name="test_hlt" type="highlittext" indexed="true"/>
+   <field name="test_hlt_off" type="highlittext" indexed="true"/>
 
    <!-- fields to test individual tokenizers and tokenfilters -->
    <field name="teststop" type="teststop" indexed="true" stored="true"/>
@@ -460,7 +460,18 @@
    <copyField source="text_fr" dest="highlight" maxChars="25" />
    <copyField source="text_en" dest="highlight" maxChars="25" />
    <copyField source="text_*" dest="highlight" maxChars="25" />
-   
+
+   <!-- test source glob matching multiple explicit fields: sku1 and sku2 -->
+   <copyField source="sku*" dest="text"/>
+   <copyField source="sku*" dest="*_s"/>
+   <copyField source="sku*" dest="*_dest_sub_s"/>
+   <copyField source="sku*" dest="dest_sub_no_ast_s"/>
+
+   <!-- test source glob that doesn't match any explicit or dynamic field -->
+   <copyField source="testing123_*" dest="text"/>
+  
+   <copyField source="*" dest="catchall_t"/>
+
  <!-- Similarity is the scoring routine for each document vs a query.
       A custom similarity may be specified here, but the default is fine
       for most applications.

Modified: lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-postingshighlight.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-postingshighlight.xml?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-postingshighlight.xml (original)
+++ lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-postingshighlight.xml Thu May 30 07:53:18 2013
@@ -26,6 +26,7 @@
     <fieldtype name="text" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
       </analyzer>
     </fieldtype>
     
@@ -33,6 +34,7 @@
     <fieldtype name="text_offsets" class="solr.TextField" storeOffsetsWithPositions="true">
       <analyzer>
         <tokenizer class="solr.MockTokenizerFactory"/>
+        <filter class="solr.LowerCaseFilterFactory"/>
       </analyzer>
     </fieldtype>
    </types>

Modified: lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-required-fields.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-required-fields.xml?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-required-fields.xml (original)
+++ lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-required-fields.xml Thu May 30 07:53:18 2013
@@ -268,7 +268,7 @@
     <fieldtype name="syn" class="solr.TextField">
       <analyzer>
           <tokenizer class="solr.MockTokenizerFactory"/>
-          <filter name="syn" class="solr.SynonymFilterFactory" synonyms="synonyms.txt"/>
+          <filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt"/>
       </analyzer>
     </fieldtype>
     
@@ -337,8 +337,8 @@
      termPositions="true" termOffsets="true"/>
 
    <!-- test highlit field settings -->
-   <field name="test_hlt" type="highlittext" indexed="true" compressed="true"/>
-   <field name="test_hlt_off" type="highlittext" indexed="true" compressed="false"/>
+   <field name="test_hlt" type="highlittext" indexed="true"/>
+   <field name="test_hlt_off" type="highlittext" indexed="true"/>
 
    <!-- fields to test individual tokenizers and tokenfilters -->
    <field name="teststop" type="teststop" indexed="true" stored="true"/>

Modified: lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml (original)
+++ lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml Thu May 30 07:53:18 2013
@@ -111,7 +111,6 @@
         <filter class="solr.StopFilterFactory"
                 ignoreCase="true"
                 words="stopwords.txt"
-                enablePositionIncrements="true"
                 />
         <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0" splitOnCaseChange="1"/>
         <filter class="solr.LowerCaseFilterFactory"/>
@@ -124,7 +123,6 @@
         <filter class="solr.StopFilterFactory"
                 ignoreCase="true"
                 words="stopwords.txt"
-                enablePositionIncrements="true"
                 />
         <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="1"/>
         <filter class="solr.LowerCaseFilterFactory"/>
@@ -141,7 +139,6 @@
         <filter class="solr.StopFilterFactory"
                 ignoreCase="true"
                 words="stopwords.txt"
-                enablePositionIncrements="true"
                 />
         <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0" splitOnCaseChange="1"/>
         <filter class="solr.LowerCaseFilterFactory"/>
@@ -154,7 +151,6 @@
         <filter class="solr.StopFilterFactory"
                 ignoreCase="true"
                 words="stopwords.txt"
-                enablePositionIncrements="true"
                 />
         <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="1"/>
         <filter class="solr.LowerCaseFilterFactory"/>
@@ -355,7 +351,7 @@
     <fieldtype name="syn" class="solr.TextField">
       <analyzer>
           <tokenizer class="solr.MockTokenizerFactory"/>
-          <filter name="syn" class="solr.SynonymFilterFactory" synonyms="synonyms.txt"/>
+          <filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt"/>
       </analyzer>
     </fieldtype>
 
@@ -469,8 +465,8 @@
      termPositions="true" termOffsets="true"/>
 
    <!-- test highlit field settings -->
-   <field name="test_hlt" type="highlittext" indexed="true" compressed="true"/>
-   <field name="test_hlt_off" type="highlittext" indexed="true" compressed="false"/>
+   <field name="test_hlt" type="highlittext" indexed="true"/>
+   <field name="test_hlt_off" type="highlittext" indexed="true"/>
 
    <!-- fields to test individual tokenizers and tokenfilters -->
    <field name="teststop" type="teststop" indexed="true" stored="true"/>
@@ -616,4 +612,13 @@
    <copyField source="src_sub_no_ast_i" dest="*_s"/>
    <copyField source="src_sub_no_ast_i" dest="*_dest_sub_s"/>
    <copyField source="src_sub_no_ast_i" dest="dest_sub_no_ast_s"/>
+
+   <!-- test source glob matching multiple explicit fields: title_stemmed and title_lettertok -->
+   <copyField source="title_*" dest="text"/>
+   <copyField source="title_*" dest="*_s"/>
+   <copyField source="title_*" dest="*_dest_sub_s"/>
+   <copyField source="title_*" dest="dest_sub_no_ast_s"/>
+
+  <solrQueryParser defaultOperator="OR"/>
+
 </schema>

Modified: lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-trie.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-trie.xml?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-trie.xml (original)
+++ lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema-trie.xml Thu May 30 07:53:18 2013
@@ -233,7 +233,7 @@
              See the Java Regular Expression documentation for more
              infomation on pattern and replacement string syntax.
 
-             http://java.sun.com/j2se/1.6.0/docs/api/java/util/regex/package-summary.html
+             http://docs.oracle.com/javase/7/docs/api/java/util/regex/package-summary.html
           -->
         <filter class="solr.PatternReplaceFilterFactory"
                 pattern="([^a-z])" replacement="" replace="all"
@@ -258,9 +258,6 @@
      type: mandatory - the name of a previously defined type from the <types> section
      indexed: true if this field should be indexed (searchable or sortable)
      stored: true if this field should be retrievable
-     compressed: [false] if this field should be stored using gzip compression
-       (this will only apply if the field type is compressable; among
-       the standard field types, only TextField and StrField are)
      multiValued: true if this field may contain multiple values per document
      omitNorms: (expert) set to true to omit the norms associated with
        this field (this disables length normalization and index-time

Modified: lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema.xml?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema.xml (original)
+++ lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema.xml Thu May 30 07:53:18 2013
@@ -204,7 +204,7 @@
     <fieldtype name="lowerpunctfilt" class="solr.TextField">
       <analyzer>
         <tokenizer class="solr.MockTokenizerFactory"/>
-        <filter name="syn" class="solr.SynonymFilterFactory" synonyms="synonyms.txt" expand="true"/>
+        <filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt" expand="true"/>
         <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="1" splitOnCaseChange="1"/>
         <filter class="solr.LowerCaseFilterFactory"/>
       </analyzer>
@@ -355,7 +355,7 @@
     <fieldtype name="syn" class="solr.TextField">
       <analyzer>
           <tokenizer class="solr.MockTokenizerFactory"/>
-          <filter name="syn" class="solr.SynonymFilterFactory" synonyms="old_synonyms.txt"/>
+          <filter class="solr.SynonymFilterFactory" synonyms="old_synonyms.txt"/>
       </analyzer>
     </fieldtype>
 
@@ -506,22 +506,22 @@
    <!-- indexed=false should not prevent omit___=true -->
    <field name="pint_i_norm" type="pint" omitNorms="true"
           indexed="false" />
-   <field name="pint_i_tf"   type="pint" omitOmitTermFreqAndPositions="true"
+   <field name="pint_i_tf"   type="pint" omitTermFreqAndPositions="true"
           indexed="false" />
    <field name="pint_i_pos"  type="pint" omitPositions="true"
           indexed="false" />
    <field name="pint_i_all"  type="pint" 
           indexed="false" 
           omitNorms="true"
-          omitOmitTermFreqAndPositions="true"
+          omitTermFreqAndPositions="true"
           omitPositions="true" />
-   <!-- omitOmitTermFreqAndPositions=false and omitPositions=true are ok -->
+   <!-- omitTermFreqAndPositions=false and omitPositions=true are ok -->
    <field name="pint_tf_pos" type="pint" indexed="true" 
-          omitOmitTermFreqAndPositions="false" omitPositions="true" />
+          omitTermFreqAndPositions="false" omitPositions="true" />
 
    <!-- test highlit field settings -->
-   <field name="test_hlt" type="highlittext" indexed="true" compressed="true"/>
-   <field name="test_hlt_off" type="highlittext" indexed="true" compressed="false"/>
+   <field name="test_hlt" type="highlittext" indexed="true"/>
+   <field name="test_hlt_off" type="highlittext" indexed="true"/>
 
    <!-- fields to test individual tokenizers and tokenfilters -->
    <field name="teststop" type="teststop" indexed="true" stored="true"/>
@@ -582,6 +582,8 @@
    
    <field name="store" type="location" indexed="true" stored="true" omitNorms="false"/>
 
+   <field name="lower" type="lowertok" indexed="false" stored="true" multiValued="true" />
+
    <!-- Dynamic field definitions.  If a field name is not found, dynamicFields
         will be used if the name matches any of the patterns.
         RESTRICTION: the glob-like pattern in the name attribute must have

Modified: lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema11.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema11.xml?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema11.xml (original)
+++ lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema11.xml Thu May 30 07:53:18 2013
@@ -224,7 +224,7 @@
              See the Java Regular Expression documentation for more
              infomation on pattern and replacement string syntax.
              
-             http://java.sun.com/j2se/1.6.0/docs/api/java/util/regex/package-summary.html
+             http://docs.oracle.com/javase/7/docs/api/java/util/regex/package-summary.html
           -->
         <filter class="solr.PatternReplaceFilterFactory"
                 pattern="([^a-z])" replacement="" replace="all"
@@ -286,6 +286,7 @@ valued. -->
                stored="false" indexed="true"
                class="solr.ExternalFileField"/>
 
+    <fieldType name="text_no_analyzer" stored="false" indexed="true" class="solr.TextField" />
  </types>
 
 
@@ -295,9 +296,6 @@ valued. -->
      type: mandatory - the name of a previously defined type from the <types> section
      indexed: true if this field should be indexed (searchable or sortable)
      stored: true if this field should be retrievable
-     compressed: [false] if this field should be stored using gzip compression
-       (this will only apply if the field type is compressable; among
-       the standard field types, only TextField and StrField are)
      multiValued: true if this field may contain multiple values per document
      omitNorms: (expert) set to true to omit the norms associated with
        this field (this disables length normalization and index-time
@@ -322,6 +320,8 @@ valued. -->
 
    <field name="eff_trie" type="eff_tfloat" />
 
+   <field name="text_no_analyzer" type="text_no_analyzer" indexed="true" />
+
    <!-- Dynamic field definitions.  If a field name is not found, dynamicFields
         will be used if the name matches any of the patterns.
         RESTRICTION: the glob-like pattern in the name attribute must have

Modified: lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema12.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema12.xml?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema12.xml (original)
+++ lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema12.xml Thu May 30 07:53:18 2013
@@ -119,7 +119,6 @@
         <filter class="solr.StopFilterFactory"
                 ignoreCase="true"
                 words="stopwords.txt"
-                enablePositionIncrements="true"
                 />
         <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0" splitOnCaseChange="1"/>
         <filter class="solr.LowerCaseFilterFactory"/>
@@ -132,7 +131,6 @@
         <filter class="solr.StopFilterFactory"
                 ignoreCase="true"
                 words="stopwords.txt"
-                enablePositionIncrements="true"
                 />
         <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="1"/>
         <filter class="solr.LowerCaseFilterFactory"/>
@@ -149,7 +147,6 @@
         <filter class="solr.StopFilterFactory"
                 ignoreCase="true"
                 words="stopwords.txt"
-                enablePositionIncrements="true"
                 />
         <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0" splitOnCaseChange="1"/>
         <filter class="solr.LowerCaseFilterFactory"/>
@@ -162,7 +159,6 @@
         <filter class="solr.StopFilterFactory"
                 ignoreCase="true"
                 words="stopwords.txt"
-                enablePositionIncrements="true"
                 />
         <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="1"/>
         <filter class="solr.LowerCaseFilterFactory"/>
@@ -357,7 +353,7 @@
     <fieldtype name="syn" class="solr.TextField">
       <analyzer>
           <tokenizer class="solr.MockTokenizerFactory"/>
-          <filter name="syn" class="solr.SynonymFilterFactory" synonyms="synonyms.txt"/>
+          <filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt"/>
       </analyzer>
     </fieldtype>
 
@@ -472,8 +468,8 @@
      termPositions="true" termOffsets="true"/>
 
    <!-- test highlit field settings -->
-   <field name="test_hlt" type="highlittext" indexed="true" compressed="true"/>
-   <field name="test_hlt_off" type="highlittext" indexed="true" compressed="false"/>
+   <field name="test_hlt" type="highlittext" indexed="true"/>
+   <field name="test_hlt_off" type="highlittext" indexed="true"/>
 
    <!-- fields to test individual tokenizers and tokenfilters -->
    <field name="teststop" type="teststop" indexed="true" stored="true"/>
@@ -596,6 +592,11 @@
    <dynamicField name="bar_copydest_*" type="ignored" multiValued="true"/>
    <dynamicField name="*_es"  type="text"    indexed="true"  stored="true"/>
 
+   <!-- for testing if score psuedofield is erroneously treated as multivalued
+        when a matching dynamic field exists 
+   -->
+   <dynamicField name="*core" type="ignored" multiValued="true" />
+
  </fields>
 
  <defaultSearchField>text</defaultSearchField>

Modified: lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema15.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema15.xml?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema15.xml (original)
+++ lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schema15.xml Thu May 30 07:53:18 2013
@@ -112,7 +112,6 @@
         <filter class="solr.StopFilterFactory"
                 ignoreCase="true"
                 words="stopwords.txt"
-                enablePositionIncrements="true"
                 />
         <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0" splitOnCaseChange="1"/>
         <filter class="solr.LowerCaseFilterFactory"/>
@@ -125,7 +124,6 @@
         <filter class="solr.StopFilterFactory"
                 ignoreCase="true"
                 words="stopwords.txt"
-                enablePositionIncrements="true"
                 />
         <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="1"/>
         <filter class="solr.LowerCaseFilterFactory"/>
@@ -142,7 +140,6 @@
         <filter class="solr.StopFilterFactory"
                 ignoreCase="true"
                 words="stopwords.txt"
-                enablePositionIncrements="true"
                 />
         <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0" splitOnCaseChange="1"/>
         <filter class="solr.LowerCaseFilterFactory"/>
@@ -155,7 +152,6 @@
         <filter class="solr.StopFilterFactory"
                 ignoreCase="true"
                 words="stopwords.txt"
-                enablePositionIncrements="true"
                 />
         <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="1"/>
         <filter class="solr.LowerCaseFilterFactory"/>
@@ -356,7 +352,7 @@
     <fieldtype name="syn" class="solr.TextField">
       <analyzer>
           <tokenizer class="solr.MockTokenizerFactory"/>
-          <filter name="syn" class="solr.SynonymFilterFactory" synonyms="synonyms.txt"/>
+          <filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt"/>
       </analyzer>
     </fieldtype>
 
@@ -470,8 +466,8 @@
      termPositions="true" termOffsets="true"/>
 
    <!-- test highlit field settings -->
-   <field name="test_hlt" type="highlittext" indexed="true" compressed="true"/>
-   <field name="test_hlt_off" type="highlittext" indexed="true" compressed="false"/>
+   <field name="test_hlt" type="highlittext" indexed="true"/>
+   <field name="test_hlt_off" type="highlittext" indexed="true"/>
 
    <!-- fields to test individual tokenizers and tokenfilters -->
    <field name="teststop" type="teststop" indexed="true" stored="true"/>

Modified: lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schemasurround.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schemasurround.xml?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schemasurround.xml (original)
+++ lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/schemasurround.xml Thu May 30 07:53:18 2013
@@ -119,7 +119,6 @@
         <filter class="solr.StopFilterFactory"
                 ignoreCase="true"
                 words="stopwords.txt"
-                enablePositionIncrements="true"
                 />
         <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0" splitOnCaseChange="1"/>
         <filter class="solr.LowerCaseFilterFactory"/>
@@ -132,7 +131,6 @@
         <filter class="solr.StopFilterFactory"
                 ignoreCase="true"
                 words="stopwords.txt"
-                enablePositionIncrements="true"
                 />
         <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="1"/>
         <filter class="solr.LowerCaseFilterFactory"/>
@@ -149,7 +147,6 @@
         <filter class="solr.StopFilterFactory"
                 ignoreCase="true"
                 words="stopwords.txt"
-                enablePositionIncrements="true"
                 />
         <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="1" catenateNumbers="1" catenateAll="0" splitOnCaseChange="1"/>
         <filter class="solr.LowerCaseFilterFactory"/>
@@ -162,7 +159,6 @@
         <filter class="solr.StopFilterFactory"
                 ignoreCase="true"
                 words="stopwords.txt"
-                enablePositionIncrements="true"
                 />
         <filter class="solr.WordDelimiterFilterFactory" generateWordParts="1" generateNumberParts="1" catenateWords="0" catenateNumbers="0" catenateAll="0" splitOnCaseChange="1"/>
         <filter class="solr.LowerCaseFilterFactory"/>
@@ -357,7 +353,7 @@
     <fieldtype name="syn" class="solr.TextField">
       <analyzer>
           <tokenizer class="solr.MockTokenizerFactory"/>
-          <filter name="syn" class="solr.SynonymFilterFactory" synonyms="synonyms.txt"/>
+          <filter class="solr.SynonymFilterFactory" synonyms="synonyms.txt"/>
       </analyzer>
     </fieldtype>
 
@@ -487,8 +483,8 @@
      termPositions="true" termOffsets="true"/>
 
    <!-- test highlit field settings -->
-   <field name="test_hlt" type="highlittext" indexed="true" compressed="true"/>
-   <field name="test_hlt_off" type="highlittext" indexed="true" compressed="false"/>
+   <field name="test_hlt" type="highlittext" indexed="true"/>
+   <field name="test_hlt_off" type="highlittext" indexed="true"/>
 
    <!-- fields to test individual tokenizers and tokenfilters -->
    <field name="teststop" type="teststop" indexed="true" stored="true"/>

Modified: lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/solrconfig-postingshighlight.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/solrconfig-postingshighlight.xml?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/solrconfig-postingshighlight.xml (original)
+++ lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/solrconfig-postingshighlight.xml Thu May 30 07:53:18 2013
@@ -22,8 +22,11 @@
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
   <dataDir>${solr.data.dir:}</dataDir>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
-  <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
-
+  <requestHandler name="standard" class="solr.StandardRequestHandler">
+    <lst name="defaults">
+      <bool name="hl.defaultSummary">false</bool>
+    </lst>
+  </requestHandler>
   <searchComponent class="solr.HighlightComponent" name="highlight">
     <highlighting class="org.apache.solr.highlight.PostingsSolrHighlighter"/>
   </searchComponent>

Modified: lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml (original)
+++ lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml Thu May 30 07:53:18 2013
@@ -390,4 +390,30 @@
     </processor>
   </updateRequestProcessorChain>
 
+  <updateRequestProcessorChain name="pre-analyzed-simple">
+    <processor class="solr.PreAnalyzedUpdateProcessorFactory">
+      <str name="fieldName">subject</str>
+      <str name="fieldName">title</str>
+      <str name="fieldName">teststop</str>
+      <str name="fieldName">nonexistent</str>
+      <str name="fieldName">ssto</str>
+      <str name="fieldName">sind</str>
+      <str name="parser">simple</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="pre-analyzed-json">
+    <processor class="solr.PreAnalyzedUpdateProcessorFactory">
+      <str name="fieldName">subject</str>
+      <str name="fieldName">title</str>
+      <str name="fieldName">teststop</str>
+      <str name="fieldName">nonexistent</str>
+      <str name="fieldName">ssto</str>
+      <str name="fieldName">sind</str>
+      <str name="parser">json</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
 </config>

Modified: lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/trivial.updateprocessor0.js
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/trivial.updateprocessor0.js?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/trivial.updateprocessor0.js (original)
+++ lucene/dev/branches/security/solr/core/src/test-files/solr/collection1/conf/trivial.updateprocessor0.js Thu May 30 07:53:18 2013
@@ -1,12 +1,14 @@
+var Assert = Packages.org.junit.Assert;
+
 function processAdd(cmd) {
     functionMessages.add("processAdd0");
-    testCase.assertNotNull(req);
-    testCase.assertNotNull(rsp);
-    testCase.assertNotNull(logger);
-    testCase.assertNotNull(cmd);
-    testCase.assertNotNull(params);
-    testCase.assertTrue(1 == params.get('intValue').intValue());  // had issues with assertTrue(1, params.get('intValue').intValue()) casting to wrong variant
-    testCase.assertTrue(params.get('boolValue').booleanValue());
+    Assert.assertNotNull(req);
+    Assert.assertNotNull(rsp);
+    Assert.assertNotNull(logger);
+    Assert.assertNotNull(cmd);
+    Assert.assertNotNull(params);
+    Assert.assertTrue(1 == params.get('intValue').intValue());  // had issues with assertTrue(1, params.get('intValue').intValue()) casting to wrong variant
+    Assert.assertTrue(params.get('boolValue').booleanValue());
 
     // Integer.valueOf is needed here to get a tru java object, because 
     // all javascript numbers are floating point (ie: java.lang.Double)
@@ -18,40 +20,40 @@ function processAdd(cmd) {
 
 function processDelete(cmd) {
     functionMessages.add("processDelete0");
-    testCase.assertNotNull(req);
-    testCase.assertNotNull(rsp);
-    testCase.assertNotNull(logger);
-    testCase.assertNotNull(cmd);
+    Assert.assertNotNull(req);
+    Assert.assertNotNull(rsp);
+    Assert.assertNotNull(logger);
+    Assert.assertNotNull(cmd);
 }
 
 function processMergeIndexes(cmd) {
     functionMessages.add("processMergeIndexes0");
-    testCase.assertNotNull(req);
-    testCase.assertNotNull(rsp);
-    testCase.assertNotNull(logger);
-    testCase.assertNotNull(cmd);
+    Assert.assertNotNull(req);
+    Assert.assertNotNull(rsp);
+    Assert.assertNotNull(logger);
+    Assert.assertNotNull(cmd);
 }
 
 function processCommit(cmd) {
     functionMessages.add("processCommit0");
-    testCase.assertNotNull(req);
-    testCase.assertNotNull(rsp);
-    testCase.assertNotNull(logger);
-    testCase.assertNotNull(cmd);
+    Assert.assertNotNull(req);
+    Assert.assertNotNull(rsp);
+    Assert.assertNotNull(logger);
+    Assert.assertNotNull(cmd);
 }
 
 function processRollback(cmd) {
     functionMessages.add("processRollback0");
-    testCase.assertNotNull(req);
-    testCase.assertNotNull(rsp);
-    testCase.assertNotNull(logger);
-    testCase.assertNotNull(cmd);
+    Assert.assertNotNull(req);
+    Assert.assertNotNull(rsp);
+    Assert.assertNotNull(logger);
+    Assert.assertNotNull(cmd);
 }
 
 function finish() {
     functionMessages.add("finish0");
-    testCase.assertNotNull(req);
-    testCase.assertNotNull(rsp);
-    testCase.assertNotNull(logger);
+    Assert.assertNotNull(req);
+    Assert.assertNotNull(rsp);
+    Assert.assertNotNull(logger);
 }
 

Modified: lucene/dev/branches/security/solr/core/src/test-files/solr/solr-no-core.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/solr/core/src/test-files/solr/solr-no-core.xml?rev=1487777&r1=1487776&r2=1487777&view=diff
==============================================================================
--- lucene/dev/branches/security/solr/core/src/test-files/solr/solr-no-core.xml (original)
+++ lucene/dev/branches/security/solr/core/src/test-files/solr/solr-no-core.xml Thu May 30 07:53:18 2013
@@ -16,26 +16,23 @@
  limitations under the License.
 -->
 
-<!--
- All (relative) paths are relative to the installation path
-  
-  persistent: Save changes made via the API to this file
-  sharedLib: path to a lib directory that will be shared across all cores
--->
-<solr persistent="${solr.xml.persist:false}">
+<solr>
+
+  <str name="shareSchema">${shareSchema:false}</str>
+
+  <solrcloud>
+    <str name="host">127.0.0.1</str>
+    <str name="hostContext">${hostContext:solr}</str>
+    <int name="hostPort">${hostPort:8983}</int>
+    <int name="zkClientTimeout">${solr.zkclienttimeout:30000}</int>
+    <int name="distribUpdateConnTimeout">${distribUpdateConnTimeout:15000}</int>
+    <int name="distribUpdateSoTimeout">${distribUpdateSoTimeout:120000}</int>
+  </solrcloud>
 
-  <!--
-  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="${hostContext:solr}" zkClientTimeout="${solr.zkclienttimeout:30000}" numShards="${numShards:3}" shareSchema="${shareSchema:false}" 
-         distribUpdateConnTimeout="${distribUpdateConnTimeout:15000}" distribUpdateSoTimeout="${distribUpdateSoTimeout:120000}">
+  <shardHandlerFactory name="shardHandlerFactory"
+    class="HttpShardHandlerFactory">
+    <int name="socketTimeout">${socketTimeout:120000}</int>
+    <int name="connTimeout">${connTimeout:15000}</int>
+  </shardHandlerFactory>
 
-    <shardHandlerFactory name="shardHandlerFactory" class="HttpShardHandlerFactory">
-      <int name="socketTimeout">${socketTimeout:120000}</int>
-      <int name="connTimeout">${connTimeout:15000}</int>
-    </shardHandlerFactory>
-  </cores>
-  
-</solr>
+</solr>
\ No newline at end of file