You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2010/10/12 08:15:05 UTC

svn commit: r1021636 [1/3] - in /lucene/dev/branches/docvalues/lucene: contrib/benchmark/conf/ contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/ contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ src/java/org/apache...

Author: simonw
Date: Tue Oct 12 06:15:03 2010
New Revision: 1021636

URL: http://svn.apache.org/viewvc?rev=1021636&view=rev
Log:
LUCENE-2186: First cut at column-stride fields (index values storage)

Added:
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/ValuesField.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Cache.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Ints.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Reader.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Values.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesAttribute.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesAttributeImpl.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesEnum.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Writer.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/FloatsRef.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/LongsRef.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ParallelArray.java   (with props)
    lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/values/
    lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/values/TestIndexValues.java   (with props)
Modified:
    lucene/dev/branches/docvalues/lucene/contrib/benchmark/conf/sort-standard.alg
    lucene/dev/branches/docvalues/lucene/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/DocMaker.java
    lucene/dev/branches/docvalues/lucene/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/SearchWithSortTask.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/AbstractField.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/Fieldable.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/CompoundFileReader.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/CompoundFileWriter.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DirectoryReader.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FieldInfo.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FieldInfos.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FieldsEnum.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/IndexFileNames.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/IndexReader.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentReader.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparator.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ReqExclScorer.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/ReqOptSumScorer.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/SortField.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ArrayUtil.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRef.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/BytesRefHash.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/IOUtils.java
    lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/packed/Packed64.java
    lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterConfig.java
    lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java
    lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/util/_TestUtil.java

Modified: lucene/dev/branches/docvalues/lucene/contrib/benchmark/conf/sort-standard.alg
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/contrib/benchmark/conf/sort-standard.alg?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/contrib/benchmark/conf/sort-standard.alg (original)
+++ lucene/dev/branches/docvalues/lucene/contrib/benchmark/conf/sort-standard.alg Tue Oct 12 06:15:03 2010
@@ -26,6 +26,7 @@ analyzer=org.apache.lucene.analysis.stan
 directory=FSDirectory
 #directory=RamDirectory
 
+doc.index.props=true
 doc.stored=true
 doc.tokenized=true
 doc.term.vector=false
@@ -66,6 +67,4 @@ log.queries=true
      } : 4
 
 } 
-
-RepSumByName
-
+RepSumByName
\ No newline at end of file

Modified: lucene/dev/branches/docvalues/lucene/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/DocMaker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/DocMaker.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/DocMaker.java (original)
+++ lucene/dev/branches/docvalues/lucene/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/DocMaker.java Tue Oct 12 06:15:03 2010
@@ -19,6 +19,7 @@ package org.apache.lucene.benchmark.byTa
 
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
@@ -28,9 +29,11 @@ import org.apache.lucene.benchmark.byTas
 import org.apache.lucene.benchmark.byTask.utils.Format;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.ValuesField;
 import org.apache.lucene.document.Field.Index;
 import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.Field.TermVector;
+import org.apache.lucene.index.values.Values;
 
 /**
  * Creates {@link Document} objects. Uses a {@link ContentSource} to generate
@@ -157,12 +160,13 @@ public class DocMaker {
   private long lastPrintedNumUniqueBytes = 0;
 
   private int printNum = 0;
+  private Map<String, Values> fieldVauleMap;
 
   // create a doc
   // use only part of the body, modify it to keep the rest (or use all if size==0).
   // reset the docdata properties so they are not added more than once.
   private Document createDocument(DocData docData, int size, int cnt) throws UnsupportedEncodingException {
-
+    Values valueType;
     final DocState ds = getDocState();
     final Document doc = reuseFields ? ds.doc : new Document();
     doc.getFields().clear();
@@ -178,6 +182,7 @@ public class DocMaker {
     name = cnt < 0 ? name : name + "_" + cnt;
     Field nameField = ds.getField(NAME_FIELD, storeVal, indexVal, termVecVal);
     nameField.setValue(name);
+    trySetIndexValues(nameField);
     doc.add(nameField);
     
     // Set DATE_FIELD
@@ -187,12 +192,14 @@ public class DocMaker {
     }
     Field dateField = ds.getField(DATE_FIELD, storeVal, indexVal, termVecVal);
     dateField.setValue(date);
+    trySetIndexValues(dateField);
     doc.add(dateField);
     
     // Set TITLE_FIELD
     String title = docData.getTitle();
     Field titleField = ds.getField(TITLE_FIELD, storeVal, indexVal, termVecVal);
     titleField.setValue(title == null ? "" : title);
+    trySetIndexValues(titleField);
     doc.add(titleField);
     
     String body = docData.getBody();
@@ -214,12 +221,15 @@ public class DocMaker {
       }
       Field bodyField = ds.getField(BODY_FIELD, bodyStoreVal, bodyIndexVal, termVecVal);
       bodyField.setValue(bdy);
+      trySetIndexValues(bodyField);
       doc.add(bodyField);
       
       if (storeBytes) {
         Field bytesField = ds.getField(BYTES_FIELD, Store.YES, Index.NOT_ANALYZED_NO_NORMS, TermVector.NO);
         bytesField.setValue(bdy.getBytes("UTF-8"));
+        trySetIndexValues(bytesField);
         doc.add(bytesField);
+        
       }
     }
 
@@ -229,6 +239,7 @@ public class DocMaker {
         for (final Map.Entry<Object,Object> entry : props.entrySet()) {
           Field f = ds.getField((String) entry.getKey(), storeVal, indexVal, termVecVal);
           f.setValue((String) entry.getValue());
+          trySetIndexValues(f);
           doc.add(f);
         }
         docData.setProps(null);
@@ -238,6 +249,12 @@ public class DocMaker {
     //System.out.println("============== Created doc "+numDocsCreated+" :\n"+doc+"\n==========");
     return doc;
   }
+  
+  private void trySetIndexValues(Field field) {
+    final Values valueType;
+    if((valueType = fieldVauleMap.get(field.name())) != null)
+      ValuesField.set(field, valueType);
+  }
 
   private void resetLeftovers() {
     leftovr.set(null);
@@ -367,6 +384,22 @@ public class DocMaker {
     resetLeftovers();
   }
   
+  private static final Map<String, Values> parseValueFields(String fields) {
+    if(fields == null)
+      return Collections.emptyMap();
+    String[] split = fields.split(";");
+    Map<String, Values> result = new HashMap<String, Values>();
+    for (String tuple : split) {
+      final String[] nameValue = tuple.split(":");
+      if (nameValue.length != 2) {
+        throw new IllegalArgumentException("illegal doc.stored.values format: "
+            + fields + " expected fieldname:ValuesType;...;...;");
+      }
+      result.put(nameValue[0].trim(), Values.valueOf(nameValue[1]));
+    }
+    return result;
+  }
+  
   /** Set the configuration parameters of this doc maker. */
   public void setConfig(Config config) {
     this.config = config;
@@ -386,6 +419,7 @@ public class DocMaker {
     boolean norms = config.get("doc.tokenized.norms", false);
     boolean bodyNorms = config.get("doc.body.tokenized.norms", true);
     boolean termVec = config.get("doc.term.vector", false);
+    fieldVauleMap = parseValueFields(config.get("doc.stored.values", null));
     storeVal = (stored ? Field.Store.YES : Field.Store.NO);
     bodyStoreVal = (bodyStored ? Field.Store.YES : Field.Store.NO);
     if (tokenized) {
@@ -423,7 +457,6 @@ public class DocMaker {
     docState = new ThreadLocal<DocState>();
     
     indexProperties = config.get("doc.index.props", false);
-
     updateDocIDLimit = config.get("doc.random.id.limit", -1);
     if (updateDocIDLimit != -1) {
       r = new Random(179);

Modified: lucene/dev/branches/docvalues/lucene/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/SearchWithSortTask.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/SearchWithSortTask.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/SearchWithSortTask.java (original)
+++ lucene/dev/branches/docvalues/lucene/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/SearchWithSortTask.java Tue Oct 12 06:15:03 2010
@@ -75,8 +75,7 @@ public class SearchWithSortTask extends 
         } else {
           throw new RuntimeException("You must specify the sort type ie page:int,subject:string");
         }
-        int type = getType(typeString);
-        sortField0 = new SortField(fieldName, type);
+        sortField0 = getSortField(fieldName, typeString);
       }
       sortFields[upto++] = sortField0;
     }
@@ -86,12 +85,26 @@ public class SearchWithSortTask extends 
       System.arraycopy(sortFields, 0, newSortFields, 0, upto);
       sortFields = newSortFields;
     }
+    
     this.sort = new Sort(sortFields);
   }
 
