Index: CHANGES.txt =================================================================== --- CHANGES.txt (revision 1033390) +++ CHANGES.txt (working copy) @@ -118,7 +118,7 @@ for example). If your code somehow depends on the old behavior, you will need to change it (e.g. using "\\" to escape '\' itself). (Sunil Kamath, Terry Yang via Robert Muir) - + Changes in Runtime Behavior * LUCENE-2650: The behavior of FSDirectory.open has changed. On 64-bit @@ -289,6 +289,12 @@ (i.e. \* or "*") Custom QueryParser sublcasses overriding getRangeQuery() will be passed null for any open endpoint. (Adriano Crestani, yonik) +* LUCENE-2742: Add native per-field codec support. CodecProvider lets you now + register a codec for each field and which is in turn recorded in the segment + and field information. Codecs are maintained on a per-segment basis and be + resolved without knowing the actual codec used for writing the segment. + (Simon Willnauer) + Optimizations * LUCENE-2410: ~20% speedup on exact (slop=0) PhraseQuery matching. Index: contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java =================================================================== --- contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java (revision 1033390) +++ contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/CreateIndexTask.java (working copy) @@ -134,6 +134,7 @@ final String defaultCodec = config.get("default.codec", null); if (defaultCodec != null) { CodecProvider.setDefaultCodec(defaultCodec); + CodecProvider.getDefault().setDefaultFieldCodec(defaultCodec); } final String mergePolicy = config.get("merge.policy", Index: contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java =================================================================== --- contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java (revision 1033390) +++ contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java (working copy) @@ -21,7 +21,6 @@ import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.document.Document; -import org.apache.lucene.document.Field; import org.apache.lucene.document.Field.Index; import org.apache.lucene.document.Field.Store; import org.apache.lucene.document.Field.TermVector; @@ -32,7 +31,6 @@ import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.LogMergePolicy; import org.apache.lucene.index.MultiFields; -import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.TermsEnum.SeekStatus; @@ -54,15 +52,14 @@ Codec appending = new AppendingCodec(); SegmentInfosWriter infosWriter = new AppendingSegmentInfosWriter(); SegmentInfosReader infosReader = new AppendingSegmentInfosReader(); - - @Override - public Codec lookup(String name) { - return appending; + public AppendingCodecProvider() { + setDefaultFieldCodec(appending.name); } @Override - public Codec getWriter(SegmentWriteState state) { + public Codec lookup(String name) { return appending; } + @Override public SegmentInfosReader getSegmentInfosReader() { return infosReader; Index: src/java/org/apache/lucene/index/CheckIndex.java =================================================================== --- src/java/org/apache/lucene/index/CheckIndex.java (revision 1033390) +++ src/java/org/apache/lucene/index/CheckIndex.java (working copy) @@ -129,8 +129,8 @@ /** Name of the segment. */ public String name; - /** Name of codec used to read this segment. */ - public String codec; + /** CodecInfo used to read this segment. */ + public SegmentCodecs codec; /** Document count (does not take deletions into account). */ public int docCount; @@ -408,7 +408,7 @@ SegmentReader reader = null; try { - final String codec = info.getCodec().name; + final SegmentCodecs codec = info.getCodecInfo(); msg(" codec=" + codec); segInfoStat.codec = codec; msg(" compound=" + info.getUseCompoundFile()); @@ -602,7 +602,7 @@ } final TermsEnum terms = fieldsEnum.terms(); - + assert terms != null; boolean hasOrd = true; final long termCountStart = status.termCount; Index: src/java/org/apache/lucene/index/DocFieldProcessor.java =================================================================== --- src/java/org/apache/lucene/index/DocFieldProcessor.java (revision 1033390) +++ src/java/org/apache/lucene/index/DocFieldProcessor.java (working copy) @@ -34,11 +34,12 @@ final class DocFieldProcessor extends DocConsumer { final DocumentsWriter docWriter; - final FieldInfos fieldInfos = new FieldInfos(); + final FieldInfos fieldInfos; final DocFieldConsumer consumer; final StoredFieldsWriter fieldsWriter; public DocFieldProcessor(DocumentsWriter docWriter, DocFieldConsumer consumer) { + this.fieldInfos = new FieldInfos(); this.docWriter = docWriter; this.consumer = consumer; consumer.setFieldInfos(fieldInfos); Index: src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java =================================================================== --- src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java (revision 1033390) +++ src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java (working copy) @@ -196,7 +196,6 @@ FieldInfo fi = fieldInfos.add(fieldName, field.isIndexed(), field.isTermVectorStored(), field.isStorePositionWithTermVector(), field.isStoreOffsetWithTermVector(), field.getOmitNorms(), false, field.getOmitTermFreqAndPositions()); - fp = new DocFieldProcessorPerField(this, fi); fp.next = fieldHash[hashPos]; fieldHash[hashPos] = fp; Index: src/java/org/apache/lucene/index/DocumentsWriter.java =================================================================== --- src/java/org/apache/lucene/index/DocumentsWriter.java (revision 1033390) +++ src/java/org/apache/lucene/index/DocumentsWriter.java (working copy) @@ -28,11 +28,9 @@ import java.util.List; import java.util.Map.Entry; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.ReentrantLock; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.Document; -import org.apache.lucene.index.codecs.Codec; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; import org.apache.lucene.search.Scorer; @@ -604,14 +602,14 @@ synchronized private void initFlushState(boolean onlyDocStore) { initSegmentName(onlyDocStore); + final SegmentCodecs info = SegmentCodecs.build(docFieldProcessor.fieldInfos, writer.codecs); flushState = new SegmentWriteState(infoStream, directory, segment, docFieldProcessor.fieldInfos, - docStoreSegment, numDocsInRAM, numDocsInStore, writer.getConfig().getTermIndexInterval(), - writer.codecs); + docStoreSegment, numDocsInRAM, numDocsInStore, writer.getConfig().getTermIndexInterval(), info); } - /** Returns the codec used to flush the last segment */ - Codec getCodec() { - return flushState.codec; + /** Returns the SegmentCodecs used to flush the last segment */ + SegmentCodecs getSegmentCodecs() { + return flushState.segmentCodecs; } /** Flush all pending docs to a new segment */ @@ -653,7 +651,7 @@ if (infoStream != null) { SegmentInfo si = new SegmentInfo(flushState.segmentName, flushState.numDocs, directory, false, -1, flushState.segmentName, - false, hasProx(), flushState.codec); + false, hasProx(), flushState.segmentCodecs); final long newSegmentSize = si.sizeInBytes(); String message = " ramUsed=" + nf.format(startNumBytesUsed/1024./1024.) + " MB" + " newFlushedSize=" + newSegmentSize + Index: src/java/org/apache/lucene/index/FieldInfo.java =================================================================== --- src/java/org/apache/lucene/index/FieldInfo.java (revision 1033390) +++ src/java/org/apache/lucene/index/FieldInfo.java (working copy) @@ -32,6 +32,7 @@ public boolean omitTermFreqAndPositions; public boolean storePayloads; // whether this field stores payloads together with term positions + public int codecId = 0; FieldInfo(String na, boolean tk, int nu, boolean storeTermVector, boolean storePositionWithTermVector, boolean storeOffsetWithTermVector, Index: src/java/org/apache/lucene/index/FieldInfos.java =================================================================== --- src/java/org/apache/lucene/index/FieldInfos.java (revision 1033390) +++ src/java/org/apache/lucene/index/FieldInfos.java (working copy) @@ -38,9 +38,10 @@ // First used in 2.9; prior to 2.9 there was no format header public static final int FORMAT_START = -2; + public static final int FORMAT_PER_FIELD_CODEC = -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_PER_FIELD_CODEC; static final int FORMAT_MINIMUM = FORMAT_START; @@ -56,7 +57,8 @@ private final HashMap byName = new HashMap(); private int format; - public FieldInfos() { } + public FieldInfos() { + } /** * Construct a FieldInfos object using the directory and the name of the file @@ -301,8 +303,8 @@ 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.writeInt(fi.codecId); output.writeByte(bits); } } @@ -321,6 +323,8 @@ for (int i = 0; i < size; i++) { String name = StringHelper.intern(input.readString()); + // if this is a previous format codec 0 will be preflex! + final int codecId = format <= FORMAT_PER_FIELD_CODEC? input.readInt():0; byte bits = input.readByte(); boolean isIndexed = (bits & IS_INDEXED) != 0; boolean storeTermVector = (bits & STORE_TERMVECTOR) != 0; @@ -329,8 +333,8 @@ boolean omitNorms = (bits & OMIT_NORMS) != 0; boolean storePayloads = (bits & STORE_PAYLOADS) != 0; boolean omitTermFreqAndPositions = (bits & OMIT_TERM_FREQ_AND_POSITIONS) != 0; - - addInternal(name, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions); + final FieldInfo addInternal = addInternal(name, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions); + addInternal.codecId = codecId; } if (input.getFilePointer() != input.length()) { Index: src/java/org/apache/lucene/index/FreqProxTermsWriter.java =================================================================== --- src/java/org/apache/lucene/index/FreqProxTermsWriter.java (revision 1033390) +++ src/java/org/apache/lucene/index/FreqProxTermsWriter.java (working copy) @@ -77,8 +77,7 @@ // Sort by field name CollectionUtil.quickSort(allFields); - // TODO: allow Lucene user to customize this codec: - final FieldsConsumer consumer = state.codec.fieldsConsumer(state); + final FieldsConsumer consumer = state.segmentCodecs.codec().fieldsConsumer(state); /* Current writer chain: Index: src/java/org/apache/lucene/index/IndexWriter.java =================================================================== --- src/java/org/apache/lucene/index/IndexWriter.java (revision 1033390) +++ src/java/org/apache/lucene/index/IndexWriter.java (working copy) @@ -623,7 +623,7 @@ // TODO: we may want to avoid doing this while // synchronized // Returns a ref, which we xfer to readerMap: - sr = SegmentReader.get(false, info.dir, info, readBufferSize, doOpenStores, termsIndexDivisor, codecs); + sr = SegmentReader.get(false, info.dir, info, readBufferSize, doOpenStores, termsIndexDivisor); if (info.dir == directory) { // Only pool if reader is not external @@ -2997,7 +2997,7 @@ SegmentInfo info = null; synchronized(this) { info = new SegmentInfo(mergedName, docCount, directory, false, -1, - null, false, merger.hasProx(), merger.getCodec()); + null, false, merger.hasProx(), merger.getSegmentCodecs()); setDiagnostics(info, "addIndexes(IndexReader...)"); segmentInfos.add(info); checkpoint(); @@ -3375,10 +3375,10 @@ directory, false, docStoreOffset, docStoreSegment, docStoreIsCompoundFile, docWriter.hasProx(), - docWriter.getCodec()); + docWriter.getSegmentCodecs()); if (infoStream != null) { - message("flush codec=" + docWriter.getCodec().name); + message("flush codec=" + docWriter.getSegmentCodecs()); } setDiagnostics(newSegment, "flush"); } @@ -4068,10 +4068,10 @@ mergedDocCount = merge.info.docCount = merger.merge(merge.mergeDocStores); // Record which codec was used to write the segment - merge.info.setCodec(merger.getCodec()); + merge.info.setSegmentCodecs(merger.getSegmentCodecs()); if (infoStream != null) { - message("merge codec=" + merger.getCodec().name); + message("merge segmentCodecs=" + merger.getSegmentCodecs()); } assert mergedDocCount == totDocCount; Index: src/java/org/apache/lucene/index/PerFieldCodecWrapper.java =================================================================== --- src/java/org/apache/lucene/index/PerFieldCodecWrapper.java (revision 0) +++ src/java/org/apache/lucene/index/PerFieldCodecWrapper.java (revision 0) @@ -0,0 +1,199 @@ +package org.apache.lucene.index; + +/** + * 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.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; + +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.TermsConsumer; +import org.apache.lucene.store.Directory; + + +/** + * Enables native per field codec support + * @lucene.internal */ +final class PerFieldCodecWrapper extends Codec { + private final SegmentCodecs segmentCodecs; + + PerFieldCodecWrapper(SegmentCodecs segmentCodecs) { + name = "PerField"; // TODO this is bogus + this.segmentCodecs = segmentCodecs; + } + + @Override + public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { + return new FieldsWriter(state); + } + + private class FieldsWriter extends FieldsConsumer { + private ArrayList consumers = new ArrayList(); + + public FieldsWriter(SegmentWriteState state) throws IOException { + assert segmentCodecs == state.segmentCodecs; + final Codec[] codecs = segmentCodecs.codecs; + for (int i = 0; i < codecs.length; i++) { + state.currentCodecId = i; // actual codec should use that to create its files + consumers.add(codecs[i].fieldsConsumer(state)); + } + } + + @Override + public TermsConsumer addField(FieldInfo field) throws IOException { + final FieldsConsumer fields = consumers.get(field.codecId); + return fields.addField(field); + } + + @Override + public void close() throws IOException { + Iterator it = consumers.iterator(); + IOException err = null; + while(it.hasNext()) { + try { + it.next().close(); + } catch (IOException ioe) { + // keep first IOException we hit but keep + // closing the rest + if (err == null) { + err = ioe; + } + } + } + if (err != null) { + throw err; + } + } + } + + private class FieldsReader extends FieldsProducer { + + private final Set fields = new TreeSet(); + private final Map codecs = new HashMap(); + public FieldsReader(Directory dir, FieldInfos fieldInfos, + SegmentInfo si, int readBufferSize, + int indexDivisor) throws IOException { + + final int fieldCount = fieldInfos.size(); + final Map producers = new HashMap(); + for(int i=0;i it; + private String current; + + public FieldsIterator() { + it = fields.iterator(); + } + + @Override + public String next() { + if (it.hasNext()) { + current = it.next(); + } else { + current = null; + } + + return current; + } + + @Override + public TermsEnum terms() throws IOException { + Terms terms = codecs.get(current).terms(current); + if (terms != null) { + return terms.iterator(); + } else { + return TermsEnum.EMPTY; + } + } + } + + @Override + public FieldsEnum iterator() throws IOException { + return new FieldsIterator(); + } + + @Override + public Terms terms(String field) throws IOException { + FieldsProducer fields = codecs.get(field); + return fields == null? null:fields.terms(field); + } + + @Override + public void close() throws IOException { + Iterator it = codecs.values().iterator(); + IOException err = null; + while(it.hasNext()) { + try { + it.next().close(); + } catch (IOException ioe) { + // keep first IOException we hit but keep + // closing the rest + if (err == null) { + err = ioe; + } + } + } + if (err != null) { + throw err; + } + } + + @Override + public void loadTermsIndex(int indexDivisor) throws IOException { + Iterator it = codecs.values().iterator(); + while(it.hasNext()) { + it.next().loadTermsIndex(indexDivisor); + } + } + } + + public FieldsProducer fieldsProducer(SegmentReadState state) + throws IOException { + return new FieldsReader(state.dir, state.fieldInfos, state.segmentInfo, state.readBufferSize, state.termsIndexDivisor); + } + + @Override + public void files(Directory dir, SegmentInfo info, Set files) throws IOException { + segmentCodecs.files(dir, info, files); + } + + @Override + public void getExtensions(Set extensions) { + for(Codec codec : segmentCodecs.codecs) { + codec.getExtensions(extensions); + } + } +} Property changes on: src/java/org/apache/lucene/index/PerFieldCodecWrapper.java ___________________________________________________________________ Added: svn:eol-style + native Added: svn:keywords + Date Author Id Revision HeadURL Index: src/java/org/apache/lucene/index/SegmentCodecs.java =================================================================== --- src/java/org/apache/lucene/index/SegmentCodecs.java (revision 0) +++ src/java/org/apache/lucene/index/SegmentCodecs.java (revision 0) @@ -0,0 +1,110 @@ +package org.apache.lucene.index; + +/** + * 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.ArrayList; +import java.util.HashSet; +import java.util.IdentityHashMap; +import java.util.Map; +import java.util.Set; + +import org.apache.lucene.index.codecs.Codec; +import org.apache.lucene.index.codecs.CodecProvider; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; + +/** + * + * @lucene.internal + */ +final class SegmentCodecs implements Cloneable { + Codec[] codecs; + final CodecProvider provider; + private final Codec codec = new PerFieldCodecWrapper(this); + + SegmentCodecs(CodecProvider provider, Codec... codecs) { + this.provider = provider; + this.codecs = codecs; + } + + Codec byField(String field) { + return provider.lookup(provider.getFieldCodec(field)); + } + + static SegmentCodecs build(FieldInfos infos, CodecProvider provider) { + final int size = infos.size(); + final Map codecRegistry = new IdentityHashMap(); + final ArrayList codecs = new ArrayList(); + + for (int i = 0; i < size; i++) { + final FieldInfo info = infos.fieldInfo(i); + if (info.isIndexed) { + final Codec fieldCodec = provider.lookup(provider.getFieldCodec(info.name)); + Integer ord = codecRegistry.get(fieldCodec); + if (ord == null) { + ord = Integer.valueOf(codecs.size()); + codecRegistry.put(fieldCodec, ord); + codecs.add(fieldCodec); + } + info.codecId = ord.intValue(); + } + } + return new SegmentCodecs(provider, codecs.toArray(Codec.EMPTY)); + + } + + Codec codec() { + return codec; + } + + void write(IndexOutput out) throws IOException { + out.writeVInt(codecs.length); + for (Codec codec : codecs) { + out.writeString(codec.name); + } + } + + void read(IndexInput in) throws IOException { + final int size = in.readVInt(); + final ArrayList list = new ArrayList(); + for (int i = 0; i < size; i++) { + final String codecName = in.readString(); + final Codec lookup = provider.lookup(codecName); + list.add(i, lookup); + } + codecs = list.toArray(Codec.EMPTY); + } + + void files(Directory dir, SegmentInfo info, Set files) + throws IOException { + final Set seen = new HashSet(); + final Codec[] codecArray = codecs; + for (Codec codec : codecArray) { + if (!seen.contains(codec)) { + seen.add(codec); + codec.files(dir, info, files); + } + } + } + + @Override + public String toString() { + return "CodecInfo [codecs=" + codecs + ", provider=" + provider + "]"; + } +} \ No newline at end of file Property changes on: src/java/org/apache/lucene/index/SegmentCodecs.java ___________________________________________________________________ Added: svn:eol-style + native Added: svn:keywords + Date Author Id Revision HeadURL Index: src/java/org/apache/lucene/index/SegmentInfo.java =================================================================== --- src/java/org/apache/lucene/index/SegmentInfo.java (revision 1033390) +++ src/java/org/apache/lucene/index/SegmentInfo.java (working copy) @@ -80,13 +80,12 @@ private boolean hasProx; // True if this segment has any fields with omitTermFreqAndPositions==false - private Codec codec; - + private SegmentCodecs segmentCodecs; private Map diagnostics; public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile, int docStoreOffset, - String docStoreSegment, boolean docStoreIsCompoundFile, boolean hasProx, Codec codec) { + String docStoreSegment, boolean docStoreIsCompoundFile, boolean hasProx, SegmentCodecs codecInfo) { this.name = name; this.docCount = docCount; this.dir = dir; @@ -96,7 +95,7 @@ this.docStoreSegment = docStoreSegment; this.docStoreIsCompoundFile = docStoreIsCompoundFile; this.hasProx = hasProx; - this.codec = codec; + this.segmentCodecs = codecInfo; delCount = 0; assert docStoreOffset == -1 || docStoreSegment != null: "dso=" + docStoreOffset + " dss=" + docStoreSegment + " docCount=" + docCount; } @@ -120,7 +119,7 @@ } isCompoundFile = src.isCompoundFile; delCount = src.delCount; - codec = src.codec; + segmentCodecs = src.segmentCodecs; } void setDiagnostics(Map diagnostics) { @@ -145,7 +144,6 @@ this.dir = dir; name = input.readString(); docCount = input.readInt(); - final String codecName; delGen = input.readLong(); docStoreOffset = input.readInt(); if (docStoreOffset != -1) { @@ -177,14 +175,15 @@ hasProx = input.readByte() == YES; // System.out.println(Thread.currentThread().getName() + ": si.read hasProx=" + hasProx + " seg=" + name); - - if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) - codecName = input.readString(); - else - codecName = "PreFlex"; - + segmentCodecs = new SegmentCodecs(codecs); + if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) { + segmentCodecs.read(input); + } else { + // codec ID on FieldInfo is 0 so it will simply use the first codec available + // TODO what todo if preflex is not available in the provider? register it or fail? + segmentCodecs.codecs = new Codec[] { codecs.lookup("PreFlex")}; + } diagnostics = input.readStringStringMap(); - codec = codecs.lookup(codecName); } /** Returns total size in bytes of all of files used by @@ -230,7 +229,7 @@ @Override public Object clone() { - SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, docStoreOffset, docStoreSegment, docStoreIsCompoundFile, hasProx, codec); + SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, docStoreOffset, docStoreSegment, docStoreIsCompoundFile, hasProx, segmentCodecs); si.isCompoundFile = isCompoundFile; si.delGen = delGen; si.delCount = delCount; @@ -242,7 +241,6 @@ si.docStoreOffset = docStoreOffset; si.docStoreSegment = docStoreSegment; si.docStoreIsCompoundFile = docStoreIsCompoundFile; - si.codec = codec; return si; } @@ -400,7 +398,7 @@ output.writeByte((byte) (isCompoundFile ? YES : NO)); output.writeInt(delCount); output.writeByte((byte) (hasProx ? 1:0)); - output.writeString(codec.name); + segmentCodecs.write(output); output.writeStringStringMap(diagnostics); } @@ -414,16 +412,16 @@ } /** Can only be called once. */ - public void setCodec(Codec codec) { - assert this.codec == null; - if (codec == null) { - throw new IllegalArgumentException("codec must be non-null"); + public void setSegmentCodecs(SegmentCodecs segmentCodecs) { + assert this.segmentCodecs == null; + if (segmentCodecs == null) { + throw new IllegalArgumentException("segmentCodecs must be non-null"); } - this.codec = codec; + this.segmentCodecs = segmentCodecs; } - Codec getCodec() { - return codec; + SegmentCodecs getCodecInfo() { + return segmentCodecs; } private void addIfExists(Set files, String fileName) throws IOException { @@ -454,7 +452,7 @@ for(String ext : IndexFileNames.NON_STORE_INDEX_EXTENSIONS) { addIfExists(fileSet, IndexFileNames.segmentFileName(name, "", ext)); } - codec.files(dir, this, fileSet); + segmentCodecs.files(dir, this, fileSet); } if (docStoreOffset != -1) { Index: src/java/org/apache/lucene/index/SegmentInfos.java =================================================================== --- src/java/org/apache/lucene/index/SegmentInfos.java (revision 1033390) +++ src/java/org/apache/lucene/index/SegmentInfos.java (working copy) @@ -308,7 +308,7 @@ SegmentInfos sis = (SegmentInfos) super.clone(); for(int i=0;i(userData); Index: src/java/org/apache/lucene/index/SegmentMerger.java =================================================================== --- src/java/org/apache/lucene/index/SegmentMerger.java (revision 1033390) +++ src/java/org/apache/lucene/index/SegmentMerger.java (working copy) @@ -184,7 +184,7 @@ fileSet.add(IndexFileNames.segmentFileName(segment, "", ext)); } - codec.files(directory, info, fileSet); + segmentWriteState.segmentCodecs.files(directory, info, fileSet); // Fieldable norm files int numFIs = fieldInfos.size(); @@ -278,7 +278,7 @@ final SegmentReader sr = (SegmentReader) readers.get(readers.size()-1); fieldInfos = (FieldInfos) sr.core.fieldInfos.clone(); } else { - fieldInfos = new FieldInfos(); // merge field names + fieldInfos = new FieldInfos();// merge field names } for (IndexReader reader : readers) { @@ -304,6 +304,7 @@ fieldInfos.add(reader.getFieldNames(FieldOption.UNINDEXED), false); } } + final SegmentCodecs codecInfo = SegmentCodecs.build(fieldInfos, this.codecs); fieldInfos.write(directory, segment + ".fnm"); int docCount = 0; @@ -357,8 +358,8 @@ } } - segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, null, docCount, 0, termIndexInterval, codecs); - + segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, null, docCount, 0, termIndexInterval, codecInfo); + return docCount; } @@ -554,15 +555,15 @@ } } - Codec getCodec() { - return codec; + SegmentCodecs getSegmentCodecs() { + assert segmentWriteState != null; + return segmentWriteState.segmentCodecs; } private final void mergeTerms() throws CorruptIndexException, IOException { // Let CodecProvider decide which codec will be used to write // the new segment: - codec = codecs.getWriter(segmentWriteState); int docBase = 0; @@ -644,7 +645,7 @@ } } starts[mergeState.readerCount] = inputDocBase; - + codec = segmentWriteState.segmentCodecs.codec(); final FieldsConsumer consumer = codec.fieldsConsumer(segmentWriteState); // NOTE: this is silly, yet, necessary -- we create a Index: src/java/org/apache/lucene/index/SegmentReader.java =================================================================== --- src/java/org/apache/lucene/index/SegmentReader.java (revision 1033390) +++ src/java/org/apache/lucene/index/SegmentReader.java (working copy) @@ -39,7 +39,6 @@ import org.apache.lucene.util.BitVector; 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.search.FieldCache; // not great (circular); used only to purge FieldCache entry on close import org.apache.lucene.util.BytesRef; @@ -90,7 +89,6 @@ final FieldInfos fieldInfos; final FieldsProducer fields; - final CodecProvider codecs; final Directory dir; final Directory cfsDir; @@ -104,17 +102,14 @@ CompoundFileReader cfsReader; CompoundFileReader storeCFSReader; - CoreReaders(SegmentReader origInstance, Directory dir, SegmentInfo si, int readBufferSize, int termsIndexDivisor, CodecProvider codecs) throws IOException { + CoreReaders(SegmentReader origInstance, Directory dir, SegmentInfo si, int readBufferSize, int termsIndexDivisor) throws IOException { if (termsIndexDivisor == 0) { throw new IllegalArgumentException("indexDivisor must be < 0 (don't load terms index) or greater than 0 (got 0)"); } segment = si.name; - if (codecs == null) { - codecs = CodecProvider.getDefault(); - } - this.codecs = codecs; + final SegmentCodecs codecInfo = si.getCodecInfo(); this.readBufferSize = readBufferSize; this.dir = dir; @@ -129,11 +124,11 @@ cfsDir = dir0; fieldInfos = new FieldInfos(cfsDir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELD_INFOS_EXTENSION)); - + this.termsIndexDivisor = termsIndexDivisor; - + // Ask codec for its Fields - fields = si.getCodec().fieldsProducer(new SegmentReadState(cfsDir, si, fieldInfos, readBufferSize, termsIndexDivisor)); + fields = codecInfo.codec().fieldsProducer(new SegmentReadState(cfsDir, si, fieldInfos, readBufferSize, termsIndexDivisor)); assert fields != null; success = true; @@ -506,7 +501,7 @@ * @throws IOException if there is a low-level IO error */ public static SegmentReader get(boolean readOnly, SegmentInfo si, int termInfosIndexDivisor) throws CorruptIndexException, IOException { - return get(readOnly, si.dir, si, BufferedIndexInput.BUFFER_SIZE, true, termInfosIndexDivisor, null); + return get(readOnly, si.dir, si, BufferedIndexInput.BUFFER_SIZE, true, termInfosIndexDivisor); } /** @@ -518,12 +513,8 @@ SegmentInfo si, int readBufferSize, boolean doOpenStores, - int termInfosIndexDivisor, - CodecProvider codecs) + int termInfosIndexDivisor) throws CorruptIndexException, IOException { - if (codecs == null) { - codecs = CodecProvider.getDefault(); - } SegmentReader instance = new SegmentReader(); instance.readOnly = readOnly; @@ -533,7 +524,7 @@ boolean success = false; try { - instance.core = new CoreReaders(instance, dir, si, readBufferSize, termInfosIndexDivisor, codecs); + instance.core = new CoreReaders(instance, dir, si, readBufferSize, termInfosIndexDivisor); if (doOpenStores) { instance.core.openDocStores(si); } Index: src/java/org/apache/lucene/index/SegmentWriteState.java =================================================================== --- src/java/org/apache/lucene/index/SegmentWriteState.java (revision 1033390) +++ src/java/org/apache/lucene/index/SegmentWriteState.java (working copy) @@ -17,13 +17,11 @@ * limitations under the License. */ -import java.util.HashSet; -import java.util.Collection; import java.io.PrintStream; +import java.util.Collection; +import java.util.HashSet; import org.apache.lucene.store.Directory; -import org.apache.lucene.index.codecs.Codec; -import org.apache.lucene.index.codecs.CodecProvider; /** * @lucene.experimental @@ -38,8 +36,8 @@ public int numDocsInStore; public final Collection flushedFiles; - // Actual codec used - final Codec codec; + final SegmentCodecs segmentCodecs; + public int currentCodecId; /** Expert: The fraction of terms in the "dictionary" which should be stored * in RAM. Smaller values use more memory, but make searching slightly @@ -59,11 +57,12 @@ * slightly smaller indexes, but slower skipping in big posting lists. */ public final int maxSkipLevels = 10; + + public SegmentWriteState(PrintStream infoStream, Directory directory, String segmentName, FieldInfos fieldInfos, String docStoreSegmentName, int numDocs, - int numDocsInStore, int termIndexInterval, - CodecProvider codecs) { + int numDocsInStore, int termIndexInterval, SegmentCodecs segmentCodecs) { this.infoStream = infoStream; this.directory = directory; this.segmentName = segmentName; @@ -72,7 +71,7 @@ this.numDocs = numDocs; this.numDocsInStore = numDocsInStore; this.termIndexInterval = termIndexInterval; - this.codec = codecs.getWriter(this); + this.segmentCodecs = segmentCodecs; flushedFiles = new HashSet(); } } Index: src/java/org/apache/lucene/index/codecs/Codec.java =================================================================== --- src/java/org/apache/lucene/index/codecs/Codec.java (revision 1033390) +++ src/java/org/apache/lucene/index/codecs/Codec.java (working copy) @@ -27,7 +27,7 @@ /** @lucene.experimental */ public abstract class Codec { - + public static final Codec[] EMPTY = new Codec[0]; /** Unique name that's used to retrieve this codec when * reading the index */ public String name; Index: src/java/org/apache/lucene/index/codecs/CodecProvider.java =================================================================== --- src/java/org/apache/lucene/index/codecs/CodecProvider.java (revision 1033390) +++ src/java/org/apache/lucene/index/codecs/CodecProvider.java (working copy) @@ -20,13 +20,13 @@ import java.util.Collection; import java.util.HashMap; import java.util.HashSet; +import java.util.Map; import java.util.Set; -import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.codecs.preflex.PreFlexCodec; import org.apache.lucene.index.codecs.pulsing.PulsingCodec; -import org.apache.lucene.index.codecs.standard.StandardCodec; import org.apache.lucene.index.codecs.simpletext.SimpleTextCodec; +import org.apache.lucene.index.codecs.standard.StandardCodec; /** Holds a set of codecs, keyed by name. You subclass * this, instantiate it, and register your codecs, then @@ -36,10 +36,13 @@ * * @lucene.experimental */ -public abstract class CodecProvider { +public class CodecProvider { private SegmentInfosWriter infosWriter = new DefaultSegmentInfosWriter(); private SegmentInfosReader infosReader = new DefaultSegmentInfosReader(); + private String defaultFieldCodec = defaultCodec; + private final Map perFieldMap = new HashMap(); + private final HashMap codecs = new HashMap(); private final Set knownExtensions = new HashSet(); @@ -86,8 +89,6 @@ return codec; } - public abstract Codec getWriter(SegmentWriteState state); - public SegmentInfosWriter getSegmentInfosWriter() { return infosWriter; } @@ -110,6 +111,62 @@ public synchronized static String getDefaultCodec() { return defaultCodec; } + + /** + * Sets the {@link Codec} for a given field. Not that setting a fields code is + * write-once. If the fields codec is already set this method will throw an + * {@link IllegalArgumentException} + * + * @param field + * the name of the field + * @param codec + * the name of the codec + * @throws IllegalArgumentException + * if the codec for the given field is already set + * + */ + public synchronized void setFieldCodec(String field, String codec) { + if (perFieldMap.containsKey(field)) + throw new IllegalArgumentException("codec for field: " + field + + " already set to " + perFieldMap.get(field)); + perFieldMap.put(field, codec); + } + + /** + * Returns the {@link Codec} name for the given field or the default codec if + * not set. + * + * @param name + * the fields name + * @return the {@link Codec} name for the given field or the default codec if + * not set. + */ + public synchronized String getFieldCodec(String name) { + final String codec; + if ((codec = perFieldMap.get(name)) == null) { + return defaultFieldCodec; + } + return codec; + } + + /** + * Returns the default {@link Codec} for this {@link CodecProvider} + * + * @return the default {@link Codec} for this {@link CodecProvider} + */ + public synchronized String getDefaultFieldCodec() { + return defaultFieldCodec; + } + + /** + * Sets the default {@link Codec} for this {@link CodecProvider} + * + * @param codec + * the codecs name + */ + public synchronized void setDefaultFieldCodec(String codec) { + defaultFieldCodec = codec; + } } class DefaultCodecProvider extends CodecProvider { @@ -118,10 +175,6 @@ register(new PreFlexCodec()); register(new PulsingCodec(1)); register(new SimpleTextCodec()); - } - - @Override - public Codec getWriter(SegmentWriteState state) { - return lookup(CodecProvider.getDefaultCodec()); + setDefaultFieldCodec(CodecProvider.getDefaultCodec()); } } Index: src/java/org/apache/lucene/index/codecs/PerFieldCodecWrapper.java =================================================================== --- src/java/org/apache/lucene/index/codecs/PerFieldCodecWrapper.java (revision 1033390) +++ src/java/org/apache/lucene/index/codecs/PerFieldCodecWrapper.java (working copy) @@ -1,238 +0,0 @@ -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.util.Map; -import java.util.HashMap; -import java.util.Set; -import java.util.HashSet; -import java.util.Iterator; -import java.util.IdentityHashMap; -import java.util.TreeMap; -import java.util.TreeSet; -import java.io.IOException; - -import org.apache.lucene.index.FieldsEnum; -import org.apache.lucene.index.TermsEnum; -import org.apache.lucene.index.Terms; -import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.FieldInfos; -import org.apache.lucene.index.SegmentInfo; -import org.apache.lucene.index.SegmentWriteState; -import org.apache.lucene.index.SegmentReadState; -import org.apache.lucene.store.Directory; - - -/** Simple Codec that dispatches field-specific codecs. - * You must ensure every field you index has a Codec, or - * the defaultCodec is non null. Also, the separate - * codecs cannot conflict on file names. - * - * @lucene.experimental */ -public class PerFieldCodecWrapper extends Codec { - private final Map fields = new IdentityHashMap(); - private final Codec defaultCodec; - - public PerFieldCodecWrapper(Codec defaultCodec) { - name = "PerField"; - this.defaultCodec = defaultCodec; - } - - public void add(String field, Codec codec) { - fields.put(field, codec); - } - - public Codec getCodec(String field) { - Codec codec = fields.get(field); - if (codec != null) { - return codec; - } else { - return defaultCodec; - } - } - - @Override - public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - return new FieldsWriter(state); - } - - private class FieldsWriter extends FieldsConsumer { - private final SegmentWriteState state; - private final Map codecs = new HashMap(); - private final Set fieldsSeen = new TreeSet(); - - public FieldsWriter(SegmentWriteState state) { - this.state = state; - } - - @Override - public TermsConsumer addField(FieldInfo field) throws IOException { - fieldsSeen.add(field.name); - Codec codec = getCodec(field.name); - - FieldsConsumer fields = codecs.get(codec); - if (fields == null) { - fields = codec.fieldsConsumer(state); - codecs.put(codec, fields); - } - return fields.addField(field); - } - - @Override - public void close() throws IOException { - Iterator it = codecs.values().iterator(); - IOException err = null; - while(it.hasNext()) { - try { - it.next().close(); - } catch (IOException ioe) { - // keep first IOException we hit but keep - // closing the rest - if (err == null) { - err = ioe; - } - } - } - if (err != null) { - throw err; - } - } - } - - private class FieldsReader extends FieldsProducer { - - private final Set fields = new TreeSet(); - private final Map codecs = new HashMap(); - - public FieldsReader(Directory dir, FieldInfos fieldInfos, - SegmentInfo si, int readBufferSize, - int indexDivisor) throws IOException { - - final int fieldCount = fieldInfos.size(); - for(int i=0;i it; - private String current; - - public FieldsIterator() { - it = fields.iterator(); - } - - @Override - public String next() { - if (it.hasNext()) { - current = it.next(); - } else { - current = null; - } - - return current; - } - - @Override - public TermsEnum terms() throws IOException { - Terms terms = codecs.get(getCodec(current)).terms(current); - if (terms != null) { - return terms.iterator(); - } else { - return null; - } - } - } - - @Override - public FieldsEnum iterator() throws IOException { - return new FieldsIterator(); - } - - @Override - public Terms terms(String field) throws IOException { - Codec codec = getCodec(field); - - FieldsProducer fields = codecs.get(codec); - assert fields != null; - return fields.terms(field); - } - - @Override - public void close() throws IOException { - Iterator it = codecs.values().iterator(); - IOException err = null; - while(it.hasNext()) { - try { - it.next().close(); - } catch (IOException ioe) { - // keep first IOException we hit but keep - // closing the rest - if (err == null) { - err = ioe; - } - } - } - if (err != null) { - throw err; - } - } - - @Override - public void loadTermsIndex(int indexDivisor) throws IOException { - Iterator it = codecs.values().iterator(); - while(it.hasNext()) { - it.next().loadTermsIndex(indexDivisor); - } - } - } - - public FieldsProducer fieldsProducer(SegmentReadState state) - throws IOException { - return new FieldsReader(state.dir, state.fieldInfos, state.segmentInfo, state.readBufferSize, state.termsIndexDivisor); - } - - @Override - public void files(Directory dir, SegmentInfo info, Set files) throws IOException { - Iterator it = fields.values().iterator(); - Set seen = new HashSet(); - while(it.hasNext()) { - final Codec codec = it.next(); - if (!seen.contains(codec)) { - seen.add(codec); - codec.files(dir, info, files); - } - } - } - - @Override - public void getExtensions(Set extensions) { - Iterator it = fields.values().iterator(); - while(it.hasNext()) { - final Codec codec = it.next(); - codec.getExtensions(extensions); - } - } -} Index: src/test/org/apache/lucene/TestExternalCodecs.java =================================================================== --- src/test/org/apache/lucene/TestExternalCodecs.java (revision 1033390) +++ src/test/org/apache/lucene/TestExternalCodecs.java (working copy) @@ -71,7 +71,10 @@ // - good improvement would be to write through to disk, // and then load into ram from disk public static class RAMOnlyCodec extends Codec { - + + public RAMOnlyCodec() { + name = "RamOnly"; + } // Postings state: static class RAMPostings extends FieldsProducer { final Map fieldToTerms = new TreeMap(); @@ -468,6 +471,7 @@ @Override public FieldsProducer fieldsProducer(SegmentReadState readState) throws IOException { + return state.get(readState.segmentInfo.name); } @@ -481,20 +485,10 @@ } public static class MyCodecs extends CodecProvider { - PerFieldCodecWrapper perField; - MyCodecs() { Codec ram = new RAMOnlyCodec(); - Codec pulsing = new PulsingReverseTermsCodec(); - perField = new PerFieldCodecWrapper(ram); - perField.add("field2", pulsing); - perField.add("id", pulsing); - register(perField); - } - - @Override - public Codec getWriter(SegmentWriteState state) { - return perField; + register(ram); + setDefaultFieldCodec(ram.name); } } @@ -617,20 +611,28 @@ // whose term sort is backwards unicode code point, and // storing "field1" as a custom entirely-in-RAM codec public void testPerFieldCodec() throws Exception { + CodecProvider provider = new MyCodecs(); + Codec pulsing = new PulsingReverseTermsCodec(); + provider.register(pulsing); + final int NUM_DOCS = 173; Directory dir = newDirectory(); IndexWriter w = new IndexWriter(dir, - newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(MockTokenizer.WHITESPACE, true, true)).setCodecProvider(new MyCodecs())); + newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(MockTokenizer.WHITESPACE, true, true)).setCodecProvider(provider)); w.setMergeFactor(3); Document doc = new Document(); // uses default codec: doc.add(newField("field1", "this field uses the standard codec as the test", Field.Store.NO, Field.Index.ANALYZED)); // uses pulsing codec: - doc.add(newField("field2", "this field uses the pulsing codec as the test", Field.Store.NO, Field.Index.ANALYZED)); + Field field2 = newField("field2", "this field uses the pulsing codec as the test", Field.Store.NO, Field.Index.ANALYZED); + provider.setFieldCodec(field2.name(), pulsing.name); + doc.add(field2); Field idField = newField("id", "", Field.Store.NO, Field.Index.NOT_ANALYZED); + provider.setFieldCodec(idField.name(), pulsing.name); + doc.add(idField); for(int i=0;i filesToDelete = merger.createCompoundFile(merged + ".cfs", info); Index: src/test/org/apache/lucene/index/TestPerFieldCodecSupport.java =================================================================== --- src/test/org/apache/lucene/index/TestPerFieldCodecSupport.java (revision 0) +++ src/test/org/apache/lucene/index/TestPerFieldCodecSupport.java (revision 0) @@ -0,0 +1,287 @@ +package org.apache.lucene.index; + +/** + * 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.List; + +import org.apache.lucene.analysis.MockAnalyzer; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.Field.Index; +import org.apache.lucene.index.CheckIndex.Status; +import org.apache.lucene.index.CheckIndex.Status.SegmentInfoStatus; +import org.apache.lucene.index.IndexWriterConfig.OpenMode; +import org.apache.lucene.index.codecs.Codec; +import org.apache.lucene.index.codecs.CodecProvider; +import org.apache.lucene.index.codecs.mocksep.MockSepCodec; +import org.apache.lucene.index.codecs.simpletext.SimpleTextCodec; +import org.apache.lucene.index.codecs.standard.StandardCodec; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.TopDocs; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util._TestUtil; +import org.junit.Test; + +/** + * + * + */ +public class TestPerFieldCodecSupport extends LuceneTestCase { + + private IndexWriter newWriter(Directory dir, IndexWriterConfig conf) + throws IOException { + LogDocMergePolicy logByteSizeMergePolicy = new LogDocMergePolicy(); + logByteSizeMergePolicy.setUseCompoundFile(false); // make sure we use plain + // files + conf.setMergePolicy(logByteSizeMergePolicy); + + final IndexWriter writer = new IndexWriter(dir, conf); + return writer; + } + + private void addDocs(IndexWriter writer, int numDocs) throws IOException { + for (int i = 0; i < numDocs; i++) { + Document doc = new Document(); + doc.add(newField("content", "aaa", Field.Store.NO, Field.Index.ANALYZED)); + writer.addDocument(doc); + } + } + + private void addDocs2(IndexWriter writer, int numDocs) throws IOException { + for (int i = 0; i < numDocs; i++) { + Document doc = new Document(); + doc.add(newField("content", "bbb", Field.Store.NO, Field.Index.ANALYZED)); + writer.addDocument(doc); + } + } + + private void addDocs3(IndexWriter writer, int numDocs) throws IOException { + for (int i = 0; i < numDocs; i++) { + Document doc = new Document(); + doc.add(newField("content", "ccc", Field.Store.NO, Field.Index.ANALYZED)); + doc.add(newField("id", "" + i, Field.Store.YES, Field.Index.ANALYZED)); + writer.addDocument(doc); + } + } + + /* + * Test is hetrogenous index segements are merge sucessfully + */ + @Test + public void testMergeUnusedPerFieldCodec() throws IOException { + Directory dir = newDirectory(); + CodecProvider provider = new MockCodecProvider(); + IndexWriterConfig iwconf = newIndexWriterConfig(TEST_VERSION_CURRENT, + new MockAnalyzer()).setOpenMode(OpenMode.CREATE).setCodecProvider( + provider); + IndexWriter writer = newWriter(dir, iwconf); + addDocs(writer, 10); + writer.commit(); + addDocs3(writer, 10); + writer.commit(); + addDocs2(writer, 10); + writer.commit(); + assertEquals(30, writer.maxDoc()); + _TestUtil.checkIndex(dir, provider); + writer.optimize(); + assertEquals(30, writer.maxDoc()); + writer.close(); + dir.close(); + } + + /* + * Test is hetrogenous index segements are merge sucessfully + */ + @Test + public void testChangeCodecAndMerge() throws IOException { + Directory dir = newDirectory(); + CodecProvider provider = new MockCodecProvider(); + IndexWriterConfig iwconf = newIndexWriterConfig(TEST_VERSION_CURRENT, + new MockAnalyzer()).setOpenMode(OpenMode.CREATE).setCodecProvider( + provider); + IndexWriter writer = newWriter(dir, iwconf); + addDocs(writer, 10); + writer.commit(); + assertQuery(new Term("content", "aaa"), dir, 10, provider); + addDocs3(writer, 10); + writer.commit(); + writer.close(); + + + + assertQuery(new Term("content", "ccc"), dir, 10, provider); + assertQuery(new Term("content", "aaa"), dir, 10, provider); + assertCodecPerField(_TestUtil.checkIndex(dir, provider), "content", + provider.lookup("MockSep")); + + iwconf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()) + .setOpenMode(OpenMode.APPEND).setCodecProvider(provider); + ((LogMergePolicy) iwconf.getMergePolicy()).setUseCompoundFile(false); + ((LogMergePolicy) iwconf.getMergePolicy()).setUseCompoundDocStore(false); + + provider = new MockCodecProvider2(); // uses standard for field content + iwconf.setCodecProvider(provider); + writer = newWriter(dir, iwconf); + // swap in new codec for currently written segments + addDocs2(writer, 10); + writer.commit(); + Codec origContentCodec = provider.lookup("MockSep"); + Codec newContentCodec = provider.lookup("Standard"); + assertHybridCodecPerField(_TestUtil.checkIndex(dir, provider), "content", + origContentCodec, origContentCodec, newContentCodec); + assertEquals(30, writer.maxDoc()); + assertQuery(new Term("content", "bbb"), dir, 10, provider); + assertQuery(new Term("content", "ccc"), dir, 10, provider); + assertQuery(new Term("content", "aaa"), dir, 10, provider); + + addDocs2(writer, 10); + writer.commit(); + assertQuery(new Term("content", "ccc"), dir, 10, provider); + assertQuery(new Term("content", "bbb"), dir, 20, provider); + assertQuery(new Term("content", "aaa"), dir, 10, provider); + assertEquals(40, writer.maxDoc()); + + writer.optimize(); + assertEquals(40, writer.maxDoc()); + writer.close(); + assertCodecPerFieldOptimized(_TestUtil.checkIndex(dir, provider), + "content", newContentCodec); + assertQuery(new Term("content", "ccc"), dir, 10, provider); + assertQuery(new Term("content", "bbb"), dir, 20, provider); + assertQuery(new Term("content", "aaa"), dir, 10, provider); + + dir.close(); + } + + public void assertCodecPerFieldOptimized(Status checkIndex, String field, + Codec codec) { + assertEquals(1, checkIndex.segmentInfos.size()); + assertEquals(codec, checkIndex.segmentInfos.get(0).codec.byField(field)); + + } + + public void assertCodecPerField(Status checkIndex, String field, Codec codec) { + for (SegmentInfoStatus info : checkIndex.segmentInfos) { + assertEquals(codec, info.codec.byField(field)); + } + } + + public void assertHybridCodecPerField(Status checkIndex, String field, + Codec... codec) throws IOException { + List segmentInfos = checkIndex.segmentInfos; + assertEquals(segmentInfos.size(), codec.length); + for (int i = 0; i < codec.length; i++) { + SegmentCodecs codecInfo = segmentInfos.get(i).codec; + FieldInfos fieldInfos = new FieldInfos(checkIndex.dir, IndexFileNames + .segmentFileName(segmentInfos.get(i).name, "", + IndexFileNames.FIELD_INFOS_EXTENSION)); + FieldInfo fieldInfo = fieldInfos.fieldInfo(field); + assertEquals("faild for segment index: " + i, codec[i], + codecInfo.codecs[fieldInfo.codecId]); + } + } + + public void assertQuery(Term t, Directory dir, int num, CodecProvider codecs) + throws CorruptIndexException, IOException { + IndexReader reader = IndexReader.open(dir, null, true, + IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, codecs); + IndexSearcher searcher = new IndexSearcher(reader); + TopDocs search = searcher.search(new TermQuery(t), num + 10); + assertEquals(num, search.totalHits); + searcher.close(); + reader.close(); + + } + + public static class MockCodecProvider extends CodecProvider { + + public MockCodecProvider() { + StandardCodec standardCodec = new StandardCodec(); + setDefaultFieldCodec(standardCodec.name); + SimpleTextCodec simpleTextCodec = new SimpleTextCodec(); + MockSepCodec mockSepCodec = new MockSepCodec(); + register(standardCodec); + register(mockSepCodec); + register(simpleTextCodec); + setFieldCodec("id", simpleTextCodec.name); + setFieldCodec("content", mockSepCodec.name); + } + } + + public static class MockCodecProvider2 extends CodecProvider { + + public MockCodecProvider2() { + StandardCodec standardCodec = new StandardCodec(); + setDefaultFieldCodec(standardCodec.name); + SimpleTextCodec simpleTextCodec = new SimpleTextCodec(); + MockSepCodec mockSepCodec = new MockSepCodec(); + register(standardCodec); + register(mockSepCodec); + register(simpleTextCodec); + setFieldCodec("id", simpleTextCodec.name); + setFieldCodec("content", standardCodec.name); + } + } + + /* + * Test per field codec support - adding fields with random codecs + */ + @Test + public void testStressPerFieldCodec() throws IOException { + Directory dir = newDirectory(random); + Index[] indexValue = new Index[] { Index.ANALYZED, Index.ANALYZED_NO_NORMS, + Index.NOT_ANALYZED, Index.NOT_ANALYZED_NO_NORMS }; + final int docsPerRound = 97; + for (int i = 0; i < 5; i++) { + CodecProvider provider = new CodecProvider(); + provider.register(new StandardCodec()); + provider.register(new SimpleTextCodec()); + // provider.register(new MockSepCodec()); // TODO enable once we have files per codec + // provider.register(new PulsingCodec()); + + for (int j = 0; j < 30 * RANDOM_MULTIPLIER; j++) { + provider.setFieldCodec("" + j, random.nextBoolean() ? "SimpleText" + : "Standard"); // TODO enable other codecs once possible + } + IndexWriterConfig config = newIndexWriterConfig(random, + TEST_VERSION_CURRENT, new MockAnalyzer()); + config.setOpenMode(OpenMode.CREATE_OR_APPEND); + config.setCodecProvider(provider); + IndexWriter writer = newWriter(dir, config); + for (int j = 0; j < docsPerRound; j++) { + final Document doc = new Document(); + for (int k = 0; k < 30 * RANDOM_MULTIPLIER; k++) { + Field field = newField("" + k, _TestUtil + .randomRealisticUnicodeString(random, 128), indexValue[random + .nextInt(indexValue.length)]); + doc.add(field); + } + writer.addDocument(doc); + } + if(random.nextBoolean()) + writer.optimize(); + writer.commit(); + assertEquals((i+1) * docsPerRound, writer.maxDoc()); + writer.close(); + _TestUtil.checkIndex(dir, provider); + } + dir.close(); + } +} \ No newline at end of file Property changes on: src/test/org/apache/lucene/index/TestPerFieldCodecSupport.java ___________________________________________________________________ Added: svn:eol-style + native Added: svn:keywords + Date Author Id Revision HeadURL Index: src/test/org/apache/lucene/index/TestSegmentMerger.java =================================================================== --- src/test/org/apache/lucene/index/TestSegmentMerger.java (revision 1033390) +++ src/test/org/apache/lucene/index/TestSegmentMerger.java (working copy) @@ -81,7 +81,7 @@ assertTrue(docsMerged == 2); //Should be able to open a new SegmentReader against the new directory SegmentReader mergedReader = SegmentReader.get(false, mergedDir, new SegmentInfo(mergedSegment, docsMerged, mergedDir, false, -1, - null, false, merger.hasProx(), merger.getCodec()), BufferedIndexInput.BUFFER_SIZE, true, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, null); + null, false, merger.hasProx(), merger.getSegmentCodecs()), BufferedIndexInput.BUFFER_SIZE, true, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR); assertTrue(mergedReader != null); assertTrue(mergedReader.numDocs() == 2); Index: src/test/org/apache/lucene/util/LuceneTestCase.java =================================================================== --- src/test/org/apache/lucene/util/LuceneTestCase.java (revision 1033390) +++ src/test/org/apache/lucene/util/LuceneTestCase.java (working copy) @@ -230,6 +230,7 @@ } CodecProvider.setDefaultCodec(codec); + cp.setDefaultFieldCodec(codec); if (codec.equals("PreFlex")) { // If we're running w/ PreFlex codec we must swap in the @@ -262,6 +263,8 @@ cp.unregister(cp.lookup("MockVariableIntBlock")); swapCodec(new PulsingCodec(1)); CodecProvider.setDefaultCodec(savedDefaultCodec); + cp.setDefaultFieldCodec(savedDefaultCodec); + } // randomly picks from core and test codecs Index: src/test/org/apache/lucene/util/_TestUtil.java =================================================================== --- src/test/org/apache/lucene/util/_TestUtil.java (revision 1033390) +++ src/test/org/apache/lucene/util/_TestUtil.java (working copy) @@ -192,12 +192,7 @@ } public static CodecProvider alwaysCodec(final Codec c) { - return new CodecProvider() { - @Override - public Codec getWriter(SegmentWriteState state) { - return c; - } - + CodecProvider p = new CodecProvider() { @Override public Codec lookup(String name) { // can't do this until we fix PreFlexRW to not @@ -209,6 +204,8 @@ } } }; + p.setDefaultFieldCodec(c.name); + return p; } /** Return a CodecProvider that can read any of the