You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rj...@apache.org on 2015/03/31 07:22:50 UTC

svn commit: r1670257 [27/39] - in /lucene/dev/branches/lucene6271: ./ dev-tools/ dev-tools/idea/.idea/libraries/ dev-tools/scripts/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneou...

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/PerSegmentSingleValuedFaceting.java Tue Mar 31 05:22:40 2015
@@ -32,6 +32,7 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.PriorityQueue;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.FacetParams;
@@ -54,12 +55,14 @@ class PerSegmentSingleValuedFaceting {
   boolean missing;
   String sort;
   String prefix;
+  BytesRef containsBR;
+  boolean ignoreCase;
 
   Filter baseSet;
 
   int nThreads;
 
-  public PerSegmentSingleValuedFaceting(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort, String prefix) {
+  public PerSegmentSingleValuedFaceting(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort, String prefix, String contains, boolean ignoreCase) {
     this.searcher = searcher;
     this.docs = docs;
     this.fieldName = fieldName;
@@ -69,6 +72,8 @@ class PerSegmentSingleValuedFaceting {
     this.missing = missing;
     this.sort = sort;
     this.prefix = prefix;
+    this.containsBR = contains != null ? new BytesRef(contains) : null;
+    this.ignoreCase = ignoreCase;
   }
 
   public void setNumThreads(int threads) {
@@ -173,16 +178,23 @@ class PerSegmentSingleValuedFaceting {
 
     while (queue.size() > 0) {
       SegFacet seg = queue.top();
-
+      
+      // if facet.contains specified, only actually collect the count if substring contained
+      boolean collect = containsBR == null || StringHelper.contains(seg.tempBR, containsBR, ignoreCase);
+      
       // we will normally end up advancing the term enum for this segment
       // while still using "val", so we need to make a copy since the BytesRef
       // may be shared across calls.
-      val.copyBytes(seg.tempBR);
-
+      if (collect) {
+        val.copyBytes(seg.tempBR);
+      }
+      
       int count = 0;
 
       do {
-        count += seg.counts[seg.pos - seg.startTermIndex];
+        if (collect) {
+          count += seg.counts[seg.pos - seg.startTermIndex];
+        }
 
         // TODO: OPTIMIZATION...
         // if mincount>0 then seg.pos++ can skip ahead to the next non-zero entry.
@@ -190,14 +202,16 @@ class PerSegmentSingleValuedFaceting {
         if (seg.pos >= seg.endTermIndex) {
           queue.pop();
           seg = queue.top();
-        }  else {
+        } else {
           seg.tempBR = seg.tenum.next();
           seg = queue.updateTop();
         }
       } while (seg != null && val.get().compareTo(seg.tempBR) == 0);
 
-      boolean stop = collector.collect(val.get(), count);
-      if (stop) break;
+      if (collect) {
+        boolean stop = collector.collect(val.get(), count);
+        if (stop) break;
+      }
     }
 
     NamedList<Integer> res = collector.getFacetCounts();
@@ -253,31 +267,28 @@ class PerSegmentSingleValuedFaceting {
         startTermIndex=-1;
         endTermIndex=si.getValueCount();
       }
-
       final int nTerms=endTermIndex-startTermIndex;
-      if (nTerms>0) {
-        // count collection array only needs to be as big as the number of terms we are
-        // going to collect counts for.
-        final int[] counts = this.counts = new int[nTerms];
-        DocIdSet idSet = baseSet.getDocIdSet(context, null);  // this set only includes live docs
-        DocIdSetIterator iter = idSet.iterator();
+      if (nTerms == 0) return;
 
+      // count collection array only needs to be as big as the number of terms we are
+      // going to collect counts for.
+      final int[] counts = this.counts = new int[nTerms];
+      DocIdSet idSet = baseSet.getDocIdSet(context, null);  // this set only includes live docs
+      DocIdSetIterator iter = idSet.iterator();
 
-        ////
+      if (prefix==null) {
+        // specialized version when collecting counts for all terms
         int doc;
-
-        if (prefix==null) {
-          // specialized version when collecting counts for all terms
-          while ((doc = iter.nextDoc()) < DocIdSetIterator.NO_MORE_DOCS) {
-            counts[1+si.getOrd(doc)]++;
-          }
-        } else {
-          // version that adjusts term numbers because we aren't collecting the full range
-          while ((doc = iter.nextDoc()) < DocIdSetIterator.NO_MORE_DOCS) {
-            int term = si.getOrd(doc);
-            int arrIdx = term-startTermIndex;
-            if (arrIdx>=0 && arrIdx<nTerms) counts[arrIdx]++;
-          }
+        while ((doc = iter.nextDoc()) < DocIdSetIterator.NO_MORE_DOCS) {
+          counts[1+si.getOrd(doc)]++;
+        }
+      } else {
+        // version that adjusts term numbers because we aren't collecting the full range
+        int doc;
+        while ((doc = iter.nextDoc()) < DocIdSetIterator.NO_MORE_DOCS) {
+          int term = si.getOrd(doc);
+          int arrIdx = term-startTermIndex;
+          if (arrIdx>=0 && arrIdx<nTerms) counts[arrIdx]++;
         }
       }
     }

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/SimpleFacets.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/SimpleFacets.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/SimpleFacets.java Tue Mar 31 05:22:40 2015
@@ -17,26 +17,6 @@
 
 package org.apache.solr.request;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.EnumSet;
-import java.util.IdentityHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executor;
-import java.util.concurrent.Future;
-import java.util.concurrent.FutureTask;
-import java.util.concurrent.RunnableFuture;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
@@ -97,6 +77,26 @@ import org.apache.solr.util.BoundedTreeS
 import org.apache.solr.util.DateMathParser;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.EnumSet;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Future;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.RunnableFuture;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
 /**
  * A class that generates simple Facet information for a request.
  *
@@ -289,7 +289,6 @@ public class SimpleFacets {
 
     String[] facetQs = params.getParams(FacetParams.FACET_QUERY);
 
-    
     if (null != facetQs && 0 != facetQs.length) {
       for (String q : facetQs) {
         parseParams(FacetParams.FACET_QUERY, q);
@@ -382,8 +381,9 @@ public class SimpleFacets {
     boolean missing = params.getFieldBool(field, FacetParams.FACET_MISSING, false);
     // default to sorting if there is a limit.
     String sort = params.getFieldParam(field, FacetParams.FACET_SORT, limit>0 ? FacetParams.FACET_SORT_COUNT : FacetParams.FACET_SORT_INDEX);
-    String prefix = params.getFieldParam(field,FacetParams.FACET_PREFIX);
-
+    String prefix = params.getFieldParam(field, FacetParams.FACET_PREFIX);
+    String contains = params.getFieldParam(field, FacetParams.FACET_CONTAINS);
+    boolean ignoreCase = params.getFieldBool(field, FacetParams.FACET_CONTAINS_IGNORE_CASE, false);
 
     NamedList<Integer> counts;
     SchemaField sf = searcher.getSchema().getField(field);
@@ -435,13 +435,13 @@ public class SimpleFacets {
     }
 
     if (params.getFieldBool(field, GroupParams.GROUP_FACET, false)) {
-      counts = getGroupedCounts(searcher, base, field, multiToken, offset,limit, mincount, missing, sort, prefix);
+      counts = getGroupedCounts(searcher, base, field, multiToken, offset,limit, mincount, missing, sort, prefix, contains, ignoreCase);
     } else {
       assert method != null;
       switch (method) {
         case ENUM:
           assert TrieField.getMainValuePrefix(ft) == null;
-          counts = getFacetTermEnumCounts(searcher, base, field, offset, limit, mincount,missing,sort,prefix);
+          counts = getFacetTermEnumCounts(searcher, base, field, offset, limit, mincount, missing, sort, prefix, contains, ignoreCase);
           break;
         case FCS:
           assert !multiToken;
@@ -450,16 +450,19 @@ public class SimpleFacets {
             if (prefix != null && !prefix.isEmpty()) {
               throw new SolrException(ErrorCode.BAD_REQUEST, FacetParams.FACET_PREFIX + " is not supported on numeric types");
             }
+            if (contains != null && !contains.isEmpty()) {
+              throw new SolrException(ErrorCode.BAD_REQUEST, FacetParams.FACET_CONTAINS + " is not supported on numeric types");
+            }
             counts = NumericFacets.getCounts(searcher, base, field, offset, limit, mincount, missing, sort);
           } else {
-            PerSegmentSingleValuedFaceting ps = new PerSegmentSingleValuedFaceting(searcher, base, field, offset,limit, mincount, missing, sort, prefix);
+            PerSegmentSingleValuedFaceting ps = new PerSegmentSingleValuedFaceting(searcher, base, field, offset,limit, mincount, missing, sort, prefix, contains, ignoreCase);
             Executor executor = threads == 0 ? directExecutor : facetExecutor;
             ps.setNumThreads(threads);
             counts = ps.getFacetCounts(executor);
           }
           break;
         case FC:
-          counts = DocValuesFacets.getCounts(searcher, base, field, offset,limit, mincount, missing, sort, prefix);
+          counts = DocValuesFacets.getCounts(searcher, base, field, offset,limit, mincount, missing, sort, prefix, contains, ignoreCase);
           break;
         default:
           throw new AssertionError();
@@ -478,7 +481,9 @@ public class SimpleFacets {
                                              int mincount,
                                              boolean missing,
                                              String sort,
-                                             String prefix) throws IOException {
+                                             String prefix,
+                                             String contains,
+                                             boolean ignoreCase) throws IOException {
     GroupingSpecification groupingSpecification = rb.getGroupingSpec();
     final String groupField  = groupingSpecification != null ? groupingSpecification.getFields()[0] : null;
     if (groupField == null) {
@@ -488,8 +493,9 @@ public class SimpleFacets {
       );
     }
 
-    BytesRef prefixBR = prefix != null ? new BytesRef(prefix) : null;
-    final TermGroupFacetCollector collector = TermGroupFacetCollector.createTermGroupFacetCollector(groupField, field, multiToken, prefixBR, 128);
+    BytesRef prefixBytesRef = prefix != null ? new BytesRef(prefix) : null;
+    BytesRef containsRef = contains != null ? new BytesRef(contains) : null;
+    final TermGroupFacetCollector collector = TermGroupFacetCollector.createTermGroupFacetCollector(groupField, field, multiToken, prefixBytesRef, 128);
     
     SchemaField sf = searcher.getSchema().getFieldOrNull(groupField);
     
@@ -519,6 +525,10 @@ public class SimpleFacets {
     List<TermGroupFacetCollector.FacetEntry> scopedEntries 
       = result.getFacetEntries(offset, limit < 0 ? Integer.MAX_VALUE : limit);
     for (TermGroupFacetCollector.FacetEntry facetEntry : scopedEntries) {
+      //:TODO:can we do contains earlier than this to make it more efficient?
+      if (containsRef != null && !StringHelper.contains(facetEntry.getValue(), containsRef, ignoreCase)) {
+        continue;
+      }
       facetFieldType.indexedToReadable(facetEntry.getValue(), charsRef);
       facetCounts.add(charsRef.toString(), facetEntry.getCount());
     }
@@ -683,7 +693,7 @@ public class SimpleFacets {
    * @see FacetParams#FACET_ZEROS
    * @see FacetParams#FACET_MISSING
    */
-  public NamedList<Integer> getFacetTermEnumCounts(SolrIndexSearcher searcher, DocSet docs, String field, int offset, int limit, int mincount, boolean missing, String sort, String prefix)
+  public NamedList<Integer> getFacetTermEnumCounts(SolrIndexSearcher searcher, DocSet docs, String field, int offset, int limit, int mincount, boolean missing, String sort, String prefix, String contains, boolean ignoreCase)
     throws IOException {
 
     /* :TODO: potential optimization...
@@ -715,10 +725,16 @@ public class SimpleFacets {
     int off=offset;
     int lim=limit>=0 ? limit : Integer.MAX_VALUE;
 
-    BytesRef startTermBytes = null;
+    BytesRef prefixTermBytes = null;
     if (prefix != null) {
       String indexedPrefix = ft.toInternal(prefix);
-      startTermBytes = new BytesRef(indexedPrefix);
+      prefixTermBytes = new BytesRef(indexedPrefix);
+    }
+    
+    BytesRef containsTermBytes = null;
+    if (contains != null) {
+      String indexedContains = ft.toInternal(contains);
+      containsTermBytes = new BytesRef(indexedContains);
     }
 
     Fields fields = r.fields();
@@ -732,8 +748,8 @@ public class SimpleFacets {
       // TODO: OPT: if seek(ord) is supported for this termsEnum, then we could use it for
       // facet.offset when sorting by index order.
 
-      if (startTermBytes != null) {
-        if (termsEnum.seekCeil(startTermBytes) == TermsEnum.SeekStatus.END) {
+      if (prefixTermBytes != null) {
+        if (termsEnum.seekCeil(prefixTermBytes) == TermsEnum.SeekStatus.END) {
           termsEnum = null;
         } else {
           term = termsEnum.term();
@@ -750,77 +766,77 @@ public class SimpleFacets {
     if (docs.size() >= mincount) {
       while (term != null) {
 
-        if (startTermBytes != null && !StringHelper.startsWith(term, startTermBytes))
+        if (prefixTermBytes != null && !StringHelper.startsWith(term, prefixTermBytes))
           break;
 
-        int df = termsEnum.docFreq();
+        if (containsTermBytes == null || StringHelper.contains(term, containsTermBytes, ignoreCase)) {
+          int df = termsEnum.docFreq();
 
-        // If we are sorting, we can use df>min (rather than >=) since we
-        // are going in index order.  For certain term distributions this can
-        // make a large difference (for example, many terms with df=1).
-        if (df>0 && df>min) {
-          int c;
-
-          if (df >= minDfFilterCache) {
-            // use the filter cache
-
-            if (deState==null) {
-              deState = new SolrIndexSearcher.DocsEnumState();
-              deState.fieldName = field;
-              deState.liveDocs = r.getLiveDocs();
-              deState.termsEnum = termsEnum;
-              deState.postingsEnum = postingsEnum;
-            }
+          // If we are sorting, we can use df>min (rather than >=) since we
+          // are going in index order.  For certain term distributions this can
+          // make a large difference (for example, many terms with df=1).
+          if (df > 0 && df > min) {
+            int c;
+
+            if (df >= minDfFilterCache) {
+              // use the filter cache
+
+              if (deState == null) {
+                deState = new SolrIndexSearcher.DocsEnumState();
+                deState.fieldName = field;
+                deState.liveDocs = r.getLiveDocs();
+                deState.termsEnum = termsEnum;
+                deState.postingsEnum = postingsEnum;
+              }
 
-            c = searcher.numDocs(docs, deState);
+              c = searcher.numDocs(docs, deState);
 
-            postingsEnum = deState.postingsEnum;
-          } else {
-            // iterate over TermDocs to calculate the intersection
+              postingsEnum = deState.postingsEnum;
+            } else {
+              // iterate over TermDocs to calculate the intersection
 
-            // TODO: specialize when base docset is a bitset or hash set (skipDocs)?  or does it matter for this?
-            // TODO: do this per-segment for better efficiency (MultiDocsEnum just uses base class impl)
-            // TODO: would passing deleted docs lead to better efficiency over checking the fastForRandomSet?
-            postingsEnum = termsEnum.postings(null, postingsEnum, PostingsEnum.NONE);
-            c=0;
-
-            if (postingsEnum instanceof MultiPostingsEnum) {
-              MultiPostingsEnum.EnumWithSlice[] subs = ((MultiPostingsEnum) postingsEnum).getSubs();
-              int numSubs = ((MultiPostingsEnum) postingsEnum).getNumSubs();
-              for (int subindex = 0; subindex<numSubs; subindex++) {
-                MultiPostingsEnum.EnumWithSlice sub = subs[subindex];
-                if (sub.postingsEnum == null) continue;
-                int base = sub.slice.start;
+              // TODO: specialize when base docset is a bitset or hash set (skipDocs)?  or does it matter for this?
+              // TODO: do this per-segment for better efficiency (MultiDocsEnum just uses base class impl)
+              // TODO: would passing deleted docs lead to better efficiency over checking the fastForRandomSet?
+              postingsEnum = termsEnum.postings(null, postingsEnum, PostingsEnum.NONE);
+              c = 0;
+
+              if (postingsEnum instanceof MultiPostingsEnum) {
+                MultiPostingsEnum.EnumWithSlice[] subs = ((MultiPostingsEnum) postingsEnum).getSubs();
+                int numSubs = ((MultiPostingsEnum) postingsEnum).getNumSubs();
+                for (int subindex = 0; subindex < numSubs; subindex++) {
+                  MultiPostingsEnum.EnumWithSlice sub = subs[subindex];
+                  if (sub.postingsEnum == null) continue;
+                  int base = sub.slice.start;
+                  int docid;
+                  while ((docid = sub.postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+                    if (fastForRandomSet.exists(docid + base)) c++;
+                  }
+                }
+              } else {
                 int docid;
-                while ((docid = sub.postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-                  if (fastForRandomSet.exists(docid+base)) c++;
+                while ((docid = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+                  if (fastForRandomSet.exists(docid)) c++;
                 }
               }
-            } else {
-              int docid;
-              while ((docid = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-                if (fastForRandomSet.exists(docid)) c++;
-              }
-            }
-            
 
-          }
-
-          if (sortByCount) {
-            if (c>min) {
-              BytesRef termCopy = BytesRef.deepCopyOf(term);
-              queue.add(new CountPair<>(termCopy, c));
-              if (queue.size()>=maxsize) min=queue.last().val;
             }
-          } else {
-            if (c >= mincount && --off<0) {
-              if (--lim<0) break;
-              ft.indexedToReadable(term, charsRef);
-              res.add(charsRef.toString(), c);
+
+            if (sortByCount) {
+              if (c > min) {
+                BytesRef termCopy = BytesRef.deepCopyOf(term);
+                queue.add(new CountPair<>(termCopy, c));
+                if (queue.size() >= maxsize) min = queue.last().val;
+              }
+            } else {
+              if (c >= mincount && --off < 0) {
+                if (--lim < 0) break;
+                ft.indexedToReadable(term, charsRef);
+                res.add(charsRef.toString(), c);
+              }
             }
           }
         }
-
         term = termsEnum.next();
       }
     }

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/SolrQueryRequest.java Tue Mar 31 05:22:40 2015
@@ -20,6 +20,7 @@ package org.apache.solr.request;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.util.RTimer;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.core.SolrCore;
@@ -63,9 +64,15 @@ public interface SolrQueryRequest {
    */
   public void close();
 
-  /** The start time of this request in milliseconds */
+  /** The start time of this request in milliseconds.
+   * Use this only if you need the absolute system time at the start of the request,
+   * getRequestTimer() provides a more accurate mechanism for timing purposes.
+   */
   public long getStartTime();
 
+  /** The timer for this request, created when the request started being processed */
+  public RTimer getRequestTimer();
+
   /** The index searcher associated with this request */
   public SolrIndexSearcher getSearcher();
 
@@ -90,6 +97,13 @@ public interface SolrQueryRequest {
    * @param params The new set of parameter
    */
   public void forward(String handler, SolrParams params,  SolrQueryResponse rsp);
+
+  /** Returns any associated JSON (or null if none) in deserialized generic form.
+   * Java classes used to represent the JSON are as follows: Map, List, String, Long, Double, Boolean
+   */
+  public Map<String,Object> getJSON();
+
+  public void setJSON(Map<String,Object> json);
 }
 
 

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java Tue Mar 31 05:22:40 2015
@@ -19,6 +19,7 @@ package org.apache.solr.request;
 
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.util.RTimer;
 import org.apache.solr.util.RefCounted;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.common.params.SolrParams;
@@ -49,11 +50,19 @@ public abstract class SolrQueryRequestBa
   protected SolrParams params;
   protected Map<Object,Object> context;
   protected Iterable<ContentStream> streams;
+  protected Map<String,Object> json;
 
-  public SolrQueryRequestBase(SolrCore core, SolrParams params) {
+  private final RTimer requestTimer;
+
+  public SolrQueryRequestBase(SolrCore core, SolrParams params, RTimer requestTimer) {
     this.core = core;
     this.schema = null == core ? null : core.getLatestSchema();
     this.params = this.origParams = params;
+    this.requestTimer = requestTimer;
+  }
+
+  public SolrQueryRequestBase(SolrCore core, SolrParams params) {
+    this(core, params, new RTimer());
   }
 
   @Override
@@ -85,6 +94,10 @@ public abstract class SolrQueryRequestBa
     return startTime;
   }
 
+  public RTimer getRequestTimer () {
+    return requestTimer;
+  }
+
   // The index searcher associated with this request
   protected RefCounted<SolrIndexSearcher> searcherHolder;
   @Override
@@ -160,4 +173,14 @@ public abstract class SolrQueryRequestBa
 
   }
 
+  @Override
+  public Map<String, Object> getJSON() {
+    return json;
+  }
+
+  @Override
+  public void setJSON(Map<String, Object> json) {
+    this.json = json;
+  }
+
 }

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/response/SolrQueryResponse.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/response/SolrQueryResponse.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/response/SolrQueryResponse.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/response/SolrQueryResponse.java Tue Mar 31 05:22:40 2015
@@ -171,44 +171,7 @@ public class SolrQueryResponse {
     return err;
   }
 
-  /**
-   * The endtime of the request in milliseconds.
-   * Used to calculate query time.
-   * @see #setEndTime(long)
-   * @see #getEndTime()
-   */
-  protected long endtime;
-
-  /**
-   * Get the time in milliseconds when the response officially finished. 
-   */
-  public long getEndTime() {
-    if (endtime==0) {
-      setEndTime();
-    }
-    return endtime;
-  }
-
-  /**
-   * Stop the timer for how long this query took.
-   * @see #setEndTime(long)
-   */
-  public long setEndTime() {
-    return setEndTime(System.currentTimeMillis());
-  }
-
-  /**
-   * Set the in milliseconds when the response officially finished. 
-   * @see #setEndTime()
-   */
-  public long setEndTime(long endtime) {
-    if (endtime!=0) {
-      this.endtime=endtime;
-    }
-    return this.endtime;
-  }
-  
-  /** Repsonse header to be logged */ 
+  /** Response header to be logged */
   public NamedList<Object> getResponseHeader() {
     @SuppressWarnings("unchecked")
     SimpleOrderedMap<Object> header = (SimpleOrderedMap<Object>) values.get("responseHeader");

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java Tue Mar 31 05:22:40 2015
@@ -21,11 +21,12 @@ import java.io.IOException;
 import java.io.Writer;
 import java.util.*;
 
-import org.apache.lucene.document.Document;
-import org.apache.lucene.index.IndexableField;
+import org.apache.solr.client.solrj.io.TupleStream;
+import org.apache.solr.client.solrj.io.Tuple;
 import org.apache.lucene.index.StorableField;
 import org.apache.lucene.index.StoredDocument;
 import org.apache.lucene.util.BytesRef;
+import org.apache.solr.common.EnumFieldValue;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.util.Base64;
@@ -186,18 +187,22 @@ public abstract class TextResponseWriter
       writeMap(name, (Map)val, false, true);
     } else if (val instanceof NamedList) {
       writeNamedList(name, (NamedList)val);
+    } else if (val instanceof TupleStream) {
+      writeTupleStream((TupleStream) val);
     } else if (val instanceof Iterable) {
       writeArray(name,((Iterable)val).iterator());
     } else if (val instanceof Object[]) {
       writeArray(name,(Object[])val);
     } else if (val instanceof Iterator) {
-      writeArray(name,(Iterator)val);
+      writeArray(name, (Iterator) val);
     } else if (val instanceof byte[]) {
       byte[] arr = (byte[])val;
       writeByteArr(name, arr, 0, arr.length);
     } else if (val instanceof BytesRef) {
       BytesRef arr = (BytesRef)val;
       writeByteArr(name, arr.bytes, arr.offset, arr.length);
+    } else if (val instanceof EnumFieldValue) {
+      writeStr(name, val.toString(), true);
     } else {
       // default... for debugging only
       writeStr(name, val.getClass().getName() + ':' + val.toString(), true);
@@ -309,6 +314,26 @@ public abstract class TextResponseWriter
     }
   }
 
+  public void writeTupleStream(TupleStream tupleStream) throws IOException {
+    tupleStream.open();
+    writeStartDocumentList("response", -1, -1, -1, null);
+    boolean isFirst = true;
+    while(true) {
+      Tuple tuple = tupleStream.read();
+      if(!isFirst) {
+        writer.write(",");
+      }
+      writeMap(null, tuple.fields, false, true);
+      isFirst = false;
+      if(tuple.EOF) {
+        break;
+      }
+    }
+    writeEndDocumentList();
+    tupleStream.close();
+  }
+
+
   /** if this form of the method is called, val is the Java string form of a double */
   public abstract void writeDouble(String name, String val) throws IOException;
 

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/response/transform/ValueSourceAugmenter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/response/transform/ValueSourceAugmenter.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/response/transform/ValueSourceAugmenter.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/response/transform/ValueSourceAugmenter.java Tue Mar 31 05:22:40 2015
@@ -96,12 +96,15 @@ public class ValueSourceAugmenter extend
       }
 
       int localId = docid - rcontext.docBase;
-      Object val = values.objectVal(localId);
-      if (val != null) {
-        doc.setField( name, val );
-      }
+      setValue(doc,values.objectVal(localId));
     } catch (IOException e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "exception at docid " + docid + " for valuesource " + valueSource, e);
     }
   }
+  
+  protected void setValue(SolrDocument doc, Object val) {
+    if(val!=null) {
+      doc.setField( name, val );
+    }
+  }
 }

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/AbstractSpatialPrefixTreeFieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/AbstractSpatialPrefixTreeFieldType.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/AbstractSpatialPrefixTreeFieldType.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/AbstractSpatialPrefixTreeFieldType.java Tue Mar 31 05:22:40 2015
@@ -17,14 +17,22 @@ package org.apache.solr.schema;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.io.Reader;
 import java.util.Map;
 