-  private int getType(String typeString) {
-    int type;
-    if (typeString.equals("float")) {
+  private SortField getSortField(String fieldName, String typeString) {
+    boolean useIndexValues = false;
+    int type = -1;
+    if (typeString.equals("intvalues")) {
+      useIndexValues = true;
+      type = SortField.INT;
+    } else if (typeString.equals("floatvalues")) {
+      useIndexValues = true;
+      type = SortField.FLOAT;
+    } else if (typeString.equals("stringvalues")) {
+      useIndexValues = true;
+      type = SortField.STRING;
+    } else if (typeString.equals("bytesvalues")) {
+      useIndexValues = true;
+      type = SortField.BYTES;
+    } else if (typeString.equals("float")) {
       type = SortField.FLOAT;
     } else if (typeString.equals("double")) {
       type = SortField.DOUBLE;
@@ -110,7 +123,10 @@ public class SearchWithSortTask extends 
     } else {
       throw new RuntimeException("Unrecognized sort field type " + typeString);
     }
-    return type;
+
+    SortField f = new SortField(fieldName, type);
+    f.setUseIndexValues(useIndexValues);
+    return f;
   }
 
   @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/AbstractField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/AbstractField.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/AbstractField.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/AbstractField.java Tue Oct 12 06:15:03 2010
@@ -19,6 +19,7 @@ import org.apache.lucene.search.PhraseQu
 import org.apache.lucene.search.spans.SpanQuery; // for javadocs
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.index.FieldInvertState;
+import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.StringHelper; // for javadocs
 
 
@@ -292,4 +293,16 @@ public abstract class AbstractField impl
     result.append('>');
     return result.toString();
   }
+  private AttributeSource source;
+  
+  public boolean hasFieldAttribute() {
+    return source != null;
+  }
+  
+  public AttributeSource getFieldAttributes() {
+    if(source == null)
+      source = new AttributeSource();
+    return source;
+  }
+
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/Fieldable.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/Fieldable.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/Fieldable.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/Fieldable.java Tue Oct 12 06:15:03 2010
@@ -20,6 +20,7 @@ import org.apache.lucene.analysis.TokenS
 import org.apache.lucene.index.FieldInvertState; // for javadocs
 import org.apache.lucene.search.PhraseQuery; // for javadocs
 import org.apache.lucene.search.spans.SpanQuery; // for javadocs
+import org.apache.lucene.util.AttributeSource;
 
 import java.io.Reader;
 import java.io.Serializable;
@@ -209,4 +210,7 @@ public interface Fieldable extends Seria
   * silently fail to find results.
   */
   void setOmitTermFreqAndPositions(boolean omitTermFreqAndPositions);
+  
+  boolean hasFieldAttribute();
+  AttributeSource getFieldAttributes();
 }

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/ValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/ValuesField.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/ValuesField.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/ValuesField.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,136 @@
+package org.apache.lucene.document;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import java.io.IOException;
+import java.io.Reader;
+import java.util.Comparator;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.document.Field.Index;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.Field.TermVector;
+import org.apache.lucene.index.values.Values;
+import org.apache.lucene.index.values.ValuesAttribute;
+import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ *
+ */
+@SuppressWarnings("serial")
+public class ValuesField extends AbstractField {
+  private final ValuesAttribute attr;
+  private final AttributeSource fieldAttributes;
+  
+
+  public ValuesField(String name) {
+    super(name, Store.NO, Index.NO, TermVector.NO);
+    fieldAttributes = getFieldAttributes();
+    attr = fieldAttributes.addAttribute(ValuesAttribute.class);
+  }
+  
+  ValuesField() {
+    this("");
+  }
+
+  public Reader readerValue() {
+    return null;
+  }
+
+  public String stringValue() {
+    return null;
+  }
+
+  public TokenStream tokenStreamValue() {
+    return tokenStream;
+  }
+
+  public void setInt(long value) {
+    attr.setType(Values.PACKED_INTS);
+    attr.ints().set(value);
+  }
+
+  public void setFloat(float value) {
+    attr.setType(Values.SIMPLE_FLOAT_4BYTE);
+    attr.floats().set(value);
+  }
+
+  public void setFloat(double value) {
+    attr.setType(Values.SIMPLE_FLOAT_8BYTE);
+    attr.floats().set(value);
+  }
+
+  public void setBytes(BytesRef value, Values type) {
+    setBytes(value, type, null);
+
+  }
+
+  public void setBytes(BytesRef value, Values type, Comparator<BytesRef> comp) {
+    attr.setType(type);
+    attr.bytes().copy(value);
+    attr.setBytesComparator(comp);
+  }
+
+  public ValuesAttribute values() {
+    return attr;
+  }
+  
+  public <T extends Fieldable> T set(T field) {
+    AttributeSource src = field.getFieldAttributes();
+    src.addAttribute(ValuesAttribute.class);
+    fieldAttributes.copyTo(field.getFieldAttributes());
+    return field;
+  }
+  
+  public static ValuesAttribute values(Fieldable fieldable) {
+    return fieldable.getFieldAttributes().addAttribute(ValuesAttribute.class);
+  }
+
+  public static <T extends Fieldable> T set(T field, Values type) {
+    if(field instanceof ValuesField)
+      return field;
+    final ValuesField valField = new ValuesField();
+    switch (type) {
+    case BYTES_FIXED_DEREF:
+    case BYTES_FIXED_SORTED:
+    case BYTES_FIXED_STRAIGHT:
+    case BYTES_VAR_DEREF:
+    case BYTES_VAR_SORTED:
+    case BYTES_VAR_STRAIGHT:
+      BytesRef ref = field.isBinary() ? new BytesRef(field.getBinaryValue(),
+          field.getBinaryOffset(), field.getBinaryLength()) : new BytesRef(field
+          .stringValue());
+      valField.setBytes(ref, type);
+      break;
+    case PACKED_INTS:
+    case PACKED_INTS_FIXED:
+      valField.setInt(Long.parseLong(field.stringValue()));
+      break;
+    case SIMPLE_FLOAT_4BYTE:
+      valField.setFloat(Float.parseFloat(field.stringValue()));
+      break;
+    case SIMPLE_FLOAT_8BYTE:
+      valField.setFloat(Double.parseDouble(field.stringValue()));
+      break;
+    default:
+      throw new IllegalArgumentException("unknown type: " + type);
+    }
+    
+    return valField.set(field);
+  }
+}

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/ValuesField.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/document/ValuesField.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/CompoundFileReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/CompoundFileReader.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/CompoundFileReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/CompoundFileReader.java Tue Oct 12 06:15:03 2010
@@ -157,7 +157,7 @@ public class CompoundFileReader extends 
             throw new IOException("Stream closed");
         
         id = IndexFileNames.stripSegmentName(id);
