You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2015/02/09 00:53:25 UTC

svn commit: r1658277 [20/38] - in /lucene/dev/branches/lucene6005: ./ dev-tools/ dev-tools/idea/solr/contrib/dataimporthandler/ dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/lucene/replicator/ dev-tools/maven/solr/ dev-tools/maven/solr/contrib/...

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java Sun Feb  8 23:53:14 2015
@@ -24,11 +24,29 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Iterator;
+import java.util.Arrays;
 
+import com.carrotsearch.hppc.IntObjectOpenHashMap;
+import com.carrotsearch.hppc.LongOpenHashSet;
+import com.carrotsearch.hppc.LongObjectOpenHashMap;
+import com.carrotsearch.hppc.cursors.IntObjectCursor;
+import com.carrotsearch.hppc.cursors.LongCursor;
+import com.carrotsearch.hppc.cursors.LongObjectCursor;
+import com.carrotsearch.hppc.IntOpenHashSet;
+import com.carrotsearch.hppc.cursors.ObjectCursor;
+import com.carrotsearch.hppc.LongObjectMap;
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.FilterLeafReader;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.MultiDocValues;
+import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.queries.TermsFilter;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.LeafCollector;
@@ -40,35 +58,37 @@ import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.TopDocsCollector;
 import org.apache.lucene.search.TopFieldCollector;
 import org.apache.lucene.search.TopScoreDocCollector;
+import org.apache.lucene.uninverting.UninvertingReader;
 import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.LongValues;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.params.ExpandParams;
-import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.TrieFloatField;
+import org.apache.solr.schema.TrieIntField;
+import org.apache.solr.schema.TrieLongField;
+import org.apache.solr.schema.TrieDoubleField;
+import org.apache.solr.schema.StrField;
 import org.apache.solr.search.CollapsingQParserPlugin;
 import org.apache.solr.search.DocIterator;
 import org.apache.solr.search.DocList;
 import org.apache.solr.search.DocSlice;
 import org.apache.solr.search.QParser;
 import org.apache.solr.search.QueryParsing;
+import org.apache.solr.search.SolrConstantScoreQuery;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
 import org.apache.solr.util.plugin.SolrCoreAware;
 
-import com.carrotsearch.hppc.IntObjectMap;
-import com.carrotsearch.hppc.IntObjectOpenHashMap;
-import com.carrotsearch.hppc.IntOpenHashSet;
-import com.carrotsearch.hppc.cursors.IntObjectCursor;
-import com.carrotsearch.hppc.cursors.ObjectCursor;
-
 /**
  * The ExpandComponent is designed to work with the CollapsingPostFilter.
  * The CollapsingPostFilter collapses a result set on a field.
@@ -118,6 +138,7 @@ public class ExpandComponent extends Sea
     SolrParams params = req.getParams();
 
     String field = params.get(ExpandParams.EXPAND_FIELD);
+    String hint = null;
     if (field == null) {
       List<Query> filters = rb.getFilters();
       if (filters != null) {
@@ -125,6 +146,7 @@ public class ExpandComponent extends Sea
           if (q instanceof CollapsingQParserPlugin.CollapsingPostFilter) {
             CollapsingQParserPlugin.CollapsingPostFilter cp = (CollapsingQParserPlugin.CollapsingPostFilter) q;
             field = cp.getField();
+            hint = cp.hint;
           }
         }
       }
@@ -183,26 +205,168 @@ public class ExpandComponent extends Sea
 
     SolrIndexSearcher searcher = req.getSearcher();
     LeafReader reader = searcher.getLeafReader();
-    SortedDocValues values = DocValues.getSorted(reader, field);
-    FixedBitSet groupBits = new FixedBitSet(values.getValueCount());
+
+    FieldType fieldType = searcher.getSchema().getField(field).getType();
+
+    SortedDocValues values = null;
+    long nullValue = 0;
+
+    if(fieldType instanceof StrField) {
+      //Get The Top Level SortedDocValues
+      if(CollapsingQParserPlugin.HINT_TOP_FC.equals(hint)) {
+        Map<String, UninvertingReader.Type> mapping = new HashMap();
+        mapping.put(field, UninvertingReader.Type.SORTED);
+        UninvertingReader uninvertingReader = new UninvertingReader(new ReaderWrapper(searcher.getLeafReader(), field), mapping);
+        values = uninvertingReader.getSortedDocValues(field);
+      } else {
+        values = DocValues.getSorted(reader, field);
+      }
+    } else {
+      //Get the nullValue for the numeric collapse field
+      String defaultValue = searcher.getSchema().getField(field).getDefaultValue();
+      if(defaultValue != null) {
+        if(fieldType instanceof TrieIntField || fieldType instanceof TrieLongField) {
+          nullValue = Long.parseLong(defaultValue);
+        } else if(fieldType instanceof TrieFloatField){
+          nullValue = Float.floatToIntBits(Float.parseFloat(defaultValue));
+        } else if(fieldType instanceof TrieDoubleField){
+          nullValue = Double.doubleToLongBits(Double.parseDouble(defaultValue));
+        }
+      } else {
+        if(fieldType instanceof TrieFloatField){
+          nullValue = Float.floatToIntBits(0.0f);
+        } else if(fieldType instanceof TrieDoubleField){
+          nullValue = Double.doubleToLongBits(0.0f);
+        }
+      }
+    }
+
+    FixedBitSet groupBits = null;
+    LongOpenHashSet groupSet = null;
     DocList docList = rb.getResults().docList;
     IntOpenHashSet collapsedSet = new IntOpenHashSet(docList.size() * 2);
 
+    //Gather the groups for the current page of documents
     DocIterator idit = docList.iterator();
-
+    int[] globalDocs = new int[docList.size()];
+    int docsIndex = -1;
     while (idit.hasNext()) {
-      int doc = idit.nextDoc();
-      int ord = values.getOrd(doc);
-      if (ord > -1) {
-        groupBits.set(ord);
-        collapsedSet.add(doc);
+      globalDocs[++docsIndex] = idit.nextDoc();
+    }
+
+    Arrays.sort(globalDocs);
+    Query groupQuery = null;
+
+    /*
+    * This code gathers the group information for the current page.
+    */
+    List<LeafReaderContext> contexts = searcher.getTopReaderContext().leaves();
+    int currentContext = 0;
+    int currentDocBase = contexts.get(currentContext).docBase;
+    int nextDocBase = (currentContext+1)<contexts.size() ? contexts.get(currentContext+1).docBase : Integer.MAX_VALUE;
+    IntObjectOpenHashMap<BytesRef> ordBytes = null;
+    if(values != null) {
+      groupBits = new FixedBitSet(values.getValueCount());
+      MultiDocValues.OrdinalMap ordinalMap = null;
+      SortedDocValues[] sortedDocValues = null;
+      LongValues segmentOrdinalMap = null;
+      SortedDocValues currentValues = null;
+      if(values instanceof  MultiDocValues.MultiSortedDocValues) {
+        ordinalMap = ((MultiDocValues.MultiSortedDocValues)values).mapping;
+        sortedDocValues = ((MultiDocValues.MultiSortedDocValues)values).values;
+        currentValues = sortedDocValues[currentContext];
+        segmentOrdinalMap = ordinalMap.getGlobalOrds(currentContext);
+      }
+      int count = 0;
+
+      ordBytes = new IntObjectOpenHashMap();
+
+      for(int i=0; i<globalDocs.length; i++) {
+        int globalDoc = globalDocs[i];
+        while(globalDoc >= nextDocBase) {
+          currentContext++;
+          currentDocBase = contexts.get(currentContext).docBase;
+          nextDocBase = (currentContext+1) < contexts.size() ? contexts.get(currentContext+1).docBase : Integer.MAX_VALUE;
+          if(ordinalMap != null) {
+            currentValues = sortedDocValues[currentContext];
+            segmentOrdinalMap = ordinalMap.getGlobalOrds(currentContext);
+          }
+        }
+
+        int contextDoc = globalDoc - currentDocBase;
+        if(ordinalMap != null) {
+          int ord = currentValues.getOrd(contextDoc);
+          if(ord > -1) {
+            ++count;
+            BytesRef ref = currentValues.lookupOrd(ord);
+            ord = (int)segmentOrdinalMap.get(ord);
+            ordBytes.put(ord, BytesRef.deepCopyOf(ref));
+            groupBits.set(ord);
+            collapsedSet.add(globalDoc);
+          }
+        } else {
+          int ord = values.getOrd(globalDoc);
+          if(ord > -1) {
+            ++count;
+            BytesRef ref = values.lookupOrd(ord);
+            ordBytes.put(ord, BytesRef.deepCopyOf(ref));
+            groupBits.set(ord);
+            collapsedSet.add(globalDoc);
+          }
+        }
+      }
+
+      if(count > 0 && count < 200) {
+        try {
+          groupQuery = getGroupQuery(field, count, ordBytes);
+        } catch(Exception e) {
+          throw new IOException(e);
+        }
+      }
+    } else {
+      groupSet = new LongOpenHashSet((int)(docList.size()*1.25));
+      NumericDocValues collapseValues = contexts.get(currentContext).reader().getNumericDocValues(field);
+      int count = 0;
+      for(int i=0; i<globalDocs.length; i++) {
+        int globalDoc = globalDocs[i];
+        while(globalDoc >= nextDocBase) {
+          currentContext++;
+          currentDocBase = contexts.get(currentContext).docBase;
+          nextDocBase = currentContext+1 < contexts.size() ? contexts.get(currentContext+1).docBase : Integer.MAX_VALUE;
+          collapseValues = contexts.get(currentContext).reader().getNumericDocValues(field);
+        }
+        int contextDoc = globalDoc - currentDocBase;
+        long value = collapseValues.get(contextDoc);
+        if(value != nullValue) {
+          ++count;
+          groupSet.add(value);
+          collapsedSet.add(globalDoc);
+        }
+      }
+
+      if(count > 0 && count < 200) {
+        groupQuery = getGroupQuery(field, fieldType, count, groupSet);
       }
     }
 
     Collector collector;
     if (sort != null)
       sort = sort.rewrite(searcher);