+import org.apache.commons.io.IOUtils;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.core.KeywordTokenizer;
 import org.apache.lucene.spatial.prefix.PrefixTreeStrategy;
 import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
 import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTreeFactory;
 import org.apache.lucene.spatial.query.SpatialArgsParser;
 import org.apache.solr.util.MapListener;
 
+import com.spatial4j.core.shape.Shape;
+
 /**
  * @see PrefixTreeStrategy
  * @lucene.experimental
@@ -62,7 +70,40 @@ public abstract class AbstractSpatialPre
     if (v != null)
       defaultFieldValuesArrayLen = Integer.valueOf(v);
   }
-
+  
+  /**
+   * This analyzer is not actually used for indexing.  It is implemented here
+   * so that the analysis UI will show reasonable tokens.
+   */
+  @Override
+  public Analyzer getIndexAnalyzer()
+  {
+    return new Analyzer() {
+      
+      @Override
+      protected TokenStreamComponents createComponents(final String fieldName) {
+        return new TokenStreamComponents(new KeywordTokenizer()) {
+          private Shape shape = null;
+          
+          protected void setReader(final Reader reader) throws IOException {
+            source.setReader(reader);
+            shape = parseShape(IOUtils.toString(reader));
+          }
+          
+          public TokenStream getTokenStream() {
+            PrefixTreeStrategy s = newSpatialStrategy(fieldName==null ? getTypeName() : fieldName);
+            return s.createIndexableFields(shape)[0].tokenStreamValue();
+          }
+        };
+      }
+    };
+  }
+  
+  @Override
+  public Analyzer getQueryAnalyzer()
+  {
+    return getIndexAnalyzer();
+  }
 
   @Override
   protected T newSpatialStrategy(String fieldName) {

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/BBoxField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/BBoxField.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/BBoxField.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/BBoxField.java Tue Mar 31 05:22:40 2015
@@ -44,6 +44,7 @@ public class BBoxField extends AbstractS
 
   private String numberTypeName;//required
   private String booleanTypeName = "boolean";
+  private boolean storeSubFields = false;
 
   private IndexSchema schema;
 
@@ -66,6 +67,11 @@ public class BBoxField extends AbstractS
     if (v != null) {
       booleanTypeName = v;
     }
+    
+    v = args.remove("storeSubFields");
+    if (v != null) {
+      storeSubFields = Boolean.valueOf(v);
+    }
   }
 
   @Override
