Index: lucene/src/test/org/apache/lucene/TestExternalCodecs.java =================================================================== --- lucene/src/test/org/apache/lucene/TestExternalCodecs.java (revision 1099621) +++ lucene/src/test/org/apache/lucene/TestExternalCodecs.java (working copy) @@ -490,6 +490,16 @@ } @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + return null; + } + + @Override + public PerDocValues docsProducer(SegmentReadState state) throws IOException { + return null; + } + + @Override public void getExtensions(Set extensions) { } Index: lucene/src/test/org/apache/lucene/search/TestSort.java =================================================================== --- lucene/src/test/org/apache/lucene/search/TestSort.java (revision 1099621) +++ lucene/src/test/org/apache/lucene/search/TestSort.java (working copy) @@ -37,7 +37,6 @@ import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.Term; import org.apache.lucene.index.codecs.CodecProvider; -import org.apache.lucene.index.codecs.docvalues.DocValuesCodecProvider; import org.apache.lucene.index.values.Type; import org.apache.lucene.queryParser.ParseException; import org.apache.lucene.search.BooleanClause.Occur; @@ -228,10 +227,8 @@ public void setUp() throws Exception { super.setUp(); - //nocommit - enable doc values by default for all tests - DocValuesCodecProvider provider = new DocValuesCodecProvider(); - provider.copyFrom(CodecProvider.getDefault()); - CodecProvider.setDefault(provider); + //nocommit - we need to be able to run this test with preflex, but without docvalues!!!! + assumeFalse("cannot work with preflex codec", CodecProvider.getDefault().getDefaultFieldCodec().equals("PreFlex")); full = getFullIndex(); searchX = getXIndex(); searchY = getYIndex(); Index: lucene/src/test/org/apache/lucene/index/values/TestDocValuesIndexing.java =================================================================== --- lucene/src/test/org/apache/lucene/index/values/TestDocValuesIndexing.java (revision 1099621) +++ lucene/src/test/org/apache/lucene/index/values/TestDocValuesIndexing.java (working copy) @@ -41,7 +41,6 @@ import org.apache.lucene.index.Term; import org.apache.lucene.index.codecs.CodecProvider; import org.apache.lucene.index.codecs.PerDocValues; -import org.apache.lucene.index.codecs.docvalues.DocValuesCodecProvider; import org.apache.lucene.index.values.DocValues.MissingValue; import org.apache.lucene.index.values.DocValues.Source; import org.apache.lucene.queryParser.ParseException; @@ -76,13 +75,10 @@ * - run RAT */ - private CodecProvider provider; - @Before public void setUp() throws Exception { super.setUp(); - provider = new DocValuesCodecProvider(); - provider.copyFrom(CodecProvider.getDefault()); + assumeFalse("cannot work with preflex codec", CodecProvider.getDefault().getDefaultFieldCodec().equals("PreFlex")); } /* @@ -105,7 +101,7 @@ writer.close(true); - IndexReader reader = IndexReader.open(dir, null, true, 1, provider); + IndexReader reader = IndexReader.open(dir, null, true, 1); assertTrue(reader.isOptimized()); IndexSearcher searcher = new IndexSearcher(reader); @@ -244,7 +240,7 @@ LogMergePolicy policy = new LogDocMergePolicy(); cfg.setMergePolicy(policy); policy.setUseCompoundFile(useCompoundFile); - cfg.setCodecProvider(provider); + //cfg.setCodecProvider(provider); return cfg; } Index: lucene/src/test/org/apache/lucene/index/TestDocTermOrds.java =================================================================== --- lucene/src/test/org/apache/lucene/index/TestDocTermOrds.java (revision 1099621) +++ lucene/src/test/org/apache/lucene/index/TestDocTermOrds.java (working copy) @@ -33,10 +33,15 @@ import org.apache.lucene.index.codecs.BlockTermsWriter; import org.apache.lucene.index.codecs.Codec; import org.apache.lucene.index.codecs.CoreCodecProvider; +import org.apache.lucene.index.codecs.DocValuesConsumer; +import org.apache.lucene.index.codecs.DefaultDocValuesProducer; import org.apache.lucene.index.codecs.FieldsConsumer; import org.apache.lucene.index.codecs.FieldsProducer; import org.apache.lucene.index.codecs.FixedGapTermsIndexReader; import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter; +import org.apache.lucene.index.codecs.PerDocConsumer; +import org.apache.lucene.index.codecs.DefaultDocValuesConsumer; +import org.apache.lucene.index.codecs.PerDocValues; import org.apache.lucene.index.codecs.PostingsReaderBase; import org.apache.lucene.index.codecs.PostingsWriterBase; import org.apache.lucene.index.codecs.TermsIndexReaderBase; @@ -196,11 +201,13 @@ StandardPostingsReader.files(dir, segmentInfo, ""+id, files); BlockTermsReader.files(dir, segmentInfo, ""+id, files); FixedGapTermsIndexReader.files(dir, segmentInfo, ""+id, files); + DefaultDocValuesConsumer.files(dir, segmentInfo, id, files); } @Override public void getExtensions(Set extensions) { getStandardExtensions(extensions); + DefaultDocValuesConsumer.getDocValuesExtensions(extensions); } public static void getStandardExtensions(Set extensions) { @@ -209,6 +216,16 @@ BlockTermsReader.getExtensions(extensions); FixedGapTermsIndexReader.getIndexExtensions(extensions); } + + @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + return new DefaultDocValuesConsumer(state, BytesRef.getUTF8SortedAsUnicodeComparator()); + } + + @Override + public PerDocValues docsProducer(SegmentReadState state) throws IOException { + return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId); + } } public void testRandom() throws Exception { Index: lucene/src/java/org/apache/lucene/index/values/PerDocFieldValues.java =================================================================== --- lucene/src/java/org/apache/lucene/index/values/PerDocFieldValues.java (revision 1099621) +++ lucene/src/java/org/apache/lucene/index/values/PerDocFieldValues.java (working copy) @@ -20,7 +20,7 @@ import org.apache.lucene.document.DocValuesField; import org.apache.lucene.document.Fieldable; -import org.apache.lucene.index.codecs.docvalues.DocValuesConsumer; +import org.apache.lucene.index.codecs.DocValuesConsumer; import org.apache.lucene.util.BytesRef; /** Index: lucene/src/java/org/apache/lucene/index/values/Writer.java =================================================================== --- lucene/src/java/org/apache/lucene/index/values/Writer.java (revision 1099621) +++ lucene/src/java/org/apache/lucene/index/values/Writer.java (working copy) @@ -20,7 +20,7 @@ import java.util.Comparator; import java.util.concurrent.atomic.AtomicLong; -import org.apache.lucene.index.codecs.docvalues.DocValuesConsumer; +import org.apache.lucene.index.codecs.DocValuesConsumer; import org.apache.lucene.store.Directory; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; Index: lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java =================================================================== --- lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java (revision 1099621) +++ lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java (working copy) @@ -30,7 +30,7 @@ import org.apache.lucene.index.DocumentsWriterPerThread.DocState; import org.apache.lucene.index.codecs.Codec; import org.apache.lucene.index.codecs.PerDocConsumer; -import org.apache.lucene.index.codecs.docvalues.DocValuesConsumer; +import org.apache.lucene.index.codecs.DocValuesConsumer; import org.apache.lucene.util.ArrayUtil; Index: lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java =================================================================== --- lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java (revision 1099621) +++ lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java (working copy) @@ -32,7 +32,7 @@ import org.apache.lucene.index.codecs.PerDocConsumer; import org.apache.lucene.index.codecs.PerDocValues; import org.apache.lucene.index.codecs.TermsConsumer; -import org.apache.lucene.index.codecs.docvalues.DocValuesConsumer; +import org.apache.lucene.index.codecs.DocValuesConsumer; import org.apache.lucene.index.values.DocValues; import org.apache.lucene.store.Directory; Index: lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java (revision 1099621) +++ lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java (working copy) @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.Set; +import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentReadState; @@ -28,8 +29,13 @@ import org.apache.lucene.index.codecs.standard.StandardPostingsWriter; import org.apache.lucene.index.codecs.PostingsReaderBase; import org.apache.lucene.index.codecs.standard.StandardPostingsReader; +import org.apache.lucene.index.codecs.DocValuesConsumer; +import org.apache.lucene.index.codecs.DefaultDocValuesProducer; import org.apache.lucene.index.codecs.FieldsConsumer; import org.apache.lucene.index.codecs.FieldsProducer; +import org.apache.lucene.index.codecs.PerDocConsumer; +import org.apache.lucene.index.codecs.DefaultDocValuesConsumer; +import org.apache.lucene.index.codecs.PerDocValues; import org.apache.lucene.index.codecs.VariableGapTermsIndexReader; import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter; import org.apache.lucene.index.codecs.BlockTermsReader; @@ -38,6 +44,7 @@ import org.apache.lucene.index.codecs.TermsIndexWriterBase; import org.apache.lucene.index.codecs.standard.StandardCodec; import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; /** This codec "inlines" the postings for terms that have * low docFreq. It wraps another codec, which is used for @@ -155,10 +162,22 @@ StandardPostingsReader.files(dir, segmentInfo, codecId, files); BlockTermsReader.files(dir, segmentInfo, codecId, files); VariableGapTermsIndexReader.files(dir, segmentInfo, codecId, files); + DefaultDocValuesConsumer.files(dir, segmentInfo, id, files); } @Override public void getExtensions(Set extensions) { StandardCodec.getStandardExtensions(extensions); + DefaultDocValuesConsumer.getDocValuesExtensions(extensions); } + + @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + return new DefaultDocValuesConsumer(state, BytesRef.getUTF8SortedAsUnicodeComparator()); + } + + @Override + public PerDocValues docsProducer(SegmentReadState state) throws IOException { + return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId); + } } Index: lucene/src/java/org/apache/lucene/index/codecs/DocValuesConsumer.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/DocValuesConsumer.java (revision 1099621) +++ lucene/src/java/org/apache/lucene/index/codecs/DocValuesConsumer.java (working copy) @@ -1,4 +1,4 @@ -package org.apache.lucene.index.codecs.docvalues; +package org.apache.lucene.index.codecs; /** * Licensed to the Apache Software Foundation (ASF) under one or more Index: lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesConsumer.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesConsumer.java (revision 1099621) +++ lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesConsumer.java (working copy) @@ -1,167 +0,0 @@ -package org.apache.lucene.index.codecs.docvalues; - -/** - * 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.Collection; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.values.DocValues; -import org.apache.lucene.index.values.PerDocFieldValues; -import org.apache.lucene.index.values.Writer; -import org.apache.lucene.util.Bits; - -/** - * Abstract API that consumes {@link PerDocFieldValues}. - * {@link DocValuesConsumer} are always associated with a specific field and - * segments. Concrete implementations of this API write the given - * {@link PerDocFieldValues} into a implementation specific format depending on - * the fields meta-data. - * - * @lucene.experimental - */ -public abstract class DocValuesConsumer { - // TODO this might need to go in the codec package since is a direct relative - // to TermsConsumer - protected final AtomicLong bytesUsed; - - /** - * Creates a new {@link DocValuesConsumer}. - * - * @param bytesUsed - * bytes-usage tracking reference used by implementation to track - * internally allocated memory. All tracked bytes must be released - * once {@link #finish(int)} has been called. - */ - protected DocValuesConsumer(AtomicLong bytesUsed) { - this.bytesUsed = bytesUsed == null ? new AtomicLong(0) : bytesUsed; - } - - /** - * Adds the given {@link PerDocFieldValues} instance to this - * {@link DocValuesConsumer} - * - * @param docID - * the document ID to add the value for. The docID must always - * increase or be 0 if it is the first call to this method. - * @param docValues - * the values to add - * @throws IOException - * if an {@link IOException} occurs - */ - public abstract void add(int docID, PerDocFieldValues docValues) - throws IOException; - - /** - * Called when the consumer of this API is doc with adding - * {@link PerDocFieldValues} to this {@link DocValuesConsumer} - * - * @param docCount - * the total number of documents in this {@link DocValuesConsumer}. - * Must be greater than or equal the last given docID to - * {@link #add(int, PerDocFieldValues)}. - * @throws IOException - */ - public abstract void finish(int docCount) throws IOException; - - /** - * Gathers files associated with this {@link DocValuesConsumer} - * - * @param files - * the of files to add the consumers files to. - */ - public abstract void files(Collection files) throws IOException; - - /** - * Merges the given {@link org.apache.lucene.index.codecs.MergeState} into - * this {@link DocValuesConsumer}. - * - * @param mergeState - * the state to merge - * @param values - * the docValues to merge in - * @throws IOException - * if an {@link IOException} occurs - */ - public void merge(org.apache.lucene.index.codecs.MergeState mergeState, - DocValues values) throws IOException { - assert mergeState != null; - // TODO we need some kind of compatibility notation for values such - // that two slightly different segments can be merged eg. fixed vs. - // variable byte len or float32 vs. float64 - int docBase = 0; - boolean merged = false; - /* - * We ignore the given DocValues here and merge from the subReaders directly - * to support bulk copies on the DocValues Writer level. if this gets merged - * with MultiDocValues the writer can not optimize for bulk-copyable data - */ - for (final IndexReader reader : mergeState.readers) { - final DocValues r = reader.docValues(mergeState.fieldInfo.name); - if (r != null) { - merged = true; - merge(new Writer.MergeState(r, docBase, reader.maxDoc(), reader - .getDeletedDocs())); - } - docBase += reader.numDocs(); - } - if (merged) { - finish(mergeState.mergedDocCount); - } - } - - /** - * Merges the given {@link MergeState} into this {@link DocValuesConsumer}. - * {@link MergeState#docBase} must always be increasing. Merging segments out - * of order is not supported. - * - * @param mergeState - * the {@link MergeState} to merge - * @throws IOException - * if an {@link IOException} occurs - */ - protected abstract void merge(MergeState mergeState) throws IOException; - - /** - * Specialized auxiliary MergeState is necessary since we don't want to - * exploit internals up to the codecs consumer. An instance of this class is - * created for each merged low level {@link IndexReader} we are merging to - * support low level bulk copies. - */ - public static class MergeState { - /** - * the source reader for this MergeState - merged values should be read from - * this instance - */ - public final DocValues reader; - /** the absolute docBase for this MergeState within the resulting segment */ - public final int docBase; - /** the number of documents in this MergeState */ - public final int docCount; - /** the deleted bits for this MergeState */ - public final Bits bits; - - public MergeState(DocValues reader, int docBase, int docCount, Bits bits) { - assert reader != null; - this.reader = reader; - this.docBase = docBase; - this.docCount = docCount; - this.bits = bits; - } - } -} Index: lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodec.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodec.java (revision 1099621) +++ lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodec.java (working copy) @@ -1,145 +0,0 @@ -package org.apache.lucene.index.codecs.docvalues; - -/** - * 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.Set; - -import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.FieldInfos; -import org.apache.lucene.index.IndexFileNames; -import org.apache.lucene.index.PerDocWriteState; -import org.apache.lucene.index.SegmentInfo; -import org.apache.lucene.index.SegmentReadState; -import org.apache.lucene.index.SegmentWriteState; -import org.apache.lucene.index.codecs.Codec; -import org.apache.lucene.index.codecs.FieldsConsumer; -import org.apache.lucene.index.codecs.FieldsProducer; -import org.apache.lucene.index.codecs.PerDocConsumer; -import org.apache.lucene.index.codecs.PerDocValues; -import org.apache.lucene.index.values.Writer; -import org.apache.lucene.store.Directory; -import org.apache.lucene.util.BytesRef; - -/** - * A codec that adds DocValues support to a given codec transparently. - * @lucene.experimental - */ -public class DocValuesCodec extends Codec { - private final Codec other; - private final Comparator comparator; - - public DocValuesCodec(Codec other, Comparator comparator) { - this.name = other.name; - this.other = other; - this.comparator = comparator; - } - - public DocValuesCodec(Codec other) { - this(other, null); - } - - @Override - public PerDocConsumer docsConsumer(final PerDocWriteState state) - throws IOException { - return new PerDocConsumer() { - public void close() throws IOException { - } - - @Override - public DocValuesConsumer addValuesField(FieldInfo field) - throws IOException { - final DocValuesConsumer consumer = Writer.create(field.getDocValues(), - docValuesId(state.segmentName, state.codecId, field.number), - // TODO can we have a compound file per segment and codec for - // docvalues? - state.directory, comparator, state.bytesUsed); - return consumer; - } - }; - } - - @Override - public PerDocValues docsProducer(SegmentReadState state) throws IOException { - return new DocValuesProducerBase(state.segmentInfo, state.dir, state.fieldInfos, state.codecId); - } - - @Override - public FieldsConsumer fieldsConsumer(SegmentWriteState state) - throws IOException { - return other.fieldsConsumer(state); - } - - @Override - public FieldsProducer fieldsProducer(SegmentReadState state) - throws IOException { - return other.fieldsProducer(state); - } - - static String docValuesId(String segmentsName, int codecID, int fieldId) { - return segmentsName + "_" + codecID + "-" + fieldId; - } - - @Override - public void files(Directory dir, SegmentInfo segmentInfo, int codecId, - Set files) throws IOException { - FieldInfos fieldInfos = segmentInfo.getFieldInfos(); - boolean indexed = false; - for (FieldInfo fieldInfo : fieldInfos) { - if (fieldInfo.getCodecId() == codecId) { - indexed |= fieldInfo.isIndexed; - if (fieldInfo.hasDocValues()) { - String filename = docValuesId(segmentInfo.name, codecId, fieldInfo.number); - switch (fieldInfo.getDocValues()) { - case BYTES_FIXED_DEREF: - case BYTES_VAR_DEREF: - case BYTES_VAR_SORTED: - case BYTES_FIXED_SORTED: - case BYTES_VAR_STRAIGHT: - files.add(IndexFileNames.segmentFileName(filename, "", - Writer.INDEX_EXTENSION)); - assert dir.fileExists(IndexFileNames.segmentFileName(filename, "", - Writer.INDEX_EXTENSION)); - case BYTES_FIXED_STRAIGHT: - case FLOAT_32: - case FLOAT_64: - case INTS: - files.add(IndexFileNames.segmentFileName(filename, "", - Writer.DATA_EXTENSION)); - assert dir.fileExists(IndexFileNames.segmentFileName(filename, "", - Writer.DATA_EXTENSION)); - break; - default: - assert false; - } - } - - } - } - if (indexed) { - other.files(dir, segmentInfo, codecId, files); - } - } - - @Override - public void getExtensions(Set extensions) { - other.getExtensions(extensions); - extensions.add(Writer.DATA_EXTENSION); - extensions.add(Writer.INDEX_EXTENSION); - } -} Index: lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesProducerBase.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesProducerBase.java (revision 1099621) +++ lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesProducerBase.java (working copy) @@ -1,152 +0,0 @@ -package org.apache.lucene.index.codecs.docvalues; - -/** - * 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.Collection; -import java.util.TreeMap; - -import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.FieldInfos; -import org.apache.lucene.index.SegmentInfo; -import org.apache.lucene.index.codecs.PerDocValues; -import org.apache.lucene.index.values.Bytes; -import org.apache.lucene.index.values.DocValues; -import org.apache.lucene.index.values.Floats; -import org.apache.lucene.index.values.Ints; -import org.apache.lucene.index.values.Type; -import org.apache.lucene.store.Directory; - -/** - * Abstract base class for FieldsProducer implementations supporting - * {@link DocValues}. - * - * @lucene.experimental - */ -public class DocValuesProducerBase extends PerDocValues { - - protected final TreeMap docValues = new TreeMap(); - - /** - * Creates a new {@link DocValuesProducerBase} instance and loads all - * {@link DocValues} instances for this segment and codec. - * - * @param si - * the segment info to load the {@link DocValues} for. - * @param dir - * the directory to load the {@link DocValues} from. - * @param fieldInfo - * the {@link FieldInfos} - * @param codecId - * the codec ID - * @throws IOException - * if an {@link IOException} occurs - */ - protected DocValuesProducerBase(SegmentInfo si, Directory dir, - FieldInfos fieldInfo, int codecId) throws IOException { - load(fieldInfo, si.name, si.docCount, dir, codecId); - } - - /** - * Returns a {@link DocValues} instance for the given field name or - * null if this field has no {@link DocValues}. - */ - @Override - public DocValues docValues(String field) throws IOException { - return docValues.get(field); - } - - // Only opens files... doesn't actually load any values - protected void load(FieldInfos fieldInfos, String segment, int docCount, - Directory dir, int codecId) throws IOException { - for (FieldInfo fieldInfo : fieldInfos) { - if (codecId == fieldInfo.getCodecId() && fieldInfo.hasDocValues()) { - final String field = fieldInfo.name; - // TODO can we have a compound file per segment and codec for docvalues? - final String id = DocValuesCodec.docValuesId(segment, codecId, fieldInfo.number); - docValues.put(field, loadDocValues(docCount, dir, id, fieldInfo - .getDocValues())); - } - } - } - - - /** - * Loads a {@link DocValues} instance depending on the given {@link Type}. - * Codecs that use different implementations for a certain {@link Type} can - * simply override this method and return their custom implementations. - * - * @param docCount - * number of documents in the segment - * @param dir - * the {@link Directory} to load the {@link DocValues} from - * @param id - * the unique file ID within the segment - * @param type - * the type to load - * @return a {@link DocValues} instance for the given type - * @throws IOException - * if an {@link IOException} occurs - * @throws IllegalArgumentException - * if the given {@link Type} is not supported - */ - protected DocValues loadDocValues(int docCount, Directory dir, String id, - Type type) throws IOException { - switch (type) { - case INTS: - return Ints.getValues(dir, id, false); - case FLOAT_32: - return Floats.getValues(dir, id, docCount); - case FLOAT_64: - return Floats.getValues(dir, id, docCount); - case BYTES_FIXED_STRAIGHT: - return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount); - case BYTES_FIXED_DEREF: - return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount); - case BYTES_FIXED_SORTED: - return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount); - case BYTES_VAR_STRAIGHT: - return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount); - case BYTES_VAR_DEREF: - return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount); - case BYTES_VAR_SORTED: - return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount); - default: - throw new IllegalStateException("unrecognized index values mode " + type); - } - } - - public void close() throws IOException { - Collection values = docValues.values(); - IOException ex = null; - for (DocValues docValues : values) { - try { - docValues.close(); - } catch (IOException e) { - ex = e; - } - } - if (ex != null) { - throw ex; - } - } - - @Override - public Collection fields() { - return docValues.keySet(); - } -} Index: lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodecProvider.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodecProvider.java (revision 1099621) +++ lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodecProvider.java (working copy) @@ -1,34 +0,0 @@ -package org.apache.lucene.index.codecs.docvalues; -/** - * 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 org.apache.lucene.index.codecs.Codec; -import org.apache.lucene.index.codecs.CodecProvider; - -/** - * Simple Codec provider that wrapps all registered codecs into a {@link DocValuesCodec} - */ -public class DocValuesCodecProvider extends CodecProvider { - - @Override - public synchronized void register(Codec codec) { - if (codec instanceof DocValuesCodec) { - super.register(codec); - } else { - super.register(new DocValuesCodec(codec)); - } - } -} Index: lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java (revision 0) +++ lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java (revision 0) @@ -0,0 +1,107 @@ +package org.apache.lucene.index.codecs; + +/** + * 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.Set; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.PerDocWriteState; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.index.values.Writer; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; + +public class DefaultDocValuesConsumer extends PerDocConsumer { + private final String segmentName; + private final int codecId; + private final Directory directory; + private final AtomicLong bytesUsed; + private final Comparator comparator; + + public DefaultDocValuesConsumer(PerDocWriteState state, Comparator comparator) { + this.segmentName = state.segmentName; + this.codecId = state.codecId; + this.bytesUsed = state.bytesUsed; + this.directory = state.directory; + this.comparator = comparator; + } + + @Override + public void close() throws IOException { + } + + @Override + public DocValuesConsumer addValuesField(FieldInfo field) throws IOException { + return Writer.create(field.getDocValues(), + docValuesId(segmentName, codecId, field.number), + // TODO can we have a compound file per segment and codec for + // docvalues? + directory, comparator, bytesUsed); + } + + public static void files(Directory dir, SegmentInfo segmentInfo, int codecId, + Set files) throws IOException { + FieldInfos fieldInfos = segmentInfo.getFieldInfos(); + boolean indexed = false; + for (FieldInfo fieldInfo : fieldInfos) { + if (fieldInfo.getCodecId() == codecId) { + indexed |= fieldInfo.isIndexed; + if (fieldInfo.hasDocValues()) { + String filename = docValuesId(segmentInfo.name, codecId, fieldInfo.number); + switch (fieldInfo.getDocValues()) { + case BYTES_FIXED_DEREF: + case BYTES_VAR_DEREF: + case BYTES_VAR_SORTED: + case BYTES_FIXED_SORTED: + case BYTES_VAR_STRAIGHT: + files.add(IndexFileNames.segmentFileName(filename, "", + Writer.INDEX_EXTENSION)); + assert dir.fileExists(IndexFileNames.segmentFileName(filename, "", + Writer.INDEX_EXTENSION)); + case BYTES_FIXED_STRAIGHT: + case FLOAT_32: + case FLOAT_64: + case INTS: + files.add(IndexFileNames.segmentFileName(filename, "", + Writer.DATA_EXTENSION)); + assert dir.fileExists(IndexFileNames.segmentFileName(filename, "", + Writer.DATA_EXTENSION)); + break; + default: + assert false; + } + } + + } + } + } + + static String docValuesId(String segmentsName, int codecID, int fieldId) { + return segmentsName + "_" + codecID + "-" + fieldId; + } + + public static void getDocValuesExtensions(Set extensions) { + extensions.add(Writer.DATA_EXTENSION); + extensions.add(Writer.INDEX_EXTENSION); + } +} Property changes on: lucene\src\java\org\apache\lucene\index\codecs\DefaultDocValuesConsumer.java ___________________________________________________________________ Added: svn:eol-style + native Index: lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java (revision 1099621) +++ lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java (working copy) @@ -20,14 +20,21 @@ import java.io.IOException; import java.util.Set; +import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.codecs.Codec; +import org.apache.lucene.index.codecs.DocValuesConsumer; +import org.apache.lucene.index.codecs.DefaultDocValuesProducer; import org.apache.lucene.index.codecs.FieldsConsumer; import org.apache.lucene.index.codecs.FieldsProducer; +import org.apache.lucene.index.codecs.PerDocConsumer; +import org.apache.lucene.index.codecs.DefaultDocValuesConsumer; +import org.apache.lucene.index.codecs.PerDocValues; import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; /** For debugging, curiosity, transparency only!! Do not * use this codec in production. @@ -63,10 +70,23 @@ @Override public void files(Directory dir, SegmentInfo segmentInfo, int id, Set files) throws IOException { files.add(getPostingsFileName(segmentInfo.name, ""+id)); + DefaultDocValuesConsumer.files(dir, segmentInfo, id, files); } @Override public void getExtensions(Set extensions) { extensions.add(POSTINGS_EXTENSION); + DefaultDocValuesConsumer.getDocValuesExtensions(extensions); } + + // nocommit: need plain text impls + @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + return new DefaultDocValuesConsumer(state, BytesRef.getUTF8SortedAsUnicodeComparator()); + } + + @Override + public PerDocValues docsProducer(SegmentReadState state) throws IOException { + return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId); + } } Index: lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java (revision 1099621) +++ lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java (working copy) @@ -20,12 +20,17 @@ import java.io.IOException; import java.util.Set; +import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.codecs.Codec; +import org.apache.lucene.index.codecs.DocValuesConsumer; import org.apache.lucene.index.codecs.FieldsConsumer; import org.apache.lucene.index.codecs.FieldsProducer; +import org.apache.lucene.index.codecs.PerDocConsumer; +import org.apache.lucene.index.codecs.DefaultDocValuesConsumer; +import org.apache.lucene.index.codecs.PerDocValues; import org.apache.lucene.index.codecs.PostingsWriterBase; import org.apache.lucene.index.codecs.PostingsReaderBase; import org.apache.lucene.index.codecs.TermsIndexWriterBase; @@ -34,7 +39,10 @@ import org.apache.lucene.index.codecs.VariableGapTermsIndexReader; import org.apache.lucene.index.codecs.BlockTermsWriter; import org.apache.lucene.index.codecs.BlockTermsReader; +import org.apache.lucene.index.codecs.DefaultDocValuesProducer; +import org.apache.lucene.index.values.Writer; import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; /** Default codec. * @lucene.experimental */ @@ -135,11 +143,13 @@ StandardPostingsReader.files(dir, segmentInfo, codecId, files); BlockTermsReader.files(dir, segmentInfo, codecId, files); VariableGapTermsIndexReader.files(dir, segmentInfo, codecId, files); + DefaultDocValuesConsumer.files(dir, segmentInfo, id, files); } @Override public void getExtensions(Set extensions) { getStandardExtensions(extensions); + DefaultDocValuesConsumer.getDocValuesExtensions(extensions); } public static void getStandardExtensions(Set extensions) { @@ -148,4 +158,14 @@ BlockTermsReader.getExtensions(extensions); VariableGapTermsIndexReader.getIndexExtensions(extensions); } + + @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + return new DefaultDocValuesConsumer(state, BytesRef.getUTF8SortedAsUnicodeComparator()); + } + + @Override + public PerDocValues docsProducer(SegmentReadState state) throws IOException { + return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId); + } } Index: lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java (revision 1099621) +++ lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java (working copy) @@ -22,11 +22,14 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.index.codecs.Codec; +import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.codecs.FieldsConsumer; import org.apache.lucene.index.codecs.FieldsProducer; +import org.apache.lucene.index.codecs.PerDocConsumer; +import org.apache.lucene.index.codecs.PerDocValues; /** Codec that reads the pre-flex-indexing postings * format. It does not provide a writer because newly @@ -78,4 +81,14 @@ extensions.add(TERMS_EXTENSION); extensions.add(TERMS_INDEX_EXTENSION); } + + @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + return null; + } + + @Override + public PerDocValues docsProducer(SegmentReadState state) throws IOException { + return null; + } } Index: lucene/src/java/org/apache/lucene/index/codecs/Codec.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/Codec.java (revision 1099621) +++ lucene/src/java/org/apache/lucene/index/codecs/Codec.java (working copy) @@ -52,13 +52,9 @@ * use; else, those files may be deleted. */ public abstract FieldsProducer fieldsProducer(SegmentReadState state) throws IOException; - public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { - return null; - } + public abstract PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException; - public PerDocValues docsProducer(SegmentReadState state) throws IOException { - return null; - } + public abstract PerDocValues docsProducer(SegmentReadState state) throws IOException; /** * Gathers files associated with this segment Index: lucene/src/java/org/apache/lucene/index/codecs/PerDocConsumer.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/PerDocConsumer.java (revision 1099621) +++ lucene/src/java/org/apache/lucene/index/codecs/PerDocConsumer.java (working copy) @@ -19,7 +19,6 @@ import java.io.IOException; import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.codecs.docvalues.DocValuesConsumer; import org.apache.lucene.index.values.DocValues; /** Index: lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java (revision 1099621) +++ lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java (working copy) @@ -1,4 +1,4 @@ -package org.apache.lucene.index.codecs.docvalues; +package org.apache.lucene.index.codecs; /** * Licensed to the Apache Software Foundation (ASF) under one or more @@ -37,12 +37,12 @@ * * @lucene.experimental */ -public class DocValuesProducerBase extends PerDocValues { +public class DefaultDocValuesProducer extends PerDocValues { protected final TreeMap docValues = new TreeMap(); /** - * Creates a new {@link DocValuesProducerBase} instance and loads all + * Creates a new {@link DefaultDocValuesProducer} instance and loads all * {@link DocValues} instances for this segment and codec. * * @param si @@ -56,7 +56,7 @@ * @throws IOException * if an {@link IOException} occurs */ - protected DocValuesProducerBase(SegmentInfo si, Directory dir, + public DefaultDocValuesProducer(SegmentInfo si, Directory dir, FieldInfos fieldInfo, int codecId) throws IOException { load(fieldInfo, si.name, si.docCount, dir, codecId); } @@ -77,7 +77,7 @@ if (codecId == fieldInfo.getCodecId() && fieldInfo.hasDocValues()) { final String field = fieldInfo.name; // TODO can we have a compound file per segment and codec for docvalues? - final String id = DocValuesCodec.docValuesId(segment, codecId, fieldInfo.number); + final String id = DefaultDocValuesConsumer.docValuesId(segment, codecId, fieldInfo.number); docValues.put(field, loadDocValues(docCount, dir, id, fieldInfo .getDocValues())); } Index: lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java =================================================================== --- lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java (revision 1099621) +++ lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java (working copy) @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.Set; +import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentReadState; @@ -33,8 +34,13 @@ import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl; import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexInput; import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexOutput; +import org.apache.lucene.index.codecs.DocValuesConsumer; +import org.apache.lucene.index.codecs.DefaultDocValuesProducer; import org.apache.lucene.index.codecs.FixedGapTermsIndexReader; import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter; +import org.apache.lucene.index.codecs.PerDocConsumer; +import org.apache.lucene.index.codecs.DefaultDocValuesConsumer; +import org.apache.lucene.index.codecs.PerDocValues; import org.apache.lucene.index.codecs.PostingsWriterBase; import org.apache.lucene.index.codecs.PostingsReaderBase; import org.apache.lucene.index.codecs.BlockTermsReader; @@ -211,11 +217,12 @@ } @Override - public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set files) { + public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set files) throws IOException { final String codecIdAsString = "" + codecId; SepPostingsReaderImpl.files(segmentInfo, codecIdAsString, files); BlockTermsReader.files(dir, segmentInfo, codecIdAsString, files); FixedGapTermsIndexReader.files(dir, segmentInfo, codecIdAsString, files); + DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files); } @Override @@ -223,5 +230,16 @@ SepPostingsWriterImpl.getExtensions(extensions); BlockTermsReader.getExtensions(extensions); FixedGapTermsIndexReader.getIndexExtensions(extensions); + DefaultDocValuesConsumer.getDocValuesExtensions(extensions); } + + @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + return new DefaultDocValuesConsumer(state, BytesRef.getUTF8SortedAsUnicodeComparator()); + } + + @Override + public PerDocValues docsProducer(SegmentReadState state) throws IOException { + return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId); + } } Index: lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java =================================================================== --- lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java (revision 1099621) +++ lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java (working copy) @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.Set; +import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentReadState; @@ -33,8 +34,13 @@ import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl; import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexInput; import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexOutput; +import org.apache.lucene.index.codecs.DocValuesConsumer; +import org.apache.lucene.index.codecs.DefaultDocValuesProducer; import org.apache.lucene.index.codecs.FixedGapTermsIndexReader; import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter; +import org.apache.lucene.index.codecs.PerDocConsumer; +import org.apache.lucene.index.codecs.DefaultDocValuesConsumer; +import org.apache.lucene.index.codecs.PerDocValues; import org.apache.lucene.index.codecs.PostingsWriterBase; import org.apache.lucene.index.codecs.PostingsReaderBase; import org.apache.lucene.index.codecs.BlockTermsReader; @@ -186,11 +192,12 @@ } @Override - public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set files) { + public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set files) throws IOException { final String codecIdAsString = "" + codecId; SepPostingsReaderImpl.files(segmentInfo, codecIdAsString, files); BlockTermsReader.files(dir, segmentInfo, codecIdAsString, files); FixedGapTermsIndexReader.files(dir, segmentInfo, codecIdAsString, files); + DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files); } @Override @@ -198,5 +205,16 @@ SepPostingsWriterImpl.getExtensions(extensions); BlockTermsReader.getExtensions(extensions); FixedGapTermsIndexReader.getIndexExtensions(extensions); + DefaultDocValuesConsumer.getDocValuesExtensions(extensions); } + + @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + return new DefaultDocValuesConsumer(state, BytesRef.getUTF8SortedAsUnicodeComparator()); + } + + @Override + public PerDocValues docsProducer(SegmentReadState state) throws IOException { + return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId); + } } Index: lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java =================================================================== --- lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java (revision 1099621) +++ lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java (working copy) @@ -20,14 +20,20 @@ import java.io.IOException; import java.util.Set; +import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.codecs.Codec; +import org.apache.lucene.index.codecs.DocValuesConsumer; +import org.apache.lucene.index.codecs.DefaultDocValuesProducer; import org.apache.lucene.index.codecs.FieldsConsumer; import org.apache.lucene.index.codecs.FieldsProducer; import org.apache.lucene.index.codecs.FixedGapTermsIndexReader; import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter; +import org.apache.lucene.index.codecs.PerDocConsumer; +import org.apache.lucene.index.codecs.DefaultDocValuesConsumer; +import org.apache.lucene.index.codecs.PerDocValues; import org.apache.lucene.index.codecs.PostingsReaderBase; import org.apache.lucene.index.codecs.PostingsWriterBase; import org.apache.lucene.index.codecs.BlockTermsReader; @@ -130,16 +136,18 @@ } @Override - public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set files) { + public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set files) throws IOException { final String codecIdAsString = "" + codecId; SepPostingsReaderImpl.files(segmentInfo, codecIdAsString, files); BlockTermsReader.files(dir, segmentInfo, codecIdAsString, files); FixedGapTermsIndexReader.files(dir, segmentInfo, codecIdAsString, files); + DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files); } @Override public void getExtensions(Set extensions) { getSepExtensions(extensions); + DefaultDocValuesConsumer.getDocValuesExtensions(extensions); } public static void getSepExtensions(Set extensions) { @@ -147,4 +155,14 @@ BlockTermsReader.getExtensions(extensions); FixedGapTermsIndexReader.getIndexExtensions(extensions); } + + @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + return new DefaultDocValuesConsumer(state, BytesRef.getUTF8SortedAsUnicodeComparator()); + } + + @Override + public PerDocValues docsProducer(SegmentReadState state) throws IOException { + return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId); + } } Index: lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java =================================================================== --- lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java (revision 1099621) +++ lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java (working copy) @@ -26,16 +26,22 @@ import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.codecs.BlockTermsReader; import org.apache.lucene.index.codecs.BlockTermsWriter; import org.apache.lucene.index.codecs.Codec; +import org.apache.lucene.index.codecs.DocValuesConsumer; +import org.apache.lucene.index.codecs.DefaultDocValuesProducer; import org.apache.lucene.index.codecs.FieldsConsumer; import org.apache.lucene.index.codecs.FieldsProducer; import org.apache.lucene.index.codecs.FixedGapTermsIndexReader; import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter; +import org.apache.lucene.index.codecs.PerDocConsumer; +import org.apache.lucene.index.codecs.DefaultDocValuesConsumer; +import org.apache.lucene.index.codecs.PerDocValues; import org.apache.lucene.index.codecs.PostingsReaderBase; import org.apache.lucene.index.codecs.PostingsWriterBase; import org.apache.lucene.index.codecs.TermStats; @@ -338,7 +344,7 @@ BlockTermsReader.files(dir, segmentInfo, codecIdAsString, files); FixedGapTermsIndexReader.files(dir, segmentInfo, codecIdAsString, files); VariableGapTermsIndexReader.files(dir, segmentInfo, codecIdAsString, files); - + DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files); // hackish! Iterator it = files.iterator(); while(it.hasNext()) { @@ -356,7 +362,19 @@ BlockTermsReader.getExtensions(extensions); FixedGapTermsIndexReader.getIndexExtensions(extensions); VariableGapTermsIndexReader.getIndexExtensions(extensions); + DefaultDocValuesConsumer.getDocValuesExtensions(extensions); extensions.add(SEED_EXT); //System.out.println("MockRandom.getExtensions return " + extensions); } + + // can we make this more evil? + @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + return new DefaultDocValuesConsumer(state, BytesRef.getUTF8SortedAsUnicodeComparator()); + } + + @Override + public PerDocValues docsProducer(SegmentReadState state) throws IOException { + return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId); + } } Index: lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingCodec.java =================================================================== --- lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingCodec.java (revision 1099621) +++ lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingCodec.java (working copy) @@ -20,13 +20,19 @@ import java.io.IOException; import java.util.Set; +import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.codecs.Codec; +import org.apache.lucene.index.codecs.DocValuesConsumer; +import org.apache.lucene.index.codecs.DefaultDocValuesProducer; import org.apache.lucene.index.codecs.FieldsConsumer; import org.apache.lucene.index.codecs.FieldsProducer; import org.apache.lucene.index.codecs.FixedGapTermsIndexReader; +import org.apache.lucene.index.codecs.PerDocConsumer; +import org.apache.lucene.index.codecs.DefaultDocValuesConsumer; +import org.apache.lucene.index.codecs.PerDocValues; import org.apache.lucene.index.codecs.standard.StandardCodec; import org.apache.lucene.index.codecs.PostingsReaderBase; import org.apache.lucene.index.codecs.standard.StandardPostingsReader; @@ -133,10 +139,22 @@ StandardPostingsReader.files(dir, segmentInfo, codecIdAsString, files); BlockTermsReader.files(dir, segmentInfo, codecIdAsString, files); FixedGapTermsIndexReader.files(dir, segmentInfo, codecIdAsString, files); + DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files); } @Override public void getExtensions(Set extensions) { StandardCodec.getStandardExtensions(extensions); + DefaultDocValuesConsumer.getDocValuesExtensions(extensions); } + + @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + return new DefaultDocValuesConsumer(state, BytesRef.getUTF8SortedAsUnicodeComparator()); + } + + @Override + public PerDocValues docsProducer(SegmentReadState state) throws IOException { + return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId); + } }