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