@@ -108,7 +114,14 @@ public class BBoxField extends AbstractS
   // note: Registering the field is probably optional; it makes it show up in the schema browser and may have other
   //  benefits.
   private void register(IndexSchema schema, String name, FieldType fieldType) {
-    SchemaField sf = new SchemaField(name, fieldType);
+    int props = fieldType.properties;
+    if(storeSubFields) {
+      props |= STORED;
+    }
+    else {
+      props &= ~STORED;
+    }
+    SchemaField sf = new SchemaField(name, fieldType, props, null);
     schema.getFields().put(sf.getName(), sf);
   }
 

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/EnumField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/EnumField.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/EnumField.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/EnumField.java Tue Mar 31 05:22:40 2015
@@ -17,12 +17,33 @@ package org.apache.solr.schema;
  * limitations under the License.
  */
 
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.xpath.XPath;
+import javax.xml.xpath.XPathConstants;
+import javax.xml.xpath.XPathExpressionException;
+import javax.xml.xpath.XPathFactory;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
 import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.StorableField;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.EnumFieldSource;
-import org.apache.lucene.search.*;
+import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.DocValuesRangeQuery;
+import org.apache.lucene.search.NumericRangeQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SortField;
 import org.apache.lucene.uninverting.UninvertingReader.Type;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -40,19 +61,6 @@ import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 import org.xml.sax.SAXException;
 
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.xpath.XPath;
-import javax.xml.xpath.XPathConstants;
-import javax.xml.xpath.XPathExpressionException;
-import javax.xml.xpath.XPathFactory;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.HashMap;
-import java.util.Locale;
-import java.util.Map;
-
 /***
  * Field type for support of string values with custom sort order.
  */
