You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2011/05/07 10:37:02 UTC

svn commit: r1100480 [2/2] - in /lucene/dev/branches/branch_3x: lucene/ lucene/docs/ lucene/docs/lucene-contrib/ lucene/docs/skin/images/ lucene/src/java/org/apache/lucene/document/ lucene/src/java/org/apache/lucene/index/ lucene/src/site/src/documenta...

Modified: lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/schema/TrieDateField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/schema/TrieDateField.java?rev=1100480&r1=1100479&r2=1100480&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/schema/TrieDateField.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/schema/TrieDateField.java Sat May  7 08:37:00 2011
@@ -17,191 +17,119 @@
 
 package org.apache.solr.schema;
 
-import org.apache.solr.common.SolrException;
-import org.apache.solr.analysis.CharFilterFactory;
-import org.apache.solr.analysis.TokenFilterFactory;
-import org.apache.solr.analysis.TokenizerChain;
-import org.apache.solr.analysis.TrieTokenizerFactory;
-import org.apache.solr.search.function.*;
+import org.apache.solr.search.function.ValueSource;
 import org.apache.solr.search.QParser;
 import org.apache.solr.response.TextResponseWriter;
 import org.apache.solr.response.XMLWriter;
 import org.apache.lucene.document.Fieldable;
-import org.apache.lucene.document.Field;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.NumericRangeQuery;
-import org.apache.lucene.util.NumericUtils;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.NumericTokenStream;
 
 import java.util.Map;
 import java.util.Date;
 import java.io.IOException;
 
 public class TrieDateField extends DateField {
-  protected int precisionStepArg = TrieField.DEFAULT_PRECISION_STEP;  // the one passed in or defaulted
-  protected int precisionStep = precisionStepArg;     // normalized
+
+  final TrieField wrappedField = new TrieField() {{
+    type = TrieTypes.DATE;
+  }};
 
   @Override
   protected void init(IndexSchema schema, Map<String, String> args) {
-    String p = args.remove("precisionStep");
-    if (p != null) {
-       precisionStepArg = Integer.parseInt(p);
-    }
-    // normalize the precisionStep
-    precisionStep = precisionStepArg;
-    if (precisionStep<=0 || precisionStep>=64) precisionStep=Integer.MAX_VALUE;
-
-    CharFilterFactory[] filterFactories = new CharFilterFactory[0];
-    TokenFilterFactory[] tokenFilterFactories = new TokenFilterFactory[0];
-    analyzer = new TokenizerChain(filterFactories, new TrieTokenizerFactory(TrieField.TrieTypes.DATE, precisionStep), tokenFilterFactories);
-    // for query time we only need one token, so we use the biggest possible precisionStep:
-    queryAnalyzer = new TokenizerChain(filterFactories, new TrieTokenizerFactory(TrieField.TrieTypes.DATE, Integer.MAX_VALUE), tokenFilterFactories);
+    wrappedField.init(schema, args);
+    analyzer = wrappedField.analyzer;
+    queryAnalyzer = wrappedField.queryAnalyzer;
   }
 
   @Override
   public Date toObject(Fieldable f) {
-    byte[] arr = f.getBinaryValue();
-    if (arr==null) throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,TrieField.badFieldString(f));
-    return new Date(TrieField.toLong(arr));
+    return (Date) wrappedField.toObject(f);
   }
 
   @Override
   public SortField getSortField(SchemaField field, boolean top) {
-    field.checkSortability();
-    return new SortField(field.getName(), FieldCache.NUMERIC_UTILS_LONG_PARSER, top);
+    return wrappedField.getSortField(field, top);
   }
 
   @Override
   public ValueSource getValueSource(SchemaField field, QParser parser) {
-    field.checkFieldCacheSource(parser);
-    return new TrieDateFieldSource(field.getName(), FieldCache.NUMERIC_UTILS_LONG_PARSER);
+    return wrappedField.getValueSource(field, parser);
+  }
 