-    GroupExpandCollector groupExpandCollector = new GroupExpandCollector(values, groupBits, collapsedSet, limit, sort);
+
+
+    Collector groupExpandCollector = null;
+
+    if(values != null) {
+      groupExpandCollector = new GroupExpandCollector(values, groupBits, collapsedSet, limit, sort);
+    } else {
+      groupExpandCollector = new NumericGroupExpandCollector(field, nullValue, groupSet, collapsedSet, limit, sort);
+    }
+
+    if(groupQuery !=  null) {
+      //Limits the results to documents that are in the same group as the documents in the page.
+      newFilters.add(groupQuery);
+    }
+
     SolrIndexSearcher.ProcessedFilter pfilter = searcher.getProcessedFilter(null, newFilters);
     if (pfilter.postFilter != null) {
       pfilter.postFilter.setLastDelegate(groupExpandCollector);
@@ -212,12 +376,11 @@ public class ExpandComponent extends Sea
     }
 
     searcher.search(query, pfilter.filter, collector);
-    IntObjectMap groups = groupExpandCollector.getGroups();
+    LongObjectMap groups = ((GroupCollector)groupExpandCollector).getGroups();
     Map<String, DocSlice> outMap = new HashMap<>();
     CharsRefBuilder charsRef = new CharsRefBuilder();
-    FieldType fieldType = searcher.getSchema().getField(field).getType();
-    for (IntObjectCursor cursor : (Iterable<IntObjectCursor>) groups) {
-      int ord = cursor.key;
+    for (LongObjectCursor cursor : (Iterable<LongObjectCursor>) groups) {
+      long groupValue = cursor.key;
       TopDocsCollector topDocsCollector = (TopDocsCollector) cursor.value;
       TopDocs topDocs = topDocsCollector.topDocs();
       ScoreDoc[] scoreDocs = topDocs.scoreDocs;
@@ -230,10 +393,21 @@ public class ExpandComponent extends Sea
           scores[i] = scoreDoc.score;
         }
         DocSlice slice = new DocSlice(0, docs.length, docs, scores, topDocs.totalHits, topDocs.getMaxScore());
-        final BytesRef bytesRef = values.lookupOrd(ord);
-        fieldType.indexedToReadable(bytesRef, charsRef);
-        String group = charsRef.toString();
-        outMap.put(group, slice);
+
+        if(fieldType instanceof StrField) {
+          final BytesRef bytesRef = ordBytes.get((int)groupValue);
+          fieldType.indexedToReadable(bytesRef, charsRef);
+          String group = charsRef.toString();
+          outMap.put(group, slice);
+        } else {
+          if(fieldType instanceof TrieIntField || fieldType instanceof TrieLongField ) {
+            outMap.put(Long.toString(groupValue), slice);
+          } else if(fieldType instanceof TrieFloatField) {
+            outMap.put(Float.toString(Float.intBitsToFloat((int)groupValue)), slice);
+          } else if(fieldType instanceof TrieDoubleField) {
+            outMap.put(Double.toString(Double.longBitsToDouble(groupValue)), slice);
+          }
+        }
       }
     }
 
@@ -306,32 +480,46 @@ public class ExpandComponent extends Sea
     rb.rsp.add("expanded", expanded);
   }
 