@@ -226,6 +234,14 @@ public class EnumField extends Primitive
    * {@inheritDoc}
    */
   @Override
+  public FieldType.NumericType getNumericType() {
+    return FieldType.NumericType.INT;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
   public Query getRangeQuery(QParser parser, SchemaField field, String min, String max, boolean minInclusive, boolean maxInclusive) {
     Integer minValue = stringValueToIntValue(min);
     Integer maxValue = stringValueToIntValue(max);
@@ -255,10 +271,7 @@ public class EnumField extends Primitive
    * {@inheritDoc}
    */
   @Override
-  public void checkSchemaField(final SchemaField field) {
-    if (field.hasDocValues() && !field.multiValued() && !(field.isRequired() || field.getDefaultValue() != null)) {
-      throw new IllegalStateException("Field " + this + " has single-valued doc values enabled, but has no default value and is not required");
-    }
+  public void checkSchemaField(SchemaField field) {
   }
 
   /**
@@ -383,6 +396,7 @@ public class EnumField extends Primitive
     newType.setStoreTermVectors(field.storeTermVector());
     newType.setStoreTermVectorOffsets(field.storeTermOffsets());
     newType.setStoreTermVectorPositions(field.storeTermPositions());
+    newType.setStoreTermVectorPayloads(field.storeTermPayloads());
     newType.setNumericType(FieldType.NumericType.INT);
     newType.setNumericPrecisionStep(DEFAULT_PRECISION_STEP);
 
@@ -394,6 +408,30 @@ public class EnumField extends Primitive
   }
 
   /**
+   * {@inheritDoc}
+   */
+  @Override
+  public List<StorableField> createFields(SchemaField sf, Object value, float boost) {
+    if (sf.hasDocValues()) {
+      List<StorableField> fields = new ArrayList<>();
+      final StorableField field = createField(sf, value, boost);
+      fields.add(field);
+
+      if (sf.multiValued()) {
+        BytesRefBuilder bytes = new BytesRefBuilder();
+        readableToIndexed(stringValueToIntValue(value.toString()).toString(), bytes);
+        fields.add(new SortedSetDocValuesField(sf.getName(), bytes.toBytesRef()));
+      } else {
+        final long bits = field.numericValue().intValue();
+        fields.add(new NumericDocValuesField(sf.getName(), bits));
+      }
+      return fields;
+    } else {
+      return Collections.singletonList(createField(sf, value, boost));
+    }
+  }
+
+  /**
    * Converting the (internal) integer value (indicating the sort order) to string (displayed) value
    * @param intVal integer value
    * @return string value

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/FieldProperties.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/FieldProperties.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/FieldProperties.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/FieldProperties.java Tue Mar 31 05:22:40 2015
@@ -17,8 +17,8 @@
 
 package org.apache.solr.schema;
 
-import java.util.Map;
 import java.util.HashMap;
+import java.util.Map;
 
 /**
  *
@@ -52,13 +52,15 @@ public abstract class FieldProperties {
   protected final static int STORE_OFFSETS       = 0x00004000;
   protected final static int DOC_VALUES          = 0x00008000;
 
+  protected final static int STORE_TERMPAYLOADS  = 0x00010000;
+
   static final String[] propertyNames = {
           "indexed", "tokenized", "stored",
           "binary", "omitNorms", "omitTermFreqAndPositions",
           "termVectors", "termPositions", "termOffsets",
           "multiValued",
           "sortMissingFirst","sortMissingLast","required", "omitPositions",
-          "storeOffsetsWithPositions", "docValues"
+          "storeOffsetsWithPositions", "docValues", "termPayloads"
   };
 
   static final Map<String,Integer> propertyMap = new HashMap<>();

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/FieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/FieldType.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/FieldType.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/FieldType.java Tue Mar 31 05:22:40 2015
@@ -17,8 +17,6 @@
 
 package org.apache.solr.schema;
 
-import static org.apache.lucene.analysis.util.AbstractAnalysisFactory.LUCENE_MATCH_VERSION_PARAM;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -40,7 +38,6 @@ import org.apache.lucene.index.IndexOpti
 import org.apache.lucene.index.StorableField;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.DocTermOrdsRewriteMethod;
 import org.apache.lucene.search.DocValuesRangeQuery;
 import org.apache.lucene.search.DocValuesRewriteMethod;
 import org.apache.lucene.search.MultiTermQuery;
@@ -68,6 +65,8 @@ import org.apache.solr.search.Sorting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.lucene.analysis.util.AbstractAnalysisFactory.LUCENE_MATCH_VERSION_PARAM;
+
 /**
  * Base class for all field types used by an index schema.
  *
@@ -269,6 +268,7 @@ public abstract class FieldType extends
     newType.setStoreTermVectors(field.storeTermVector());
     newType.setStoreTermVectorOffsets(field.storeTermOffsets());
     newType.setStoreTermVectorPositions(field.storeTermPositions());
+    newType.setStoreTermVectorPayloads(field.storeTermPayloads());
 
     return createField(field.getName(), val, newType, boost);
   }
@@ -732,9 +732,9 @@ public abstract class FieldType extends
    */
   public MultiTermQuery.RewriteMethod getRewriteMethod(QParser parser, SchemaField field) {
     if (!field.indexed() && field.hasDocValues()) {
-      return field.multiValued() ? new DocTermOrdsRewriteMethod() : new DocValuesRewriteMethod();
+      return new DocValuesRewriteMethod();
     } else {
-      return MultiTermQuery.CONSTANT_SCORE_FILTER_REWRITE;
+      return MultiTermQuery.CONSTANT_SCORE_REWRITE;
     }
   }
 

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/IndexSchema.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/IndexSchema.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/IndexSchema.java Tue Mar 31 05:22:40 2015
@@ -755,7 +755,7 @@ public class IndexSchema {
   }
 
   /** Returns true if the given name has exactly one asterisk either at the start or end of the name */
-  private static boolean isValidFieldGlob(String name) {
+  protected static boolean isValidFieldGlob(String name) {
     if (name.startsWith("*") || name.endsWith("*")) {
       int count = 0;
       for (int pos = 0 ; pos < name.length() && -1 != (pos = name.indexOf('*', pos)) ; ++pos) ++count;
@@ -935,7 +935,7 @@ public class IndexSchema {
     }
   }
 
-  private void registerExplicitSrcAndDestFields(String source, int maxChars, SchemaField destSchemaField, SchemaField sourceSchemaField) {
+  protected void registerExplicitSrcAndDestFields(String source, int maxChars, SchemaField destSchemaField, SchemaField sourceSchemaField) {
     List<CopyField> copyFieldList = copyFieldsMap.get(source);
     if (copyFieldList == null) {
       copyFieldList = new ArrayList<>();
@@ -1108,6 +1108,8 @@ public class IndexSchema {
       this.destDynamicBase = destDynamicBase;
     }
 
+    public DynamicField getDestination() { return destination; }
+
     public String getDestFieldName() { return destination.getRegex(); }
 
     /**
@@ -1295,7 +1297,7 @@ public class IndexSchema {
       if (df.matches(fieldName)) return df.prototype.getType();
     }
     return null;
-  };
+  }
 
 
   /**
@@ -1416,10 +1418,11 @@ public class IndexSchema {
     List<SimpleOrderedMap<Object>> copyFieldProperties = new ArrayList<>();
     SortedMap<String,List<CopyField>> sortedCopyFields = new TreeMap<>(copyFieldsMap);
     for (List<CopyField> copyFields : sortedCopyFields.values()) {
+      copyFields = new ArrayList<>(copyFields);
       Collections.sort(copyFields, new Comparator<CopyField>() {
         @Override
         public int compare(CopyField cf1, CopyField cf2) {
-          // sources are all be the same, just sorting by destination here
+          // sources are all the same, just sorting by destination here
           return cf1.getDestination().getName().compareTo(cf2.getDestination().getName());
         }
       });
@@ -1494,12 +1497,12 @@ public class IndexSchema {
    * {@link #getSchemaUpdateLock()}.
    *
    * @param newField the SchemaField to add 
-   * @param persist to persist the schema or not or not
+   * @param persist to persist the schema or not
    * @return a new IndexSchema based on this schema with newField added
    * @see #newField(String, String, Map)
    */
   public IndexSchema addField(SchemaField newField, boolean persist) {
-    return addFields(Collections.singletonList(newField),Collections.EMPTY_MAP,persist );
+    return addFields(Collections.singletonList(newField), Collections.emptyMap(), persist);
   }
 
   public IndexSchema addField(SchemaField newField) {
@@ -1552,13 +1555,51 @@ public class IndexSchema {
 
 
   /**
+   * Copies this schema, deletes the named fields from the copy.
+   * <p>
+   * The schema will not be persisted.
+   * <p>
+   * Requires synchronizing on the object returned by
+   * {@link #getSchemaUpdateLock()}.
+   *
+   * @param names the names of the fields to delete
+   * @return a new IndexSchema based on this schema with the named fields deleted
+   */
+  public IndexSchema deleteFields(Collection<String> names) {
+    String msg = "This IndexSchema is not mutable.";
+    log.error(msg);
+    throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+  }
+
+  /**
+   * Copies this schema, deletes the named field from the copy, creates a new field 
+   * with the same name using the given args, then rebinds any referring copy fields
+   * to the replacement field.
+   *
+   * <p>
+   * The schema will not be persisted.
+   * <p>
+   * Requires synchronizing on the object returned by {@link #getSchemaUpdateLock()}.
+   *
+   * @param fieldName The name of the field to be replaced
+   * @param replacementFieldType  The field type of the replacement field                                   
+   * @param replacementArgs Initialization params for the replacement field
+   * @return a new IndexSchema based on this schema with the named field replaced
+   */
+  public IndexSchema replaceField(String fieldName, FieldType replacementFieldType, Map<String,?> replacementArgs) {
+    String msg = "This IndexSchema is not mutable.";
+    log.error(msg);
+    throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+  }
+
+  /**
    * Copies this schema, adds the given dynamic fields to the copy,
    * Requires synchronizing on the object returned by
    * {@link #getSchemaUpdateLock()}.
    *
    * @param newDynamicFields the SchemaFields to add
    * @param copyFieldNames 0 or more names of targets to copy this field to.  The target fields must already exist.
-   * @param persist to persist the schema or not or not
+   * @param persist to persist the schema or not
    * @return a new IndexSchema based on this schema with newDynamicFields added
    * @see #newDynamicField(String, String, Map)
    */
@@ -1572,20 +1613,78 @@ public class IndexSchema {
   }
 
   /**
-   * Copies this schema and adds the new copy fields to the copy
+   * Copies this schema, deletes the named dynamic fields from the copy.
+   * <p>
+   * The schema will not be persisted.
+   * <p>
+   * Requires synchronizing on the object returned by
+   * {@link #getSchemaUpdateLock()}.
+   *
+   * @param fieldNamePatterns the names of the dynamic fields to delete
+   * @return a new IndexSchema based on this schema with the named dynamic fields deleted
+   */
+  public IndexSchema deleteDynamicFields(Collection<String> fieldNamePatterns) {
+    String msg = "This IndexSchema is not mutable.";
+    log.error(msg);
+    throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+  }
+
+  /**
+   * Copies this schema, deletes the named dynamic field from the copy, creates a new dynamic
+   * field with the same field name pattern using the given args, then rebinds any referring
+   * dynamic copy fields to the replacement dynamic field.
+   *
+   * <p>
+   * The schema will not be persisted.
+   * <p>
+   * Requires synchronizing on the object returned by {@link #getSchemaUpdateLock()}.
+   *
+   * @param fieldNamePattern The glob for the dynamic field to be replaced
+   * @param replacementFieldType  The field type of the replacement dynamic field                                   
+   * @param replacementArgs Initialization params for the replacement dynamic field
+   * @return a new IndexSchema based on this schema with the named dynamic field replaced
+   */
+  public ManagedIndexSchema replaceDynamicField
+      (String fieldNamePattern, FieldType replacementFieldType, Map<String,?> replacementArgs) {
+    String msg = "This IndexSchema is not mutable.";
+    log.error(msg);
+    throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+  }
+
+    /**
+     * Copies this schema and adds the new copy fields to the copy
+     * Requires synchronizing on the object returned by
+     * {@link #getSchemaUpdateLock()}.
+     *
+     * @param copyFields Key is the name of the source field name, value is a collection of target field names.  Fields must exist.
+     * @param persist to persist the schema or not
+     * @return The new Schema with the copy fields added
+     */
+  public IndexSchema addCopyFields(Map<String, Collection<String>> copyFields, boolean persist) {
+    String msg = "This IndexSchema is not mutable.";
+    log.error(msg);
+    throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+  }
+
+  /**
+   * Copies this schema and deletes the given copy fields from the copy.
+   * <p>
+   * The schema will not be persisted.
+   * <p>
    * Requires synchronizing on the object returned by
    * {@link #getSchemaUpdateLock()}.
    *
-   * @param copyFields Key is the name of the source field name, value is a collection of target field names.  Fields must exist.
-   * @param persist to persist the schema or not or not
-   * @return The new Schema with the copy fields added
+   * @param copyFields Key is the name of the source field name, value is a collection of target field names. 
+   *                   Each corresponding copy field directives must exist.
+   * @return The new Schema with the copy fields deleted
    */
-  public IndexSchema addCopyFields(Map<String, Collection<String>> copyFields, boolean persist){
+  public IndexSchema deleteCopyFields(Map<String, Collection<String>> copyFields) {
     String msg = "This IndexSchema is not mutable.";
     log.error(msg);
     throw new SolrException(ErrorCode.SERVER_ERROR, msg);
   }
 
+
   /**
    * Returns a SchemaField if the given fieldName does not already 
    * exist in this schema, and does not match any dynamic fields 
@@ -1623,7 +1722,7 @@ public class IndexSchema {
   }
 
   /**
-   * Returns the schema update lock that should be synchronzied on
+   * Returns the schema update lock that should be synchronized on
    * to update the schema.  Only applicable to mutable schemas.
    *
    * @return the schema update lock object to synchronize on
@@ -1640,7 +1739,7 @@ public class IndexSchema {
    * {@link #getSchemaUpdateLock()}.
    *
    * @param fieldTypeList a list of FieldTypes to add
-   * @param persist to persist the schema or not or not
+   * @param persist to persist the schema or not
    * @return a new IndexSchema based on this schema with the new types added
    * @see #newFieldType(String, String, Map)
    */
@@ -1648,6 +1747,43 @@ public class IndexSchema {
     String msg = "This IndexSchema is not mutable.";
     log.error(msg);
     throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+  }
+
+  /**
+   * Copies this schema, deletes the named field types from the copy.
+   * <p>
+   * The schema will not be persisted.
+   * <p>
+   * Requires synchronizing on the object returned by {@link #getSchemaUpdateLock()}.
+   *
+   * @param names the names of the field types to delete
+   * @return a new IndexSchema based on this schema with the named field types deleted
+   */
+  public IndexSchema deleteFieldTypes(Collection<String> names) {
+    String msg = "This IndexSchema is not mutable.";
+    log.error(msg);
+    throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+  }
+
+  /**
+   * Copies this schema, deletes the named field type from the copy, creates a new field type 
+   * with the same name using the given args, rebuilds fields and dynamic fields of the given
+   * type, then rebinds any referring copy fields to the rebuilt fields.
+   * 
+   * <p>
+   * The schema will not be persisted.
+   * <p>
+   * Requires synchronizing on the object returned by {@link #getSchemaUpdateLock()}.
+   *  
+   * @param typeName The name of the field type to be replaced
+   * @param replacementClassName The class name of the replacement field type
+   * @param replacementArgs Initialization params for the replacement field type
+   * @return a new IndexSchema based on this schema with the named field type replaced
+   */
+  public IndexSchema replaceFieldType(String typeName, String replacementClassName, Map<String,Object> replacementArgs) {
+    String msg = "This IndexSchema is not mutable.";
+    log.error(msg);
+    throw new SolrException(ErrorCode.SERVER_ERROR, msg);
   }
 
   /**