+  /**
+   * @return the precisionStep used to index values into the field
+   */
+  public int getPrecisionStep() {
+    return wrappedField.getPrecisionStep();
   }
 
   @Override
   public void write(XMLWriter xmlWriter, String name, Fieldable f) throws IOException {
-    byte[] arr = f.getBinaryValue();
-    if (arr==null) {
-      xmlWriter.writeStr(name, TrieField.badFieldString(f));
-      return;
-    }
-
-    xmlWriter.writeDate(name,new Date(TrieField.toLong(arr)));
+    wrappedField.write(xmlWriter, name, f);
   }
 
   @Override
   public void write(TextResponseWriter writer, String name, Fieldable f) throws IOException {
-    byte[] arr = f.getBinaryValue();
-    if (arr==null) {
-      writer.writeStr(name, TrieField.badFieldString(f),true);
-      return;
-    }
-
-    writer.writeDate(name,new Date(TrieField.toLong(arr)));
+    wrappedField.write(writer, name, f);
   }
 
   @Override
   public boolean isTokenized() {
-    return true;
+    return wrappedField.isTokenized();
   }
 
-  /**
-   * @return the precisionStep used to index values into the field
-   */
-  public int getPrecisionStep() {
-    return precisionStepArg;
+  @Override
+  public boolean multiValuedFieldCache() {
+    return wrappedField.multiValuedFieldCache();
   }
 
-
-
   @Override
   public String storedToReadable(Fieldable f) {
-    return toExternal(f);
+    return wrappedField.storedToReadable(f);
   }
 
   @Override
   public String readableToIndexed(String val) {  
-    return NumericUtils.longToPrefixCoded(super.parseMath(null, val).getTime());
+    return wrappedField.readableToIndexed(val);
   }
 
   @Override
   public String toInternal(String val) {
-    return readableToIndexed(val);
+    return wrappedField.toInternal(val);
   }
 
   @Override
   public String toExternal(Fieldable f) {
-    byte[] arr = f.getBinaryValue();
-    if (arr==null) return TrieField.badFieldString(f);
-     return super.toExternal(new Date(TrieField.toLong(arr)));
+    return wrappedField.toExternal(f);
   }
 
   @Override
   public String indexedToReadable(String indexedForm) {
-    return super.toExternal( new Date(NumericUtils.prefixCodedToLong(indexedForm)) );
+    return wrappedField.indexedToReadable(indexedForm);
   }
 
   @Override
   public String storedToIndexed(Fieldable f) {
-    // TODO: optimize to remove redundant string conversion
-    return readableToIndexed(storedToReadable(f));
+    return wrappedField.storedToIndexed(f);
   }
 
   @Override
-  public Field createField(SchemaField field, String externalVal, float boost) {
-    boolean indexed = field.indexed();
-    boolean stored = field.stored();
-
-    if (!indexed && !stored) {
-      if (log.isTraceEnabled())
-        log.trace("Ignoring unindexed/unstored field: " + field);
-      return null;
-    }
-
-    int ps = precisionStep;
-
-    byte[] arr=null;
-    TokenStream ts=null;
-
-    long time = super.parseMath(null, externalVal).getTime();
-    if (stored) arr = TrieField.toArr(time);
-    if (indexed) ts = new NumericTokenStream(ps).setLongValue(time);
-
-    Field f;
-    if (stored) {
-      f = new Field(field.getName(), arr);
-      if (indexed) f.setTokenStream(ts);
-    } else {
-      f = new Field(field.getName(), ts);
-    }
-
-    // term vectors aren't supported
-
-    f.setOmitNorms(field.omitNorms());
-    f.setOmitTermFreqAndPositions(field.omitTf());
-    f.setBoost(boost);
-    return f;
+  public Fieldable createField(SchemaField field, String externalVal, float boost) {
+    return wrappedField.createField(field, externalVal, boost);
   }
 
   @Override
   public Query getRangeQuery(QParser parser, SchemaField field, String min, String max, boolean minInclusive, boolean maxInclusive) {
-    return getRangeQuery(parser, field,
-            min==null ? null : super.parseMath(null,min),
-            max==null ? null : super.parseMath(null,max),
-            minInclusive, maxInclusive);
+    return wrappedField.getRangeQuery(parser, field, min, max, minInclusive, maxInclusive);
   }
   
   @Override
   public Query getRangeQuery(QParser parser, SchemaField sf, Date min, Date max, boolean minInclusive, boolean maxInclusive) {
-    int ps = precisionStep;
-    Query query = NumericRangeQuery.newLongRange(sf.getName(), ps,
+    return NumericRangeQuery.newLongRange(sf.getName(), wrappedField.precisionStep,
               min == null ? null : min.getTime(),
               max == null ? null : max.getTime(),
               minInclusive, maxInclusive);
-
-    return query;
   }
 }

Modified: lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/schema/TrieField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/schema/TrieField.java?rev=1100480&r1=1100479&r2=1100480&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/schema/TrieField.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/schema/TrieField.java Sat May  7 08:37:00 2011
@@ -18,6 +18,7 @@ package org.apache.solr.schema;
 
 import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericField;
 import org.apache.lucene.search.*;
 import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.analysis.TokenStream;
@@ -35,17 +36,17 @@ import java.util.Map;
 import java.util.Date;
 
 /**
- * Provides field types to support for Lucene's Trie Range Queries.
+ * Provides field types to support for Lucene's {@link NumericField}.
  * See {@link org.apache.lucene.search.NumericRangeQuery} for more details.
  * It supports integer, float, long, double and date types.
  * <p/>
  * For each number being added to this field, multiple terms are generated as per the algorithm described in the above
- * link. The possible number of terms increases dramatically with higher precision steps (factor 2^precisionStep). For
+ * link. The possible number of terms increases dramatically with lower precision steps. For
  * the fast range search to work, trie fields must be indexed.
  * <p/>
  * Trie fields are sortable in numerical order and can be used in function queries.
  * <p/>
- * Note that if you use a precisionStep of 32 for int/float and 64 for long/double, then multiple terms will not be
+ * Note that if you use a precisionStep of 32 for int/float and 64 for long/double/date, then multiple terms will not be
  * generated, range search will be no faster than any other number field, but sorting will still be possible.
  *
  * @version $Id$
@@ -94,21 +95,28 @@ public class TrieField extends FieldType
 
   @Override
   public Object toObject(Fieldable f) {
-    byte[] arr = f.getBinaryValue();
-    if (arr==null) return badFieldString(f);
-    switch (type) {
-      case INTEGER:
-        return toInt(arr);
-      case FLOAT:
-        return toFloat(arr);
-      case LONG:
-        return toLong(arr);
-      case DOUBLE:
-        return toDouble(arr);
-      case DATE:
-        return new Date(toLong(arr));
-      default:
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown type for trie field: " + f.name());
+    if (f instanceof NumericField) {
+      final Number val = ((NumericField) f).getNumericValue();
+      if (val==null) return badFieldString(f);
+      return (type == TrieTypes.DATE) ? new Date(val.longValue()) : val;
+    } else {
+      // the following code is "deprecated" and only to support pre-3.2 indexes using the old BinaryField encoding:
+      final byte[] arr = f.getBinaryValue();
+      if (arr==null) return badFieldString(f);
+      switch (type) {
+        case INTEGER:
+          return toInt(arr);
+        case FLOAT:
+          return Float.intBitsToFloat(toInt(arr));
+        case LONG:
+          return toLong(arr);
+        case DOUBLE:
+          return Double.longBitsToDouble(toLong(arr));
+        case DATE:
+          return new Date(toLong(arr));
+        default:
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown type for trie field: " + f.name());
+      }
     }
   }
 
@@ -152,59 +160,12 @@ public class TrieField extends FieldType
 
   @Override
   public void write(XMLWriter xmlWriter, String name, Fieldable f) throws IOException {
-    byte[] arr = f.getBinaryValue();
-    if (arr==null) {
-      xmlWriter.writeStr(name, badFieldString(f));
-      return;
-    }
-
-    switch (type) {
-      case INTEGER:
-        xmlWriter.writeInt(name,toInt(arr));
-        break;
-      case FLOAT:
-        xmlWriter.writeFloat(name,toFloat(arr));
-        break;
-      case LONG:
-        xmlWriter.writeLong(name,toLong(arr));
-        break;
-      case DOUBLE:
-        xmlWriter.writeDouble(name,toDouble(arr));
-        break;
-      case DATE:
-        xmlWriter.writeDate(name,new Date(toLong(arr)));
-        break;
-      default:
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown type for trie field: " + f.name());
-    }
+    xmlWriter.writeVal(name, toObject(f));
   }
 
   @Override
   public void write(TextResponseWriter writer, String name, Fieldable f) throws IOException {
-    byte[] arr = f.getBinaryValue();
-    if (arr==null) {
-      writer.writeStr(name, badFieldString(f),true);
-      return;
-    }
-    switch (type) {
-      case INTEGER:
-        writer.writeInt(name,toInt(arr));
-        break;
-      case FLOAT:
-        writer.writeFloat(name,toFloat(arr));
-        break;
-      case LONG:
-        writer.writeLong(name,toLong(arr));
-        break;
-      case DOUBLE:
-        writer.writeDouble(name,toDouble(arr));
-        break;
-      case DATE:
-        writer.writeDate(name,new Date(toLong(arr)));
-        break;
-      default:
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown type for trie field: " + f.name());
-    }
+    writer.writeVal(name, toObject(f));
   }
 
   @Override
@@ -273,56 +234,18 @@ public class TrieField extends FieldType
     return query;
   }
 
-
+  @Deprecated
   static int toInt(byte[] arr) {
     return (arr[0]<<24) | ((arr[1]&0xff)<<16) | ((arr[2]&0xff)<<8) | (arr[3]&0xff);
   }
   
+  @Deprecated
   static long toLong(byte[] arr) {
     int high = (arr[0]<<24) | ((arr[1]&0xff)<<16) | ((arr[2]&0xff)<<8) | (arr[3]&0xff);
     int low = (arr[4]<<24) | ((arr[5]&0xff)<<16) | ((arr[6]&0xff)<<8) | (arr[7]&0xff);
     return (((long)high)<<32) | (low&0x0ffffffffL);
   }
 
-  static float toFloat(byte[] arr) {
-    return Float.intBitsToFloat(toInt(arr));
-  }
-
-  static double toDouble(byte[] arr) {
-    return Double.longBitsToDouble(toLong(arr));
-  }
-
-  static byte[] toArr(int val) {
-    byte[] arr = new byte[4];
-    arr[0] = (byte)(val>>>24);
-    arr[1] = (byte)(val>>>16);
-    arr[2] = (byte)(val>>>8);
-    arr[3] = (byte)(val);
-    return arr;
-  }
-
-  static byte[] toArr(long val) {
-    byte[] arr = new byte[8];
-    arr[0] = (byte)(val>>>56);
-    arr[1] = (byte)(val>>>48);
-    arr[2] = (byte)(val>>>40);
-    arr[3] = (byte)(val>>>32);
-    arr[4] = (byte)(val>>>24);
-    arr[5] = (byte)(val>>>16);
-    arr[6] = (byte)(val>>>8);
-    arr[7] = (byte)(val);
-    return arr;
-  }
-
-  static byte[] toArr(float val) {
-    return toArr(Float.floatToRawIntBits(val));
-  }
-
-  static byte[] toArr(double val) {
-    return toArr(Double.doubleToRawLongBits(val));
-  }
-
-
   @Override
   public String storedToReadable(Fieldable f) {
     return toExternal(f);
@@ -346,7 +269,6 @@ public class TrieField extends FieldType
     }
   }
 
-
   @Override
   public String toInternal(String val) {
     return readableToIndexed(val);
@@ -360,22 +282,9 @@ public class TrieField extends FieldType
 
   @Override
   public String toExternal(Fieldable f) {
-    byte[] arr = f.getBinaryValue();
-    if (arr==null) return badFieldString(f);
-    switch (type) {
-      case INTEGER:
-        return Integer.toString(toInt(arr));
-      case FLOAT:
-        return Float.toString(toFloat(arr));
-      case LONG:
-        return Long.toString(toLong(arr));
-      case DOUBLE:
-        return Double.toString(toDouble(arr));
-      case DATE:
-        return dateField.formatDate(new Date(toLong(arr)));
-      default:
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown type for trie field: " + f.name());
-    }
+    return (type == TrieTypes.DATE)
+      ? dateField.toExternal((Date) toObject(f)) 
+      : toObject(f).toString();
   }
 
   @Override
@@ -390,7 +299,7 @@ public class TrieField extends FieldType
       case DOUBLE:
         return Double.toString( NumericUtils.sortableLongToDouble(NumericUtils.prefixCodedToLong(indexedForm)) );
       case DATE:
-        return dateField.formatDate( new Date(NumericUtils.prefixCodedToLong(indexedForm)) );
+        return dateField.toExternal( new Date(NumericUtils.prefixCodedToLong(indexedForm)) );
       default:
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown type for trie field: " + type);
     }
@@ -398,12 +307,58 @@ public class TrieField extends FieldType
 
   @Override
   public String storedToIndexed(Fieldable f) {
-    // TODO: optimize to remove redundant string conversion
-    return readableToIndexed(storedToReadable(f));
+    if (f instanceof NumericField) {
+      final Number val = ((NumericField) f).getNumericValue();
+      if (val==null)
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Invalid field contents: "+f.name());
+      switch (type) {
+        case INTEGER:
+          return NumericUtils.intToPrefixCoded(val.intValue());
+        case FLOAT:
+          return NumericUtils.intToPrefixCoded(NumericUtils.floatToSortableInt(val.floatValue()));
+        case LONG: //fallthrough!
+        case DATE:
+          return NumericUtils.longToPrefixCoded(val.longValue());
+        case DOUBLE:
+          return NumericUtils.longToPrefixCoded(NumericUtils.doubleToSortableLong(val.doubleValue()));
+        default:
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown type for trie field: " + f.name());
+      }
+    } else {
+      // the following code is "deprecated" and only to support pre-3.2 indexes using the old BinaryField encoding:
+      final byte[] arr = f.getBinaryValue();
+      if (arr==null)
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Invalid field contents: "+f.name());
+      switch (type) {
+        case INTEGER:
+          return NumericUtils.intToPrefixCoded(toInt(arr));
+        case FLOAT: {
+          // WARNING: Code Duplication! Keep in sync with o.a.l.util.NumericUtils!
+          // copied from NumericUtils to not convert to/from float two times
+          // code in next 2 lines is identical to: int v = NumericUtils.floatToSortableInt(Float.intBitsToFloat(toInt(arr)));
+          int v = toInt(arr);
+          if (v<0) v ^= 0x7fffffff;
+          return NumericUtils.intToPrefixCoded(v);
+        }
+        case LONG: //fallthrough!
+        case DATE:
+          return NumericUtils.longToPrefixCoded(toLong(arr));
+        case DOUBLE: {
+          // WARNING: Code Duplication! Keep in sync with o.a.l.util.NumericUtils!
+          // copied from NumericUtils to not convert to/from double two times
+          // code in next 2 lines is identical to: long v = NumericUtils.doubleToSortableLong(Double.longBitsToDouble(toLong(arr)));
+          long v = toLong(arr);
+          if (v<0) v ^= 0x7fffffffffffffffL;
+          return NumericUtils.longToPrefixCoded(v);
+        }
+        default:
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown type for trie field: " + f.name());
+      }
+    }
   }
 
   @Override
-  public Field createField(SchemaField field, String externalVal, float boost) {
+  public Fieldable createField(SchemaField field, String externalVal, float boost) {
     boolean indexed = field.indexed();
     boolean stored = field.stored();
 
@@ -413,52 +368,27 @@ public class TrieField extends FieldType
       return null;
     }
 
-    int ps = precisionStep;
-
-    byte[] arr=null;
-    TokenStream ts=null;
-    // String indexedVal = indexed && precisionStep==0 ? readableToIndexed(externalVal) : null;
-
+    final NumericField f = new NumericField(field.getName(), precisionStep, stored ? Field.Store.YES : Field.Store.NO, indexed);
     switch (type) {
       case INTEGER:
-        int i = Integer.parseInt(externalVal);
-        if (stored) arr = toArr(i);
-        if (indexed) ts = new NumericTokenStream(ps).setIntValue(i);
+        f.setIntValue(Integer.parseInt(externalVal));
         break;
       case FLOAT:
-        float f = Float.parseFloat(externalVal);
-        if (stored) arr = toArr(f);
-        if (indexed) ts = new NumericTokenStream(ps).setFloatValue(f);
+        f.setFloatValue(Float.parseFloat(externalVal));
         break;
       case LONG:
-        long l = Long.parseLong(externalVal);
-        if (stored) arr = toArr(l);
-        if (indexed) ts = new NumericTokenStream(ps).setLongValue(l);
+        f.setLongValue(Long.parseLong(externalVal));
         break;
       case DOUBLE:
-        double d = Double.parseDouble(externalVal);
-        if (stored) arr = toArr(d);
-        if (indexed) ts = new NumericTokenStream(ps).setDoubleValue(d);
+        f.setDoubleValue(Double.parseDouble(externalVal));
         break;
       case DATE:
-        long time = dateField.parseMath(null, externalVal).getTime();
-        if (stored) arr = toArr(time);
-        if (indexed) ts = new NumericTokenStream(ps).setLongValue(time);
+        f.setLongValue(dateField.parseMath(null, externalVal).getTime());
         break;
       default:
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown type for trie field: " + type);
     }
 
-    Field f;
-    if (stored) {
-      f = new Field(field.getName(), arr);
-      if (indexed) f.setTokenStream(ts);
-    } else {
-      f = new Field(field.getName(), ts);
-    }
-
-    // term vectors aren't supported
-
     f.setOmitNorms(field.omitNorms());
     f.setOmitTermFreqAndPositions(field.omitTf());
     f.setBoost(boost);
@@ -482,14 +412,12 @@ public class TrieField extends FieldType
    * that indexes multiple precisions per value.
    */
   public static String getMainValuePrefix(FieldType ft) {
-    if (ft instanceof TrieDateField) {
-      int step = ((TrieDateField)ft).getPrecisionStep();
-      if (step <= 0 || step >=64) return null;
-      return LONG_PREFIX;
-    } else if (ft instanceof TrieField) {
-      TrieField trie = (TrieField)ft;
-      if (trie.precisionStep  == Integer.MAX_VALUE) return null;
-
+    if (ft instanceof TrieDateField)
+      ft = ((TrieDateField) ft).wrappedField;
+    if (ft instanceof TrieField) {
+      final TrieField trie = (TrieField)ft;
+      if (trie.precisionStep  == Integer.MAX_VALUE)
+        return null;
       switch (trie.type) {
         case INTEGER:
         case FLOAT:

Modified: lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/update/AddUpdateCommand.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/update/AddUpdateCommand.java?rev=1100480&r1=1100479&r2=1100480&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/update/AddUpdateCommand.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/update/AddUpdateCommand.java Sat May  7 08:37:00 2011
@@ -18,7 +18,7 @@
 package org.apache.solr.update;
 
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.index.Term;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.SolrInputField;
@@ -72,7 +72,7 @@ public class AddUpdateCommand extends Up
        if (sf != null) {
          if (doc != null) {
            schema.getUniqueKeyField();
-           Field storedId = doc.getField(sf.getName());
+           Fieldable storedId = doc.getField(sf.getName());
            indexedId = sf.getType().storedToIndexed(storedId);
          }
          if (solrDoc != null) {

Modified: lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/update/DocumentBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/update/DocumentBuilder.java?rev=1100480&r1=1100479&r2=1100480&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/update/DocumentBuilder.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/update/DocumentBuilder.java Sat May  7 08:37:00 2011
@@ -23,7 +23,6 @@ import java.util.HashMap;
 import java.util.List;
 
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Fieldable;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrException;
@@ -73,7 +72,7 @@ public class DocumentBuilder {
         }
       }
     } else {
-      Field field = sfield.createField(val, boost);
+      Fieldable field = sfield.createField(val, boost);
       if (field != null) {
         if (!sfield.multiValued()) {
           String oldValue = map.put(sfield.getName(), val);
@@ -161,7 +160,7 @@ public class DocumentBuilder {
     // default value are defacto 'required' fields.  
     List<String> missingFields = null;
     for (SchemaField field : schema.getRequiredFields()) {
-      if (doc.getField(field.getName() ) == null) {
+      if (doc.getFieldable(field.getName() ) == null) {
         if (field.getDefaultValue() != null) {
           addField(doc, field, field.getDefaultValue(), 1.0f);
         } else {
@@ -201,7 +200,7 @@ public class DocumentBuilder {
         if (f != null) doc.add(f); // null fields are not added
       }
     } else {
-      Field f = field.createField(val, boost);
+      Fieldable f = field.createField(val, boost);
       if (f != null) doc.add(f);  // null fields are not added
     }
   }
@@ -258,7 +257,7 @@ public class DocumentBuilder {
         if (sfield != null && sfield.getType() instanceof BinaryField) {
           isBinaryField = true;
           BinaryField binaryField = (BinaryField) sfield.getType();
-          Field f = binaryField.createField(sfield,v,boost);
+          Fieldable f = binaryField.createField(sfield,v,boost);
           if(f != null){
             out.add(f);
           }
@@ -297,7 +296,7 @@ public class DocumentBuilder {
             if (destinationField.getType() instanceof BinaryField) {
               BinaryField binaryField = (BinaryField) destinationField.getType();
               //TODO: safe to assume that binary fields only create one?
-              fields = new Field[]{binaryField.createField(destinationField, v, boost)};
+              fields = new Fieldable[]{binaryField.createField(destinationField, v, boost)};
             }
           } else {
             fields = destinationField.createFields(cf.getLimitedValue(val), boost);
@@ -327,7 +326,7 @@ public class DocumentBuilder {
     // Now validate required fields or add default values
     // fields with default values are defacto 'required'
     for (SchemaField field : schema.getRequiredFields()) {
-      if (out.getField(field.getName() ) == null) {
+      if (out.getFieldable(field.getName() ) == null) {
         if (field.getDefaultValue() != null) {
           addField(out, field, field.getDefaultValue(), 1.0f);
         } 

Modified: lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/update/UpdateHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/update/UpdateHandler.java?rev=1100480&r1=1100479&r2=1100480&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/update/UpdateHandler.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/update/UpdateHandler.java Sat May  7 08:37:00 2011
@@ -21,7 +21,6 @@ package org.apache.solr.update;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.Scorer;
@@ -125,7 +124,7 @@ public abstract class UpdateHandler impl
 
   protected final String getIndexedIdOptional(Document doc) {
     if (idField == null) return null;
-    Field f = doc.getField(idField.getName());
+    Fieldable f = doc.getFieldable(idField.getName());
     if (f == null) return null;
     return idFieldType.storedToIndexed(f);
   }

Modified: lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/BasicFunctionalityTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/BasicFunctionalityTest.java?rev=1100480&r1=1100479&r2=1100480&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/BasicFunctionalityTest.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/BasicFunctionalityTest.java Sat May  7 08:37:00 2011
@@ -27,6 +27,7 @@ import javax.xml.parsers.DocumentBuilder
 import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.index.LogMergePolicy;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.Query;
@@ -370,7 +371,7 @@ public class BasicFunctionalityTest exte
     
     IndexSchema ischema = new IndexSchema(solrConfig, getSchemaFile(), null);
     SchemaField f; // Solr field type
-    Field luf; // Lucene field
+    Fieldable luf; // Lucene field
 
     f = ischema.getField("test_basictv");
     luf = f.createField("test", 0f);
@@ -573,7 +574,7 @@ public class BasicFunctionalityTest exte
 
     DocList dl = (DocList) rsp.getValues().get("response");
     org.apache.lucene.document.Document d = req.getSearcher().doc(dl.iterator().nextDoc());
-    // ensure field is not lazy
+    // ensure field is not lazy, only works for Non-Numeric fields currently (if you change schema behind test, this may fail)
     assertTrue( d.getFieldable("test_hlt") instanceof Field );
     assertTrue( d.getFieldable("title") instanceof Field );
     req.close();

Modified: lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/handler/MoreLikeThisHandlerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/handler/MoreLikeThisHandlerTest.java?rev=1100480&r1=1100479&r2=1100480&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/handler/MoreLikeThisHandlerTest.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/handler/MoreLikeThisHandlerTest.java Sat May  7 08:37:00 2011
@@ -88,7 +88,7 @@ public class MoreLikeThisHandlerTest ext
 
     params.put(CommonParams.Q, new String[]{"id:42"});
     params.put(MoreLikeThisParams.MLT, new String[]{"true"});
-    params.put(MoreLikeThisParams.SIMILARITY_FIELDS, new String[]{"name,subword,foo_ti"});
+    params.put(MoreLikeThisParams.SIMILARITY_FIELDS, new String[]{"name,subword"});
     params.put(MoreLikeThisParams.INTERESTING_TERMS,new String[]{"details"});
     params.put(MoreLikeThisParams.MIN_TERM_FREQ,new String[]{"1"});
     params.put(MoreLikeThisParams.MIN_DOC_FREQ,new String[]{"1"});

Modified: lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/update/DocumentBuilderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/update/DocumentBuilderTest.java?rev=1100480&r1=1100479&r2=1100480&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/update/DocumentBuilderTest.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/update/DocumentBuilderTest.java Sat May  7 08:37:00 2011
@@ -75,8 +75,8 @@ public class DocumentBuilderTest extends
     doc.addField( "home", "2.2,3.3", 1.0f );
     Document out = DocumentBuilder.toDocument( doc, core.getSchema() );
     assertNotNull( out.get( "home" ) );//contains the stored value and term vector, if there is one
-    assertNotNull( out.getField( "home_0" + FieldType.POLY_FIELD_SEPARATOR + "double" ) );
-    assertNotNull( out.getField( "home_1" + FieldType.POLY_FIELD_SEPARATOR + "double" ) );
+    assertNotNull( out.getFieldable( "home_0" + FieldType.POLY_FIELD_SEPARATOR + "double" ) );
+    assertNotNull( out.getFieldable( "home_1" + FieldType.POLY_FIELD_SEPARATOR + "double" ) );
   }
 
 }

Modified: lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/update/TestIndexingPerformance.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/update/TestIndexingPerformance.java?rev=1100480&r1=1100479&r2=1100480&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/update/TestIndexingPerformance.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/update/TestIndexingPerformance.java Sat May  7 08:37:00 2011
@@ -19,6 +19,7 @@ package org.apache.solr.update;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Fieldable;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.util.AbstractSolrTestCase;
@@ -98,7 +99,7 @@ public class TestIndexingPerformance ext
         for (int j=0; j<fields.length; j+=2) {
           String field = fields[j];
           String val = fields[j+1];
-          Field f = schema.getField(field).createField(val, 1.0f);
+          Fieldable f = schema.getField(field).createField(val, 1.0f);
           add.doc.add(f);
         }
       }