-  private class GroupExpandCollector implements Collector {
+  private class GroupExpandCollector implements Collector, GroupCollector {
     private SortedDocValues docValues;
-    private IntObjectMap<Collector> groups;
-    private int docBase;
+    private MultiDocValues.OrdinalMap ordinalMap;
+    private SortedDocValues segmentValues;
+    private LongValues segmentOrdinalMap;
+    private MultiDocValues.MultiSortedDocValues multiSortedDocValues;
+
+    private LongObjectMap<Collector> groups;
     private FixedBitSet groupBits;
     private IntOpenHashSet collapsedSet;
 
     public GroupExpandCollector(SortedDocValues docValues, FixedBitSet groupBits, IntOpenHashSet collapsedSet, int limit, Sort sort) throws IOException {
       int numGroups = collapsedSet.size();
-      groups = new IntObjectOpenHashMap<>(numGroups * 2);
+      groups = new LongObjectOpenHashMap<>(numGroups * 2);
       DocIdSetIterator iterator = new BitSetIterator(groupBits, 0); // cost is not useful here
       int group;
       while ((group = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-        Collector collector = (sort == null) ? TopScoreDocCollector.create(limit, true) : TopFieldCollector.create(sort, limit, false, false, false, true);
+        Collector collector = (sort == null) ? TopScoreDocCollector.create(limit) : TopFieldCollector.create(sort, limit, false, false, false);
         groups.put(group, collector);
       }
 
       this.collapsedSet = collapsedSet;
       this.groupBits = groupBits;
       this.docValues = docValues;
+      if(docValues instanceof MultiDocValues.MultiSortedDocValues) {
+        this.multiSortedDocValues = (MultiDocValues.MultiSortedDocValues)docValues;
+        this.ordinalMap = multiSortedDocValues.mapping;
+      }
     }
 
     public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
       final int docBase = context.docBase;
-      final IntObjectMap<LeafCollector> leafCollectors = new IntObjectOpenHashMap<>();
-      for (IntObjectCursor<Collector> entry : groups) {
+
+      if(ordinalMap != null) {
+        this.segmentValues = this.multiSortedDocValues.values[context.ord];
+        this.segmentOrdinalMap = ordinalMap.getGlobalOrds(context.ord);
+      }
+
+      final LongObjectMap<LeafCollector> leafCollectors = new LongObjectOpenHashMap<>();
+      for (LongObjectCursor<Collector> entry : groups) {
         leafCollectors.put(entry.key, entry.value.getLeafCollector(context));
       }
       return new LeafCollector() {
@@ -345,27 +533,139 @@ public class ExpandComponent extends Sea
 
         @Override
         public void collect(int docId) throws IOException {
-          int doc = docId + docBase;
-          int ord = docValues.getOrd(doc);
-          if (ord > -1 && groupBits.get(ord) && !collapsedSet.contains(doc)) {
+          int globalDoc = docId + docBase;
+          int ord = -1;
+          if(ordinalMap != null) {
+            ord = segmentValues.getOrd(docId);
+            if(ord > -1) {
+              ord = (int)segmentOrdinalMap.get(ord);
+            }
+          } else {
+            ord = docValues.getOrd(globalDoc);
+          }
+
+          if (ord > -1 && groupBits.get(ord) && !collapsedSet.contains(globalDoc)) {
             LeafCollector c = leafCollectors.get(ord);
             c.collect(docId);
           }
         }
+      };
+    }
+
+    public LongObjectMap<Collector> getGroups() {
+      return groups;
+    }
+  }
+
+  private class NumericGroupExpandCollector implements Collector, GroupCollector {
+    private NumericDocValues docValues;
+
+
+    private String field;
+    private LongObjectOpenHashMap<Collector> groups;
+
+    private IntOpenHashSet collapsedSet;
+    private long nullValue;
+
+    public NumericGroupExpandCollector(String field, long nullValue, LongOpenHashSet groupSet, IntOpenHashSet collapsedSet, int limit, Sort sort) throws IOException {
+      int numGroups = collapsedSet.size();
+      this.nullValue = nullValue;
+      groups = new LongObjectOpenHashMap(numGroups * 2);
+      Iterator<LongCursor> iterator = groupSet.iterator();
+      while (iterator.hasNext()) {
+        LongCursor cursor = iterator.next();
+        Collector collector = (sort == null) ? TopScoreDocCollector.create(limit) : TopFieldCollector.create(sort, limit, false, false, false);
+        groups.put(cursor.value, collector);
+      }
+
+      this.field = field;
+      this.collapsedSet = collapsedSet;
+    }
+
+    public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+      final int docBase = context.docBase;
+      this.docValues = context.reader().getNumericDocValues(this.field);
+
+      final LongObjectOpenHashMap<LeafCollector> leafCollectors = new LongObjectOpenHashMap<>();
+
+      for (LongObjectCursor<Collector> entry : groups) {
+        leafCollectors.put(entry.key, entry.value.getLeafCollector(context));
+      }
+
+      return new LeafCollector() {
 
         @Override
-        public boolean acceptsDocsOutOfOrder() {
-          return false;
+        public void setScorer(Scorer scorer) throws IOException {
+          for (ObjectCursor<LeafCollector> c : leafCollectors.values()) {
+            c.value.setScorer(scorer);
+          }
+        }
+
+        @Override
+        public void collect(int docId) throws IOException {
+          long value = docValues.get(docId);
+          if (value != nullValue && leafCollectors.containsKey(value) && !collapsedSet.contains(docId + docBase)) {
+            LeafCollector c = leafCollectors.lget();
+            c.collect(docId);
+          }
         }
       };
     }
 
-    public IntObjectMap<Collector> getGroups() {
+    public LongObjectOpenHashMap<Collector> getGroups() {
       return groups;
     }
 
   }
 
+  private interface GroupCollector {
+    public LongObjectMap getGroups();
+
+  }
+
+  private Query getGroupQuery(String fname,
+                           FieldType ft,
+                           int size,
+                           LongOpenHashSet groupSet) {
+
+    BytesRef[] bytesRefs = new BytesRef[size];
+    BytesRefBuilder term = new BytesRefBuilder();
+    Iterator<LongCursor> it = groupSet.iterator();
+    int index = -1;
+    String stringVal =  null;
+    while (it.hasNext()) {
+      LongCursor cursor = it.next();
+      if(ft instanceof TrieIntField || ft instanceof TrieLongField) {
+        stringVal = Long.toString(cursor.value);
+      } else {
+        if(ft instanceof TrieFloatField) {
+          stringVal = Float.toString(Float.intBitsToFloat((int)cursor.value));
+        } else {
+          stringVal = Double.toString(Double.longBitsToDouble(cursor.value));
+        }
+      }
+      ft.readableToIndexed(stringVal, term);
+      bytesRefs[++index] = term.toBytesRef();
+    }
+
+    return new SolrConstantScoreQuery(new TermsFilter(fname, bytesRefs));
+  }
+
+  private Query getGroupQuery(String fname,
+                              int size,
+                              IntObjectOpenHashMap<BytesRef> ordBytes) throws Exception {
+
+    BytesRef[] bytesRefs = new BytesRef[size];
+    int index = -1;
+    Iterator<IntObjectCursor<BytesRef>>it = ordBytes.iterator();
+    while (it.hasNext()) {
+      IntObjectCursor<BytesRef> cursor = it.next();
+      bytesRefs[++index] = cursor.value;
+    }
+    return new SolrConstantScoreQuery(new TermsFilter(fname, bytesRefs));
+  }
+
+
   ////////////////////////////////////////////
   ///  SolrInfoMBean
   ////////////////////////////////////////////
@@ -385,4 +685,49 @@ public class ExpandComponent extends Sea
       throw new RuntimeException(e);
     }
   }
+
+  private class ReaderWrapper extends FilterLeafReader {
+
+    private String field;
+
+    public ReaderWrapper(LeafReader leafReader, String field) {
+      super(leafReader);
+      this.field = field;
+    }
+
+    public SortedDocValues getSortedDocValues(String field) {
+      return null;
+    }
+
+    public Object getCoreCacheKey() {
+      return in.getCoreCacheKey();
+    }
+
+    public FieldInfos getFieldInfos() {
+      Iterator<FieldInfo> it = in.getFieldInfos().iterator();
+      List<FieldInfo> newInfos = new ArrayList();
+      while(it.hasNext()) {
+        FieldInfo fieldInfo = it.next();
+
+        if(fieldInfo.name.equals(field)) {
+          FieldInfo f = new FieldInfo(fieldInfo.name,
+              fieldInfo.number,
+              fieldInfo.hasVectors(),
+              fieldInfo.hasNorms(),
+              fieldInfo.hasPayloads(),
+              fieldInfo.getIndexOptions(),
+              DocValuesType.NONE,
+              fieldInfo.getDocValuesGen(),
+              fieldInfo.attributes());
+          newInfos.add(f);
+
+        } else {
+          newInfos.add(fieldInfo);
+        }
+      }
+      FieldInfos infos = new FieldInfos(newInfos.toArray(new FieldInfo[newInfos.size()]));
+      return infos;
+    }
+  }
+
 }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/HighlightComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/HighlightComponent.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/HighlightComponent.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/HighlightComponent.java Sun Feb  8 23:53:14 2015
@@ -92,7 +92,6 @@ public class HighlightComponent extends
       PluginInfo pluginInfo = core.getSolrConfig().getPluginInfo(SolrHighlighter.class.getName()); //TODO deprecated configuration remove later
       if (pluginInfo != null) {
         highlighter = core.createInitInstance(pluginInfo, SolrHighlighter.class, null, DefaultSolrHighlighter.class.getName());
-        highlighter.initalize(core.getSolrConfig());
       } else {
         DefaultSolrHighlighter defHighlighter = new DefaultSolrHighlighter(core);
         defHighlighter.init(PluginInfo.EMPTY_INFO);

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java Sun Feb  8 23:53:14 2015
@@ -152,11 +152,8 @@ public class HttpShardHandler extends Sh
           if (urls.size() <= 1) {
             String url = urls.get(0);
             srsp.setShardAddress(url);
-            SolrClient client = new HttpSolrClient(url, httpClient);
-            try {
+            try (SolrClient client = new HttpSolrClient(url, httpClient)) {
               ssr.nl = client.request(req);
-            } finally {
-              client.shutdown();
             }
           } else {
             LBHttpSolrClient.Rsp rsp = httpShardHandlerFactory.makeLoadBalancedRequest(req, urls);
@@ -282,7 +279,6 @@ public class HttpShardHandler extends Sh
 
         clusterState =  zkController.getClusterState();
         String shardKeys =  params.get(ShardParams._ROUTE_);
-        if(shardKeys == null) shardKeys = params.get(ShardParams.SHARD_KEYS);//eprecated
 
         // This will be the complete list of slices we need to query for this request.
         slices = new HashMap<>();

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java Sun Feb  8 23:53:14 2015
@@ -18,6 +18,8 @@ package org.apache.solr.handler.componen
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.http.client.HttpClient;
+import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.http.impl.client.DefaultHttpRequestRetryHandler;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.impl.LBHttpSolrClient;
@@ -27,6 +29,7 @@ import org.apache.solr.common.util.Execu
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.URLUtil;
+import org.apache.solr.core.ConfigSolr;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.slf4j.Logger;
@@ -66,8 +69,8 @@ public class HttpShardHandlerFactory ext
   protected HttpClient defaultClient;
   private LBHttpSolrClient loadbalancer;
   //default values:
-  int soTimeout = 0; 
-  int connectionTimeout = 0; 
+  int soTimeout = ConfigSolr.DEFAULT_DISTRIBUPDATESOTIMEOUT;
+  int connectionTimeout = ConfigSolr.DEFAULT_DISTRIBUPDATECONNTIMEOUT;
   int maxConnectionsPerHost = 20;
   int maxConnections = 10000;
   int corePoolSize = 0;
@@ -75,6 +78,7 @@ public class HttpShardHandlerFactory ext
   int keepAliveTime = 5;
   int queueSize = -1;
   boolean accessPolicy = false;
+  boolean useRetries = false;
 
   private String scheme = null;
 
@@ -97,6 +101,10 @@ public class HttpShardHandlerFactory ext
 
   // Configure if the threadpool favours fairness over throughput
   static final String INIT_FAIRNESS_POLICY = "fairnessPolicy";
+  
+  // Turn on retries for certain IOExceptions, many of which can happen
+  // due to connection pooling limitations / races
+  static final String USE_RETRIES = "useRetries";
 
   /**
    * Get {@link ShardHandler} that uses the default http client.
@@ -129,6 +137,7 @@ public class HttpShardHandlerFactory ext
     this.keepAliveTime = getParameter(args, MAX_THREAD_IDLE_TIME, keepAliveTime);
     this.queueSize = getParameter(args, INIT_SIZE_OF_QUEUE, queueSize);
     this.accessPolicy = getParameter(args, INIT_FAIRNESS_POLICY, accessPolicy);
+    this.useRetries = getParameter(args, USE_RETRIES, useRetries);
     
     // magic sysprop to make tests reproducible: set by SolrTestCaseJ4.
     String v = System.getProperty("tests.shardhandler.randomSeed");
@@ -153,8 +162,18 @@ public class HttpShardHandlerFactory ext
     clientParams.set(HttpClientUtil.PROP_MAX_CONNECTIONS, maxConnections);
     clientParams.set(HttpClientUtil.PROP_SO_TIMEOUT, soTimeout);
     clientParams.set(HttpClientUtil.PROP_CONNECTION_TIMEOUT, connectionTimeout);
-    clientParams.set(HttpClientUtil.PROP_USE_RETRY, false);
+    if (!useRetries) {
+      clientParams.set(HttpClientUtil.PROP_USE_RETRY, false);
+    }
     this.defaultClient = HttpClientUtil.createClient(clientParams);
+    
+    // must come after createClient
+    if (useRetries) {
+      // our default retry handler will never retry on IOException if the request has been sent already,
+      // but for these read only requests we can use the standard DefaultHttpRequestRetryHandler rules
+      ((DefaultHttpClient) this.defaultClient).setHttpRequestRetryHandler(new DefaultHttpRequestRetryHandler());
+    }
+    
     this.loadbalancer = createLoadbalancer(defaultClient);
   }
 
@@ -189,7 +208,7 @@ public class HttpShardHandlerFactory ext
       } finally {
         
         if (loadbalancer != null) {
-          loadbalancer.shutdown();
+          loadbalancer.close();
         }
       }
     }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java Sun Feb  8 23:53:14 2015
@@ -23,6 +23,7 @@ import org.apache.lucene.index.ReaderUti
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.LeafFieldComparator;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Scorer;
@@ -598,7 +599,8 @@ public class QueryComponent extends Sear
         // :TODO: would be simpler to always serialize every position of SortField[]
         if (type==SortField.Type.SCORE || type==SortField.Type.DOC) continue;
 
-        FieldComparator comparator = null;
+        FieldComparator<?> comparator = null;
+        LeafFieldComparator leafComparator = null;
         Object[] vals = new Object[nDocs];
 
         int lastIdx = -1;
@@ -621,12 +623,12 @@ public class QueryComponent extends Sear
 
           if (comparator == null) {
             comparator = sortField.getComparator(1,0);
-            comparator = comparator.setNextReader(currentLeaf);
+            leafComparator = comparator.getLeafComparator(currentLeaf);
           }
 
           doc -= currentLeaf.docBase;  // adjust for what segment this is in
-          comparator.setScorer(new FakeScorer(doc, score));
-          comparator.copy(0, doc);
+          leafComparator.setScorer(new FakeScorer(doc, score));
+          leafComparator.copy(0, doc);
           Object val = comparator.value(0);
           if (null != ft) val = ft.marshalSortValue(val);
           vals[position] = val;

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java Sun Feb  8 23:53:14 2015
@@ -36,6 +36,7 @@ import org.apache.lucene.search.DocIdSet
 import org.apache.lucene.search.FieldComparator;
 import org.apache.lucene.search.FieldComparatorSource;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SimpleFieldComparator;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TermQuery;
@@ -631,7 +632,7 @@ public class QueryElevationComponent ext
 
   @Override
   public FieldComparator<Integer> newComparator(String fieldname, final int numHits, int sortPos, boolean reversed) throws IOException {
-    return new FieldComparator<Integer>() {
+    return new SimpleFieldComparator<Integer>() {
       private final int[] values = new int[numHits];
       private int bottomVal;
       private int topVal;
@@ -677,13 +678,13 @@ public class QueryElevationComponent ext
       }
 
       @Override
-      public FieldComparator setNextReader(LeafReaderContext context) throws IOException {
+      protected void doSetNextReader(LeafReaderContext context) throws IOException {
         //convert the ids to Lucene doc ids, the ordSet and termValues needs to be the same size as the number of elevation docs we have
         ordSet.clear();
         Fields fields = context.reader().fields();
-        if (fields == null) return this;
+        if (fields == null) return;
         Terms terms = fields.terms(idField);
-        if (terms == null) return this;
+        if (terms == null) return;
         termsEnum = terms.iterator(termsEnum);
         BytesRefBuilder term = new BytesRefBuilder();
         Bits liveDocs = context.reader().getLiveDocs();
@@ -701,7 +702,6 @@ public class QueryElevationComponent ext
             }
           }
         }
-        return this;
       }
 
       @Override

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java Sun Feb  8 23:53:14 2015
@@ -177,7 +177,7 @@ public class RealTimeGetComponent extend
 
        int docid = searcher.getFirstMatch(new Term(idField.getName(), idBytes.get()));
        if (docid < 0) continue;
-       StoredDocument luceneDocument = searcher.doc(docid);
+       StoredDocument luceneDocument = searcher.doc(docid, rsp.getReturnFields().getLuceneFieldNames());
        SolrDocument doc = toSolrDoc(luceneDocument,  core.getLatestSchema());
        if( transformer != null ) {
          transformer.transform(doc, docid);
@@ -598,9 +598,7 @@ public class RealTimeGetComponent extend
           // TODO: do any kind of validation here?
           updates.add(o);
 
-        } catch (SolrException e) {
-          log.warn("Exception reading log for updates", e);
-        } catch (ClassCastException e) {
+        } catch (SolrException | ClassCastException e) {
           log.warn("Exception reading log for updates", e);
         }
       }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/component/SuggestComponent.java Sun Feb  8 23:53:14 2015
@@ -17,6 +17,7 @@ package org.apache.solr.handler.componen
  * limitations under the License.
  */
 
+import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -29,6 +30,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.search.suggest.Lookup;
 import org.apache.lucene.search.suggest.Lookup.LookupResult;
@@ -77,6 +79,9 @@ public class SuggestComponent extends Se
   /** SolrConfig label to identify boolean value to build suggesters on optimize */
   private static final String BUILD_ON_OPTIMIZE_LABEL = "buildOnOptimize";
   
+  /** SolrConfig label to identify boolean value to build suggesters on optimize */
+  private static final String BUILD_ON_STARTUP_LABEL = "buildOnStartup";
+  
   @SuppressWarnings("unchecked")
   protected NamedList initParams;
   
@@ -128,14 +133,22 @@ public class SuggestComponent extends Se
               throw new RuntimeException("More than one dictionary is missing name.");
             }
           }
-          
-          // Register event listeners for this Suggester
-          core.registerFirstSearcherListener(new SuggesterListener(core, suggester, false, false));
+          boolean buildOnStartup;
+          Object buildOnStartupObj = suggesterParams.get(BUILD_ON_STARTUP_LABEL);
+          if (buildOnStartupObj == null) {
+            File storeFile = suggester.getStoreFile();
+            buildOnStartup = storeFile == null || !storeFile.exists();
+          } else {
+            buildOnStartup = Boolean.parseBoolean((String) buildOnStartupObj);
+          }
           boolean buildOnCommit = Boolean.parseBoolean((String) suggesterParams.get(BUILD_ON_COMMIT_LABEL));
           boolean buildOnOptimize = Boolean.parseBoolean((String) suggesterParams.get(BUILD_ON_OPTIMIZE_LABEL));
-          if (buildOnCommit || buildOnOptimize) {
-            LOG.info("Registering newSearcher listener for suggester: " + suggester.getName());
-            core.registerNewSearcherListener(new SuggesterListener(core, suggester, buildOnCommit, buildOnOptimize));
+          
+          if (buildOnCommit || buildOnOptimize || buildOnStartup) {
+            SuggesterListener listener = new SuggesterListener(core, suggester, buildOnCommit, buildOnOptimize, buildOnStartup, core.isReloaded());
+            LOG.info("Registering searcher listener for suggester: " + suggester.getName() + " - " + listener);
+            core.registerFirstSearcherListener(listener);
+            core.registerNewSearcherListener(listener);
           }
         }
       }
@@ -448,12 +461,23 @@ public class SuggestComponent extends Se
     private final SolrSuggester suggester;
     private final boolean buildOnCommit;
     private final boolean buildOnOptimize;
+    private final boolean buildOnStartup;
+    
+    // On core reload, immediately after the core is created a new searcher is opened, causing the suggester
+    // to trigger a "buildOnCommit". The only event that we want to trigger in that situation is "buildOnStartup"
+    // so if buildOnCommit is true and this is a core being reloaded, we will skip the first time this listener 
+    // is called. 
+    private final AtomicLong callCount = new AtomicLong(0);
+    private final boolean isCoreReload;
+    
 
-    public SuggesterListener(SolrCore core, SolrSuggester checker, boolean buildOnCommit, boolean buildOnOptimize) {
+    public SuggesterListener(SolrCore core, SolrSuggester checker, boolean buildOnCommit, boolean buildOnOptimize, boolean buildOnStartup, boolean isCoreReload) {
       this.core = core;
       this.suggester = checker;
       this.buildOnCommit = buildOnCommit;
       this.buildOnOptimize = buildOnOptimize;
+      this.buildOnStartup = buildOnStartup;
+      this.isCoreReload = isCoreReload;
     }
 
     @Override
@@ -462,24 +486,23 @@ public class SuggestComponent extends Se
     @Override
     public void newSearcher(SolrIndexSearcher newSearcher,
                             SolrIndexSearcher currentSearcher) {
-      if (currentSearcher == null) {
-        // firstSearcher event
-        try {
-          LOG.info("Loading suggester index for: " + suggester.getName());
-          suggester.reload(core, newSearcher);
-        } catch (IOException e) {
-          log.error("Exception in reloading suggester index for: " + suggester.getName(), e);
+      long thisCallCount = callCount.incrementAndGet();
+      if (isCoreReload && thisCallCount == 1) {
+        LOG.info("Skipping first newSearcher call for suggester " + suggester + " in core reload");
+        return;
+      } else if (thisCallCount == 1 || (isCoreReload && thisCallCount == 2)) {
+        if (buildOnStartup) {
+          LOG.info("buildOnStartup: " + suggester.getName());
+          buildSuggesterIndex(newSearcher);
         }
       } else {
-        // newSearcher event
         if (buildOnCommit)  {
+          LOG.info("buildOnCommit: " + suggester.getName());
           buildSuggesterIndex(newSearcher);
         } else if (buildOnOptimize) {
           if (newSearcher.getIndexReader().leaves().size() == 1)  {
+            LOG.info("buildOnOptimize: " + suggester.getName());
             buildSuggesterIndex(newSearcher);
-          } else  {
-            LOG.info("Index is not optimized therefore skipping building suggester index for: " 
-                    + suggester.getName());
           }
         }
       }
@@ -488,7 +511,6 @@ public class SuggestComponent extends Se
 
     private void buildSuggesterIndex(SolrIndexSearcher newSearcher) {
       try {
-        LOG.info("Building suggester index for: " + suggester.getName());
         suggester.build(core, newSearcher);
       } catch (Exception e) {
         log.error("Exception in building suggester index for: " + suggester.getName(), e);
@@ -500,6 +522,14 @@ public class SuggestComponent extends Se
 
     @Override
     public void postSoftCommit() {}
+
+    @Override
+    public String toString() {
+      return "SuggesterListener [core=" + core + ", suggester=" + suggester
+          + ", buildOnCommit=" + buildOnCommit + ", buildOnOptimize="
+          + buildOnOptimize + ", buildOnStartup=" + buildOnStartup
+          + ", isCoreReload=" + isCoreReload + "]";
+    }
     
   }
 }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java Sun Feb  8 23:53:14 2015
@@ -16,18 +16,6 @@
  */
 package org.apache.solr.highlight;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.ListIterator;
-import java.util.Map;
-import java.util.Set;
-
 import org.apache.lucene.analysis.CachingTokenFilter;
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
@@ -59,7 +47,6 @@ import org.apache.solr.common.params.Sol
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.core.PluginInfo;
-import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.IndexSchema;
@@ -71,6 +58,18 @@ import org.apache.solr.util.plugin.Plugi
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+
 /**
  * 
  * @since solr 1.3
@@ -160,38 +159,7 @@ public class DefaultSolrHighlighter exte
     if(boundaryScanner == null) boundaryScanner = new SimpleBoundaryScanner();
     boundaryScanners.put("", boundaryScanner);
     boundaryScanners.put(null, boundaryScanner);
-    
-    initialized = true;
-  }
-  //just for back-compat with the deprecated method
-  private boolean initialized = false;
-  @Override
-  @Deprecated
-  public void initalize( SolrConfig config) {
-    if (initialized) return;
-    SolrFragmenter frag = new GapFragmenter();
-    fragmenters.put("", frag);
-    fragmenters.put(null, frag);
-
-    SolrFormatter fmt = new HtmlFormatter();
-    formatters.put("", fmt);
-    formatters.put(null, fmt);    
-
-    SolrEncoder enc = new DefaultEncoder();
-    encoders.put("", enc);
-    encoders.put(null, enc);    
 
-    SolrFragListBuilder fragListBuilder = new SimpleFragListBuilder();
-    fragListBuilders.put( "", fragListBuilder );
-    fragListBuilders.put( null, fragListBuilder );
-    
-    SolrFragmentsBuilder fragsBuilder = new ScoreOrderFragmentsBuilder();
-    fragmentsBuilders.put( "", fragsBuilder );
-    fragmentsBuilders.put( null, fragsBuilder );
-    
-    SolrBoundaryScanner boundaryScanner = new SimpleBoundaryScanner();
-    boundaryScanners.put("", boundaryScanner);
-    boundaryScanners.put(null, boundaryScanner);
   }
 
   /**
@@ -240,16 +208,11 @@ public class DefaultSolrHighlighter exte
    * @param request The SolrQueryRequest
    */
   private QueryScorer getSpanQueryScorer(Query query, String fieldName, TokenStream tokenStream, SolrQueryRequest request) {
-    boolean reqFieldMatch = request.getParams().getFieldBool(fieldName, HighlightParams.FIELD_MATCH, false);
-    boolean highlightMultiTerm = request.getParams().getBool(HighlightParams.HIGHLIGHT_MULTI_TERM, true);
-    QueryScorer scorer;
-    if (reqFieldMatch) {
-      scorer = new QueryScorer(query, fieldName);
-    }
-    else {
-      scorer = new QueryScorer(query, null);
-    }
-    scorer.setExpandMultiTermQuery(highlightMultiTerm);
+    QueryScorer scorer = new QueryScorer(query,
+        request.getParams().getFieldBool(fieldName, HighlightParams.FIELD_MATCH, false) ? fieldName : null);
+    scorer.setExpandMultiTermQuery(request.getParams().getBool(HighlightParams.HIGHLIGHT_MULTI_TERM, true));
+    scorer.setUsePayloads(request.getParams().getFieldBool(fieldName, HighlightParams.PAYLOADS,
+        request.getSearcher().getLeafReader().getFieldInfos().fieldInfo(fieldName).hasPayloads()));
     return scorer;
   }
 
@@ -608,7 +571,7 @@ public class DefaultSolrHighlighter exte
       if (summaries.length > 0) 
       docSummaries.add(fieldName, summaries);
     }
-    // no summeries made, copy text from alternate field
+    // no summaries made, copy text from alternate field
     if (summaries == null || summaries.length == 0) {
       alternateField( docSummaries, params, doc, fieldName );
     }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/highlight/PostingsSolrHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/highlight/PostingsSolrHighlighter.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/highlight/PostingsSolrHighlighter.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/highlight/PostingsSolrHighlighter.java Sun Feb  8 23:53:14 2015
@@ -107,9 +107,6 @@ import java.util.Set;
  * @lucene.experimental 
  */
 public class PostingsSolrHighlighter extends SolrHighlighter implements PluginInfoInitialized {
-
-  @Override
-  public void initalize(SolrConfig config) {}
   
   @Override
   public void init(PluginInfo info) {}

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/highlight/SolrHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/highlight/SolrHighlighter.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/highlight/SolrHighlighter.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/highlight/SolrHighlighter.java Sun Feb  8 23:53:14 2015
@@ -16,21 +16,20 @@ package org.apache.solr.highlight;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.lucene.search.Query;
 import org.apache.solr.common.params.HighlightParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.core.SolrConfig;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.search.DocList;
 import org.apache.solr.util.SolrPluginUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
 
 public abstract class SolrHighlighter
 {
@@ -38,9 +37,6 @@ public abstract class SolrHighlighter
   public static int DEFAULT_PHRASE_LIMIT = 5000;
   public static Logger log = LoggerFactory.getLogger(SolrHighlighter.class);
 
-  @Deprecated
-  public abstract void initalize( SolrConfig config );
-
   /**
    * Check whether Highlighting is enabled for this request.
    * @param params The params controlling Highlighting

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/parser/CharStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/parser/CharStream.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/parser/CharStream.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/parser/CharStream.java Sun Feb  8 23:53:14 2015
@@ -27,22 +27,6 @@ interface CharStream {
    */
   char readChar() throws java.io.IOException;
 
-  @Deprecated
-  /**
-   * Returns the column position of the character last read.
-   * @deprecated
-   * @see #getEndColumn
-   */
-  int getColumn();
-
-  @Deprecated
-  /**
-   * Returns the line number of the character last read.
-   * @deprecated
-   * @see #getEndLine
-   */
-  int getLine();
-
   /**
    * Returns the column number of the last character for current token (being
    * matched after the last call to BeginTOken).

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/parser/FastCharStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/parser/FastCharStream.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/parser/FastCharStream.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/parser/FastCharStream.java Sun Feb  8 23:53:14 2015
@@ -108,15 +108,6 @@ public final class FastCharStream implem
     }
   }
 
-  @Override
-  public final int getColumn() {
-    return bufferStart + bufferPosition;
-  }
-  @Override
-  public final int getLine() {
-    return 1;
-  }
-  @Override
   public final int getEndColumn() {
     return bufferStart + bufferPosition;
   }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java Sun Feb  8 23:53:14 2015
@@ -150,13 +150,9 @@ public abstract class SolrQueryParserBas
       Query res = TopLevelQuery(null);  // pass null so we can tell later if an explicit field was provided or not
       return res!=null ? res : newBooleanQuery(false);
     }
-    catch (ParseException tme) {
+    catch (ParseException | TokenMgrError tme) {
       throw new SyntaxError("Cannot parse '" +query+ "': " + tme.getMessage(), tme);
-    }
-    catch (TokenMgrError tme) {
-      throw new SyntaxError("Cannot parse '" +query+ "': " + tme.getMessage(), tme);
-    }
-    catch (BooleanQuery.TooManyClauses tmc) {
+    } catch (BooleanQuery.TooManyClauses tmc) {
       throw new SyntaxError("Cannot parse '" +query+ "': too many boolean clauses", tmc);
     }
   }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/request/SimpleFacets.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/request/SimpleFacets.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/request/SimpleFacets.java Sun Feb  8 23:53:14 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.DocsEnum;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.LeafReader;
@@ -95,6 +75,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.
  *

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java Sun Feb  8 23:53:14 2015
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
-import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
@@ -36,7 +35,6 @@ import com.google.common.cache.CacheBuil
 import com.spatial4j.core.context.SpatialContext;
 import com.spatial4j.core.context.SpatialContextFactory;
 import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.io.LegacyShapeReadWriterFormat;
 import com.spatial4j.core.shape.Point;
 import com.spatial4j.core.shape.Rectangle;
 import com.spatial4j.core.shape.Shape;
@@ -59,6 +57,7 @@ import org.apache.solr.common.params.Sol
 import org.apache.solr.response.TextResponseWriter;
 import org.apache.solr.search.QParser;
 import org.apache.solr.search.SpatialOptions;
+import org.apache.solr.util.DistanceUnits;
 import org.apache.solr.util.MapListener;
 import org.apache.solr.util.SpatialUtils;
 import org.slf4j.Logger;
@@ -71,7 +70,7 @@ import org.slf4j.LoggerFactory;
  */
 public abstract class AbstractSpatialFieldType<T extends SpatialStrategy> extends FieldType implements SpatialQueryable {
 
-  /** A local-param with one of "none" (default), "distance", or "recipDistance". */
+  /** A local-param with one of "none" (default), "distance", "recipDistance" or supported values in ({@link DistanceUnits#getSupportedUnits()}. */
   public static final String SCORE_PARAM = "score";
   /** A local-param boolean that can be set to false to only return the
    * FunctionQuery (score), and thus not do filtering.
@@ -90,6 +89,10 @@ public abstract class AbstractSpatialFie
 
   private final Cache<String, T> fieldStrategyCache = CacheBuilder.newBuilder().build();
 
+  protected DistanceUnits distanceUnits;
+  @Deprecated
+  protected String units; // for back compat; hopefully null
+
   protected final Set<String> supportedScoreModes;
 
   protected AbstractSpatialFieldType() {
@@ -101,6 +104,7 @@ public abstract class AbstractSpatialFie
     set.add(NONE);
     set.add(DISTANCE);
     set.add(RECIP_DISTANCE);
+    set.addAll(DistanceUnits.getSupportedUnits());
     set.addAll(moreScoreModes);
     supportedScoreModes = Collections.unmodifiableSet(set);
   }
@@ -109,30 +113,58 @@ public abstract class AbstractSpatialFie
   protected void init(IndexSchema schema, Map<String, String> args) {
     super.init(schema, args);
 
-    String units = args.remove("units");
-    if (!"degrees".equals(units))
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-          "Must specify units=\"degrees\" on field types with class "+getClass().getSimpleName());
-
-    //replace legacy rect format with ENVELOPE
-    String wbStr = args.get("worldBounds");
-    if (wbStr != null && !wbStr.toUpperCase(Locale.ROOT).startsWith("ENVELOPE")) {
-      log.warn("Using old worldBounds format? Should use ENVELOPE(xMin, xMax, yMax, yMin).");
-      String[] parts = wbStr.split(" ");//"xMin yMin xMax yMax"
-      if (parts.length == 4) {
-        args.put("worldBounds",
-            "ENVELOPE(" + parts[0] + ", " + parts[2] + ", " + parts[3] + ", " + parts[1] + ")");
-      } //else likely eventual exception
-    }
-
     //Solr expects us to remove the parameters we've used.
     MapListener<String, String> argsWrap = new MapListener<>(args);
     ctx = SpatialContextFactory.makeSpatialContext(argsWrap, schema.getResourceLoader().getClassLoader());
     args.keySet().removeAll(argsWrap.getSeenKeys());
 
+    final String unitsErrMsg = "units parameter is deprecated, please use distanceUnits instead for field types with class " +
+        getClass().getSimpleName();
+    this.units = args.remove("units");//deprecated
+    if (units != null) {
+      if ("degrees".equals(units)) {
+        log.warn(unitsErrMsg);
+      } else {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, unitsErrMsg);
+      }
+    }
+
+    final String distanceUnitsStr = args.remove("distanceUnits");
+    if (distanceUnitsStr == null) {
+      if (units != null) {
+        this.distanceUnits = DistanceUnits.BACKCOMPAT;
+      } else {
+        this.distanceUnits = ctx.isGeo() ? DistanceUnits.KILOMETERS : DistanceUnits.DEGREES;
+      }
+    } else {
+      // If both units and distanceUnits was specified
+      if (units != null) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, unitsErrMsg);
+      }
+      this.distanceUnits = parseDistanceUnits(distanceUnitsStr);
+      if (this.distanceUnits == null)
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+            "Must specify distanceUnits as one of "+ DistanceUnits.getSupportedUnits() +
+                " on field types with class "+getClass().getSimpleName());
+    }
+
     argsParser = newSpatialArgsParser();
   }
 
+  /** if {@code str} is non-null, returns {@link org.apache.solr.util.DistanceUnits#valueOf(String)}
+   * (which will return null if not found),
+   * else returns {@link #distanceUnits} (only null before initialized in {@code init()}.
+   * @param str maybe null
+   * @return maybe null
+   */
+  public DistanceUnits parseDistanceUnits(String str) {
+    if (str == null) {
+      return this.distanceUnits;
+    } else {
+      return DistanceUnits.valueOf(str);
+    }
+  }
+
   protected SpatialArgsParser newSpatialArgsParser() {
     return new SpatialArgsParser() {
       @Override
@@ -192,18 +224,17 @@ public abstract class AbstractSpatialFie
   protected Shape parseShape(String str) {
     if (str.length() == 0)
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "empty string shape");
-    //In Solr trunk we only support "lat, lon" (or x y) as an additional format; in v4.0 we do the
-    // weird Circle & Rect formats too (Spatial4j LegacyShapeReadWriterFormat).
-    try {
-      Shape shape = LegacyShapeReadWriterFormat.readShapeOrNull(str, ctx);
-      if (shape != null)
-        return shape;
-      return ctx.readShapeFromWkt(str);
-    } catch (Exception e) {
-      String message = e.getMessage();
-      if (!message.contains(str))
-        message = "Couldn't parse shape '" + str + "' because: " + message;
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, message, e);
+    if (Character.isLetter(str.charAt(0))) {//WKT starts with a letter
+      try {
+        return ctx.readShapeFromWkt(str);
+      } catch (Exception e) {
+        String message = e.getMessage();
+        if (!message.contains(str))
+          message = "Couldn't parse shape '" + str + "' because: " + message;
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, message, e);
+      }
+    } else {
+      return SpatialUtils.parsePointSolrException(str, ctx);
     }
   }
 
@@ -281,7 +312,12 @@ public abstract class AbstractSpatialFie
 
   protected SpatialArgs parseSpatialArgs(QParser parser, String externalVal) {
     try {
-      return argsParser.parse(externalVal, ctx);
+      SpatialArgs args = argsParser.parse(externalVal, ctx);
+      // Convert parsed args.distErr to degrees (using distanceUnits)
+      if (args.getDistErr() != null) {
+        args.setDistErr(args.getDistErr() * distanceUnits.multiplierFromThisUnitToDegrees());
+      }
+      return args;
     } catch (SolrException e) {
       throw e;
     } catch (Exception e) {
@@ -315,6 +351,11 @@ public abstract class AbstractSpatialFie
     return new FilteredQuery(functionQuery, filter);
   }
 
+  @Override
+  public double getSphereRadius() {
+      return distanceUnits.getEarthRadius();
+  }
+
   /** The set of values supported for the score local-param. Not null. */
   public Set<String> getSupportedScoreModes() {
     return supportedScoreModes;
@@ -324,21 +365,31 @@ public abstract class AbstractSpatialFie
     if (score == null) {
       return null;
     }
-    switch (score) {
-      case NONE:
+
+    final double multiplier; // default multiplier for degrees
+
+    switch(score) {
       case "":
+      case NONE:
         return null;
-      case DISTANCE:
-        double multiplier = 1.0;//TODO support units=kilometers
-        return strategy.makeDistanceValueSource(spatialArgs.getShape().getCenter(), multiplier);
       case RECIP_DISTANCE:
         return strategy.makeRecipDistanceValueSource(spatialArgs.getShape());
+      case DISTANCE:
+        multiplier = distanceUnits.multiplierFromDegreesToThisUnit();
+        break;
       default:
+        DistanceUnits du = parseDistanceUnits(score);
+        if (du != null) {
+          multiplier = du.multiplierFromDegreesToThisUnit();
+        } else {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-            "'score' local-param must be one of " + supportedScoreModes);
+              "'score' local-param must be one of " + supportedScoreModes + ", it was: " + score);
     }
   }
 
+    return strategy.makeDistanceValueSource(spatialArgs.getShape().getCenter(), multiplier);
+  }
+
   /**
    * Gets the cached strategy for this field, creating it if necessary
    * via {@link #newSpatialStrategy(String)}.
@@ -368,6 +419,10 @@ public abstract class AbstractSpatialFie
     throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Sorting not supported on SpatialField: " + field.getName()+
       ", instead try sorting by query.");
   }
+
+  public DistanceUnits getDistanceUnits() {
+    return this.distanceUnits;
+  }
 }
 
 

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/AbstractSpatialPrefixTreeFieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/AbstractSpatialPrefixTreeFieldType.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/AbstractSpatialPrefixTreeFieldType.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/AbstractSpatialPrefixTreeFieldType.java Sun Feb  8 23:53:14 2015
@@ -17,14 +17,14 @@ package org.apache.solr.schema;
  * limitations under the License.
  */
 
+import java.util.Map;
+
 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 java.util.Map;
-
 /**
  * @see PrefixTreeStrategy
  * @lucene.experimental
@@ -42,6 +42,13 @@ public abstract class AbstractSpatialPre
   protected void init(IndexSchema schema, Map<String, String> args) {
     super.init(schema, args);
 
+    // Convert the maxDistErr to degrees (based on distanceUnits) since Lucene spatial layer depends on degrees
+    if(args.containsKey(SpatialPrefixTreeFactory.MAX_DIST_ERR)) {
+      double maxDistErrOriginal = Double.parseDouble(args.get(SpatialPrefixTreeFactory.MAX_DIST_ERR));
+      args.put(SpatialPrefixTreeFactory.MAX_DIST_ERR, 
+          Double.toString(maxDistErrOriginal * distanceUnits.multiplierFromThisUnitToDegrees()));
+    }
+
     //Solr expects us to remove the parameters we've used.
     MapListener<String, String> argsWrap = new MapListener<>(args);
     grid = SpatialPrefixTreeFactory.makeSPT(argsWrap, schema.getResourceLoader().getClassLoader(), ctx);

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/BBoxField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/BBoxField.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/BBoxField.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/BBoxField.java Sun Feb  8 23:53:14 2015
@@ -17,6 +17,12 @@ package org.apache.solr.schema;
  * limitations under the License.
  */
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
 import com.spatial4j.core.shape.Rectangle;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.queries.function.ValueSource;
@@ -27,12 +33,6 @@ import org.apache.lucene.spatial.util.Sh
 import org.apache.solr.common.SolrException;
 import org.apache.solr.search.QParser;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-
 public class BBoxField extends AbstractSpatialFieldType<BBoxStrategy> implements SchemaAware {
   private static final String PARAM_QUERY_TARGET_PROPORTION = "queryTargetProportion";
   private static final String PARAM_MIN_SIDE_LENGTH = "minSideLength";
@@ -140,6 +140,7 @@ public class BBoxField extends AbstractS
     if (scoreParam == null) {
       return null;
     }
+
     switch (scoreParam) {
       //TODO move these to superclass after LUCENE-5804 ?
       case OVERLAP_RATIO:
@@ -160,10 +161,12 @@ public class BBoxField extends AbstractS
             queryTargetProportion, minSideLength);
 
       case AREA:
-        return new ShapeAreaValueSource(strategy.makeShapeValueSource(), ctx, ctx.isGeo());
+        return new ShapeAreaValueSource(strategy.makeShapeValueSource(), ctx, ctx.isGeo(),
+            distanceUnits.multiplierFromDegreesToThisUnit() * distanceUnits.multiplierFromDegreesToThisUnit());
 
       case AREA2D:
-        return new ShapeAreaValueSource(strategy.makeShapeValueSource(), ctx, false);
+        return new ShapeAreaValueSource(strategy.makeShapeValueSource(), ctx, false,
+            distanceUnits.multiplierFromDegreesToThisUnit() * distanceUnits.multiplierFromDegreesToThisUnit());
 
       default:
         return super.getValueSourceFromSpatialArgs(parser, field, spatialArgs, scoreParam, strategy);

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/CollationField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/CollationField.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/CollationField.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/CollationField.java Sun Feb  8 23:53:14 2015
@@ -180,11 +180,8 @@ public class CollationField extends Fiel
      input = loader.openResource(fileName);
      String rules = IOUtils.toString(input, "UTF-8");
      return new RuleBasedCollator(rules);
-    } catch (IOException e) {
-      // io error
-      throw new RuntimeException(e);
-    } catch (ParseException e) {
-      // invalid rules
+    } catch (IOException | ParseException e) {
+      // io error or invalid rules
       throw new RuntimeException(e);
     } finally {
       IOUtils.closeQuietly(input);

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/CurrencyField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/CurrencyField.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/CurrencyField.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/CurrencyField.java Sun Feb  8 23:53:14 2015
@@ -836,13 +836,7 @@ class FileExchangeRateProvider implement
           
           addRate(tmpRates, fromCurrency, toCurrency, exchangeRate);
         }
-      } catch (SAXException e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error parsing currency config.", e);
-      } catch (IOException e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error parsing currency config.", e);
-      } catch (ParserConfigurationException e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error parsing currency config.", e);
-      } catch (XPathExpressionException e) {
+      } catch (SAXException | XPathExpressionException | ParserConfigurationException | IOException e) {
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error parsing currency config.", e);
       }
     } catch (IOException e) {

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/DateRangeField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/DateRangeField.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/DateRangeField.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/DateRangeField.java Sun Feb  8 23:53:14 2015
@@ -54,12 +54,7 @@ public class DateRangeField extends Abst
 
   @Override
   protected void init(IndexSchema schema, Map<String, String> args) {
-    super.init(schema, addDegrees(args));
-  }
-
-  private Map<String, String> addDegrees(Map<String, String> args) {
-    args.put("units", "degrees");//HACK!
-    return args;
+    super.init(schema, args);
   }
 
   @Override

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/EnumField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/EnumField.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/EnumField.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/EnumField.java Sun Feb  8 23:53:14 2015
@@ -126,13 +126,7 @@ public class EnumField extends Primitive
           enumStringToIntMap.put(valueStr, i);
         }
       }
-      catch (ParserConfigurationException e) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Error parsing enums config.", e);
-      }
-      catch (SAXException e) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Error parsing enums config.", e);
-      }
-      catch (XPathExpressionException e) {
+      catch (ParserConfigurationException | XPathExpressionException | SAXException e) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Error parsing enums config.", e);
       }
     }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/GeoHashField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/GeoHashField.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/GeoHashField.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/GeoHashField.java Sun Feb  8 23:53:14 2015
@@ -17,16 +17,18 @@
 
 package org.apache.solr.schema;
 
+import java.io.IOException;
+
+import com.spatial4j.core.context.SpatialContext;
+import com.spatial4j.core.distance.DistanceUtils;
+import com.spatial4j.core.io.GeohashUtils;
+import com.spatial4j.core.shape.Point;
+import org.apache.lucene.index.StorableField;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.LiteralValueSource;
-import org.apache.lucene.index.StorableField;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.uninverting.UninvertingReader.Type;
-
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.io.GeohashUtils;
-import com.spatial4j.core.shape.Point;
 import org.apache.solr.response.TextResponseWriter;
 import org.apache.solr.search.QParser;
 import org.apache.solr.search.SolrConstantScoreQuery;
@@ -35,9 +37,6 @@ import org.apache.solr.search.function.V
 import org.apache.solr.search.function.distance.GeohashHaversineFunction;
 import org.apache.solr.util.SpatialUtils;
 
-
-import java.io.IOException;
-
 /**
  * This is a class that represents a <a
  * href="http://en.wikipedia.org/wiki/Geohash">Geohash</a> field. The field is
@@ -94,4 +93,9 @@ public class GeoHashField extends FieldT
     return new StrFieldSource(field.name);
   }
 
+  @Override
+  public double getSphereRadius() {
+    return DistanceUtils.EARTH_MEAN_RADIUS_KM;
+  }
+
 }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/IndexSchema.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/IndexSchema.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/IndexSchema.java Sun Feb  8 23:53:14 2015
@@ -901,37 +901,49 @@ public class IndexSchema {
       String msg = "copyField dest :'" + dest + "' is not an explicit field and doesn't match a dynamicField.";
       throw new SolrException(ErrorCode.SERVER_ERROR, msg);
     }
-    if (sourceIsDynamicFieldReference || sourceIsGlob) {
-      if (null != destDynamicField) { // source: glob or no-asterisk dynamic field ref; dest: dynamic field ref
+    if (sourceIsGlob) {
+      if (null != destDynamicField) { // source: glob ; dest: dynamic field ref
         registerDynamicCopyField(new DynamicCopy(source, destDynamicField, maxChars, sourceDynamicBase, destDynamicBase));
         incrementCopyFieldTargetCount(destSchemaField);
-      } else {                        // source: glob or no-asterisk dynamic field ref; dest: explicit field
+      } else {                        // source: glob ; dest: explicit field
         destDynamicField = new DynamicField(destSchemaField);
         registerDynamicCopyField(new DynamicCopy(source, destDynamicField, maxChars, sourceDynamicBase, null));
         incrementCopyFieldTargetCount(destSchemaField);
       }
-    } else {                          
-      if (null != destDynamicField) { // source: explicit field; dest: dynamic field reference
+    } else if (sourceIsDynamicFieldReference) {
+        if (null != destDynamicField) {  // source: no-asterisk dynamic field ref ; dest: dynamic field ref
+          registerDynamicCopyField(new DynamicCopy(source, destDynamicField, maxChars, sourceDynamicBase, destDynamicBase));
+          incrementCopyFieldTargetCount(destSchemaField);
+        } else {                        // source: no-asterisk dynamic field ref ; dest: explicit field
+          sourceSchemaField = getField(source);
+          registerExplicitSrcAndDestFields(source, maxChars, destSchemaField, sourceSchemaField);
+        }
+    } else {
+      if (null != destDynamicField) { // source: explicit field ; dest: dynamic field reference
         if (destDynamicField.pattern instanceof DynamicReplacement.DynamicPattern.NameEquals) {
           // Dynamic dest with no asterisk is acceptable
           registerDynamicCopyField(new DynamicCopy(source, destDynamicField, maxChars, sourceDynamicBase, destDynamicBase));
           incrementCopyFieldTargetCount(destSchemaField);
-        } else {
+        } else {                    // source: explicit field ; dest: dynamic field with an asterisk
           String msg = "copyField only supports a dynamic destination with an asterisk "
                      + "if the source also has an asterisk";
           throw new SolrException(ErrorCode.SERVER_ERROR, msg);
         }
-      } else {                        // source & dest: explicit fields 
-        List<CopyField> copyFieldList = copyFieldsMap.get(source);
-        if (copyFieldList == null) {
-          copyFieldList = new ArrayList<>();
-          copyFieldsMap.put(source, copyFieldList);
-        }
-        copyFieldList.add(new CopyField(sourceSchemaField, destSchemaField, maxChars));
-        incrementCopyFieldTargetCount(destSchemaField);
+      } else {                        // source & dest: explicit fields
+        registerExplicitSrcAndDestFields(source, maxChars, destSchemaField, sourceSchemaField);
       }
     }
   }
+
+  private void registerExplicitSrcAndDestFields(String source, int maxChars, SchemaField destSchemaField, SchemaField sourceSchemaField) {
+    List<CopyField> copyFieldList = copyFieldsMap.get(source);
+    if (copyFieldList == null) {
+      copyFieldList = new ArrayList<>();
+      copyFieldsMap.put(source, copyFieldList);
+    }
+    copyFieldList.add(new CopyField(sourceSchemaField, destSchemaField, maxChars));
+    incrementCopyFieldTargetCount(destSchemaField);
+  }
   
   private void incrementCopyFieldTargetCount(SchemaField dest) {
     copyFieldTargetCounts.put(dest, copyFieldTargetCounts.containsKey(dest) ? copyFieldTargetCounts.get(dest) + 1 : 1);

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/LatLonType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/LatLonType.java?rev=1658277&r1=1658276&r2=1658277&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/LatLonType.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/schema/LatLonType.java Sun Feb  8 23:53:14 2015
@@ -22,11 +22,13 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import com.spatial4j.core.context.SpatialContext;
+import com.spatial4j.core.distance.DistanceUtils;
 import com.spatial4j.core.shape.Point;
-
+import com.spatial4j.core.shape.Rectangle;
 import org.apache.lucene.document.FieldType;
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.StorableField;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
@@ -49,11 +51,6 @@ import org.apache.solr.search.ExtendedQu
 import org.apache.solr.search.PostFilter;
 import org.apache.solr.search.QParser;
 import org.apache.solr.search.SpatialOptions;
-
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.shape.Rectangle;
-
 import org.apache.solr.util.SpatialUtils;
 
 
@@ -256,6 +253,11 @@ public class LatLonType extends Abstract
     throw new UnsupportedOperationException("LatLonType uses multiple fields.  field=" + field.getName());
   }
 
+  @Override
+  public double getSphereRadius() {
+    return DistanceUtils.EARTH_MEAN_RADIUS_KM;
+  }
+
 }
 
 class LatLonValueSource extends VectorValueSource {