-        FileEntry entry = entries.get(id);
+        final FileEntry entry = entries.get(id);
         if (entry == null)
             throw new IOException("No sub-file with id " + id + " found");
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/CompoundFileWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/CompoundFileWriter.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/CompoundFileWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/CompoundFileWriter.java Tue Oct 12 06:15:03 2010
@@ -49,9 +49,13 @@ import java.io.IOException;
  */
 final class CompoundFileWriter {
 
-    private static final class FileEntry {
+    static final class FileEntry {
+	
+	FileEntry(String file) {
+	    this.file = file;
+	}
         /** source file */
-        String file;
+        final String file;
 
         /** temporary holder for the start of directory entry for this file */
         long directoryOffset;
@@ -128,10 +132,7 @@ final class CompoundFileWriter {
         if (! ids.add(file))
             throw new IllegalArgumentException(
                 "File " + file + " already added");
-
-        FileEntry entry = new FileEntry();
-        entry.file = file;
-        entries.add(entry);
+        entries.add(new FileEntry(file));
     }
 
     /** Merge files with the extensions added up to now.

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DirectoryReader.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DirectoryReader.java Tue Oct 12 06:15:03 2010
@@ -36,7 +36,14 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.store.Lock;
 import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.values.Reader;
+import org.apache.lucene.index.values.Values;
+import org.apache.lucene.index.values.ValuesEnum;
+import org.apache.lucene.index.values.Reader.Source;
+import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.FloatsRef;
+import org.apache.lucene.util.LongsRef;
 import org.apache.lucene.util.ReaderUtil;
 import org.apache.lucene.util.BytesRef;
 
@@ -990,7 +997,264 @@ class DirectoryReader extends IndexReade
 
     return commits;
   }
+  
+  public Reader getIndexValues(String field) {
+    ensureOpen();
+    if (subReaders.length == 1) {
+      return subReaders[0].getIndexValues(field);
+    } 
+    return new MultiValueReader(field);
+  }
+  
+  private class MultiValueReader extends Reader {
+
+    private String id;
+    private Values value;
+
+    public MultiValueReader(String id) {
+      this.id = id;
+      for (SegmentReader reader : subReaders) {
+        FieldInfo fieldInfo = reader.fieldInfos().fieldInfo(id);
+        if(fieldInfo != null){
+          value = fieldInfo.getIndexValues();
+          break;
+        }
+      }
+    }
+
+    @Override
+    public ValuesEnum getEnum(AttributeSource source) throws IOException {
+      return new MultiValuesEnum(id, value);
+    }
 
+    @Override
+    public Source load() throws IOException {
+      return new MultiSource(id);
+    }
+
+    public void close() throws IOException {
+      //      
+    }
+    
+  }
+  
+  private class MultiValuesEnum extends ValuesEnum {
+    private int numDocs_ = 0;
+    private int pos = -1;
+    private int start = 0;
+    private final String id;
+    private final ValuesEnum[] enumCache;
+    private ValuesEnum current;
+
+    protected MultiValuesEnum(String id, Values enumType) {
+      super(enumType);
+      enumCache = new ValuesEnum[subReaders.length];
+      this.id = id;
+    }
+
+    @Override
+    public void close() throws IOException {
+      for (ValuesEnum valuesEnum : enumCache) {
+        if(valuesEnum != null)
+          valuesEnum.close();
+      }
+    }
+
+    @Override
+    public int advance( int target) throws IOException {
+      int n = target - start;
+      do {
+        if(target >= maxDoc)
+          return pos = NO_MORE_DOCS;
+        if (n >= numDocs_) {
+          int idx = readerIndex(target);
+          if (enumCache[idx] == null) {
+            try {
+              Reader indexValues = subReaders[idx].getIndexValues(id);
+              if (indexValues != null) // nocommit does that work with default
+                // values?
+                enumCache[idx] = indexValues.getEnum(this.attributes());
+              else
+                enumCache[idx] = new DummyEnum(this.attributes(),
+                    subReaders[idx].maxDoc(), attr.type());
+            } catch (IOException ex) {
+              // nocommit what to do here?
+              throw new RuntimeException(ex);
+            }
+          }
+          current = enumCache[idx];
+          start = starts[idx];
+          numDocs_ = subReaders[idx].maxDoc();
+          n = target - start;
+        }
+        target = start+numDocs_;
+      } while ((n = current.advance(n)) == NO_MORE_DOCS);
+      return pos = start+current.docID();
+    }
+
+
+    @Override
+    public int docID() {
+      return pos;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return advance(pos+1);
+    }
+  }
+  
+  private class MultiSource extends Source {
+    private int numDocs_ = 0;
+    private int start = 0;
+    private Source current;
+    private final String id;
+    
+    MultiSource(String id) {
+      this.id = id;
+    }
+    
+    public long ints(int docID) {
+      int n = docID - start;
+      if(n >= numDocs_) {
+        int idx = readerIndex(docID);
+        try{
+        current = subReaders[idx].getIndexValuesCache().getInts(id);
+        if(current == null) //nocommit does that work with default values?
+          current = new DummySource();
+        }catch(IOException ex) {
+          // nocommit what to do here?
+          throw new RuntimeException(ex);
+        }
+        start = starts[idx];
+        numDocs_ = subReaders[idx].maxDoc();
+        n = docID - start;
+      }
+      return current.ints(n);
+    }
+
+    public double floats(int docID) {
+      int n = docID - start;
+      if(n >= numDocs_) {
+          int idx = readerIndex(docID);
+          try{
+          current = subReaders[idx].getIndexValuesCache().getFloats(id);
+          if(current == null) //nocommit does that work with default values?
+            current = new DummySource();
+          }catch(IOException ex) {
+            // nocommit what to do here?
+            throw new RuntimeException(ex);
+          }
+          numDocs_ = subReaders[idx].maxDoc();
+
+          start = starts[idx];
+          n = docID - start;
+      }
+      return current.floats(n);
+    }
+
+    public BytesRef bytes(int docID) {
+      int n = docID - start;
+      if(n >= numDocs_) {
+        int idx = readerIndex(docID);
+        try{
+        current = subReaders[idx].getIndexValuesCache().getBytes(id);
+        if(current == null) //nocommit does that work with default values?
+          current = new DummySource();
+        }catch(IOException ex) {
+          // nocommit what to do here?
+          throw new RuntimeException(ex);
+        }
+        numDocs_ = subReaders[idx].maxDoc();
+        start = starts[idx];
+        n = docID - start;
+      }
+      return current.bytes(n);
+    }
+    
+    public long ramBytesUsed() {
+      return current.ramBytesUsed();
+    }
+    
+  }
+
+  private static class DummySource extends Source {
+    private final BytesRef ref = new BytesRef();
+    @Override
+    public BytesRef bytes(int docID) {
+      return ref;
+    }
+
+    
+    @Override
+    public double floats(int docID) {
+      return 0.0d;
+    }
+
+    @Override
+    public long ints(int docID) {
+      return 0;
+    }
+
+    public long ramBytesUsed() {
+      return 0;
+    }
+  }
+  
+  private static class DummyEnum extends ValuesEnum {
+    private int pos = -1;
+    private final int maxDoc;
+    
+    public DummyEnum(AttributeSource source, int maxDoc, Values type) {
+      super(source, type);
+      this.maxDoc = maxDoc;
+      switch (type) {
+      case BYTES_VAR_STRAIGHT:
+      case BYTES_FIXED_STRAIGHT:
+      case BYTES_FIXED_DEREF:
+      case BYTES_FIXED_SORTED:
+      case BYTES_VAR_DEREF:
+      case BYTES_VAR_SORTED:
+        // nocommit - this is not correct for Fixed_straight
+        BytesRef bytes = attr.bytes();
+        bytes.length = 0;
+        bytes.offset = 0;
+        break;
+      case PACKED_INTS:
+      case PACKED_INTS_FIXED:
+        LongsRef ints = attr.ints();
+        ints.set(0);
+        break;
+
+      case SIMPLE_FLOAT_4BYTE:
+      case SIMPLE_FLOAT_8BYTE:
+        FloatsRef floats = attr.floats();
+        floats.set(0d);
+        break;
+      default:
+        throw new IllegalArgumentException("unknown Values type: " + type);
+      }
+    }
+    @Override
+    public void close() throws IOException {
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      return pos = (pos < maxDoc ? target: NO_MORE_DOCS);
+    }
+    @Override
+    public int docID() {
+      return pos;
+    }
+    @Override
+    public int nextDoc() throws IOException {
+      return advance(pos+1);
+    }
+    
+  }
+  
+  
   private static final class ReaderCommit extends IndexCommit {
     private String segmentsFileName;
     Collection<String> files;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java Tue Oct 12 06:15:03 2010
@@ -17,8 +17,19 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.index.values.Ints;
+import org.apache.lucene.index.values.Floats;
+import org.apache.lucene.index.values.Bytes;
+import org.apache.lucene.index.values.ValuesAttribute;
+import org.apache.lucene.index.values.Writer;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FloatsRef;
+import org.apache.lucene.util.LongsRef;
+
 import java.io.IOException;
 import java.util.Collection;
+import java.util.Comparator;
 import java.util.Map;
 import java.util.HashMap;
 
@@ -37,6 +48,153 @@ final class DocFieldProcessor extends Do
   final FieldInfos fieldInfos = new FieldInfos();
   final DocFieldConsumer consumer;
   final StoredFieldsWriter fieldsWriter;
+  final private Map<String,IndexValuesProcessor> indexValues = new HashMap<String,IndexValuesProcessor>();
+
+  synchronized IndexValuesProcessor getProcessor(Directory dir, String segment, String name, ValuesAttribute attr, FieldInfo fieldInfo)
+    throws IOException {
+    if(attr == null)
+      return null;
+    IndexValuesProcessor p = indexValues.get(name);
+    if (p == null) {
+        org.apache.lucene.index.values.Values v = attr.type();
+        final String id = segment + "_" + fieldInfo.number;
+        switch(v) {
+        case PACKED_INTS:
+          p = new IntValuesProcessor(dir, id, false);
+          break;
+        case PACKED_INTS_FIXED:
+          p = new IntValuesProcessor(dir, id, true);
+          break;
+        case SIMPLE_FLOAT_4BYTE:
+          p = new FloatValuesProcessor(dir, id, 4);
+          break;
+        case SIMPLE_FLOAT_8BYTE:
+          p = new FloatValuesProcessor(dir, id, 8);
+          break;
+        case BYTES_FIXED_STRAIGHT:
+          p = new BytesValuesProcessor(dir, id, true, null, Bytes.Mode.STRAIGHT);
+          break;
+        case BYTES_FIXED_DEREF:
+          p = new BytesValuesProcessor(dir, id, true, null, Bytes.Mode.DEREF);
+          break;
+        case BYTES_FIXED_SORTED:
+          p = new BytesValuesProcessor(dir, id, true, attr.bytesComparator(), Bytes.Mode.SORTED);
+          break;
+        case BYTES_VAR_STRAIGHT:
+          p = new BytesValuesProcessor(dir, id, false, null, Bytes.Mode.STRAIGHT);
+          break;
+        case BYTES_VAR_DEREF:
+          p = new BytesValuesProcessor(dir, id, false, null, Bytes.Mode.DEREF);
+          break;
+        case BYTES_VAR_SORTED:
+          p = new BytesValuesProcessor(dir, id, false, attr.bytesComparator(), Bytes.Mode.SORTED);
+          break;
+        }
+        fieldInfo.setIndexValues(v);
+        indexValues.put(name, p);
+    }
+
+    return p;
+  }
+
+  static abstract class IndexValuesProcessor {
+    public abstract void add(int docID, String name, ValuesAttribute attr) throws IOException;
+    public abstract void finish(int docCount) throws IOException;
+    public abstract void files(Collection<String> files) throws IOException;
+  }
+
+  static class FloatValuesProcessor extends IndexValuesProcessor {
+    private final Writer writer;
+    private final String id;
+
+    public FloatValuesProcessor(Directory dir, String id, int precision) throws IOException {
+      this.id = id;
+      writer = Floats.getWriter(dir, id, precision);
+    }
+
+    @Override
+    public void add(int docID, String name, ValuesAttribute attr) throws IOException {
+        final FloatsRef floats = attr.floats();
+        if(floats != null) {
+          writer.add(docID, floats.get());
+          return;
+        }
+      throw new IllegalArgumentException("could not extract float/double from field " + name);
+    }
+
+    @Override
+    public void finish(int docCount) throws IOException {
+      writer.finish(docCount);
+    }
+
+    @Override
+    public void files(Collection<String> files) {
+      Floats.files(id, files);
+    }
+  }
+
+  static class IntValuesProcessor extends IndexValuesProcessor {
+    private final Writer writer;
+    private final String id;
+
+    public IntValuesProcessor(Directory dir, String id, boolean fixedArray) throws IOException {
+      this.id = id;
+      writer = Ints.getWriter(dir, id, fixedArray);
+    }
+
+    @Override
+      public void add(int docID, String name, ValuesAttribute attr) throws IOException {
+        final LongsRef ints = attr.ints();
+        if(ints != null) {
+          writer.add(docID, ints.get());
+          return;
+        }
+      throw new IllegalArgumentException("could not extract int/long from field " + name);
+    }
+
+    @Override
+    public void finish(int docCount) throws IOException {
+      writer.finish(docCount);
+    }
+
+    @Override
+    public void files(Collection<String> files) throws IOException {
+      Ints.files(id, files);
+    }
+  }
+
+  static class BytesValuesProcessor extends IndexValuesProcessor {
+    private final Writer writer;
+    private final String id;
+    private final Directory dir;
+
+    public BytesValuesProcessor(Directory dir, String id, boolean fixedSize, Comparator<BytesRef> comp, Bytes.Mode mode) throws IOException {
+      this.id = id;
+      writer = Bytes.getWriter(dir, id, mode,comp, fixedSize);
+      this.dir = dir;
+    }
+
+    // nocommit -- make this thread private and not sync'd
+    @Override
+    public synchronized void add(int docID, String name, ValuesAttribute attr) throws IOException {
+      final BytesRef bytes = attr.bytes();
+      if(bytes != null) {
+        writer.add(docID, bytes);
+        return;
+      }
+      throw new IllegalArgumentException("could not extract byte[] from field " + name);
+    }
+
+    @Override
+    public void finish(int docCount) throws IOException {
+      writer.finish(docCount);
+    }
+
+    @Override
+    public void files(Collection<String> files) throws IOException {
+      Bytes.files(dir, id, files);
+    }
+  }
 
   public DocFieldProcessor(DocumentsWriter docWriter, DocFieldConsumer consumer) {
     this.docWriter = docWriter;
@@ -63,6 +221,14 @@ final class DocFieldProcessor extends Do
     fieldsWriter.flush(state);
     consumer.flush(childThreadsAndFields, state);
 
+    for(IndexValuesProcessor p : indexValues.values()) {
+      if (p != null) {
+        p.finish(state.numDocs);
+        p.files(state.flushedFiles);
+      }
+    }
+    indexValues.clear();
+
     // Important to save after asking consumer to flush so
     // consumer can alter the FieldInfo* if necessary.  EG,
     // FreqProxTermsWriter does this with

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java Tue Oct 12 06:15:03 2010
@@ -20,10 +20,16 @@ package org.apache.lucene.index;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
+import java.util.Map.Entry;
 import java.io.IOException;
+
+import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.index.values.ValuesAttribute;
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.RamUsageEstimator;
 
 /**
@@ -243,10 +249,24 @@ final class DocFieldProcessorPerThread e
     // enabled; we could save [small amount of] CPU
     // here.
     quickSort(fields, 0, fieldCount-1);
+   
 
-    for(int i=0;i<fieldCount;i++)
-      fields[i].consumer.processFields(fields[i].fields, fields[i].fieldCount);
-
+    for(int i=0;i<fieldCount;i++) {
+      final DocFieldProcessorPerField perField = fields[i];
+      final Fieldable fieldable = perField.fields[0];
+      perField.consumer.processFields(perField.fields, perField.fieldCount);
+      if(!fieldable.hasFieldAttribute())
+        continue;
+      final AttributeSource attrSource = fieldable.getFieldAttributes();
+      if(!attrSource.hasAttribute(ValuesAttribute.class))
+        continue;
+      final ValuesAttribute attribute = attrSource.getAttribute(ValuesAttribute.class);
+      final DocFieldProcessor.IndexValuesProcessor processor = docFieldProcessor
+          .getProcessor(docState.docWriter.directory,
+              docState.docWriter.segment, fieldable.name(), attribute, perField.fieldInfo);
+      if (processor != null)
+        processor.add(docState.docID, fieldable.name(), attribute);
+    }
     if (docState.maxTermPrefix != null && docState.infoStream != null) {
       docState.infoStream.println("WARNING: document contains at least one immense term (whose UTF8 encoding is longer than the max length " + DocumentsWriter.MAX_TERM_LENGTH_UTF8 + "), all of which were skipped.  Please correct the analyzer to not produce such terms.  The prefix of the first immense term is: '" + docState.maxTermPrefix + "...'"); 
       docState.maxTermPrefix = null;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FieldInfo.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FieldInfo.java Tue Oct 12 06:15:03 2010
@@ -1,5 +1,7 @@
 package org.apache.lucene.index;
 
+import org.apache.lucene.index.values.Values;
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -22,6 +24,8 @@ public final class FieldInfo {
   public String name;
   public boolean isIndexed;
   public int number;
+  Values indexValues;
+
 
   // true if term vector for this field should be stored
   boolean storeTermVector;
@@ -88,4 +92,18 @@ public final class FieldInfo {
       }
     }
   }
+
+  void setIndexValues(Values v) {
+    if (indexValues != null) {
+      if (indexValues != v) {
+        throw new IllegalArgumentException("indexValues is already set to " + indexValues + "; cannot change to " + v);
+      }
+    } else{
+	   indexValues = v;
+    }
+  }
+
+  Values getIndexValues() {
+    return indexValues;
+  }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FieldInfos.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FieldInfos.java Tue Oct 12 06:15:03 2010
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.index.values.Values;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
@@ -39,8 +40,11 @@ public final class FieldInfos {
   // First used in 2.9; prior to 2.9 there was no format header
   public static final int FORMAT_START = -2;
 
+  // Records index values for this field
+  public static final int FORMAT_INDEX_VALUES = -3;
+
   // whenever you add a new format, make it 1 smaller (negative version logic)!
-  static final int FORMAT_CURRENT = FORMAT_START;
+  static final int FORMAT_CURRENT = FORMAT_INDEX_VALUES;
   
   static final int FORMAT_MINIMUM = FORMAT_START;
   
@@ -301,9 +305,51 @@ public final class FieldInfos {
       if (fi.omitNorms) bits |= OMIT_NORMS;
       if (fi.storePayloads) bits |= STORE_PAYLOADS;
       if (fi.omitTermFreqAndPositions) bits |= OMIT_TERM_FREQ_AND_POSITIONS;
-      
+
       output.writeString(fi.name);
       output.writeByte(bits);
+
+      final byte b;
+
+      if (fi.indexValues == null) {
+        b = 0;
+      } else {
+        switch(fi.indexValues) {
+        case PACKED_INTS:
+          b = 1;
+          break;
+        case SIMPLE_FLOAT_4BYTE:
+          b = 2;
+          break;
+        case SIMPLE_FLOAT_8BYTE:
+          b = 3;
+          break;
+        case BYTES_FIXED_STRAIGHT:
+          b = 4;
+          break;
+        case BYTES_FIXED_DEREF:
+          b = 5;
+          break;
+        case BYTES_FIXED_SORTED:
+          b = 6;
+          break;
+        case BYTES_VAR_STRAIGHT:
+          b = 7;
+          break;
+        case BYTES_VAR_DEREF:
+          b = 8;
+          break;
+        case BYTES_VAR_SORTED:
+          b = 9;
+          break;
+        case PACKED_INTS_FIXED:
+          b = 10;
+          break;
+        default:
+          throw new IllegalStateException("unhandled indexValues type " + fi.indexValues);
+        }
+      }
+      output.writeByte(b);
     }
   }
 
@@ -330,7 +376,49 @@ public final class FieldInfos {
       boolean storePayloads = (bits & STORE_PAYLOADS) != 0;
       boolean omitTermFreqAndPositions = (bits & OMIT_TERM_FREQ_AND_POSITIONS) != 0;
       
-      addInternal(name, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
+      FieldInfo fi = addInternal(name, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
+
+      if (format <= FORMAT_INDEX_VALUES) {
+        final byte b = input.readByte();
+
+        switch(b) {
+        case 0:
+          fi.indexValues = null;
+          break;
+        case 1:
+          fi.indexValues = Values.PACKED_INTS;
+          break;
+        case 2:
+          fi.indexValues = Values.SIMPLE_FLOAT_4BYTE;
+          break;
+        case 3:
+          fi.indexValues = Values.SIMPLE_FLOAT_8BYTE;
+          break;
+        case 4:
+          fi.indexValues = Values.BYTES_FIXED_STRAIGHT;
+          break;
+        case 5:
+          fi.indexValues = Values.BYTES_FIXED_DEREF;
+          break;
+        case 6:
+          fi.indexValues = Values.BYTES_FIXED_SORTED;
+          break;
+        case 7:
+          fi.indexValues = Values.BYTES_VAR_STRAIGHT;
+          break;
+        case 8:
+          fi.indexValues = Values.BYTES_VAR_DEREF;
+          break;
+        case 9:
+          fi.indexValues = Values.BYTES_VAR_SORTED;
+          break;
+        case 10:
+          fi.indexValues = Values.PACKED_INTS_FIXED;
+          break;
+        default:
+          throw new IllegalStateException("unhandled indexValues type " + b);
+        }
+      }
     }
 
     if (input.getFilePointer() != input.length()) {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FieldsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FieldsEnum.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FieldsEnum.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FieldsEnum.java Tue Oct 12 06:15:03 2010
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 
+import org.apache.lucene.index.values.ValuesEnum;
 import org.apache.lucene.util.AttributeSource;
 
 /** Enumerates indexed fields.  You must first call {@link
@@ -55,7 +56,7 @@ public abstract class FieldsEnum {
    *  null this method should not be called. This method
    *  will not return null. */
   public abstract TermsEnum terms() throws IOException;
-
+  
   public final static FieldsEnum[] EMPTY_ARRAY = new FieldsEnum[0];
 
   /** Provides zero fields */

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java Tue Oct 12 06:15:03 2010
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldSelector;
+import org.apache.lucene.index.values.ValuesEnum;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.search.FieldCache; // not great (circular); used only to purge FieldCache entry on close

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/IndexFileNames.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/IndexFileNames.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/IndexFileNames.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/IndexFileNames.java Tue Oct 12 06:15:03 2010
@@ -78,6 +78,12 @@ public final class IndexFileNames {
 
   /** Extension of separate norms */
   public static final String SEPARATE_NORMS_EXTENSION = "s";
+  
+  /** Extension of Column-Stride Filed data files */
+  public static final String CSF_DATA_EXTENSION = "dat";
+  
+  /** Extension of Column-Stride Filed index files */
+  public static final String CSF_INDEX_EXTENSION = "idx";
 
   /**
    * This array contains all filename extensions used by
@@ -98,6 +104,8 @@ public final class IndexFileNames {
     GEN_EXTENSION,
     NORMS_EXTENSION,
     COMPOUND_FILE_STORE_EXTENSION,
+    CSF_DATA_EXTENSION,
+    CSF_INDEX_EXTENSION
   };
 
   public static final String[] STORE_INDEX_EXTENSIONS = new String[] {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/IndexReader.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/IndexReader.java Tue Oct 12 06:15:03 2010
@@ -21,6 +21,8 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.FieldSelector;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.values.Cache;
+import org.apache.lucene.index.values.Reader;
 import org.apache.lucene.store.*;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -1373,6 +1375,17 @@ public abstract class IndexReader implem
     throw new UnsupportedOperationException("This reader does not support this method.");
   }
 
+  // nocommit -- should this expose the iterator API via Fields and access Source only via getIndexValuesCache?
+  public Reader getIndexValues(String field) {
+    throw new UnsupportedOperationException();
+  }
+
+  private final Cache indexValuesCache = new Cache(this);
+
+  // nocommit -- don't expose readers if we have this?
+  public Cache getIndexValuesCache() {
+    return indexValuesCache;
+  }
 
   private Fields fields;
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentInfo.java Tue Oct 12 06:15:03 2010
@@ -23,6 +23,7 @@ import org.apache.lucene.store.IndexInpu
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
+
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
@@ -31,6 +32,7 @@ import java.util.Set;
 import java.util.HashSet;
 import java.util.HashMap;
 import java.util.ArrayList;
+import java.util.regex.Pattern;
 
 /**
  * Information about a segment such as it's name, directory, and files related
@@ -476,7 +478,12 @@ public final class SegmentInfo {
     if (delFileName != null && (delGen >= YES || dir.fileExists(delFileName))) {
       fileSet.add(delFileName);
     }
-
+    //nocommit - is there a better way to get all the dat / idx files?
+    for(String file : dir.listAll()) {
+      if(file.startsWith(name) && (file.endsWith("dat") || file.endsWith("idx"))){
+        fileSet.add(file);
+      }
+    }
     if (normGen != null) {
       for (int i = 0; i < normGen.length; i++) {
         long gen = normGen[i];

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Tue Oct 12 06:15:03 2010
@@ -31,6 +31,12 @@ import org.apache.lucene.index.codecs.Co
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.MergeState;
 import org.apache.lucene.index.codecs.FieldsConsumer;
+import org.apache.lucene.index.values.Bytes;
+import org.apache.lucene.index.values.Ints;
+import org.apache.lucene.index.values.Reader;
+import org.apache.lucene.index.values.Floats;
+import org.apache.lucene.index.values.Values;
+import org.apache.lucene.index.values.Writer;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
@@ -157,6 +163,8 @@ final class SegmentMerger {
     if (mergeDocStores && fieldInfos.hasVectors())
       mergeVectors();
 
+    mergeIndexValues();
+
     return mergedDocs;
   }
 
@@ -170,6 +178,12 @@ final class SegmentMerger {
       reader.close();
     }
   }
+  
+  private void addIfExists(Set<String> files, String file, Directory dir) throws IOException{
+    if(dir.fileExists(file)){
+      files.add(file);
+    }
+  }
 
   final List<String> createCompoundFile(String fileName, final SegmentInfo info)
           throws IOException {
@@ -183,13 +197,20 @@ final class SegmentMerger {
                              !ext.equals(IndexFileNames.FIELDS_INDEX_EXTENSION)))
         fileSet.add(IndexFileNames.segmentFileName(segment, "", ext));
     }
-
     codec.files(directory, info, fileSet);
     
     // Fieldable norm files
-    int numFIs = fieldInfos.size();
+    final int numFIs = fieldInfos.size();
     for (int i = 0; i < numFIs; i++) {
-      FieldInfo fi = fieldInfos.fieldInfo(i);
+      final FieldInfo fi = fieldInfos.fieldInfo(i);
+      // Index Values aka. CSF
+      if (fi.indexValues != null) {
+        addIfExists(fileSet, IndexFileNames.segmentFileName(segment, Integer
+            .toString(fi.number), IndexFileNames.CSF_DATA_EXTENSION), directory);
+        addIfExists(fileSet, IndexFileNames.segmentFileName(segment, Integer
+            .toString(fi.number), IndexFileNames.CSF_INDEX_EXTENSION),
+            directory);
+      }
       if (fi.isIndexed && !fi.omitNorms) {
         fileSet.add(IndexFileNames.segmentFileName(segment, "", IndexFileNames.NORMS_EXTENSION));
         break;
@@ -288,10 +309,18 @@ final class SegmentMerger {
         int numReaderFieldInfos = readerFieldInfos.size();
         for (int j = 0; j < numReaderFieldInfos; j++) {
           FieldInfo fi = readerFieldInfos.fieldInfo(j);
-          fieldInfos.add(fi.name, fi.isIndexed, fi.storeTermVector,
-              fi.storePositionWithTermVector, fi.storeOffsetWithTermVector,
-              !reader.hasNorms(fi.name), fi.storePayloads,
-              fi.omitTermFreqAndPositions);
+          FieldInfo merged = fieldInfos.add(fi.name, fi.isIndexed, fi.storeTermVector,
+                                            fi.storePositionWithTermVector, fi.storeOffsetWithTermVector,
+                                            !reader.hasNorms(fi.name), fi.storePayloads,
+                                            fi.omitTermFreqAndPositions);
+          final Values fiIndexValues = fi.indexValues;
+          final Values mergedIndexValues = merged.indexValues;
+          if (mergedIndexValues == null) {
+            merged.setIndexValues(fiIndexValues);
+          } else if (mergedIndexValues != fiIndexValues) {
+            // nocommit -- what to do?
+            throw new IllegalStateException("cannot merge field " + fi.name + " indexValues changed from " + mergedIndexValues + " to " + fiIndexValues);
+          }
         }
       } else {
         addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION_OFFSET), true, true, true, false, false);
@@ -302,6 +331,8 @@ final class SegmentMerger {
         addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.STORES_PAYLOADS), false, false, false, true, false);
         addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.INDEXED), false, false, false, false, false);
         fieldInfos.add(reader.getFieldNames(FieldOption.UNINDEXED), false);
+
+        // nocommit -- how should we handle index values here?
       }
     }
     fieldInfos.write(directory, segment + ".fnm");
@@ -362,6 +393,77 @@ final class SegmentMerger {
     return docCount;
   }
 
+  private void mergeIndexValues() throws IOException {
+    final int numFields = fieldInfos.size();
+    for (int i = 0; i < numFields; i++) {
+      final FieldInfo fieldInfo = fieldInfos.fieldInfo(i);
+      final Values v = fieldInfo.indexValues;
+      // nocommit we need some kind of compatibility notation for values such
+      // that two slighly different segments can be merged eg. fixed vs.
+      // variable byte len or float32 vs. float64
+
+      if (v != null) {
+        int docBase = 0;
+        final List<Writer.MergeState> mergeStates = new ArrayList<Writer.MergeState>();
+        for (IndexReader reader : readers) {
+          Reader r = reader.getIndexValues(fieldInfo.name);
+          if (r != null) {
+            mergeStates.add(new Writer.MergeState(r, docBase, reader
+                .maxDoc(), reader.getDeletedDocs()));
+          }
+          docBase += reader.numDocs();
+        }
+        if (mergeStates.isEmpty()) {
+          continue;
+        }
+        final String id = segment + "_" + fieldInfo.number;
+        final Writer writer;
+        switch (v) {
+        case PACKED_INTS:
+        case PACKED_INTS_FIXED:
+          writer = Ints.getWriter(directory, id, true);
+          break;
+        case SIMPLE_FLOAT_4BYTE:
+          writer = Floats.getWriter(directory, id, 4);
+          break;
+        case SIMPLE_FLOAT_8BYTE:
+          writer = Floats.getWriter(directory, id, 8);
+          break;
+        case BYTES_FIXED_STRAIGHT:
+          writer = Bytes.getWriter(directory, id,
+              Bytes.Mode.STRAIGHT, null, true);
+          break;
+        case BYTES_FIXED_DEREF:
+          writer = Bytes.getWriter(directory, id,
+              Bytes.Mode.DEREF, null, true);
+          break;
+        case BYTES_FIXED_SORTED:
+          // nocommit -- enable setting Comparator
+          writer = Bytes.getWriter(directory, id,
+              Bytes.Mode.SORTED, null, true);
+          break;
+        case BYTES_VAR_STRAIGHT:
+          writer = Bytes.getWriter(directory, id,
+              Bytes.Mode.STRAIGHT, null, false);
+          break;
+        case BYTES_VAR_DEREF:
+          writer = Bytes.getWriter(directory, id,
+              Bytes.Mode.DEREF, null, false);
+          break;
+        case BYTES_VAR_SORTED:
+          // nocommit -- enable setting Comparator
+          writer = Bytes.getWriter(directory, id,
+              Bytes.Mode.SORTED, null, false);
+          break;
+        default:
+          continue;
+        }
+        writer.add(mergeStates);
+        writer.finish(mergedDocs);
+      }
+    }
+  }
+
   private int copyFieldsWithDeletions(final FieldsWriter fieldsWriter, final IndexReader reader,
                                       final FieldsReader matchingFieldsReader)
     throws IOException, MergeAbortedException, CorruptIndexException {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1021636&r1=1021635&r2=1021636&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentReader.java Tue Oct 12 06:15:03 2010
@@ -30,6 +30,7 @@ import java.util.Set;
 
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldSelector;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.store.BufferedIndexInput;
@@ -41,6 +42,11 @@ import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.CloseableThreadLocal;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.FieldsProducer;
+import org.apache.lucene.index.values.Bytes;
+import org.apache.lucene.index.values.Ints;
+import org.apache.lucene.index.values.Reader;
+import org.apache.lucene.index.values.Floats;
+import org.apache.lucene.index.values.Values;
 import org.apache.lucene.search.FieldCache; // not great (circular); used only to purge FieldCache entry on close
 import org.apache.lucene.util.BytesRef;
 
@@ -135,7 +141,7 @@ public class SegmentReader extends Index
         // Ask codec for its Fields
         fields = si.getCodec().fieldsProducer(new SegmentReadState(cfsDir, si, fieldInfos, readBufferSize, termsIndexDivisor));
         assert fields != null;
-
+        openIndexValuesReaders(cfsDir, si);
         success = true;
       } finally {
         if (!success) {
@@ -150,6 +156,57 @@ public class SegmentReader extends Index
       this.origInstance = origInstance;
     }
 
+    final Map<String,Reader> indexValues = new HashMap<String,Reader>();
+
+    // Only opens files... doesn't actually load any values
+    private void openIndexValuesReaders(Directory dir, SegmentInfo si) throws IOException {
+      final int numFields = fieldInfos.size();
+      for(int i=0;i<numFields;i++) {
+        final FieldInfo fieldInfo = fieldInfos.fieldInfo(i);
+        final Values v = fieldInfo.getIndexValues();
+        final String field = fieldInfo.name;
+        final String id = IndexFileNames.segmentFileName(segment, Integer
+            .toString(fieldInfo.number), "");
+        // nocommit - externalize the filenames 
+        if (v != null && dir.fileExists(id+".dat")) {
+          switch(v) {
+          case PACKED_INTS:
+            indexValues.put(field, Ints.getReader(dir, id, false));
+            break;
+          case PACKED_INTS_FIXED:
+            indexValues.put(field, Ints.getReader(dir, id, true));
+            break;
+          case SIMPLE_FLOAT_4BYTE:
+            indexValues.put(field, Floats.getReader(dir, id, si.docCount));
+            break;
+          case SIMPLE_FLOAT_8BYTE:
+            indexValues.put(field, Floats.getReader(dir, id, si.docCount));
+            break;
+          case BYTES_FIXED_STRAIGHT:
+            indexValues.put(field, Bytes.getReader(dir, id, Bytes.Mode.STRAIGHT, true, si.docCount));
+            break;
+          case BYTES_FIXED_DEREF:
+            indexValues.put(field, Bytes.getReader(dir, id, Bytes.Mode.DEREF, true, si.docCount));
+            break;
+          case BYTES_FIXED_SORTED:
+            indexValues.put(field, Bytes.getReader(dir, id, Bytes.Mode.SORTED, true, si.docCount));
+            break;
+          case BYTES_VAR_STRAIGHT:
+            indexValues.put(field, Bytes.getReader(dir, id, Bytes.Mode.STRAIGHT, false, si.docCount));
+            break;
+          case BYTES_VAR_DEREF:
+            indexValues.put(field, Bytes.getReader(dir, id, Bytes.Mode.DEREF, false, si.docCount));
+            break;
+          case BYTES_VAR_SORTED:
+            indexValues.put(field, Bytes.getReader(dir, id, Bytes.Mode.SORTED, false, si.docCount));
+            break;
+          default:
+            throw new IllegalStateException("unrecognized index values mode " + v);
+          }
+        }
+      }
+    }
+
     synchronized TermVectorsReader getTermVectorsReaderOrig() {
       return termVectorsReaderOrig;
     }
@@ -167,9 +224,7 @@ public class SegmentReader extends Index
     }
 
     synchronized void decRef() throws IOException {
-
       if (ref.decrementAndGet() == 0) {
-
         if (fields != null) {
           fields.close();
         }
@@ -198,9 +253,17 @@ public class SegmentReader extends Index
         if (origInstance != null) {
           FieldCache.DEFAULT.purge(origInstance);
         }
+        closeIndexValuesReaders();
+      }
+    }
+
+    private void closeIndexValuesReaders() throws IOException {
+      for (Reader reader : indexValues.values()) {
+        reader.close();
       }
     }
 
+
     synchronized void openDocStores(SegmentInfo si) throws IOException {
 
       assert si.name.equals(segment);
@@ -1281,4 +1344,9 @@ public class SegmentReader extends Index
   public int getTermInfosIndexDivisor() {
     return core.termsIndexDivisor;
   }
+
+  @Override
+  public Reader getIndexValues(String field) {
+    return core.indexValues.get(field);
+  }
 }

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,301 @@
+package org.apache.lucene.index.values;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Base class for specific Bytes Reader/Writer implementations */
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.values.Reader.SortedSource;
+import org.apache.lucene.index.values.Reader.Source;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.ByteBlockPool;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CodecUtil;
+
+/**
+ * Provides concrete Writer/Reader impls for byte[] value per document. There
+ * are 6 package-private impls of this, for all combinations of
+ * STRAIGHT/DEREF/SORTED X fixed/not fixed.
+ * 
+ * <p>
+ * NOTE: The total amount of byte[] data stored (across a single segment) cannot
+ * exceed 2GB.
+ * </p>
+ * <p>
+ * NOTE: Each byte[] must be <= 32768 bytes in length
+ * </p>
+ */
+//nocommit - add mmap version 
+//nocommti - add bulk copy where possible
+public final class Bytes {
+
+  // don't instantiate!
+  private Bytes() {
+  }
+
+  public static enum Mode {
+    STRAIGHT, DEREF, SORTED
+  };
+
+  public static void files(Directory dir, String id, Collection<String> files)
+      throws IOException {
+    files.add(IndexFileNames.segmentFileName(id, "",
+        IndexFileNames.CSF_DATA_EXTENSION));
+    final String idxFile = IndexFileNames.segmentFileName(id, "",
+        IndexFileNames.CSF_INDEX_EXTENSION);
+    if (dir.fileExists(idxFile)) {
+      files.add(idxFile);
+    }
+  }
+
+  // nocommit -- i shouldn't have to specify fixed? can
+  // track itself & do the write thing at write time?
+  public static Writer getWriter(Directory dir, String id, Mode mode,
+      Comparator<BytesRef> comp, boolean fixedSize) throws IOException {
+
+    if (comp == null) {
+      comp = BytesRef.getUTF8SortedAsUnicodeComparator();
+    }
+
+    if (fixedSize) {
+      if (mode == Mode.STRAIGHT) {
+        return new FixedStraightBytesImpl.Writer(dir, id);
+      } else if (mode == Mode.DEREF) {
+        return new FixedDerefBytesImpl.Writer(dir, id);
+      } else if (mode == Mode.SORTED) {
+        return new FixedSortedBytesImpl.Writer(dir, id, comp);
+      }
+    } else {
+      if (mode == Mode.STRAIGHT) {
+        return new VarStraightBytesImpl.Writer(dir, id);
+      } else if (mode == Mode.DEREF) {
+        return new VarDerefBytesImpl.Writer(dir, id);
+      } else if (mode == Mode.SORTED) {
+        return new VarSortedBytesImpl.Writer(dir, id, comp);
+      }
+    }
+
+    throw new IllegalArgumentException("");
+  }
+
+  // nocommit -- I can peek @ header to determing fixed/mode?
+  public static Reader getReader(Directory dir, String id, Mode mode,
+      boolean fixedSize, int maxDoc) throws IOException {
+    if (fixedSize) {
+      if (mode == Mode.STRAIGHT) {
+        try {
+          return new FixedStraightBytesImpl.Reader(dir, id, maxDoc);
+        } catch (IOException e) {
+          throw e;
+        }
+      } else if (mode == Mode.DEREF) {
+        try {
+          return new FixedDerefBytesImpl.Reader(dir, id, maxDoc);
+        } catch (IOException e) {
+          throw e;
+        }
+      } else if (mode == Mode.SORTED) {
+        return new FixedSortedBytesImpl.Reader(dir, id, maxDoc);
+      }
+    } else {
+      if (mode == Mode.STRAIGHT) {
+        return new VarStraightBytesImpl.Reader(dir, id, maxDoc);
+      } else if (mode == Mode.DEREF) {
+        return new VarDerefBytesImpl.Reader(dir, id, maxDoc);
+      } else if (mode == Mode.SORTED) {
+        return new VarSortedBytesImpl.Reader(dir, id, maxDoc);
+      }
+    }
+
+    throw new IllegalArgumentException("");
+  }
+
+  static abstract class BytesBaseSource extends Source {
+    protected final IndexInput datIn;
+    protected final IndexInput idxIn;
+    protected final BytesRef defaultValue = new BytesRef();
+
+    protected BytesBaseSource(IndexInput datIn, IndexInput idxIn) {
+      this.datIn = datIn;
+      this.idxIn = idxIn;
+    }
+
+    public void close() throws IOException {
+      if (datIn != null)
+        datIn.close();
+      if (idxIn != null) // if straight
+        idxIn.close();
+
+    }
+  }
+
+  static abstract class BytesBaseSortedSource extends SortedSource {
+    protected final IndexInput datIn;
+    protected final IndexInput idxIn;
+    protected final BytesRef defaultValue = new BytesRef();
+
+    protected BytesBaseSortedSource(IndexInput datIn, IndexInput idxIn) {
+      this.datIn = datIn;
+      this.idxIn = idxIn;
+    }
+
+    public void close() throws IOException {
+      if (datIn != null)
+        datIn.close();
+      if (idxIn != null) // if straight
+        idxIn.close();
+
+    }
+  }
+
+  static abstract class BytesWriterBase extends Writer {
+
+    private final Directory dir;
+    private final String id;
+    protected IndexOutput idxOut;
+    protected IndexOutput datOut;
+    protected BytesRef bytesRef;
+    private String codecName;
+    private int version;
+    protected final ByteBlockPool pool;
+    protected final AtomicLong bytesUsed;
+
+    protected BytesWriterBase(Directory dir, String id, String codecName,
+        int version, boolean initIndex, boolean initData, ByteBlockPool pool, AtomicLong bytesUsed) throws IOException {
+      this.dir = dir;
+      this.id = id;
+      this.codecName = codecName;
+      this.version = version;
+      this.pool = pool;
+      this.bytesUsed = bytesUsed;
+      if (initData)
+        initDataOut();
+      if (initIndex)
+        initIndexOut();
+    }
+
+    protected void initDataOut() throws IOException {
+      datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
+          IndexFileNames.CSF_DATA_EXTENSION));
+      CodecUtil.writeHeader(datOut, codecName, version);
+    }
+
+    protected void initIndexOut() throws IOException {
+      idxOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
+          IndexFileNames.CSF_INDEX_EXTENSION));
+      CodecUtil.writeHeader(idxOut, codecName, version);
+    }
+
+    public long ramBytesUsed() {
+      return bytesUsed.get();
+    }
+
+    /**
+     * Must be called only with increasing docIDs. It's OK for some docIDs to be
+     * skipped; they will be filled with 0 bytes.
+     */
+    @Override
+    public abstract void add(int docID, BytesRef bytes) throws IOException;
+
+    @Override
+    public synchronized void finish(int docCount) throws IOException {
+      if (datOut != null)
+        datOut.close();
+      if (idxOut != null)
+        idxOut.close();
+      if(pool != null)
+        pool.reset();
+    }
+
+    @Override
+    protected void add(int docID) throws IOException {
+      add(docID, bytesRef);
+    }
+
+    @Override
+    protected void setNextAttribute(ValuesAttribute attr) {
+      bytesRef = attr.bytes();
+      assert bytesRef != null;
+    }
+  }
+
+  /**
+   * Opens all necessary files, but does not read any data in until you call
+   * {@link #load}.
+   */
+   static abstract class BytesReaderBase extends Reader {
+    protected final IndexInput idxIn;
+    protected final IndexInput datIn;
+    protected final int version;
+    protected final String id;
+
+    protected BytesReaderBase(Directory dir, String id, String codecName,
+        int maxVersion, boolean doIndex) throws IOException {
+      this.id = id;
+      datIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
+          IndexFileNames.CSF_DATA_EXTENSION));
+      version = CodecUtil.checkHeader(datIn, codecName, maxVersion, maxVersion);
+
+      if (doIndex) {
+        idxIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
+            IndexFileNames.CSF_INDEX_EXTENSION));
+        final int version2 = CodecUtil.checkHeader(idxIn, codecName,
+            maxVersion, maxVersion);
+        assert version == version2;
+      } else {
+        idxIn = null;
+      }
+    }
+
+    protected final IndexInput cloneData() {
+      assert !isClosed.get():printEx();
+      // is never NULL
+      return (IndexInput) datIn.clone();
+    }
+
+    protected final IndexInput cloneIndex() {
+      assert !isClosed.get():printEx();
+      return idxIn == null ? null : (IndexInput) idxIn.clone();
+    }
+    private final AtomicBoolean isClosed = new AtomicBoolean(false);
+    Exception ex;
+    public void close() throws IOException {
+      assert !isClosed.getAndSet(true);
+      ex =new Exception();
+      if (datIn != null) {
+        datIn.close();
+      }
+      if (idxIn != null) {
+        idxIn.close();
+      }
+    }
+    
+    private String printEx() {
+      ex.printStackTrace();
+      return ex.getMessage();
+    }
+  }
+
+}
\ No newline at end of file

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Cache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Cache.java?rev=1021636&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Cache.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Cache.java Tue Oct 12 06:15:03 2010
@@ -0,0 +1,116 @@
+package org.apache.lucene.index.values;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.values.Reader.SortedSource;
+import org.apache.lucene.index.values.Reader.Source;
+import org.apache.lucene.util.BytesRef;
+
+public class Cache {
+  final IndexReader r;
+  // TODO(simonw): use WeakHashMaps instead here?
+  final Map<String, Source> ints = new HashMap<String, Source>();
+  final Map<String, Source> floats = new HashMap<String, Source>();
+  final Map<String, Source> bytes = new HashMap<String, Source>();
+  final Map<String, SortedSource> sortedBytes = new HashMap<String, SortedSource>();
+
+  public Cache(IndexReader r) {
+    this.r = r;
+  }
+
+  synchronized public Source getInts(String id) throws IOException {
+    Source s = ints.get(id);
+    if (s == null) {
+      final Reader indexValues = r.getIndexValues(id);
+      if (indexValues == null) {
+        return null;
+      }
+      s = indexValues.load();
+      ints.put(id, s);
+    }
+
+    return s;
+  }
+
+  synchronized public Source getFloats(String id) throws IOException {
+    Source s = floats.get(id);
+    if (s == null) {
+      final Reader indexValues = r.getIndexValues(id);
+      if (indexValues == null) {
+        return null;
+      }
+      s = indexValues.load();
+      floats.put(id, s);
+    }
+
+    return s;
+  }
+
+  synchronized public SortedSource getSortedBytes(String id,
+      Comparator<BytesRef> comp) throws IOException {
+    SortedSource s = sortedBytes.get(id);
+    if (s == null) {
+      final Reader indexValues = r.getIndexValues(id);
+      if (indexValues == null) {
+        return null;
+      }
+      s = indexValues.loadSorted(comp);
+      sortedBytes.put(id, s);
+    } else {
+      // TODO(simonw): verify comp is the same!
+    }
+
+    return s;
+  }
+
+  synchronized public Source getBytes(String id) throws IOException {
+    Source s = bytes.get(id);
+    if (s == null) {
+      final Reader indexValues = r.getIndexValues(id);
+      if (indexValues == null) {
+        return null;
+      }
+      s = indexValues.load();
+      bytes.put(id, s);
+    }
+
+    return s;
+  }
+
+  public void purgeInts(String id) {
+    ints.remove(id);
+  }
+
+  public void purgeFloats(String id) {
+    floats.remove(id);
+  }
+
+  public void purgeBytes(String id) {
+    bytes.remove(id);
+  }
+
+  public void purgeSortedBytes(String id) {
+    sortedBytes.remove(id);
+  }
+}

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Cache.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Cache.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL