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 [26/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/handler/component/StatsField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/handler/component/StatsField.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/handler/component/StatsField.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/handler/component/StatsField.java Tue Mar 31 05:22:40 2015
@@ -19,30 +19,32 @@ package org.apache.solr.handler.componen
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
 import java.util.IdentityHashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.*;
 import org.apache.lucene.queries.function.FunctionQuery;
 import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.valuesource.QueryValueSource;
 import org.apache.lucene.queries.function.valuesource.FieldCacheSource;
-
+import org.apache.lucene.queries.function.valuesource.QueryValueSource;
+import org.apache.lucene.search.Query;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.StatsParams;
 import org.apache.solr.common.util.StrUtils;
-import org.apache.solr.request.SolrQueryRequest; // jdocs
 import org.apache.solr.request.DocValuesStats;
+import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.DocIterator;
@@ -60,6 +62,106 @@ import org.apache.solr.search.SyntaxErro
  * @see StatsComponent
  */
 public class StatsField {
+  
+  /**
+   * An enumeration representing the sumer set of all possible stat values that can be computed.
+   * Each of these enum values can be specified as a local param in a <code>stats.field</code> 
+   * (eg: <code>stats.field={!min=true mean=true}my_field_name</code>) but not all enum values 
+   * are valid for all field types (eg: <code>mean</code> is meaningless for String fields)
+   *
+   * @lucene.internal
+   * @lucene.experimental
+   */
+  public static enum Stat {
+    min(true),
+    max(true),
+    missing(true),
+    sum(true),
+    count(true),
+    mean(false, sum, count),
+    sumOfSquares(true),
+    stddev(false, sum, count, sumOfSquares),
+    calcdistinct(true),
+    percentiles(true){
+      /** special for percentiles **/
+      boolean parseParams(StatsField sf) {
+        String percentileParas = sf.localParams.get(this.name());
+        if (percentileParas != null) {
+          List<Double> percentiles = new ArrayList<Double>();
+          try {
+            for (String percentile : StrUtils.splitSmart(percentileParas, ',')) {
+              percentiles.add(Double.parseDouble(percentile));
+            }
+            if (!percentiles.isEmpty()) {
+              sf.percentilesList.addAll(percentiles);
+              sf.tdigestCompression = sf.localParams.getDouble("tdigestCompression", 
+                                                               sf.tdigestCompression);
+              return true;
+            }
+          } catch (NumberFormatException e) {
+            throw new SolrException(ErrorCode.BAD_REQUEST, "Unable to parse "
+                + StatsParams.STATS_FIELD + " local params: " + sf.localParams + " due to: "
+                + e.getMessage(), e);
+          }
+
+        }
+        return false;
+      }
+    };
+
+    private final List<Stat> distribDeps;
+    
+    /**
+     * Sole constructor for Stat enum values
+     * @param deps the set of stat values, other then this one, which are a distributed 
+     *        dependency and must be computed and returned by each individual shards in 
+     *        order to compute <i>this</i> stat over the entire distributed result set.
+     * @param selfDep indicates that when computing this stat across a distributed result 
+     *        set, each shard must compute this stat <i>in addition to</i> any other 
+     *        distributed dependences.
+     * @see #getDistribDeps
+     */
+    Stat(boolean selfDep, Stat... deps) {
+      distribDeps = new ArrayList<Stat>(deps.length+1);
+      distribDeps.addAll(Arrays.asList(deps));
+      if (selfDep) { 
+        distribDeps.add(this);
+      }
+    }
+    
+    /**
+     * Given a String, returns the corrisponding Stat enum value if any, otherwise returns null.
+     */
+    public static Stat forName(String paramKey) {
+      try {
+        return Stat.valueOf(paramKey);
+      } catch (IllegalArgumentException e) {
+        return null;
+      }
+    }
+    
+    /**
+     * The stats that must be computed and returned by each shard involved in a distributed 
+     * request in order to compute the overall value for this stat across the entire distributed 
+     * result set.  A Stat instance may include itself in the <code>getDistribDeps()</code> result,
+     * but that is not always the case.
+     */
+    public EnumSet<Stat> getDistribDeps() {
+      return EnumSet.copyOf(this.distribDeps);
+    }
+    
+    /** return value of true means user is requesting this stat */
+    boolean parseParams(StatsField sf) {
+      return sf.localParams.getBool(this.name(), false);
+    }
+    
+  }
+
+  /**
+   * The set of stats computed by default when no localparams are used to specify explicit stats 
+   */
+  public final static Set<Stat> DEFAULT_STATS = Collections.<Stat>unmodifiableSet
+    (EnumSet.of(Stat.min, Stat.max, Stat.missing, Stat.sum, Stat.count, Stat.mean, Stat.sumOfSquares, Stat.stddev));
 
   private final SolrIndexSearcher searcher;
   private final ResponseBuilder rb;
@@ -68,11 +170,18 @@ public class StatsField {
   private final ValueSource valueSource; // may be null if simple field stats
   private final SchemaField schemaField; // may be null if function/query stats
   private final String key;
-  private final boolean calcDistinct; // TODO: put this inside localParams ? SOLR-6349 ?
+  private final boolean  topLevelCalcDistinct;
   private final String[] facets;
   private final List<String> tagList;
   private final List<String> excludeTagList;
-
+  private final EnumSet<Stat> statsToCalculate = EnumSet.noneOf(Stat.class);
+  private final EnumSet<Stat> statsInResponse = EnumSet.noneOf(Stat.class);
+  private final List<Double> percentilesList= new ArrayList<Double>();
+  private final boolean isShard;
+  
+  private double tdigestCompression = 100.0D;
+  
+  
   /**
    * @param rb the current request/response
    * @param statsParam the raw {@link StatsParams#STATS_FIELD} string
@@ -84,6 +193,7 @@ public class StatsField {
 
     SolrParams params = rb.req.getParams();
     try {
+      isShard = params.getBool("isShard", false);
       SolrParams localParams = QueryParsing.getLocalParams(originalParam, params);
       if (null == localParams) {
         // simplest possible input: bare string (field name)
@@ -91,8 +201,9 @@ public class StatsField {
         customParams.add(QueryParsing.V, originalParam);
         localParams = customParams;
       }
-      this.localParams = localParams;
 
+      this.localParams = localParams;
+      
       String parserName = localParams.get(QueryParsing.TYPE);
       SchemaField sf = null;
       ValueSource vs = null;
@@ -141,11 +252,12 @@ public class StatsField {
                                                // default to entire original param str.
                                                originalParam));
 
-    
-    this.calcDistinct = null == schemaField
-      ? params.getBool(StatsParams.STATS_CALC_DISTINCT, false) 
-      : params.getFieldBool(schemaField.getName(), StatsParams.STATS_CALC_DISTINCT, false);
+    this.topLevelCalcDistinct = null == schemaField
+        ? params.getBool(StatsParams.STATS_CALC_DISTINCT, false) 
+        : params.getFieldBool(schemaField.getName(), StatsParams.STATS_CALC_DISTINCT, false);
 
+    populateStatsSets();
+        
     String[] facets = params.getFieldParams(key, StatsParams.STATS_FACET);
     this.facets = (null == facets) ? new String[0] : facets;
     String tagStr = localParams.get(CommonParams.TAG);
@@ -269,6 +381,12 @@ public class StatsField {
    */
   public StatsValues computeLocalStatsValues(DocSet base) throws IOException {
 
+    if (statsToCalculate.isEmpty()) { 
+      // perf optimization for the case where we compute nothing
+      // ie: stats.field={!min=$domin}myfield&domin=false
+      return StatsValuesFactory.createStatsValues(this);
+    }
+
     if (null != schemaField 
         && (schemaField.multiValued() || schemaField.getType().multiValuedFieldCache())) {
 
@@ -360,15 +478,6 @@ public class StatsField {
     return valueSource;
   }
 
-  /**
-   * Whether or not the effective value of the {@link StatsParams#STATS_CALC_DISTINCT} param
-   * is true or false for this StatsField
-   */
-  public boolean getCalcDistinct() {
-    return calcDistinct;
-  }
-
-
   public List<String> getTagList() {
     return tagList;
   }
@@ -377,4 +486,67 @@ public class StatsField {
     return "StatsField<" + originalParam + ">";
   }
 
+  /**
+   * A helper method which inspects the {@link #localParams} associated with this StatsField, 
+   * and uses them to populate the {@link #statsInResponse} and {@link #statsToCalculate} data 
+   * structures
+   */
+  private void populateStatsSets() {
+    boolean statSpecifiedByLocalParam = false;
+    // local individual stat
+    Iterator<String> itParams = localParams.getParameterNamesIterator();
+    
+    while (itParams.hasNext()) {
+      String paramKey = itParams.next();
+      Stat stat = Stat.forName(paramKey);
+      if (stat != null) {
+        statSpecifiedByLocalParam = true;
+        if (stat.parseParams(this)) {
+          statsInResponse.add(stat);
+          statsToCalculate.addAll(stat.getDistribDeps());
+        }
+      }
+    }
+
+    // if no individual stat setting. 
+    if ( ! statSpecifiedByLocalParam ) {
+      statsInResponse.addAll(DEFAULT_STATS);
+      for (Stat stat : statsInResponse) {
+        statsToCalculate.addAll(stat.getDistribDeps());
+      }
+    }
+
+    // calcDistinct has special "default" behavior using top level CalcDistinct param
+    if (topLevelCalcDistinct && localParams.getBool(Stat.calcdistinct.toString(), true)) {
+      statsInResponse.add(Stat.calcdistinct);
+      statsToCalculate.addAll(Stat.calcdistinct.getDistribDeps());
+    }
+  }
+
+  public boolean calculateStats(Stat stat) {
+    return statsToCalculate.contains(stat);
+  }
+  
+  public boolean includeInResponse(Stat stat) {
+    if (isShard) {
+      return statsToCalculate.contains(stat);
+    }
+   
+    if (statsInResponse.contains(stat)) {
+      return true;
+    }
+    return false;
+  }
+
+  public List<Double> getPercentilesList() {
+    return percentilesList;
+  }
+  
+  public boolean getIsShard() {
+    return isShard;
+  }
+  
+  public double getTdigestCompression() {
+    return tdigestCompression;
+  }
 }

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java Tue Mar 31 05:22:40 2015
@@ -19,6 +19,7 @@ package org.apache.solr.handler.componen
 
 import java.io.IOException;
 import java.util.*;
+import java.nio.ByteBuffer;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
@@ -28,28 +29,36 @@ import org.apache.solr.common.EnumFieldV
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.handler.component.StatsField.Stat;
 import org.apache.solr.schema.*;
 
+import com.tdunning.math.stats.AVLTreeDigest;
+
 /**
- * Factory class for creating instance of {@link org.apache.solr.handler.component.StatsValues}
+ * Factory class for creating instance of 
+ * {@link org.apache.solr.handler.component.StatsValues}
  */
 public class StatsValuesFactory {
 
   /**
-   * Creates an instance of StatsValues which supports values from the specified {@link StatsField}
+   * Creates an instance of StatsValues which supports values from the specified 
+   * {@link StatsField}
    *
-   * @param statsField {@link StatsField} whose statistics will be created by the resulting {@link StatsValues}
-   * @return Instance of {@link StatsValues} that will create statistics from values from the specified {@link StatsField}
+   * @param statsField
+   *          {@link StatsField} whose statistics will be created by the
+   *          resulting {@link StatsValues}
+   * @return Instance of {@link StatsValues} that will create statistics from
+   *         values from the specified {@link StatsField}
    */
   public static StatsValues createStatsValues(StatsField statsField) {
-
+    
     final SchemaField sf = statsField.getSchemaField();
-
+    
     if (null == sf) {
       // function stats
       return new NumericStatsValues(statsField);
-    } 
-
+    }
+    
     final FieldType fieldType = sf.getType(); // TODO: allow FieldType to provide impl.
     
     if (TrieDateField.class.isInstance(fieldType)) {
@@ -61,76 +70,90 @@ public class StatsValuesFactory {
     } else if (sf.getType().getClass().equals(EnumField.class)) {
       return new EnumStatsValues(statsField);
     } else {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field type " + fieldType + " is not currently supported");
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          "Field type " + fieldType + " is not currently supported");
     }
   }
 }
 
 /**
- * Abstract implementation of {@link org.apache.solr.handler.component.StatsValues} 
- * that provides the default behavior for most StatsValues implementations.
+ * Abstract implementation of
+ * {@link org.apache.solr.handler.component.StatsValues} that provides the
+ * default behavior for most StatsValues implementations.
  *
- * There are very few requirements placed on what statistics concrete implementations 
- * should collect, with the only required statistics being the minimum and maximum values.
+ * There are very few requirements placed on what statistics concrete
+ * implementations should collect, with the only required statistics being the
+ * minimum and maximum values.
  */
 abstract class AbstractStatsValues<T> implements StatsValues {
   private static final String FACETS = "facets";
-
+  
   /** Tracks all data about tthe stats we need to collect */
   final protected StatsField statsField;
 
-  /** 
-   * local copy to save method dispatch in tight loops 
-   * @see StatsField#getCalcDistinct
-   */
-  final protected boolean calcDistinct;
-
   /** may be null if we are collecting stats directly from a function ValueSource */
   final protected SchemaField sf;
-  /** may be null if we are collecting stats directly from a function ValueSource */
+  /**
+   * may be null if we are collecting stats directly from a function ValueSource
+   */
   final protected FieldType ft;
 
+  // final booleans from StatsField to allow better inlining & JIT optimizing
+  final protected boolean computeCount;
+  final protected boolean computeMissing;
+  final protected boolean computeCalcDistinct;
+  final protected boolean computeMin;
+  final protected boolean computeMax;
+  final protected boolean computeMinOrMax;
+
   /** 
    * Either a function value source to collect from, or the ValueSource associated 
    * with a single valued field we are collecting from.  Will be null until/unless 
    * {@link #setNextReader} is called at least once
    */
   private ValueSource valueSource;
-  /** 
-   * Context to use when retrieving FunctionValues, will be null until/unless 
+  /**
+   * Context to use when retrieving FunctionValues, will be null until/unless
    * {@link #setNextReader} is called at least once
    */
   private Map vsContext;
-  /** 
-   * Values to collect, will be null until/unless {@link #setNextReader} is called 
-   * at least once 
+  /**
+   * Values to collect, will be null until/unless {@link #setNextReader} is
+   * called at least once
    */
   protected FunctionValues values;
-
+  
   protected T max;
   protected T min;
   protected long missing;
   protected long count;
   protected long countDistinct;
-  protected Set<T> distinctValues;
+  protected final Set<T> distinctValues;
+  
+  // facetField facetValue
+  protected Map<String,Map<String, StatsValues>> facets = new HashMap<>();
   
-  // facetField   facetValue
-  protected Map<String, Map<String, StatsValues>> facets = new HashMap<>();
-
   protected AbstractStatsValues(StatsField statsField) {
     this.statsField = statsField;
-    this.calcDistinct = statsField.getCalcDistinct();
-    this.distinctValues = new TreeSet<>();
+    this.computeCount = statsField.calculateStats(Stat.count);
+    this.computeMissing = statsField.calculateStats(Stat.missing);
+    this.computeCalcDistinct = statsField.calculateStats(Stat.calcdistinct);
+    this.computeMin = statsField.calculateStats(Stat.min);
+    this.computeMax = statsField.calculateStats(Stat.max);
+    this.computeMinOrMax = computeMin || computeMax;
+      
+    this.distinctValues = computeCalcDistinct ? new TreeSet<>() : null;
 
     // alternatively, we could refactor a common base class that doesn't know/care
     // about either SchemaField or ValueSource - but then there would be a lot of
     // duplicate code between "NumericSchemaFieldStatsValues" and 
     // "NumericValueSourceStatsValues" which would have diff parent classes
     //
-    // part of the complexity here being that the StatsValues API serves two 
-    // masters: collecting concrete Values from things like DocValuesStats and 
-    // the distributed aggregation logic, but also collecting docIds which it then
-    // uses to go out and pull concreate values from the ValueSource 
+    // part of the complexity here being that the StatsValues API serves two
+    // masters: collecting concrete Values from things like DocValuesStats and
+    // the distributed aggregation logic, but also collecting docIds which it
+    // then
+    // uses to go out and pull concreate values from the ValueSource
     // (from a func, or single valued field)
     if (null != statsField.getSchemaField()) {
       assert null == statsField.getValueSource();
@@ -143,27 +166,33 @@ abstract class AbstractStatsValues<T> im
       this.ft = null;
     }
   }
-
+  
   /**
    * {@inheritDoc}
    */
   @Override
   public void accumulate(NamedList stv) {
-    count += (Long) stv.get("count");
-    missing += (Long) stv.get("missing");
-    if (calcDistinct) {
+    if (computeCount) {
+      count += (Long) stv.get("count");
+    }
+    if (computeMissing) {
+      missing += (Long) stv.get("missing");
+    }
+    if (computeCalcDistinct) {
       distinctValues.addAll((Collection<T>) stv.get("distinctValues"));
       countDistinct = distinctValues.size();
     }
-
-    updateMinMax((T) stv.get("min"), (T) stv.get("max"));
+    
+    if (computeMinOrMax) {
+      updateMinMax((T) stv.get("min"), (T) stv.get("max"));
+    }
     updateTypeSpecificStats(stv);
-
+    
     NamedList f = (NamedList) stv.get(FACETS);
     if (f == null) {
       return;
     }
-
+    
     for (int i = 0; i < f.size(); i++) {
       String field = f.getName(i);
       NamedList vals = (NamedList) f.getVal(i);
@@ -183,37 +212,45 @@ abstract class AbstractStatsValues<T> im
       }
     }
   }
-
+  
   /**
    * {@inheritDoc}
    */
   @Override
   public void accumulate(BytesRef value, int count) {
     if (null == ft) {
-      throw new IllegalStateException("Can't collect & convert BytesRefs on stats that do't use a a FieldType: " + statsField);
+      throw new IllegalStateException(
+          "Can't collect & convert BytesRefs on stats that do't use a a FieldType: "
+              + statsField);
     }
-    T typedValue = (T)ft.toObject(sf, value);
+    T typedValue = (T) ft.toObject(sf, value);
     accumulate(typedValue, count);
   }
 
   public void accumulate(T value, int count) { 
-    this.count += count;
-    if (calcDistinct) {
+    if (computeCount) {
+      this.count += count;
+    }
+    if (computeCalcDistinct) {
       distinctValues.add(value);
       countDistinct = distinctValues.size();
     }
-    updateMinMax(value, value);
+    if (computeMinOrMax) {
+      updateMinMax(value, value);
+    }
     updateTypeSpecificStats(value, count);
   }
-
+  
   /**
    * {@inheritDoc}
    */
   @Override
   public void missing() {
-    missing++;
+    if (computeMissing) {
+      missing++;
+    }
   }
-   
+  
   /**
    * {@inheritDoc}
    */
@@ -221,7 +258,7 @@ abstract class AbstractStatsValues<T> im
   public void addMissing(int count) {
     missing += count;
   }
-
+  
   /**
    * {@inheritDoc}
    */
@@ -229,7 +266,7 @@ abstract class AbstractStatsValues<T> im
   public void addFacet(String facetName, Map<String, StatsValues> facetValues) {
     facets.put(facetName, facetValues);
   }
-
+  
   /**
    * {@inheritDoc}
    */
@@ -237,151 +274,248 @@ abstract class AbstractStatsValues<T> im
   public NamedList<?> getStatsValues() {
     NamedList<Object> res = new SimpleOrderedMap<>();
 
-    res.add("min", min);
-    res.add("max", max);
-    res.add("count", count);
-    res.add("missing", missing);
-    if (calcDistinct) {
+    if (statsField.includeInResponse(Stat.min)) {
+      res.add("min", min);
+    }
+    if (statsField.includeInResponse(Stat.max)) {
+      res.add("max", max);
+    }
+    if (statsField.includeInResponse(Stat.count)) {
+      res.add("count", count);
+    }
+    if (statsField.includeInResponse(Stat.missing)) {
+      res.add("missing", missing);
+    }
+    if (statsField.includeInResponse(Stat.calcdistinct)) {
       res.add("distinctValues", distinctValues);
       res.add("countDistinct", countDistinct);
     }
-
+    
     addTypeSpecificStats(res);
-
-     // add the facet stats
-    NamedList<NamedList<?>> nl = new SimpleOrderedMap<>();
-    for (Map.Entry<String, Map<String, StatsValues>> entry : facets.entrySet()) {
-      NamedList<NamedList<?>> nl2 = new SimpleOrderedMap<>();
-      nl.add(entry.getKey(), nl2);
-      for (Map.Entry<String, StatsValues> e2 : entry.getValue().entrySet()) {
-        nl2.add(e2.getKey(), e2.getValue().getStatsValues());
+    
+    if (!facets.isEmpty()) {
+      
+      // add the facet stats
+      NamedList<NamedList<?>> nl = new SimpleOrderedMap<>();
+      for (Map.Entry<String,Map<String,StatsValues>> entry : facets.entrySet()) {
+        NamedList<NamedList<?>> nl2 = new SimpleOrderedMap<>();
+        nl.add(entry.getKey(), nl2);
+        for (Map.Entry<String,StatsValues> e2 : entry.getValue().entrySet()) {
+          nl2.add(e2.getKey(), e2.getValue().getStatsValues());
+        }
       }
+
+      res.add(FACETS, nl);
     }
-    res.add(FACETS, nl);
+
     return res;
   }
-
+  
   /**
    * {@inheritDoc}
    */
-   public void setNextReader(LeafReaderContext ctx) throws IOException {
+  public void setNextReader(LeafReaderContext ctx) throws IOException {
     if (valueSource == null) {
       // first time we've collected local values, get the right ValueSource
       valueSource = (null == ft) 
-        ? statsField.getValueSource()
+        ? statsField.getValueSource() 
         : ft.getValueSource(sf, null);
       vsContext = ValueSource.newContext(statsField.getSearcher());
     }
     values = valueSource.getValues(vsContext, ctx);
   }
-
+  
   /**
    * Updates the minimum and maximum statistics based on the given values
    *
-   * @param min Value that the current minimum should be updated against
-   * @param max Value that the current maximum should be updated against
+   * @param min
+   *          Value that the current minimum should be updated against
+   * @param max
+   *          Value that the current maximum should be updated against
    */
   protected abstract void updateMinMax(T min, T max);
-
+  
   /**
    * Updates the type specific statistics based on the given value
    *
-   * @param value Value the statistics should be updated against
-   * @param count Number of times the value is being accumulated
+   * @param value
+   *          Value the statistics should be updated against
+   * @param count
+   *          Number of times the value is being accumulated
    */
   protected abstract void updateTypeSpecificStats(T value, int count);
-
+  
   /**
    * Updates the type specific statistics based on the values in the given list
    *
-   * @param stv List containing values the current statistics should be updated against
+   * @param stv
+   *          List containing values the current statistics should be updated
+   *          against
    */
   protected abstract void updateTypeSpecificStats(NamedList stv);
-
+  
   /**
    * Add any type specific statistics to the given NamedList
    *
-   * @param res NamedList to add the type specific statistics too
+   * @param res
+   *          NamedList to add the type specific statistics too
    */
   protected abstract void addTypeSpecificStats(NamedList<Object> res);
 }
 
- /**
+/**
  * Implementation of StatsValues that supports Double values
  */
 class NumericStatsValues extends AbstractStatsValues<Number> {
-
+  
   double sum;
   double sumOfSquares;
+  
+  AVLTreeDigest tdigest;
+
+  double minD; // perf optimization, only valid if (null != this.min)
+  double maxD; // perf optimization, only valid if (null != this.max)
+  
+  final protected boolean computeSum;
+  final protected boolean computeSumOfSquares;
+  final protected boolean computePercentiles;
 
   public NumericStatsValues(StatsField statsField) {
     super(statsField);
-  }
 
+    this.computeSum = statsField.calculateStats(Stat.sum);
+    this.computeSumOfSquares = statsField.calculateStats(Stat.sumOfSquares);
+    
+    this.computePercentiles = statsField.calculateStats(Stat.percentiles);
+    if ( computePercentiles ) {
+      
+      tdigest = new AVLTreeDigest(statsField.getTdigestCompression()); 
+    }
+  }
+  
   @Override
   public void accumulate(int docID) {
     if (values.exists(docID)) {
-      accumulate((Number) values.objectVal(docID), 1);
+      Number value = (Number) values.objectVal(docID);
+      accumulate(value, 1);
     } else {
       missing();
     }
   }
-
+  
   /**
    * {@inheritDoc}
    */
   @Override
   public void updateTypeSpecificStats(NamedList stv) {
-    sum += ((Number)stv.get("sum")).doubleValue();
-    sumOfSquares += ((Number)stv.get("sumOfSquares")).doubleValue();
+    if (computeSum) {
+      sum += ((Number) stv.get("sum")).doubleValue();
+    }
+    if (computeSumOfSquares) {
+      sumOfSquares += ((Number) stv.get("sumOfSquares")).doubleValue();
+    }
+    
+    if (computePercentiles) {      
+      byte[] data = (byte[]) stv.get("percentiles");
+      ByteBuffer buf = ByteBuffer.wrap(data);
+      tdigest.add(AVLTreeDigest.fromBytes(buf));
+    }
   }
-
+  
   /**
    * {@inheritDoc}
    */
   @Override
   public void updateTypeSpecificStats(Number v, int count) {
     double value = v.doubleValue();
-    sumOfSquares += (value * value * count); // for std deviation
-    sum += value * count;
+    if (computeSumOfSquares) {
+      sumOfSquares += (value * value * count); // for std deviation
+    }
+    if (computeSum) {
+      sum += value * count;
+    }
+    if (computePercentiles) {
+      tdigest.add(value, count);
+    }
   }
-
-   /**
+  
+  /**
    * {@inheritDoc}
    */
   @Override
   protected void updateMinMax(Number min, Number max) {
-    if (null == min) {
-      assert null == max : "min is null but max isn't ? ==> " + max;
-      return; // No-Op
-    }
-
-    assert null != max : "max is null but min isn't ? ==> " + min;
-
-    // we always use the double value, because that way the response Object class is 
+    // we always use the double values, because that way the response Object class is 
     // consistent regardless of whether we only have 1 value or many that we min/max
     //
     // TODO: would be nice to have subclasses for each type of Number ... breaks backcompat
-    double minD = min.doubleValue();
-    double maxD = max.doubleValue();
 
-    this.min = (null == this.min) ? minD : Math.min(this.min.doubleValue(), minD);
-    this.max = (null == this.max) ? maxD : Math.max(this.max.doubleValue(), maxD);
+    if (computeMin) { // nested if to encourage JIT to optimize aware final var?
+      if (null != min) {
+        double minD = min.doubleValue();
+        if (null == this.min || minD < this.minD) {
+          // Double for result & cached primitive doulbe to minimize unboxing in future comparisons
+          this.min = this.minD = minD;
+        }
+      }
+    }
+    if (computeMax) { // nested if to encourage JIT to optimize aware final var?
+      if (null != max) {
+        double maxD = max.doubleValue();
+        if (null == this.max || this.maxD < maxD) {
+          // Double for result & cached primitive doulbe to minimize unboxing in future comparisons
+          this.max = this.maxD = maxD;
+        }
+      }
+    }
   }
-
+  
   /**
-   * Adds sum, sumOfSquares, mean and standard deviation statistics to the given NamedList
+   * Adds sum, sumOfSquares, mean, stddev, and percentiles to the given
+   * NamedList
    *
-   * @param res NamedList to add the type specific statistics too
+   * @param res
+   *          NamedList to add the type specific statistics too
    */
   @Override
   protected void addTypeSpecificStats(NamedList<Object> res) {
-    res.add("sum", sum);
-    res.add("sumOfSquares", sumOfSquares);
-    res.add("mean", sum / count);
-    res.add("stddev", getStandardDeviation());
+    if (statsField.includeInResponse(Stat.sum)) {
+      res.add("sum", sum);
+    }
+    if (statsField.includeInResponse(Stat.sumOfSquares)) {
+      res.add("sumOfSquares", sumOfSquares);
+    }
+    if (statsField.includeInResponse(Stat.mean)) {
+      res.add("mean", sum / count);
+    }
+    if (statsField.includeInResponse(Stat.stddev)) {
+      res.add("stddev", getStandardDeviation());
+    }
+    if (statsField.includeInResponse(Stat.percentiles)) {
+      if (statsField.getIsShard()) {
+        // as of current t-digest version, smallByteSize() internally does a full conversion in 
+        // order to determine what the size is (can't be precomputed?) .. so rather then
+        // serialize to a ByteBuffer twice, allocate the max possible size buffer,
+        // serialize once, and then copy only the byte[] subset that we need, and free up the buffer
+        ByteBuffer buf = ByteBuffer.allocate(tdigest.byteSize()); // upper bound
+        tdigest.asSmallBytes(buf);
+        res.add("percentiles", Arrays.copyOf(buf.array(), buf.position()) );
+      } else {
+        NamedList<Object> percentileNameList = new NamedList<Object>();
+        for (Double percentile : statsField.getPercentilesList()) {
+          // Empty document set case
+          if (tdigest.size() == 0) {
+            percentileNameList.add(percentile.toString(), null);
+          } else {
+            Double cutoff = tdigest.quantile(percentile / 100);
+            percentileNameList.add(percentile.toString(), cutoff);
+          }
+        }
+        res.add("percentiles", percentileNameList);
+      }
+    }
   }
-
+  
+  
   /**
    * Calculates the standard deviation statistic
    *
@@ -391,8 +525,9 @@ class NumericStatsValues extends Abstrac
     if (count <= 1.0D) {
       return 0.0D;
     }
-
+    
     return Math.sqrt(((count * sumOfSquares) - (sum * sum)) / (count * (count - 1.0D)));
+                     
   }
 }
 
@@ -400,11 +535,11 @@ class NumericStatsValues extends Abstrac
  * Implementation of StatsValues that supports EnumField values
  */
 class EnumStatsValues extends AbstractStatsValues<EnumFieldValue> {
-
+  
   public EnumStatsValues(StatsField statsField) {
     super(statsField);
   }
-
+  
   /**
    * {@inheritDoc}
    */
@@ -419,21 +554,27 @@ class EnumStatsValues extends AbstractSt
       missing();
     }
   }
-
+  
   /**
    * {@inheritDoc}
    */
   protected void updateMinMax(EnumFieldValue min, EnumFieldValue max) {
-    if (max != null) {
-      if (max.compareTo(this.max) > 0)
-        this.max = max;
+    if (computeMin) { // nested if to encourage JIT to optimize aware final var?
+      if (null != min) {
+        if (null == this.min || (min.compareTo(this.min) < 0)) {
+          this.min = min;
+        }
+      }
+    }
+    if (computeMax) { // nested if to encourage JIT to optimize aware final var?
+      if (null != max) {
+        if (null == this.max || (max.compareTo(this.max) > 0)) {
+          this.max = max;
+        }
+      }
     }
-    if (this.min == null)
-      this.min = min;
-    else if (this.min.compareTo(min) > 0)
-      this.min = min;
   }
-
+  
   /**
    * {@inheritDoc}
    */
@@ -441,7 +582,7 @@ class EnumStatsValues extends AbstractSt
   protected void updateTypeSpecificStats(NamedList stv) {
     // No type specific stats
   }
-
+  
   /**
    * {@inheritDoc}
    */
@@ -449,7 +590,7 @@ class EnumStatsValues extends AbstractSt
   protected void updateTypeSpecificStats(EnumFieldValue value, int count) {
     // No type specific stats
   }
-
+  
   /**
    * Adds no type specific statistics
    */
@@ -457,23 +598,26 @@ class EnumStatsValues extends AbstractSt
   protected void addTypeSpecificStats(NamedList<Object> res) {
     // Add no statistics
   }
-
-
+  
 }
 
 /**
- * /**
- * Implementation of StatsValues that supports Date values
+ * /** Implementation of StatsValues that supports Date values
  */
 class DateStatsValues extends AbstractStatsValues<Date> {
-
+  
   private long sum = 0;
   double sumOfSquares = 0;
+  
+  final protected boolean computeSum;
+  final protected boolean computeSumOfSquares;
 
   public DateStatsValues(StatsField statsField) {
     super(statsField);
+    this.computeSum = statsField.calculateStats(Stat.sum);
+    this.computeSumOfSquares = statsField.calculateStats(Stat.sumOfSquares);
   }
-
+  
   @Override
   public void accumulate(int docID) {
     if (values.exists(docID)) {
@@ -482,64 +626,76 @@ class DateStatsValues extends AbstractSt
       missing();
     }
   }
-
+  
   /**
    * {@inheritDoc}
    */
   @Override
   protected void updateTypeSpecificStats(NamedList stv) {
-    Date date = (Date) stv.get("sum");
-    if (date != null) {
-      sum += date.getTime();
-      sumOfSquares += ((Number)stv.get("sumOfSquares")).doubleValue();
+    if (computeSum) {
+      sum += ((Date) stv.get("sum")).getTime();
+    }
+    if (computeSumOfSquares) {
+      sumOfSquares += ((Number) stv.get("sumOfSquares")).doubleValue();
     }
   }
-
+  
   /**
    * {@inheritDoc}
    */
   @Override
   public void updateTypeSpecificStats(Date v, int count) {
     long value = v.getTime();
-    sumOfSquares += (value * value * count); // for std deviation
-    sum += value * count;
+    if (computeSumOfSquares) {
+      sumOfSquares += (value * value * count); // for std deviation
+    }
+    if (computeSum) {
+      sum += value * count;
+    }
   }
-
-   /**
+  
+  /**
    * {@inheritDoc}
    */
   @Override
   protected void updateMinMax(Date min, Date max) {
-    if(null != min && (this.min==null || this.min.after(min))) {
-      this.min = min;
+    if (computeMin) { // nested if to encourage JIT to optimize aware final var?
+      if (null != min && (this.min==null || this.min.after(min))) {
+        this.min = min;
+      }
     }
-    if(null != max && (this.max==null || this.max.before(max))) {
-      this.max = max;
+    if (computeMax) { // nested if to encourage JIT to optimize aware final var?
+      if (null != max && (this.max==null || this.max.before(max))) {
+        this.max = max;
+      }
     }
   }
-
+  
   /**
    * Adds sum and mean statistics to the given NamedList
    *
-   * @param res NamedList to add the type specific statistics too
+   * @param res
+   *          NamedList to add the type specific statistics too
    */
   @Override
   protected void addTypeSpecificStats(NamedList<Object> res) {
-    if(sum<=0) {
-      return; // date==0 is meaningless
+    if (statsField.includeInResponse(Stat.sum)) {
+      res.add("sum", new Date(sum));
+    }
+    if (statsField.includeInResponse(Stat.mean)) {
+      res.add("mean", (count > 0) ? new Date(sum / count) : null);
+    }
+    if (statsField.includeInResponse(Stat.sumOfSquares)) {
+      res.add("sumOfSquares", sumOfSquares);
     }
-    res.add("sum", new Date(sum));
-    if (count > 0) {
-      res.add("mean", new Date(sum / count));
+    if (statsField.includeInResponse(Stat.stddev)) {
+      res.add("stddev", getStandardDeviation());
     }
-    res.add("sumOfSquares", sumOfSquares);
-    res.add("stddev", getStandardDeviation());
   }
   
-
-  
   /**
-   * Calculates the standard deviation.  For dates, this is really the MS deviation
+   * Calculates the standard deviation. For dates, this is really the MS
+   * deviation
    *
    * @return Standard deviation statistic
    */
@@ -547,7 +703,8 @@ class DateStatsValues extends AbstractSt
     if (count <= 1) {
       return 0.0D;
     }
-    return Math.sqrt(((count * sumOfSquares) - (sum * sum)) / (count * (count - 1.0D)));
+    return Math.sqrt(((count * sumOfSquares) - (sum * sum))
+        / (count * (count - 1.0D)));
   }
 }
 
@@ -555,24 +712,25 @@ class DateStatsValues extends AbstractSt
  * Implementation of StatsValues that supports String values
  */
 class StringStatsValues extends AbstractStatsValues<String> {
-
+  
   public StringStatsValues(StatsField statsField) {
     super(statsField);
   }
-
+  
   @Override
   public void accumulate(int docID) {
     if (values.exists(docID)) {
       String value = values.strVal(docID);
-      if (value != null)
+      if (value != null) {
         accumulate(value, 1);
-      else
+      } else { 
         missing();
+      }
     } else {
       missing();
     }
   }
-
+  
   /**
    * {@inheritDoc}
    */
@@ -580,7 +738,7 @@ class StringStatsValues extends Abstract
   protected void updateTypeSpecificStats(NamedList stv) {
     // No type specific stats
   }
-
+  
   /**
    * {@inheritDoc}
    */
@@ -588,16 +746,20 @@ class StringStatsValues extends Abstract
   protected void updateTypeSpecificStats(String value, int count) {
     // No type specific stats
   }
-
-   /**
+  
+  /**
    * {@inheritDoc}
    */
   @Override
   protected void updateMinMax(String min, String max) {
-    this.max = max(this.max, max);
-    this.min = min(this.min, min);
+    if (computeMin) { // nested if to encourage JIT to optimize aware final var?
+      this.min = min(this.min, min);
+    }
+    if (computeMax) { // nested if to encourage JIT to optimize aware final var?
+      this.max = max(this.max, max);
+    }
   }
-
+  
   /**
    * Adds no type specific statistics
    */
@@ -605,13 +767,17 @@ class StringStatsValues extends Abstract
   protected void addTypeSpecificStats(NamedList<Object> res) {
     // Add no statistics
   }
-
-  /** 
-   * Determines which of the given Strings is the maximum, as computed by {@link String#compareTo(String)}
+  
+  /**
+   * Determines which of the given Strings is the maximum, as computed by
+   * {@link String#compareTo(String)}
    *
-   * @param str1 String to compare against b
-   * @param str2 String compared against a
-   * @return str1 if it is considered greater by {@link String#compareTo(String)}, str2 otherwise
+   * @param str1
+   *          String to compare against b
+   * @param str2
+   *          String compared against a
+   * @return str1 if it is considered greater by
+   *         {@link String#compareTo(String)}, str2 otherwise
    */
   private static String max(String str1, String str2) {
     if (str1 == null) {
@@ -621,13 +787,17 @@ class StringStatsValues extends Abstract
     }
     return (str1.compareTo(str2) > 0) ? str1 : str2;
   }
-
+  
   /**
-   * Determines which of the given Strings is the minimum, as computed by {@link String#compareTo(String)}
+   * Determines which of the given Strings is the minimum, as computed by
+   * {@link String#compareTo(String)}
    *
-   * @param str1 String to compare against b
-   * @param str2 String compared against a
-   * @return str1 if it is considered less by {@link String#compareTo(String)}, str2 otherwise
+   * @param str1
+   *          String to compare against b
+   * @param str2
+   *          String compared against a
+   * @return str1 if it is considered less by {@link String#compareTo(String)},
+   *         str2 otherwise
    */
   private static String min(String str1, String str2) {
     if (str1 == null) {

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java Tue Mar 31 05:22:40 2015
@@ -11,10 +11,10 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
@@ -24,6 +24,7 @@ import org.apache.solr.common.SolrExcept
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.TermVectorParams;
+import org.apache.solr.common.util.Base64;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.IndexSchema;
@@ -58,7 +59,7 @@ import org.apache.solr.util.plugin.SolrC
  * Return term vectors for the documents in a query result set.
  * <p>
  * Info available:
- * term, frequency, position, offset, IDF.
+ * term, frequency, position, offset, payloads, IDF.
  * <p>
  * <b>Note</b> Returning IDF can be expensive.
  * 
@@ -153,6 +154,7 @@ public class TermVectorComponent extends
     allFields.termFreq = params.getBool(TermVectorParams.TF, false);
     allFields.positions = params.getBool(TermVectorParams.POSITIONS, false);
     allFields.offsets = params.getBool(TermVectorParams.OFFSETS, false);
+    allFields.payloads = params.getBool(TermVectorParams.PAYLOADS, false);
     allFields.docFreq = params.getBool(TermVectorParams.DF, false);
     allFields.tfIdf = params.getBool(TermVectorParams.TF_IDF, false);
     //boolean cacheIdf = params.getBool(TermVectorParams.IDF, false);
@@ -161,6 +163,7 @@ public class TermVectorComponent extends
       allFields.termFreq = true;
       allFields.positions = true;
       allFields.offsets = true;
+      allFields.payloads = true;
       allFields.docFreq = true;
       allFields.tfIdf = true;
     }
@@ -171,6 +174,7 @@ public class TermVectorComponent extends
     List<String>  noTV = new ArrayList<>();
     List<String>  noPos = new ArrayList<>();
     List<String>  noOff = new ArrayList<>();
+    List<String>  noPay = new ArrayList<>();
 
     Set<String> fields = getFields(rb);
     if ( null != fields ) {
@@ -207,6 +211,10 @@ public class TermVectorComponent extends
             if (option.offsets && !sf.storeTermOffsets() && !fieldIsUniqueKey){
               noOff.add(field);
             }
+            option.payloads = params.getFieldBool(field, TermVectorParams.PAYLOADS, allFields.payloads);
+            if (option.payloads && !sf.storeTermPayloads() && !fieldIsUniqueKey){
+              noPay.add(field);
+            }
           } else {//field doesn't have term vectors
             if (!fieldIsUniqueKey) noTV.add(field);
           }
@@ -234,6 +242,10 @@ public class TermVectorComponent extends
       warnings.add("noOffsets", noOff);
       hasWarnings = true;
     }
+    if (!noPay.isEmpty()) {
+      warnings.add("noPayloads", noPay);
+      hasWarnings = true;
+    }
     if (hasWarnings) {
       termVectors.add("warnings", warnings);
     }
@@ -341,22 +353,27 @@ public class TermVectorComponent extends
         termInfo.add("tf", freq);
       }
 
-      dpEnum = termsEnum.postings(null, dpEnum, PostingsEnum.ALL);
-      boolean useOffsets = false;
-      boolean usePositions = false;
+      int dpEnumFlags = 0;
+      dpEnumFlags |= fieldOptions.positions ? PostingsEnum.POSITIONS : 0;
+      //payloads require offsets
+      dpEnumFlags |= (fieldOptions.offsets || fieldOptions.payloads) ? PostingsEnum.OFFSETS : 0;
+      dpEnumFlags |= fieldOptions.payloads ? PostingsEnum.PAYLOADS : 0;
+      dpEnum = termsEnum.postings(null, dpEnum, dpEnumFlags);
+
+      boolean atNextDoc = false;
       if (dpEnum != null) {
         dpEnum.nextDoc();
-        usePositions = fieldOptions.positions;
-        useOffsets = fieldOptions.offsets;
+        atNextDoc = true;
       }
 
-      NamedList<Integer> positionsNL = null;
-      NamedList<Number> theOffsets = null;
+      if (atNextDoc && dpEnumFlags != 0) {
+        NamedList<Integer> positionsNL = null;
+        NamedList<Number> theOffsets = null;
+        NamedList<String> thePayloads = null;
 
-      if (usePositions || useOffsets) {
         for (int i = 0; i < freq; i++) {
           final int pos = dpEnum.nextPosition();
-          if (usePositions && pos >= 0) {
+          if (fieldOptions.positions && pos >= 0) {
             if (positionsNL == null) {
               positionsNL = new NamedList<>();
               termInfo.add("positions", positionsNL);
@@ -364,19 +381,24 @@ public class TermVectorComponent extends
             positionsNL.add("position", pos);
           }
 
-          if (useOffsets && theOffsets == null) {
-            if (dpEnum.startOffset() == -1) {
-              useOffsets = false;
-            } else {
+          int startOffset = fieldOptions.offsets ? dpEnum.startOffset() : -1;
+          if (startOffset >= 0) {
+            if (theOffsets == null) {
               theOffsets = new NamedList<>();
               termInfo.add("offsets", theOffsets);
             }
-          }
-
-          if (theOffsets != null) {
             theOffsets.add("start", dpEnum.startOffset());
             theOffsets.add("end", dpEnum.endOffset());
           }
+
+          BytesRef payload = fieldOptions.payloads ? dpEnum.getPayload() : null;
+          if (payload != null) {
+            if (thePayloads == null) {
+              thePayloads = new NamedList<>();
+              termInfo.add("payloads", thePayloads);
+            }
+            thePayloads.add("payload", Base64.byteArrayToBase64(payload.bytes, payload.offset, payload.length));
+          }
         }
       }
       
@@ -472,5 +494,5 @@ public class TermVectorComponent extends
 
 class FieldOptions {
   String fieldName;
-  boolean termFreq, positions, offsets, docFreq, tfIdf;
+  boolean termFreq, positions, offsets, payloads, docFreq, tfIdf;
 }

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java Tue Mar 31 05:22:40 2015
@@ -94,9 +94,9 @@ public class JsonLoader extends ContentS
 
     @Override
     public void load(SolrQueryRequest req,
-        SolrQueryResponse rsp,
-        ContentStream stream,
-        UpdateRequestProcessor processor) throws Exception {
+                     SolrQueryResponse rsp,
+                     ContentStream stream,
+                     UpdateRequestProcessor processor) throws Exception {
 
       Reader reader = null;
       try {
@@ -116,7 +116,7 @@ public class JsonLoader extends ContentS
     @SuppressWarnings("fallthrough")
     void processUpdate(Reader reader) throws IOException {
       String path = (String) req.getContext().get("path");
-      if(UpdateRequestHandler.DOC_PATH.equals(path) ||   "false".equals( req.getParams().get("json.command"))){
+      if (UpdateRequestHandler.DOC_PATH.equals(path) || "false".equals(req.getParams().get("json.command"))) {
         String split = req.getParams().get("split");
         String[] f = req.getParams().getParams("f");
         handleSplitMode(split, f, reader);
@@ -124,68 +124,62 @@ public class JsonLoader extends ContentS
       }
       parser = new JSONParser(reader);
       int ev = parser.nextEvent();
-      while( ev != JSONParser.EOF ) {
+      while (ev != JSONParser.EOF) {
 
-        switch( ev )
-        {
+        switch (ev) {
           case JSONParser.ARRAY_START:
             handleAdds();
             break;
 
-        case JSONParser.STRING:
-          if( parser.wasKey() ) {
-            String v = parser.getString();
-            if( v.equals( UpdateRequestHandler.ADD ) ) {
-              int ev2 = parser.nextEvent();
-              if (ev2 == JSONParser.OBJECT_START) {
-                processor.processAdd( parseAdd() );
-              } else if (ev2 == JSONParser.ARRAY_START) {
-                handleAdds();
+          case JSONParser.STRING:
+            if (parser.wasKey()) {
+              String v = parser.getString();
+              if (v.equals(UpdateRequestHandler.ADD)) {
+                int ev2 = parser.nextEvent();
+                if (ev2 == JSONParser.OBJECT_START) {
+                  processor.processAdd(parseAdd());
+                } else if (ev2 == JSONParser.ARRAY_START) {
+                  handleAdds();
+                } else {
+                  assertEvent(ev2, JSONParser.OBJECT_START);
+                }
+              } else if (v.equals(UpdateRequestHandler.COMMIT)) {
+                CommitUpdateCommand cmd = new CommitUpdateCommand(req, false);
+                cmd.waitSearcher = true;
+                parseCommitOptions(cmd);
+                processor.processCommit(cmd);
+              } else if (v.equals(UpdateRequestHandler.OPTIMIZE)) {
+                CommitUpdateCommand cmd = new CommitUpdateCommand(req, true);
+                cmd.waitSearcher = true;
+                parseCommitOptions(cmd);
+                processor.processCommit(cmd);
+              } else if (v.equals(UpdateRequestHandler.DELETE)) {
+                handleDeleteCommand();
+              } else if (v.equals(UpdateRequestHandler.ROLLBACK)) {
+                processor.processRollback(parseRollback());
               } else {
-                assertEvent(ev2, JSONParser.OBJECT_START);
+                throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown command '" + v + "' at [" + parser.getPosition() + "]");
               }
+              break;
             }
-            else if( v.equals( UpdateRequestHandler.COMMIT ) ) {
-              CommitUpdateCommand cmd = new CommitUpdateCommand(req,  false );
-              cmd.waitSearcher = true;
-              parseCommitOptions( cmd );
-              processor.processCommit( cmd );
-            }
-            else if( v.equals( UpdateRequestHandler.OPTIMIZE ) ) {
-              CommitUpdateCommand cmd = new CommitUpdateCommand(req, true );
-              cmd.waitSearcher = true;
-              parseCommitOptions( cmd );
-              processor.processCommit( cmd );
-            }
-            else if( v.equals( UpdateRequestHandler.DELETE ) ) {
-              handleDeleteCommand();
-            }
-            else if( v.equals( UpdateRequestHandler.ROLLBACK ) ) {
-              processor.processRollback( parseRollback() );
-            }
-            else {
-              throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown command '"+v+"' at ["+parser.getPosition()+"]" );
-            }
-            break;
-          }
-          // fall through
+            // fall through
 
-        case JSONParser.LONG:
-        case JSONParser.NUMBER:
-        case JSONParser.BIGNUMBER:
-        case JSONParser.BOOLEAN:
-        case JSONParser.NULL:
-          log.info( "Can't have a value here. Unexpected "
-              +JSONParser.getEventString(ev)+" at ["+parser.getPosition()+"]" );
+          case JSONParser.LONG:
+          case JSONParser.NUMBER:
+          case JSONParser.BIGNUMBER:
+          case JSONParser.BOOLEAN:
+          case JSONParser.NULL:
+            log.info("Can't have a value here. Unexpected "
+                + JSONParser.getEventString(ev) + " at [" + parser.getPosition() + "]");
 
-        case JSONParser.OBJECT_START:
-        case JSONParser.OBJECT_END:
-        case JSONParser.ARRAY_END:
-          break;
+          case JSONParser.OBJECT_START:
+          case JSONParser.OBJECT_END:
+          case JSONParser.ARRAY_END:
+            break;
 
-        default:
-          log.info("Noggit UNKNOWN_EVENT_ID: "+ev);
-          break;
+          default:
+            log.info("Noggit UNKNOWN_EVENT_ID: " + ev);
+            break;
         }
         // read the next event
         ev = parser.nextEvent();
@@ -197,7 +191,7 @@ public class JsonLoader extends ContentS
       if (fields == null || fields.length == 0) fields = new String[]{"$FQN:/**"};
       final boolean echo = "true".equals(req.getParams().get("echo"));
       final String srcField = req.getParams().get("srcField");
-      final boolean mapUniqueKeyOnly = req.getParams().getBool("mapUniqueKeyOnly",false);
+      final boolean mapUniqueKeyOnly = req.getParams().getBool("mapUniqueKeyOnly", false);
       if (srcField != null) {
         if (!"/".equals(split))
           throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Raw data can be stored only if split=/");
@@ -227,7 +221,7 @@ public class JsonLoader extends ContentS
             cmd.overwrite = overwrite;
             cmd.solrDoc = new SolrInputDocument();
             for (Map.Entry<String, Object> entry : copy.entrySet()) {
-              cmd.solrDoc.setField(entry.getKey(),entry.getValue());
+              cmd.solrDoc.setField(entry.getKey(), entry.getValue());
             }
             try {
               processor.processAdd(cmd);
@@ -241,23 +235,24 @@ public class JsonLoader extends ContentS
 
     private Map<String, Object> getDocMap(Map<String, Object> record, JSONParser parser, String srcField, boolean mapUniqueKeyOnly) {
       Map result = record;
-      if(srcField != null && parser instanceof RecordingJSONParser){
+      if (srcField != null && parser instanceof RecordingJSONParser) {
         //if srcFIeld specified extract it out first
         result = new LinkedHashMap(record);
         RecordingJSONParser rjp = (RecordingJSONParser) parser;
         result.put(srcField, rjp.getBuf());
         rjp.resetBuf();
       }
-      if(mapUniqueKeyOnly){
+      if (mapUniqueKeyOnly) {
         SchemaField sf = req.getSchema().getUniqueKeyField();
-        if(sf == null) throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No uniqueKey specified in schema");
+        if (sf == null)
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No uniqueKey specified in schema");
         String df = req.getParams().get(CommonParams.DF);
-        if(df == null)throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No 'df' specified in request");
+        if (df == null) throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No 'df' specified in request");
         Map copy = new LinkedHashMap();
         String uniqueField = (String) record.get(sf.getName());
-        if(uniqueField == null) uniqueField = UUID.randomUUID().toString().toLowerCase(Locale.ROOT);
-        copy.put(sf.getName(),uniqueField);
-        if(srcField != null && result.containsKey(srcField)){
+        if (uniqueField == null) uniqueField = UUID.randomUUID().toString().toLowerCase(Locale.ROOT);
+        copy.put(sf.getName(), uniqueField);
+        if (srcField != null && result.containsKey(srcField)) {
           copy.put(srcField, result.remove(srcField));
         }
         copy.put(df, result.values());
@@ -269,27 +264,6 @@ public class JsonLoader extends ContentS
     }
 
 
-
-    /*private void handleStreamingSingleDocs() throws IOException
-    {
-      while( true ) {
-        int ev = parser.nextEvent();
-        if(ev == JSONParser.EOF) return;
-        if(ev == JSONParser.OBJECT_START) {
-          assertEvent(ev, JSONParser.OBJECT_START);
-          AddUpdateCommand cmd = new AddUpdateCommand(req);
-          cmd.commitWithin = commitWithin;
-          cmd.overwrite = overwrite;
-          cmd.solrDoc = parseDoc(ev);
-          processor.processAdd(cmd);
-        } else if(ev == JSONParser.ARRAY_START){
-          handleAdds();
-        } else{
-          throw  new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unexpected event :"+ev);
-        }
-      }
-    }*/
-
     //
     // "delete":"id"
     // "delete":["id1","id2"]
@@ -325,8 +299,8 @@ public class JsonLoader extends ContentS
           return null;
         default:
           throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-              "Expected primitive JSON value but got: "+JSONParser.getEventString( ev  )
-                  +" at ["+parser.getPosition()+"]" );
+              "Expected primitive JSON value but got: " + JSONParser.getEventString(ev)
+                  + " at [" + parser.getPosition() + "]");
       }
     }
 
@@ -345,7 +319,7 @@ public class JsonLoader extends ContentS
 
     void handleDeleteArray(int ev) throws IOException {
       assert ev == JSONParser.ARRAY_START;
-      for (;;) {
+      for (; ; ) {
         ev = parser.nextEvent();
         if (ev == JSONParser.ARRAY_END) return;
         handleSingleDelete(ev);
@@ -358,60 +332,54 @@ public class JsonLoader extends ContentS
       DeleteUpdateCommand cmd = new DeleteUpdateCommand(req);
       cmd.commitWithin = commitWithin;
 
-      while( true ) {
+      while (true) {
         ev = parser.nextEvent();
-        if( ev == JSONParser.STRING ) {
+        if (ev == JSONParser.STRING) {
           String key = parser.getString();
-          if( parser.wasKey() ) {
-            if( "id".equals( key ) ) {
+          if (parser.wasKey()) {
+            if ("id".equals(key)) {
               cmd.setId(getString(parser.nextEvent()));
-            } else if( "query".equals(key) ) {
+            } else if ("query".equals(key)) {
               cmd.setQuery(parser.getString());
-            } else if( "commitWithin".equals(key) ) {
-              cmd.commitWithin = (int)parser.getLong();
-            } else if( "_version_".equals(key) ) {
+            } else if ("commitWithin".equals(key)) {
+              cmd.commitWithin = (int) parser.getLong();
+            } else if ("_version_".equals(key)) {
               cmd.setVersion(parser.getLong());
             } else if ("_route_".equals(key)) {
               cmd.setRoute(parser.getString());
             } else {
-              throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown key '"+key+"' at ["+parser.getPosition()+"]" );
+              throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown key '" + key + "' at [" + parser.getPosition() + "]");
             }
-          }
-          else {
+          } else {
             throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
                 "invalid string: " + key
-                    +" at ["+parser.getPosition()+"]" );
+                    + " at [" + parser.getPosition() + "]");
           }
-        }
-        else if( ev == JSONParser.OBJECT_END ) {
-          if( cmd.getId() == null && cmd.getQuery() == null ) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Missing id or query for delete at ["+parser.getPosition()+"]" );
+        } else if (ev == JSONParser.OBJECT_END) {
+          if (cmd.getId() == null && cmd.getQuery() == null) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Missing id or query for delete at [" + parser.getPosition() + "]");
           }
 
           processor.processDelete(cmd);
           return;
-        }
-        else {
+        } else {
           throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-              "Got: "+JSONParser.getEventString( ev  )
-                  +" at ["+parser.getPosition()+"]" );
+              "Got: " + JSONParser.getEventString(ev)
+                  + " at [" + parser.getPosition() + "]");
         }
       }
     }
 
 
-
-
     RollbackUpdateCommand parseRollback() throws IOException {
-      assertNextEvent( JSONParser.OBJECT_START );
-      assertNextEvent( JSONParser.OBJECT_END );
+      assertNextEvent(JSONParser.OBJECT_START);
+      assertNextEvent(JSONParser.OBJECT_END);
       return new RollbackUpdateCommand(req);
     }
 
-    void parseCommitOptions(CommitUpdateCommand cmd ) throws IOException
-    {
-      assertNextEvent( JSONParser.OBJECT_START );
-      final Map<String,Object> map = (Map)ObjectBuilder.getVal(parser);
+    void parseCommitOptions(CommitUpdateCommand cmd) throws IOException {
+      assertNextEvent(JSONParser.OBJECT_START);
+      final Map<String, Object> map = (Map) ObjectBuilder.getVal(parser);
 
       // SolrParams currently expects string values...
       SolrParams p = new SolrParams() {
@@ -437,65 +405,56 @@ public class JsonLoader extends ContentS
       RequestHandlerUtils.updateCommit(cmd, p);
     }
 
-    AddUpdateCommand parseAdd() throws IOException
-    {
+    AddUpdateCommand parseAdd() throws IOException {
       AddUpdateCommand cmd = new AddUpdateCommand(req);
       cmd.commitWithin = commitWithin;
       cmd.overwrite = overwrite;
 
       float boost = 1.0f;
 
-      while( true ) {
+      while (true) {
         int ev = parser.nextEvent();
-        if( ev == JSONParser.STRING ) {
-          if( parser.wasKey() ) {
+        if (ev == JSONParser.STRING) {
+          if (parser.wasKey()) {
             String key = parser.getString();
-            if( "doc".equals( key ) ) {
-              if( cmd.solrDoc != null ) {
+            if ("doc".equals(key)) {
+              if (cmd.solrDoc != null) {
                 throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Multiple documents in same"
-                    + " add command at ["+parser.getPosition()+"]" );
+                    + " add command at [" + parser.getPosition() + "]");
               }
-              ev = assertNextEvent( JSONParser.OBJECT_START );
-              cmd.solrDoc = parseDoc( ev );
-            }
-            else if( UpdateRequestHandler.OVERWRITE.equals( key ) ) {
+              ev = assertNextEvent(JSONParser.OBJECT_START);
+              cmd.solrDoc = parseDoc(ev);
+            } else if (UpdateRequestHandler.OVERWRITE.equals(key)) {
               cmd.overwrite = parser.getBoolean(); // reads next boolean
+            } else if (UpdateRequestHandler.COMMIT_WITHIN.equals(key)) {
+              cmd.commitWithin = (int) parser.getLong();
+            } else if ("boost".equals(key)) {
+              boost = Float.parseFloat(parser.getNumberChars().toString());
+            } else {
+              throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown key '" + key + "' at [" + parser.getPosition() + "]");
             }
-            else if( UpdateRequestHandler.COMMIT_WITHIN.equals( key ) ) {
-              cmd.commitWithin = (int)parser.getLong();
-            }
-            else if( "boost".equals( key ) ) {
-              boost = Float.parseFloat( parser.getNumberChars().toString() );
-            }
-            else {
-              throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown key '"+key+"' at ["+parser.getPosition()+"]" );
-            }
-          }
-          else {
+          } else {
             throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
                 "Should be a key "
-                +" at ["+parser.getPosition()+"]" );
+                    + " at [" + parser.getPosition() + "]");
           }
-        }
-        else if( ev == JSONParser.OBJECT_END ) {
-          if( cmd.solrDoc == null ) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Missing solr document at ["+parser.getPosition()+"]" );
+        } else if (ev == JSONParser.OBJECT_END) {
+          if (cmd.solrDoc == null) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Missing solr document at [" + parser.getPosition() + "]");
           }
-          cmd.solrDoc.setDocumentBoost( boost );
+          cmd.solrDoc.setDocumentBoost(boost);
           return cmd;
-        }
-        else {
+        } else {
           throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-              "Got: "+JSONParser.getEventString( ev  )
-              +" at ["+parser.getPosition()+"]" );
+              "Got: " + JSONParser.getEventString(ev)
+                  + " at [" + parser.getPosition() + "]");
         }
       }
     }
 
 
-    void handleAdds() throws IOException
-    {
-      while( true ) {
+    void handleAdds() throws IOException {
+      while (true) {
         AddUpdateCommand cmd = new AddUpdateCommand(req);
         cmd.commitWithin = commitWithin;
         cmd.overwrite = overwrite;
@@ -510,19 +469,18 @@ public class JsonLoader extends ContentS
     }
 
 
-    int assertNextEvent(int expected ) throws IOException
-    {
+    int assertNextEvent(int expected) throws IOException {
       int got = parser.nextEvent();
       assertEvent(got, expected);
       return got;
     }
 
     void assertEvent(int ev, int expected) {
-      if( ev != expected ) {
+      if (ev != expected) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-            "Expected: "+JSONParser.getEventString( expected  )
-            +" but got "+JSONParser.getEventString( ev )
-            +" at ["+parser.getPosition()+"]" );
+            "Expected: " + JSONParser.getEventString(expected)
+                + " but got " + JSONParser.getEventString(ev)
+                + " at [" + parser.getPosition() + "]");
       }
     }
 
@@ -531,17 +489,17 @@ public class JsonLoader extends ContentS
       assert ev == JSONParser.OBJECT_START;
 
       SolrInputDocument sdoc = new SolrInputDocument();
-      for (;;) {
+      for (; ; ) {
         ev = parser.nextEvent();
         if (ev == JSONParser.OBJECT_END) {
           return sdoc;
         }
         String fieldName = parser.getString();
 
-        if(fieldName.equals(JsonLoader.CHILD_DOC_KEY)) {
+        if (fieldName.equals(JsonLoader.CHILD_DOC_KEY)) {
           ev = parser.nextEvent();
           assertEvent(ev, JSONParser.ARRAY_START);
-          while( (ev = parser.nextEvent()) != JSONParser.ARRAY_END ) {
+          while ((ev = parser.nextEvent()) != JSONParser.ARRAY_END) {
             assertEvent(ev, JSONParser.OBJECT_START);
 
             sdoc.addChildDocument(parseDoc(ev));
@@ -569,28 +527,28 @@ public class JsonLoader extends ContentS
       }
     }
 
-    private void parseExtendedFieldValue(SolrInputField sif, int ev)  throws IOException {
+    private void parseExtendedFieldValue(SolrInputField sif, int ev) throws IOException {
       assert ev == JSONParser.OBJECT_START;
 
       float boost = 1.0f;
       Object normalFieldValue = null;
       Map<String, Object> extendedInfo = null;
 
-      for (;;) {
+      for (; ; ) {
         ev = parser.nextEvent();
         switch (ev) {
           case JSONParser.STRING:
             String label = parser.getString();
             if ("boost".equals(label)) {
               ev = parser.nextEvent();
-              if( ev != JSONParser.NUMBER &&
+              if (ev != JSONParser.NUMBER &&
                   ev != JSONParser.LONG &&
-                  ev != JSONParser.BIGNUMBER ) {
+                  ev != JSONParser.BIGNUMBER) {
                 throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Boost should have number. "
-                    + "Unexpected "+JSONParser.getEventString(ev)+" at ["+parser.getPosition()+"], field="+sif.getName() );
+                    + "Unexpected " + JSONParser.getEventString(ev) + " at [" + parser.getPosition() + "], field=" + sif.getName());
               }
 
-              boost = (float)parser.getDouble();
+              boost = (float) parser.getDouble();
             } else if ("value".equals(label)) {
               normalFieldValue = parseNormalFieldValue(parser.nextEvent(), sif.getName());
             } else {
@@ -608,7 +566,7 @@ public class JsonLoader extends ContentS
           case JSONParser.OBJECT_END:
             if (extendedInfo != null) {
               if (normalFieldValue != null) {
-                extendedInfo.put("value",normalFieldValue);
+                extendedInfo.put("value", normalFieldValue);
               }
               sif.setValue(extendedInfo, boost);
             } else {
@@ -618,7 +576,7 @@ public class JsonLoader extends ContentS
 
           default:
             throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Error parsing JSON extended field value. "
-                + "Unexpected "+JSONParser.getEventString(ev)+" at ["+parser.getPosition()+"], field="+sif.getName() );
+                + "Unexpected " + JSONParser.getEventString(ev) + " at [" + parser.getPosition() + "], field=" + sif.getName());
         }
       }
     }
@@ -654,7 +612,7 @@ public class JsonLoader extends ContentS
           return parseArrayFieldValue(ev, fieldName);
         default:
           throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Error parsing JSON field value. "
-              + "Unexpected "+JSONParser.getEventString(ev)+" at ["+parser.getPosition()+"], field="+fieldName );
+              + "Unexpected " + JSONParser.getEventString(ev) + " at [" + parser.getPosition() + "], field=" + fieldName);
       }
     }
 
@@ -663,7 +621,7 @@ public class JsonLoader extends ContentS
       assert ev == JSONParser.ARRAY_START;
 
       ArrayList lst = new ArrayList(2);
-      for (;;) {
+      for (; ; ) {
         ev = parser.nextEvent();
         if (ev == JSONParser.ARRAY_END) {
           return lst;

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/CharStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/CharStream.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/CharStream.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/CharStream.java Tue Mar 31 05:22:40 2015
@@ -27,6 +27,22 @@ 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).
@@ -96,4 +112,4 @@ interface CharStream {
   void Done();
 
 }
-/* JavaCC - OriginalChecksum=a81c9280a3ec4578458c607a9d95acb4 (do not edit this line) */
+/* JavaCC - OriginalChecksum=48b70e7c01825c8f301c7362bf1028d8 (do not edit this line) */

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/FastCharStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/FastCharStream.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/FastCharStream.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/FastCharStream.java Tue Mar 31 05:22:40 2015
@@ -108,6 +108,15 @@ 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/lucene6271/solr/core/src/java/org/apache/solr/parser/ParseException.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/ParseException.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/ParseException.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/ParseException.java Tue Mar 31 05:22:40 2015
@@ -184,4 +184,4 @@ public class ParseException extends Exce
    }
 
 }
-/* JavaCC - OriginalChecksum=d7aa203ee92ebbb23011a23311e60537 (do not edit this line) */
+/* JavaCC - OriginalChecksum=25e1ae9ad9614c4ce31c4b83f8a7397b (do not edit this line) */

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/QueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/QueryParser.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/QueryParser.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/QueryParser.java Tue Mar 31 05:22:40 2015
@@ -100,7 +100,7 @@ public class QueryParser extends SolrQue
   }
 
   final public Query Query(String field) throws ParseException, SyntaxError {
-  List<BooleanClause> clauses = new ArrayList<>();
+  List<BooleanClause> clauses = new ArrayList<BooleanClause>();
   Query q, firstQuery=null;
   int conj, mods;
     mods = Modifiers();
@@ -581,7 +581,7 @@ public class QueryParser extends SolrQue
       return (jj_ntk = jj_nt.kind);
   }
 
-  private java.util.List<int[]> jj_expentries = new java.util.ArrayList<>();
+  private java.util.List<int[]> jj_expentries = new java.util.ArrayList<int[]>();
   private int[] jj_expentry;
   private int jj_kind = -1;
   private int[] jj_lasttokens = new int[100];

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/QueryParser.jj
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/QueryParser.jj?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/QueryParser.jj (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/QueryParser.jj Tue Mar 31 05:22:40 2015
@@ -103,7 +103,7 @@ PARSER_END(QueryParser)
 }
 
 <Boost> TOKEN : {
- <NUMBER:    ("-")? (<_NUM_CHAR>)+ ( "." (<_NUM_CHAR>)+ )? > : DEFAULT
+ <NUMBER:    ("=")?("-")? (<_NUM_CHAR>)+ ( "." (<_NUM_CHAR>)+ )? > : DEFAULT
 }
 
 <Range> TOKEN : {

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/QueryParserTokenManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/QueryParserTokenManager.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/QueryParserTokenManager.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/QueryParserTokenManager.java Tue Mar 31 05:22:40 2015
@@ -692,7 +692,7 @@ private int jjMoveStringLiteralDfa0_0()
 private int jjMoveNfa_0(int startState, int curPos)
 {
    int startsAt = 0;
-   jjnewStateCnt = 4;
+   jjnewStateCnt = 5;
    int i = 1;
    jjstateSet[0] = startState;
    int kind = 0x7fffffff;
@@ -712,28 +712,34 @@ private int jjMoveNfa_0(int startState,
                   {
                      if (kind > 29)
                         kind = 29;
-                     jjCheckNAddTwoStates(1, 2);
+                     jjCheckNAddTwoStates(2, 3);
                   }
                   else if (curChar == 45)
-                     jjCheckNAdd(1);
+                     jjCheckNAdd(2);
+                  else if (curChar == 61)
+                     jjCheckNAddTwoStates(1, 2);
                   break;
                case 1:
+                  if (curChar == 45)
+                     jjCheckNAdd(2);
+                  break;
+               case 2:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
                   if (kind > 29)
                      kind = 29;
-                  jjCheckNAddTwoStates(1, 2);
+                  jjCheckNAddTwoStates(2, 3);
                   break;
-               case 2:
+               case 3:
                   if (curChar == 46)
-                     jjCheckNAdd(3);
+                     jjCheckNAdd(4);
                   break;
-               case 3:
+               case 4:
                   if ((0x3ff000000000000L & l) == 0L)
                      break;
                   if (kind > 29)
                      kind = 29;
-                  jjCheckNAdd(3);
+                  jjCheckNAdd(4);
                   break;
                default : break;
             }
@@ -772,7 +778,7 @@ private int jjMoveNfa_0(int startState,
          kind = 0x7fffffff;
       }
       ++curPos;
-      if ((i = jjnewStateCnt) == (startsAt = 4 - (jjnewStateCnt = startsAt)))
+      if ((i = jjnewStateCnt) == (startsAt = 5 - (jjnewStateCnt = startsAt)))
          return curPos;
       try { curChar = input_stream.readChar(); }
       catch(java.io.IOException e) { return curPos; }

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java Tue Mar 31 05:22:40 2015
@@ -29,6 +29,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.AutomatonQuery;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.ConstantScoreQuery;
 import org.apache.lucene.search.FuzzyQuery;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.MultiPhraseQuery;
@@ -78,7 +79,7 @@ public abstract class SolrQueryParserBas
   /** The default operator that parser uses to combine query terms */
   Operator operator = OR_OPERATOR;
 
-  MultiTermQuery.RewriteMethod multiTermRewriteMethod = MultiTermQuery.CONSTANT_SCORE_FILTER_REWRITE;
+  MultiTermQuery.RewriteMethod multiTermRewriteMethod = MultiTermQuery.CONSTANT_SCORE_REWRITE;
   boolean allowLeadingWildcard = true;
 
   String defaultField;
@@ -290,7 +291,7 @@ public abstract class SolrQueryParserBas
 
 
   /**
-   * By default QueryParser uses {@link org.apache.lucene.search.MultiTermQuery#CONSTANT_SCORE_FILTER_REWRITE}
+   * By default QueryParser uses {@link org.apache.lucene.search.MultiTermQuery#CONSTANT_SCORE_REWRITE}
    * when creating a PrefixQuery, WildcardQuery or RangeQuery. This implementation is generally preferable because it
    * a) Runs faster b) Does not have the scarcity of terms unduly influence score
    * c) avoids any "TooManyBooleanClauses" exception.
@@ -552,13 +553,28 @@ public abstract class SolrQueryParserBas
 
   // called from parser
   Query handleBoost(Query q, Token boost) {
-    if (boost != null) {
-      float boostVal = Float.parseFloat(boost.image);
-      // avoid boosting null queries, such as those caused by stop words
-      if (q != null) {
-        q.setBoost(q.getBoost() * boostVal);
+    // q==null check is to avoid boosting null queries, such as those caused by stop words
+    if (boost == null || boost.image.length()==0 || q == null) {
+      return q;
+    }
+
+    if (boost.image.charAt(0) == '=') {
+      // syntax looks like foo:x^=3
+      float val = Float.parseFloat(boost.image.substring(1));
+      Query newQ = q;
+      if (// q instanceof FilterQuery ||  // TODO: fix this when FilterQuery is introduced to avoid needless wrapping: SOLR-7219
+          q instanceof ConstantScoreQuery) {
+        newQ.setBoost(val);
+      } else {
+        newQ = new ConstantScoreQuery(q);
+        newQ.setBoost(val);
       }
+      return newQ;
     }
+
+    float boostVal = Float.parseFloat(boost.image);
+    q.setBoost(q.getBoost() * boostVal);
+
     return q;
   }
 

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/Token.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/Token.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/Token.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/Token.java Tue Mar 31 05:22:40 2015
@@ -128,4 +128,4 @@ public class Token implements java.io.Se
   }
 
 }
-/* JavaCC - OriginalChecksum=9036cc9068ac33d9c79403fe5349f705 (do not edit this line) */
+/* JavaCC - OriginalChecksum=f463ad6fd3205ca07166de02ee86b907 (do not edit this line) */

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/TokenMgrError.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/TokenMgrError.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/TokenMgrError.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/parser/TokenMgrError.java Tue Mar 31 05:22:40 2015
@@ -144,4 +144,4 @@ public class TokenMgrError extends Error
     this(LexicalError(EOFSeen, lexState, errorLine, errorColumn, errorAfter, curChar), reason);
   }
 }
-/* JavaCC - OriginalChecksum=59744072870f5c27fc8796947fc3b06f (do not edit this line) */
+/* JavaCC - OriginalChecksum=200a46f65c1a0f71a7f037b35f4e934e (do not edit this line) */

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java Tue Mar 31 05:22:40 2015
@@ -34,6 +34,7 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.solr.common.params.FacetParams;
 import org.apache.solr.common.util.NamedList;
@@ -58,7 +59,7 @@ import org.apache.solr.util.LongPriority
 public class DocValuesFacets {
   private DocValuesFacets() {}
   
-  public static NamedList<Integer> getCounts(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort, String prefix) throws IOException {
+  public static NamedList<Integer> getCounts(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort, String prefix, String contains, boolean ignoreCase) throws IOException {
     SchemaField schemaField = searcher.getSchema().getField(fieldName);
     FieldType ft = schemaField.getType();
     NamedList<Integer> res = new NamedList<>();
@@ -97,6 +98,8 @@ public class DocValuesFacets {
       prefixRef = new BytesRefBuilder();
       prefixRef.copyChars(prefix);
     }
+    
+    final BytesRef containsBR = contains != null ? new BytesRef(contains) : null;
 
     int startTermIndex, endTermIndex;
     if (prefix!=null) {
@@ -170,6 +173,12 @@ public class DocValuesFacets {
         int min=mincount-1;  // the smallest value in the top 'N' values
         for (int i=(startTermIndex==-1)?1:0; i<nTerms; i++) {
           int c = counts[i];
+          if (containsBR != null) {
+            final BytesRef term = si.lookupOrd(startTermIndex+i);
+            if (!StringHelper.contains(term, containsBR, ignoreCase)) {
+              continue;
+            }
+          }
           if (c>min) {
             // NOTE: we use c>min rather than c>=min as an optimization because we are going in
             // index order, so we already know that the keys are ordered.  This can be very
@@ -203,18 +212,28 @@ public class DocValuesFacets {
       } else {
         // add results in index order
         int i=(startTermIndex==-1)?1:0;
-        if (mincount<=0) {
-          // if mincount<=0, then we won't discard any terms and we know exactly
-          // where to start.
+        if (mincount<=0 && containsBR == null) {
+          // if mincount<=0 and we're not examining the values for contains, then
+          // we won't discard any terms and we know exactly where to start.
           i+=off;
           off=0;
         }
 
         for (; i<nTerms; i++) {          
           int c = counts[i];
-          if (c<mincount || --off>=0) continue;
+          if (c<mincount) continue;
+          BytesRef term = null;
+          if (containsBR != null) {
+            term = si.lookupOrd(startTermIndex+i);
+            if (!StringHelper.contains(term, containsBR, ignoreCase)) {
+              continue;
+            }
+          }
+          if (--off>=0) continue;
           if (--lim<0) break;
-          final BytesRef term = si.lookupOrd(startTermIndex+i);
+          if (term == null) {
+            term = si.lookupOrd(startTermIndex+i);
+          }
           ft.indexedToReadable(term, charsRef);
           res.add(charsRef.toString(), c);
         }

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/DocValuesStats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/DocValuesStats.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/DocValuesStats.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/request/DocValuesStats.java Tue Mar 31 05:22:40 2015
@@ -56,7 +56,6 @@ public class DocValuesStats {
   public static StatsValues getCounts(SolrIndexSearcher searcher, StatsField statsField, DocSet docs, String[] facet) throws IOException {
 
     final SchemaField schemaField = statsField.getSchemaField(); 
-    final boolean calcDistinct = statsField.getCalcDistinct();
 
     assert null != statsField.getSchemaField()
       : "DocValuesStats requires a StatsField using a SchemaField";