diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/FilteringCodec.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/FilteringCodec.java new file mode 100644 index 0000000..9bf02d6 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/FilteringCodec.java @@ -0,0 +1,117 @@ +package org.apache.lucene.codecs.filtering; +/** + * 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.codecs.Codec; +import org.apache.lucene.codecs.DocValuesFormat; +import org.apache.lucene.codecs.FieldInfosFormat; +import org.apache.lucene.codecs.LiveDocsFormat; +import org.apache.lucene.codecs.NormsFormat; +import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.SegmentInfosFormat; +import org.apache.lucene.codecs.StoredFieldsFormat; +import org.apache.lucene.codecs.TermVectorsFormat; +import org.apache.lucene.codecs.filtering.impl.FilteringDocValuesFormat; +import org.apache.lucene.codecs.filtering.impl.FilteringFieldInfosFormat; +import org.apache.lucene.codecs.filtering.impl.FilteringLiveDocsFormat; +import org.apache.lucene.codecs.filtering.impl.FilteringNormsFormat; +import org.apache.lucene.codecs.filtering.impl.FilteringPostingsFormat; +import org.apache.lucene.codecs.filtering.impl.FilteringSegmentInfosFormat; +import org.apache.lucene.codecs.filtering.impl.FilteringStoredFieldsFormat; +import org.apache.lucene.codecs.filtering.impl.FilteringTermVectorsFormat; + +/** + * This codec allows filtering of data written to the index. It doesn't change + * the data encoding (it uses the original codec, and records the original + * codec's name) but it allows to skip or modify on the fly some of the data. + * The details of this filtering process are determined by a subclass of + * {@link WriteFilter}, which by default passes all data transparently. + */ +public class FilteringCodec extends Codec { + public final Codec originalCodec; + public final WriteFilter writeFilter; + + private final FilteringPostingsFormat postingsFormat; + private final FilteringDocValuesFormat docValuesFormat; + private final FilteringStoredFieldsFormat storedFieldsFormat; + private final FilteringTermVectorsFormat termVectorsFormat; + private final FilteringFieldInfosFormat fieldInfosFormat; + private final FilteringSegmentInfosFormat segmentInfosFormat; + private final FilteringNormsFormat normsFormat; + private final FilteringLiveDocsFormat liveDocsFormat; + + /** + * Create a filtering codec. + * @param originalCodec original codec used to encode the data. + * @param writeFilter subclass of {@link WriteFilter}, or null for no + * filtering. + */ + public FilteringCodec(Codec originalCodec, WriteFilter writeFilter) { + super(originalCodec.getName()); + this.originalCodec = originalCodec; + this.writeFilter = writeFilter; + postingsFormat = new FilteringPostingsFormat(this); + docValuesFormat = new FilteringDocValuesFormat(this); + storedFieldsFormat = new FilteringStoredFieldsFormat(this); + termVectorsFormat = new FilteringTermVectorsFormat(this); + fieldInfosFormat = new FilteringFieldInfosFormat(this); + segmentInfosFormat = new FilteringSegmentInfosFormat(this); + normsFormat = new FilteringNormsFormat(this); + liveDocsFormat = new FilteringLiveDocsFormat(this); + } + + @Override + public PostingsFormat postingsFormat() { + return postingsFormat; + } + + @Override + public DocValuesFormat docValuesFormat() { + return docValuesFormat; + } + + @Override + public StoredFieldsFormat storedFieldsFormat() { + return storedFieldsFormat; + } + + @Override + public TermVectorsFormat termVectorsFormat() { + return termVectorsFormat; + } + + @Override + public FieldInfosFormat fieldInfosFormat() { + return fieldInfosFormat; + } + + @Override + public SegmentInfosFormat segmentInfosFormat() { + return segmentInfosFormat; + } + + @Override + public NormsFormat normsFormat() { + return normsFormat; + } + + @Override + public LiveDocsFormat liveDocsFormat() { + return liveDocsFormat; + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/WriteFilter.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/WriteFilter.java new file mode 100644 index 0000000..1eacc55 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/WriteFilter.java @@ -0,0 +1,234 @@ +package org.apache.lucene.codecs.filtering; +/** + * 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.codecs.PostingsConsumer; +import org.apache.lucene.codecs.TermStats; +import org.apache.lucene.codecs.TermsConsumer; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.MutableBits; + +/** + * This class implements a strategy for filtering. Individual methods are applied + * in different contexts and from different threads, so care should be taken that + * subclasses are thread-safe. + */ +public class WriteFilter { + + // fields + /** + * Decide whether to add a postings for a field identified by this {@link FieldInfo}. + * Default implementation always returns true. + * @param field field info + * @return true if this field's postings should be added, false otherwise. + */ + public boolean addField(final FieldInfo field) { + return true; + } + + // terms + /** + * Decide whether to add postings for this term. Returns true by default. + *

IMPORTANT: if this method returns false + * then you MUST override {@link #filterFinishTerm(TermsConsumer, FieldInfo, BytesRef, TermStats)} + * and you MUST NOT call {@link TermsConsumer#finishTerm(BytesRef, TermStats)} there for this term. + * @param info field info for this term + * @param text term + * @return if true then add postings for this term, if false then ignore + * postings for this term (see the note above!) + */ + public boolean startTerm(final FieldInfo info, final BytesRef text) { + return true; + } + + /** + * Finish adding postings for a term. See notes for {@link #startTerm(FieldInfo, BytesRef)}. + * @param original original {@link TermsConsumer}. If {@link #startTerm(FieldInfo, BytesRef)} + * returned true for this term, then you should call {@link TermsConsumer#finishTerm(BytesRef, TermStats)} + * using this instance (this is the default implementation). + * @param info field info for this term + * @param text term + * @param stats term statistics + * @throws IOException + */ + public void filterFinishTerm(final TermsConsumer original, final FieldInfo info, + final BytesRef text, TermStats stats) throws IOException { + original.finishTerm(text, stats); + } + + /** + * This method is called to finish adding all postings for all terms for a + * given field. You may adjust overall statistics in this call before + * calling {@link TermsConsumer#finish(long, long, int)} on the original + * instance. + * @param original original {@link TermsConsumer} + * @param info field info + * @param sumTotalTermFreq total sum of all term frequencies across all documents + * @param sumDocFreq sum of document frequencies for all terms + * @param docCount number of documents with this field + * @throws IOException + */ + public void filterFinishTerms(final TermsConsumer original, final FieldInfo info, + long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException { + original.finish(sumTotalTermFreq, sumDocFreq, docCount); + } + + /** + * Decide whether to add postings for a particular document. By default it calls + * the original {@link PostingsConsumer#startDoc(int, int)}. Note that if you already + * skipped postings for this field in {@link #startTerm(FieldInfo, BytesRef)} then + * this method won't be called for this term. + *

IMPORTANT: if this method doesn't call the original then you MUST override + * {@link #filterFinishDoc(PostingsConsumer, BytesRef, int)} and you MUST NOT call + * the original {@link PostingsConsumer#finishDoc()} for the same term. + * @param original original {@link PostingsConsumer} + * @param info field info + * @param text term + * @param docID internal document id for which the postings are to follow + * @param termDocFreq original term document frequency - if some postings will be skipped + * then you should adjust this number accordingly. + * @throws IOException + */ + public void filterStartDoc(final PostingsConsumer original, final FieldInfo info, + final BytesRef text, int docID, int termDocFreq) throws IOException { + original.startDoc(docID, termDocFreq); + } + + /** + * Finish adding term's postings for a document. By default it calls the original + * {@link PostingsConsumer#finishDoc()}. IMPORTANT: see note in + * {@link #filterStartDoc(PostingsConsumer, FieldInfo, BytesRef, int, int)}! + * @param original original {@link PostingsConsumer} + * @param info field info + * @param text term + * @param docID internal doc id for which to finish add postings + * @throws IOException + */ + public void filterFinishDoc(final PostingsConsumer original, final FieldInfo info, + final BytesRef text, int docID) throws IOException { + original.finishDoc(); + } + + /** + * Filter position data. Default implementation calls + * {@link PostingsConsumer#addPosition(int, BytesRef, int, int)} using provided + * arguments. You may safely skip calling this method, or adjust the arguments. + * @param original original {@link PostingsConsumer} + * @param info field info + * @param text current term + * @param docID current internal doc id + * @param position position + * @param payload payload data + * @param startOffset start offset + * @param endOffset end offset + * @throws IOException + */ + public void filterAddPosition(final PostingsConsumer original, + final FieldInfo info, final BytesRef text, final int docID, + int position, BytesRef payload, int startOffset, int endOffset) throws IOException { + original.addPosition(position, payload, startOffset, endOffset); + } + + // -------- FieldInfos ---------- + /** + * Modify {@link FieldInfos}. You can e.g. skip fields, modify their type, + * drop/create norms, drop/create doc values, etc. + * Default implementation passes the original unchanged. + * @param directory directory + * @param segmentName name of the current segment being written + * @param infos original field infos + * @return possibly modified field infos + */ + public FieldInfos filterFieldInfos(final Directory directory, final String segmentName, + FieldInfos infos) { + return infos; + } + + // -------- DocValues ----------- + + /** + * Decide whether to add a doc value for a particular doc. Returns true by default. + * @param docID document id + * @param value doc value + * @param norms true if the value represents Lucene norms + * @return true if the value should be added, false otherwise. + */ + public boolean addDocValue(final int docID, final IndexableField value, final boolean norms) { + return true; + } + + /** + * Filter or modify doc value for a doc. + * @param docID document id + * @param value original value + * @param norms true if the value represents Lucene norms + * @return the original or modified value. Must never return null. + * @see #addDocValue(int, IndexableField, boolean) + */ + public IndexableField filterDocValue(final int docID, IndexableField value, final boolean norms) { + return value; + } + + // -------- LiveDocs ---------- + /** + * Filter live documents. Default implementation returns the original live docs + * bitset. + * @param bits original bit set + * @param dir directory of the index + * @param info segment info + * @return original or possibly modified bitset. May return null to indicate + * that all docs are live. + */ + public MutableBits filterLiveDocs(MutableBits bits, final Directory dir, + SegmentInfo info) { + return bits; + } + + // StoredFieldsWriter + /** + * Filter stored fields for a document. You can modify values of fields or + * drop them altogether, so long as they match the original FieldInfo-s. Default + * implementation simply returns the original list. + * @param original original list of stored field values and corresponding field infos + * @return possibly modified list of field values and corresponding field infos, or null + * to drop all stored fields for this document. + */ + public List filterStoredFields(List original) { + return original; + } + + /** + * Simple container for a pair of field info and stored field value. + */ + public static final class FilteredField { + public final FieldInfo info; + public IndexableField field; + + public FilteredField(final FieldInfo info, IndexableField field) { + this.info = info; + this.field = field; + } + } +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesConsumer.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesConsumer.java new file mode 100644 index 0000000..a995c08 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesConsumer.java @@ -0,0 +1,53 @@ +package org.apache.lucene.codecs.filtering.impl; +/** + * 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 org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.index.DocValues.Type; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.IndexableField; + +public class FilteringDocValuesConsumer extends DocValuesConsumer { + FilteringPerDocConsumer filtering; + DocValuesConsumer original; + boolean norms; + int filteredDocCount = 0; + + FilteringDocValuesConsumer(FilteringPerDocConsumer filtering, + Type type, FieldInfo field, boolean norms) throws IOException { + this.filtering = filtering; + this.original = filtering.original.addValuesField(type, field); + this.norms = norms; + } + + @Override + public void add(int docID, IndexableField value) throws IOException { + if (filtering.filter.addDocValue(docID, value, norms)) { + filteredDocCount = docID; + IndexableField val = filtering.filter.filterDocValue(docID, value, norms); + original.add(docID, val); + } + } + + @Override + public void finish(int docCount) throws IOException { + original.finish(docCount); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesFormat.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesFormat.java new file mode 100644 index 0000000..0ccf198 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringDocValuesFormat.java @@ -0,0 +1,59 @@ +package org.apache.lucene.codecs.filtering.impl; +/** + * 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.Set; + +import org.apache.lucene.codecs.DocValuesFormat; +import org.apache.lucene.codecs.PerDocConsumer; +import org.apache.lucene.codecs.PerDocProducer; +import org.apache.lucene.codecs.filtering.FilteringCodec; +import org.apache.lucene.index.PerDocWriteState; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.index.SegmentReadState; + +public class FilteringDocValuesFormat extends DocValuesFormat { + FilteringCodec filtering; + DocValuesFormat original; + + public FilteringDocValuesFormat(FilteringCodec filtering) { + this.filtering = filtering; + original = filtering.originalCodec.docValuesFormat(); + } + + @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + if (filtering.writeFilter == null) { + return original.docsConsumer(state); + } else { + PerDocConsumer consumer = original.docsConsumer(state); + return new FilteringPerDocConsumer(filtering.writeFilter, consumer, false); + } + } + + @Override + public PerDocProducer docsProducer(SegmentReadState state) throws IOException { + return original.docsProducer(state); + } + + @Override + public void files(SegmentInfo info, Set files) throws IOException { + original.files(info, files); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosFormat.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosFormat.java new file mode 100644 index 0000000..885fca4 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosFormat.java @@ -0,0 +1,56 @@ +package org.apache.lucene.codecs.filtering.impl; +/** + * 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.Set; + +import org.apache.lucene.codecs.FieldInfosFormat; +import org.apache.lucene.codecs.FieldInfosReader; +import org.apache.lucene.codecs.FieldInfosWriter; +import org.apache.lucene.codecs.filtering.FilteringCodec; +import org.apache.lucene.index.SegmentInfo; + +public class FilteringFieldInfosFormat extends FieldInfosFormat { + FilteringCodec filtering; + FieldInfosFormat original; + + public FilteringFieldInfosFormat(FilteringCodec filtering) { + this.filtering = filtering; + this.original = filtering.originalCodec.fieldInfosFormat(); + } + + @Override + public FieldInfosReader getFieldInfosReader() throws IOException { + return original.getFieldInfosReader(); + } + + @Override + public FieldInfosWriter getFieldInfosWriter() throws IOException { + if (filtering.writeFilter == null) { + return original.getFieldInfosWriter(); + } else { + return new FilteringFieldInfosWriter(this); + } + } + + @Override + public void files(SegmentInfo info, Set files) throws IOException { + original.files(info, files); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosWriter.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosWriter.java new file mode 100644 index 0000000..08a5787 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldInfosWriter.java @@ -0,0 +1,43 @@ +package org.apache.lucene.codecs.filtering.impl; +/** + * 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 org.apache.lucene.codecs.FieldInfosWriter; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; + +class FilteringFieldInfosWriter extends FieldInfosWriter { + FilteringFieldInfosFormat filtering; + FieldInfosWriter original; + + FilteringFieldInfosWriter(FilteringFieldInfosFormat filtering) throws IOException { + this.filtering = filtering; + this.original = filtering.original.getFieldInfosWriter(); + } + + @Override + public void write(Directory directory, String segmentName, FieldInfos infos, + IOContext context) throws IOException { + FieldInfos filteredInfos = filtering.filtering.writeFilter.filterFieldInfos(directory, + segmentName, infos); + original.write(directory, segmentName, filteredInfos, context); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldsConsumer.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldsConsumer.java new file mode 100644 index 0000000..bf6e5c6 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringFieldsConsumer.java @@ -0,0 +1,54 @@ +package org.apache.lucene.codecs.filtering.impl; +/** + * 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 org.apache.lucene.codecs.FieldsConsumer; +import org.apache.lucene.codecs.TermsConsumer; +import org.apache.lucene.codecs.filtering.WriteFilter; +import org.apache.lucene.index.FieldInfo; + +class FilteringFieldsConsumer extends FieldsConsumer { + WriteFilter filter; + FieldsConsumer original; + + FilteringFieldsConsumer(WriteFilter filter, FieldsConsumer original) { + this.filter = filter; + this.original = original; + } + + @Override + public TermsConsumer addField(FieldInfo field) throws IOException { + if (filter == null) { + return original.addField(field); + } else { + if (filter.addField(field)) { + TermsConsumer consumer = original.addField(field); + return new FilteringTermsConsumer(filter, field, consumer); + } else { + return NoopTermsConsumer.INSTANCE; + } + } + } + + @Override + public void close() throws IOException { + original.close(); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringLiveDocsFormat.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringLiveDocsFormat.java new file mode 100644 index 0000000..e96528a --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringLiveDocsFormat.java @@ -0,0 +1,72 @@ +package org.apache.lucene.codecs.filtering.impl; +/** + * 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.Set; + +import org.apache.lucene.codecs.LiveDocsFormat; +import org.apache.lucene.codecs.filtering.FilteringCodec; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.MutableBits; + +public class FilteringLiveDocsFormat extends LiveDocsFormat { + FilteringCodec filtering; + LiveDocsFormat original; + + public FilteringLiveDocsFormat(FilteringCodec filtering) { + this.filtering = filtering; + this.original = filtering.originalCodec.liveDocsFormat(); + } + + @Override + public MutableBits newLiveDocs(int size) throws IOException { + return original.newLiveDocs(size); + } + + @Override + public MutableBits newLiveDocs(Bits existing) throws IOException { + return original.newLiveDocs(existing); + } + + @Override + public Bits readLiveDocs(Directory dir, SegmentInfo info, IOContext context) + throws IOException { + return original.readLiveDocs(dir, info, context); + } + + @Override + public void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfo info, + IOContext context) throws IOException { + if (filtering.writeFilter == null) { + original.writeLiveDocs(bits, dir, info, context); + } else { + MutableBits filteredBits = + filtering.writeFilter.filterLiveDocs(bits, dir, info); + original.writeLiveDocs(filteredBits, dir, info, context); + } + } + + @Override + public void files(SegmentInfo info, Set files) throws IOException { + original.files(info, files); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringNormsFormat.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringNormsFormat.java new file mode 100644 index 0000000..9789796 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringNormsFormat.java @@ -0,0 +1,59 @@ +package org.apache.lucene.codecs.filtering.impl; +/** + * 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.Set; + +import org.apache.lucene.codecs.NormsFormat; +import org.apache.lucene.codecs.PerDocConsumer; +import org.apache.lucene.codecs.PerDocProducer; +import org.apache.lucene.codecs.filtering.FilteringCodec; +import org.apache.lucene.index.PerDocWriteState; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.index.SegmentReadState; + +public class FilteringNormsFormat extends NormsFormat { + FilteringCodec filtering; + NormsFormat original; + + public FilteringNormsFormat(FilteringCodec filtering) { + this.filtering = filtering; + this.original = filtering.originalCodec.normsFormat(); + } + + @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + if (filtering.writeFilter == null) { + return original.docsConsumer(state); + } else { + PerDocConsumer consumer = original.docsConsumer(state); + return new FilteringPerDocConsumer(filtering.writeFilter, consumer, true); + } + } + + @Override + public PerDocProducer docsProducer(SegmentReadState state) throws IOException { + return original.docsProducer(state); + } + + @Override + public void files(SegmentInfo info, Set files) throws IOException { + original.files(info, files); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPerDocConsumer.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPerDocConsumer.java new file mode 100644 index 0000000..78cd431 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPerDocConsumer.java @@ -0,0 +1,54 @@ +package org.apache.lucene.codecs.filtering.impl; +/** + * 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 org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.codecs.PerDocConsumer; +import org.apache.lucene.codecs.filtering.WriteFilter; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.DocValues.Type; + +class FilteringPerDocConsumer extends PerDocConsumer { + WriteFilter filter; + PerDocConsumer original; + boolean norms; + + FilteringPerDocConsumer(WriteFilter filter, PerDocConsumer original, boolean norms) { + this.filter = filter; + this.original = original; + this.norms = norms; + } + + @Override + public void close() throws IOException { + original.close(); + } + + @Override + public DocValuesConsumer addValuesField(Type type, FieldInfo field) + throws IOException { + return new FilteringDocValuesConsumer(this, type, field, norms); + } + + @Override + public void abort() { + original.abort(); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsConsumer.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsConsumer.java new file mode 100644 index 0000000..6b8a8e8 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsConsumer.java @@ -0,0 +1,59 @@ +package org.apache.lucene.codecs.filtering.impl; +/** + * 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 org.apache.lucene.codecs.PostingsConsumer; +import org.apache.lucene.codecs.filtering.WriteFilter; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.util.BytesRef; + +public class FilteringPostingsConsumer extends PostingsConsumer { + WriteFilter filter; + FieldInfo info; + BytesRef text; + PostingsConsumer original; + int curDocID; + + public FilteringPostingsConsumer(WriteFilter filter, PostingsConsumer original, + FieldInfo info, BytesRef text) throws IOException { + this.filter = filter; + this.info = info; + this.text = text; + this.original = original; + } + + @Override + public void startDoc(int docID, int termDocFreq) throws IOException { + curDocID = docID; + filter.filterStartDoc(original, info, text, docID, termDocFreq); + } + + @Override + public void addPosition(int position, BytesRef payload, int startOffset, + int endOffset) throws IOException { + + filter.filterAddPosition(original, info, text, curDocID, position, payload, + startOffset, endOffset); + } + + @Override + public void finishDoc() throws IOException { + filter.filterFinishDoc(original, info, text, curDocID); + } +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsFormat.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsFormat.java new file mode 100644 index 0000000..deff339 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringPostingsFormat.java @@ -0,0 +1,63 @@ +package org.apache.lucene.codecs.filtering.impl; +/** + * 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.Set; + +import org.apache.lucene.codecs.FieldsConsumer; +import org.apache.lucene.codecs.FieldsProducer; +import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.filtering.FilteringCodec; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SegmentWriteState; + +public class FilteringPostingsFormat extends PostingsFormat { + FilteringCodec filtering; + PostingsFormat original; + + public FilteringPostingsFormat(FilteringCodec filtering) { + super(filtering.originalCodec.getName()); + this.filtering = filtering; + this.original = filtering.originalCodec.postingsFormat(); + } + + @Override + public FieldsConsumer fieldsConsumer(SegmentWriteState state) + throws IOException { + if (filtering.writeFilter == null) { + return original.fieldsConsumer(state); + } else { + FieldsConsumer consumer = original.fieldsConsumer(state); + return new FilteringFieldsConsumer(filtering.writeFilter, consumer); + } + } + + @Override + public FieldsProducer fieldsProducer(SegmentReadState state) + throws IOException { + return original.fieldsProducer(state); + } + + @Override + public void files(SegmentInfo segmentInfo, String segmentSuffix, + Set files) throws IOException { + original.files(segmentInfo, segmentSuffix, files); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringSegmentInfosFormat.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringSegmentInfosFormat.java new file mode 100644 index 0000000..4a1b6e5 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringSegmentInfosFormat.java @@ -0,0 +1,47 @@ +package org.apache.lucene.codecs.filtering.impl; +/** + * 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.codecs.SegmentInfosFormat; +import org.apache.lucene.codecs.SegmentInfosReader; +import org.apache.lucene.codecs.SegmentInfosWriter; +import org.apache.lucene.codecs.filtering.FilteringCodec; + +public class FilteringSegmentInfosFormat extends SegmentInfosFormat { + FilteringCodec filtering; + SegmentInfosFormat original; + + public FilteringSegmentInfosFormat(FilteringCodec filtering) { + this.filtering = filtering; + this.original = filtering.originalCodec.segmentInfosFormat(); + } + + // XXX + @Override + public SegmentInfosReader getSegmentInfosReader() { + SegmentInfosReader reader = original.getSegmentInfosReader(); + return reader; + } + + // XXX + @Override + public SegmentInfosWriter getSegmentInfosWriter() { + SegmentInfosWriter writer = original.getSegmentInfosWriter(); + return writer; + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsFormat.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsFormat.java new file mode 100644 index 0000000..2cc1020 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsFormat.java @@ -0,0 +1,62 @@ +package org.apache.lucene.codecs.filtering.impl; +/** + * 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.Set; + +import org.apache.lucene.codecs.StoredFieldsFormat; +import org.apache.lucene.codecs.StoredFieldsReader; +import org.apache.lucene.codecs.StoredFieldsWriter; +import org.apache.lucene.codecs.filtering.FilteringCodec; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; + +public class FilteringStoredFieldsFormat extends StoredFieldsFormat { + FilteringCodec filtering; + StoredFieldsFormat original; + + public FilteringStoredFieldsFormat(FilteringCodec filtering) { + this.filtering = filtering; + this.original = filtering.originalCodec.storedFieldsFormat(); + } + + @Override + public StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si, + FieldInfos fn, IOContext context) throws IOException { + return original.fieldsReader(directory, si, fn, context); + } + + @Override + public StoredFieldsWriter fieldsWriter(Directory directory, String segment, + IOContext context) throws IOException { + StoredFieldsWriter writer = original.fieldsWriter(directory, segment, context); + if (filtering.writeFilter == null) { + return writer; + } else { + return new FilteringStoredFieldsWriter(filtering.writeFilter, writer); + } + } + + @Override + public void files(SegmentInfo info, Set files) throws IOException { + original.files(info, files); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsWriter.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsWriter.java new file mode 100644 index 0000000..dc679f8 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringStoredFieldsWriter.java @@ -0,0 +1,93 @@ +package org.apache.lucene.codecs.filtering.impl; +/** + * 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.List; + +import org.apache.lucene.codecs.StoredFieldsWriter; +import org.apache.lucene.codecs.filtering.WriteFilter; +import org.apache.lucene.codecs.filtering.WriteFilter.FilteredField; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.IndexableField; + +public class FilteringStoredFieldsWriter extends StoredFieldsWriter { + WriteFilter filter; + StoredFieldsWriter original; + List fields = null; + + public FilteringStoredFieldsWriter(WriteFilter filter, StoredFieldsWriter original) { + this.filter = filter; + this.original = original; + } + + @Override + public void close() throws IOException { + if (fields != null) { + flush(); + } + original.close(); + } + + @Override + public void startDocument(int numStoredFields) throws IOException { + if (fields != null) { + flush(); + } + fields = new ArrayList(numStoredFields); + } + + @Override + public void writeField(FieldInfo info, IndexableField field) + throws IOException { + fields.add(new FilteredField(info, field)); + } + + @Override + public void abort() { + original.abort(); + fields = null; + } + + private void flush() throws IOException { + List filteredFields = filter.filterStoredFields(fields); + if (filteredFields == null || filteredFields.isEmpty()) { + return; + } + List outputFields = new ArrayList(filteredFields.size()); + for (FilteredField ff : filteredFields) { + if (ff != null && ff.field != null) { + outputFields.add(ff); + } + } + original.startDocument(outputFields.size()); + for (FilteredField ff : outputFields) { + original.writeField(ff.info, ff.field); + } + fields = null; + } + + @Override + public void finish(int numDocs) throws IOException { + if (fields != null) { + flush(); + } + original.finish(numDocs); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermVectorsFormat.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermVectorsFormat.java new file mode 100644 index 0000000..cd92a54 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermVectorsFormat.java @@ -0,0 +1,62 @@ +package org.apache.lucene.codecs.filtering.impl; +/** + * 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.Set; + +import org.apache.lucene.codecs.TermVectorsFormat; +import org.apache.lucene.codecs.TermVectorsReader; +import org.apache.lucene.codecs.TermVectorsWriter; +import org.apache.lucene.codecs.filtering.FilteringCodec; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; + +public class FilteringTermVectorsFormat extends TermVectorsFormat { + FilteringCodec filtering; + TermVectorsFormat original; + + public FilteringTermVectorsFormat(FilteringCodec filtering) { + this.filtering = filtering; + this.original = filtering.originalCodec.termVectorsFormat(); + } + + // XXX + @Override + public TermVectorsReader vectorsReader(Directory directory, + SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) + throws IOException { + TermVectorsReader reader = original.vectorsReader(directory, segmentInfo, fieldInfos, context); + return reader; + } + + // XXX + @Override + public TermVectorsWriter vectorsWriter(Directory directory, String segment, + IOContext context) throws IOException { + TermVectorsWriter writer = original.vectorsWriter(directory, segment, context); + return writer; + } + + @Override + public void files(SegmentInfo info, Set files) throws IOException { + original.files(info, files); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermsConsumer.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermsConsumer.java new file mode 100644 index 0000000..11b5674 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/FilteringTermsConsumer.java @@ -0,0 +1,66 @@ +package org.apache.lucene.codecs.filtering.impl; +/** + * 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 org.apache.lucene.codecs.PostingsConsumer; +import org.apache.lucene.codecs.TermStats; +import org.apache.lucene.codecs.TermsConsumer; +import org.apache.lucene.codecs.filtering.WriteFilter; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.util.BytesRef; + +public class FilteringTermsConsumer extends TermsConsumer { + WriteFilter filter; + TermsConsumer original; + FieldInfo info; + + public FilteringTermsConsumer(WriteFilter filter, FieldInfo info, TermsConsumer original) throws IOException { + this.filter = filter; + this.original = original; + this.info = info; + } + + @Override + public PostingsConsumer startTerm(final BytesRef text) throws IOException { + if (!filter.startTerm(info, text)) { + return NoopPostingsConsumer.INSTANCE; + } + PostingsConsumer consumer = original.startTerm(text); + FilteringPostingsConsumer fpc = new FilteringPostingsConsumer(filter, consumer, info, text); + return fpc; + } + + @Override + public void finishTerm(final BytesRef text, TermStats stats) throws IOException { + filter.filterFinishTerm(original, info, text, stats); + } + + @Override + public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) + throws IOException { + filter.filterFinishTerms(original, info, sumTotalTermFreq, sumDocFreq, docCount); + } + + @Override + public Comparator getComparator() throws IOException { + return original.getComparator(); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopDocValuesConsumer.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopDocValuesConsumer.java new file mode 100644 index 0000000..e838b1b --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopDocValuesConsumer.java @@ -0,0 +1,37 @@ +package org.apache.lucene.codecs.filtering.impl; +/** + * 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 org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.index.IndexableField; + +class NoopDocValuesConsumer extends DocValuesConsumer { + static final DocValuesConsumer INSTANCE = new NoopDocValuesConsumer(); + + @Override + public void add(int docID, IndexableField value) throws IOException { + // no-op + } + + @Override + public void finish(int docCount) throws IOException { + // no-op + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopPostingsConsumer.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopPostingsConsumer.java new file mode 100644 index 0000000..c314d40 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopPostingsConsumer.java @@ -0,0 +1,44 @@ +package org.apache.lucene.codecs.filtering.impl; +/** + * 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 org.apache.lucene.codecs.PostingsConsumer; +import org.apache.lucene.util.BytesRef; + +public class NoopPostingsConsumer extends PostingsConsumer { + + public static final NoopPostingsConsumer INSTANCE = new NoopPostingsConsumer(); + + @Override + public void startDoc(int docID, int termDocFreq) throws IOException { + // noop + } + + @Override + public void addPosition(int position, BytesRef payload, int startOffset, + int endOffset) throws IOException { + // noop + } + + @Override + public void finishDoc() throws IOException { + // noop + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopTermsConsumer.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopTermsConsumer.java new file mode 100644 index 0000000..24bd3ab --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/filtering/impl/NoopTermsConsumer.java @@ -0,0 +1,50 @@ +package org.apache.lucene.codecs.filtering.impl; +/** + * 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 org.apache.lucene.codecs.PostingsConsumer; +import org.apache.lucene.codecs.TermStats; +import org.apache.lucene.codecs.TermsConsumer; +import org.apache.lucene.util.BytesRef; + +public class NoopTermsConsumer extends TermsConsumer { + public static final NoopTermsConsumer INSTANCE = new NoopTermsConsumer(); + + @Override + public PostingsConsumer startTerm(BytesRef text) throws IOException { + return NoopPostingsConsumer.INSTANCE; + } + + @Override + public void finishTerm(BytesRef text, TermStats stats) throws IOException { + // noop + } + + @Override + public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) + throws IOException { + // noop + } + + @Override + public Comparator getComparator() throws IOException { + return BytesRef.getUTF8SortedAsUnicodeComparator(); + } +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/TeeCodec.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/TeeCodec.java new file mode 100644 index 0000000..4d0720e --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/TeeCodec.java @@ -0,0 +1,128 @@ +package org.apache.lucene.codecs.tee; +/** + * 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.Map; +import java.util.Set; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.DocValuesFormat; +import org.apache.lucene.codecs.FieldInfosFormat; +import org.apache.lucene.codecs.LiveDocsFormat; +import org.apache.lucene.codecs.NormsFormat; +import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.SegmentInfosFormat; +import org.apache.lucene.codecs.StoredFieldsFormat; +import org.apache.lucene.codecs.TermVectorsFormat; +import org.apache.lucene.codecs.tee.impl.TeeDocValuesFormat; +import org.apache.lucene.codecs.tee.impl.TeeFieldInfosFormat; +import org.apache.lucene.codecs.tee.impl.TeeLiveDocsFormat; +import org.apache.lucene.codecs.tee.impl.TeeNormsFormat; +import org.apache.lucene.codecs.tee.impl.TeePostingsFormat; +import org.apache.lucene.codecs.tee.impl.TeeSegmentInfosFormat; +import org.apache.lucene.codecs.tee.impl.TeeStoredFieldsFormat; +import org.apache.lucene.codecs.tee.impl.TeeTermVectorsFormat; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.store.Directory; + +/** + * This codec implements a tee mechanism, that is it mirrors all index data + * written to it to multiple outputs, possible using different codec per output. + * + * Note: this codec is primarily intended as an output mechanism. The reading + * part uses just the main codec ignoring other codecs, and may not reflect + * correctly the state of data in all outputs. + */ +public class TeeCodec extends Codec { + public Map outputs; + public Codec mainCodec; + + private final PostingsFormat postingsFormat; + private final DocValuesFormat docValuesFormat; + private final FieldInfosFormat fieldInfosFormat; + private final LiveDocsFormat liveDocsFormat; + private final NormsFormat normsFormat; + private final SegmentInfosFormat segmentInfosFormat; + private final StoredFieldsFormat storedFieldsFormat; + private final TermVectorsFormat termVectorsFormat; + + /** + * Create a TeeCodec. + * @param mainCodec main codec used for the managing the index. + * @param outputs a map of outputs where the data written to the main index + * will be mirrored, possibly using different codecs. + */ + public TeeCodec(Codec mainCodec, Map outputs) { + super(mainCodec.getName()); + this.mainCodec = mainCodec; + this.outputs = outputs; + postingsFormat = new TeePostingsFormat(mainCodec, outputs); + docValuesFormat = new TeeDocValuesFormat(mainCodec, outputs); + fieldInfosFormat = new TeeFieldInfosFormat(mainCodec, outputs); + liveDocsFormat = new TeeLiveDocsFormat(mainCodec, outputs); + normsFormat = new TeeNormsFormat(mainCodec, outputs); + segmentInfosFormat = new TeeSegmentInfosFormat(mainCodec, outputs); + storedFieldsFormat = new TeeStoredFieldsFormat(mainCodec, outputs); + termVectorsFormat = new TeeTermVectorsFormat(mainCodec, outputs); + } + + @Override + public PostingsFormat postingsFormat() { + return postingsFormat; + } + + @Override + public DocValuesFormat docValuesFormat() { + return docValuesFormat; + } + + @Override + public StoredFieldsFormat storedFieldsFormat() { + return storedFieldsFormat; + } + + @Override + public TermVectorsFormat termVectorsFormat() { + return termVectorsFormat; + } + + @Override + public FieldInfosFormat fieldInfosFormat() { + return fieldInfosFormat; + } + + @Override + public SegmentInfosFormat segmentInfosFormat() { + return segmentInfosFormat; + } + + @Override + public NormsFormat normsFormat() { + return normsFormat; + } + + @Override + public LiveDocsFormat liveDocsFormat() { + return liveDocsFormat; + } + + @Override + public void files(SegmentInfo info, Set files) throws IOException { + mainCodec.files(info, files); + } +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/TeeDirectory.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/TeeDirectory.java new file mode 100644 index 0000000..6adbde0 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/TeeDirectory.java @@ -0,0 +1,190 @@ +package org.apache.lucene.codecs.tee; +/** + * 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.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.lucene.codecs.tee.impl.TeeIndexOutput; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; + +/** + * This class provides a 1:1 mirroring mechanism at the level of + * {@link Directory}. Unlike the {@link TeeCodec} it doesn't allow you to + * change the encoding of data, instead it simply copies byte by byte all + * data written to the main directory. + *

You can specify a list of suffixes for file names to be mirrored. Files with + * names that don't match the list will be exempt from the mirroring process. If the list + * is empty or null then all files will be mirrored. + *

The main Directory is used for all read operations. + *

Note: existing data in the Directory won't be mirrored + * unless the initialSync is set to true - then in the constructor all + * existing data (from files with matching names) in the main directory will be copied, + * regardless whether files with the same names already exist in the secondary + * directories. See also the note above about name matching. + */ +public class TeeDirectory extends Directory { + private Set suffixes; + private Collection dirs; + private Directory main; + private boolean debug = false; + + /** + * Create a TeeDirectory. + * @param main main directory + * @param dirs secondary directories. + * @param suffixes a list of file name suffixes that are subject to the mirroring. + * See the note above. + * @param initialSync if true then the data from already existing files (that + * match suffixes) will be copied to all secondary directories. If false then only newly + * created files will be copied. + * @throws IOException if an exception occurs during initial sync. + */ + public TeeDirectory(Directory main, Collection dirs, + Set suffixes, boolean initialSync) throws IOException { + this.main = main; + this.lockFactory = main.getLockFactory(); + this.dirs = dirs; + if (suffixes == null || suffixes.isEmpty()) { + this.suffixes = Collections.emptySet(); + } else { + this.suffixes = suffixes; + } + if (initialSync) { + initialSync(); + } + } + + private void initialSync() throws IOException { + String[] files = main.listAll(); + for (String name : files) { + if (!suffixMatch(name)) { + continue; + } + // copy the file + Map outs = new HashMap(); + for (Directory d : dirs) { + outs.put(d, d.createOutput(name, IOContext.DEFAULT)); + } + TeeIndexOutput tee = new TeeIndexOutput(outs); + IndexInput in = main.openInput(name, IOContext.READ); + in.copyBytes(tee, main.fileLength(name)); + tee.close(); + } + + } + + private boolean suffixMatch(String name) { + if (suffixes.isEmpty()) { + return true; + } + for (String s : suffixes) { + if (name.endsWith(s)) { + return true; + } + } + return false; + } + + @Override + public String[] listAll() throws IOException { + return main.listAll(); + } + + @Override + public boolean fileExists(String name) throws IOException { + return main.fileExists(name); + } + + @Override + public synchronized void deleteFile(String name) throws IOException { + main.deleteFile(name); + if (suffixMatch(name)) { + for (Directory d : dirs) { + d.deleteFile(name); + } + } + } + + @Override + public long fileLength(String name) throws IOException { + return main.fileLength(name); + } + + @Override + public synchronized IndexOutput createOutput(String name, IOContext context) + throws IOException { + if (debug) { + new Exception("-createOutput " + name).printStackTrace(); + } + IndexOutput mainOutput = main.createOutput(name, context); + if (suffixMatch(name)) { + Map outs = new HashMap(); + outs.put(main, mainOutput); + for (Directory d : dirs) { + outs.put(d, d.createOutput(name, context)); + } + return new TeeIndexOutput(outs); + } else { + return mainOutput; + } + } + + @Override + public synchronized void sync(Collection names) throws IOException { + main.sync(names); + if (suffixes.isEmpty()) { + for (Directory d : dirs) { + d.sync(names); + } + } else { + Set syncNames = new HashSet(); + for (String n : names) { + if (suffixMatch(n)) { + syncNames.add(n); + } + } + if (!syncNames.isEmpty()) { + for (Directory d : dirs) { + d.sync(syncNames); + } + } + } + } + + @Override + public IndexInput openInput(String name, IOContext context) + throws IOException { + return main.openInput(name, context); + } + + @Override + public synchronized void close() throws IOException { + main.close(); + for (Directory d : dirs) { + d.close(); + } + } +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesConsumer.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesConsumer.java new file mode 100644 index 0000000..fa9e04d --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesConsumer.java @@ -0,0 +1,60 @@ +package org.apache.lucene.codecs.tee.impl; +/** + * 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.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.codecs.PerDocConsumer; +import org.apache.lucene.index.DocValues.Type; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.store.Directory; + +public class TeeDocValuesConsumer extends DocValuesConsumer { + DocValuesConsumer mainDVConsumer; + Map dvConsumers; + + public TeeDocValuesConsumer(PerDocConsumer mainConsumer, + Map consumers, Type type, FieldInfo field) throws IOException { + mainDVConsumer = mainConsumer.addValuesField(type, field); + dvConsumers = new HashMap(); + for (Entry e : consumers.entrySet()) { + dvConsumers.put(e.getKey(), e.getValue().addValuesField(type, field)); + } + } + + @Override + public void add(int docID, IndexableField value) throws IOException { + mainDVConsumer.add(docID, value); + for (DocValuesConsumer c : dvConsumers.values()) { + c.add(docID, value); + } + } + + @Override + public void finish(int docCount) throws IOException { + mainDVConsumer.finish(docCount); + for (DocValuesConsumer c : dvConsumers.values()) { + c.finish(docCount); + } + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesFormat.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesFormat.java new file mode 100644 index 0000000..4336341 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeDocValuesFormat.java @@ -0,0 +1,76 @@ +package org.apache.lucene.codecs.tee.impl; +/** + * 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.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.DocValuesFormat; +import org.apache.lucene.codecs.PerDocConsumer; +import org.apache.lucene.codecs.PerDocProducer; +import org.apache.lucene.index.PerDocWriteState; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.store.Directory; + +public class TeeDocValuesFormat extends DocValuesFormat { + Codec main; + DocValuesFormat mainFormat; + Map outputs; + + public TeeDocValuesFormat(Codec main, Map outputs) { + this.main = main; + this.outputs = outputs; + mainFormat = main.docValuesFormat(); + } + + @Override + public synchronized PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + PerDocConsumer mainConsumer = mainFormat.docsConsumer(state); + // create states for each output + HashMap consumers = new HashMap(); + for (Entry e : outputs.entrySet()) { + PerDocWriteState s = new PerDocWriteState(state.infoStream, + e.getKey(), state.segmentName, state.fieldInfos, state.bytesUsed, + state.segmentSuffix, state.context); + PerDocConsumer c = e.getValue().docValuesFormat().docsConsumer(s); + // consumer may be null if output codec doesn't support it + if (c != null) { + consumers.put(e.getKey(), c); + } + } + return new TeePerDocConsumer(mainConsumer, consumers); + } + + /** + * Reads always from the main codec. + */ + @Override + public PerDocProducer docsProducer(SegmentReadState state) throws IOException { + return mainFormat.docsProducer(state); + } + + @Override + public void files(SegmentInfo info, Set files) throws IOException { + mainFormat.files(info, files); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosFormat.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosFormat.java new file mode 100644 index 0000000..a2ec9da --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosFormat.java @@ -0,0 +1,59 @@ +package org.apache.lucene.codecs.tee.impl; +/** + * 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.Map; +import java.util.Set; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.FieldInfosFormat; +import org.apache.lucene.codecs.FieldInfosReader; +import org.apache.lucene.codecs.FieldInfosWriter; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.store.Directory; + +public class TeeFieldInfosFormat extends FieldInfosFormat { + Codec main; + FieldInfosFormat mainFormat; + Map outputs; + + public TeeFieldInfosFormat(Codec main, Map outputs) { + this.main = main; + this.outputs = outputs; + mainFormat = main.fieldInfosFormat(); + } + + /** + * Read always the original output. + */ + @Override + public FieldInfosReader getFieldInfosReader() throws IOException { + return mainFormat.getFieldInfosReader(); + } + + @Override + public synchronized FieldInfosWriter getFieldInfosWriter() throws IOException { + return new TeeFieldInfosWriter(mainFormat, outputs); + } + + @Override + public void files(SegmentInfo info, Set files) throws IOException { + mainFormat.files(info, files); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosWriter.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosWriter.java new file mode 100644 index 0000000..921da52 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldInfosWriter.java @@ -0,0 +1,54 @@ +package org.apache.lucene.codecs.tee.impl; +/** + * 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.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.FieldInfosFormat; +import org.apache.lucene.codecs.FieldInfosWriter; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; + +public class TeeFieldInfosWriter extends FieldInfosWriter { + FieldInfosWriter main; + Map writers; + + public TeeFieldInfosWriter(FieldInfosFormat mainFormat, + Map outputs) throws IOException { + main = mainFormat.getFieldInfosWriter(); + writers = new HashMap(); + for (Entry e : outputs.entrySet()) { + writers.put(e.getKey(), e.getValue().fieldInfosFormat().getFieldInfosWriter()); + } + } + + @Override + public void write(Directory directory, String segmentName, FieldInfos infos, + IOContext context) throws IOException { + main.write(directory, segmentName, infos, context); + // write to each output directory + for (Entry e : writers.entrySet()) { + e.getValue().write(e.getKey(), segmentName, infos, context); + } + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldsConsumer.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldsConsumer.java new file mode 100644 index 0000000..4b97ced --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeFieldsConsumer.java @@ -0,0 +1,57 @@ +package org.apache.lucene.codecs.tee.impl; +/** + * 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.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.FieldsConsumer; +import org.apache.lucene.codecs.TermsConsumer; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.store.Directory; + +public class TeeFieldsConsumer extends FieldsConsumer { + FieldsConsumer mainConsumer; + Map consumers; + + public TeeFieldsConsumer(FieldsConsumer mainConsumer, + HashMap consumers) { + this.mainConsumer = mainConsumer; + this.consumers = consumers; + } + + @Override + public synchronized TermsConsumer addField(FieldInfo field) throws IOException { + TermsConsumer mainTConsumer = mainConsumer.addField(field); + HashMap tConsumers = new HashMap(); + for (Entry e : consumers.entrySet()) { + tConsumers.put(e.getKey(), e.getValue().addField(field)); + } + return new TeeTermsConsumer(mainTConsumer, tConsumers); + } + + @Override + public synchronized void close() throws IOException { + mainConsumer.close(); + for (FieldsConsumer f : consumers.values()) { + f.close(); + } + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeIndexOutput.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeIndexOutput.java new file mode 100644 index 0000000..9921183 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeIndexOutput.java @@ -0,0 +1,143 @@ +package org.apache.lucene.codecs.tee.impl; +/** + * 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.Iterator; +import java.util.Map; + +import org.apache.lucene.codecs.SegmentInfosWriter; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IndexOutput; + +/** + * Copy the main output to many secondary outputs. + */ +public class TeeIndexOutput extends IndexOutput { + + SegmentInfosWriter main; + Map outs; + Map writers; + + /** + * Create an output that writes to multiple destinations. + * @param outs + */ + public TeeIndexOutput(Map outs) { + this(null, outs, null); + } + + /** + * Create an output that writes to multiple destinations. This + * constructor is used in the context of two-phase commit for + * segment infos. + * @param main main writer + * @param outs outputs + * @param writers writers that manage these outputs + */ + TeeIndexOutput(SegmentInfosWriter main, Map outs, + Map writers) { + this.main = main; + this.outs = outs; + this.writers = writers; + } + + /** + * Return the outputs. + */ + Map getOutputs() { + return outs; + } + + /** + * Return the writers. + */ + Map getWriters() { + return writers; + } + + /** + * Return the main writer. + */ + SegmentInfosWriter getMain() { + return main; + } + + @Override + public synchronized void setLength(long length) throws IOException { + for (IndexOutput io : outs.values()) { + io.setLength(length); + } + } + + @Override + public synchronized void flush() throws IOException { + for (IndexOutput io : outs.values()) { + io.flush(); + } + } + + @Override + public synchronized void close() throws IOException { + for (IndexOutput io : outs.values()) { + io.close(); + } + } + + @Override + public long getFilePointer() { + // should be the same + Iterator it = outs.values().iterator(); + long ptr = it.next().getFilePointer(); + if (outs.size() > 1) { + assert it.next().getFilePointer() == ptr; + } + return ptr; + } + + @Override + public synchronized void seek(long pos) throws IOException { + for (IndexOutput io : outs.values()) { + io.seek(pos); + } + } + + @Override + public long length() throws IOException { + // should be the same + Iterator it = outs.values().iterator(); + long len = it.next().length(); + if (outs.size() > 1) { + assert it.next().length() == len; + } + return len; + } + + @Override + public synchronized void writeByte(byte b) throws IOException { + for (IndexOutput io : outs.values()) { + io.writeByte(b); + } + } + + @Override + public synchronized void writeBytes(byte[] b, int offset, int length) throws IOException { + for (IndexOutput io : outs.values()) { + io.writeBytes(b, offset, length); + } + } +} \ No newline at end of file diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeLiveDocsFormat.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeLiveDocsFormat.java new file mode 100644 index 0000000..7fc6ba5 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeLiveDocsFormat.java @@ -0,0 +1,76 @@ +package org.apache.lucene.codecs.tee.impl; +/** + * 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.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.LiveDocsFormat; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.MutableBits; + +public class TeeLiveDocsFormat extends LiveDocsFormat { + LiveDocsFormat mainFormat; + Map formats; + + public TeeLiveDocsFormat(Codec main, Map outputs) { + this.mainFormat = main.liveDocsFormat(); + this.formats = new HashMap(); + for (Entry e : outputs.entrySet()) { + formats.put(e.getKey(), e.getValue().liveDocsFormat()); + } + } + + @Override + public MutableBits newLiveDocs(int size) throws IOException { + return mainFormat.newLiveDocs(size); + } + + @Override + public MutableBits newLiveDocs(Bits existing) throws IOException { + return mainFormat.newLiveDocs(existing); + } + + @Override + public Bits readLiveDocs(Directory dir, SegmentInfo info, IOContext context) + throws IOException { + return mainFormat.readLiveDocs(dir, info, context); + } + + @Override + public synchronized void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfo info, + IOContext context) throws IOException { + mainFormat.writeLiveDocs(bits, dir, info, context); + // write to each outputs directory + for (Entry e : formats.entrySet()) { + e.getValue().writeLiveDocs(bits, e.getKey(), info, context); + } + } + + @Override + public void files(SegmentInfo info, Set files) throws IOException { + mainFormat.files(info, files); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeNormsFormat.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeNormsFormat.java new file mode 100644 index 0000000..4972e6f --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeNormsFormat.java @@ -0,0 +1,78 @@ +package org.apache.lucene.codecs.tee.impl; +/** + * 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.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.NormsFormat; +import org.apache.lucene.codecs.PerDocConsumer; +import org.apache.lucene.codecs.PerDocProducer; +import org.apache.lucene.index.PerDocWriteState; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.store.Directory; + +public class TeeNormsFormat extends NormsFormat { + Codec main; + NormsFormat mainFormat; + Map outputs; + + public TeeNormsFormat(Codec main, Map outputs) { + this.main = main; + this.outputs = outputs; + mainFormat = main.normsFormat(); + } + + + @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + PerDocConsumer mainConsumer = mainFormat.docsConsumer(state); + // create states for each output + HashMap consumers = new HashMap(); + for (Entry e : outputs.entrySet()) { + PerDocWriteState s = new PerDocWriteState(state.infoStream, + e.getKey(), state.segmentName, state.fieldInfos, state.bytesUsed, + state.segmentSuffix, state.context); + //System.err.println(state.segmentName + "/" + state.segmentSuffix); + PerDocConsumer c = e.getValue().docValuesFormat().docsConsumer(s); + if (c != null) { + consumers.put(e.getKey(), c); + } + } + return new TeePerDocConsumer(mainConsumer, consumers); + } + + @Override + public PerDocProducer docsProducer(SegmentReadState state) throws IOException { + return mainFormat.docsProducer(state); + } + + @Override + public void files(SegmentInfo info, Set files) throws IOException { + HashSet dbg = new HashSet(); + mainFormat.files(info, dbg); + //System.err.println("-files " + dbg); + mainFormat.files(info, files); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePerDocConsumer.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePerDocConsumer.java new file mode 100644 index 0000000..0729c88 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePerDocConsumer.java @@ -0,0 +1,83 @@ +package org.apache.lucene.codecs.tee.impl; +/** + * 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.Map; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.codecs.PerDocConsumer; +import org.apache.lucene.index.AtomicReader; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.DocValues.Type; +import org.apache.lucene.store.Directory; + +public class TeePerDocConsumer extends PerDocConsumer { + PerDocConsumer mainConsumer; + Map consumers; + + public TeePerDocConsumer(PerDocConsumer mainConsumer, + Map consumers) throws IOException { + this.mainConsumer = mainConsumer; + this.consumers = consumers; + } + + @Override + public synchronized void close() throws IOException { + mainConsumer.close(); + for (Entry e : consumers.entrySet()) { + e.getValue().close(); + } + } + + @Override + public synchronized DocValuesConsumer addValuesField(Type type, FieldInfo field) + throws IOException { + return new TeeDocValuesConsumer(mainConsumer, consumers, type, field); + } + + @Override + public synchronized void abort() { + mainConsumer.abort(); + for (Entry e : consumers.entrySet()) { + e.getValue().abort(); + } + } + + // HACKY: this is trappy about perdocconsumer API + // must delete since all norms impl this differently... + // we should restructure + @Override + public boolean canMerge(FieldInfo info) { + return mainConsumer.canMerge(info); + } + + @Override + public DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info) throws IOException { + return mainConsumer.getDocValuesForMerge(reader, info); + } + + @Override + public Type getDocValuesType(FieldInfo info) { + return mainConsumer.getDocValuesType(info); + } + + + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsConsumer.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsConsumer.java new file mode 100644 index 0000000..65a169d --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsConsumer.java @@ -0,0 +1,61 @@ +package org.apache.lucene.codecs.tee.impl; +/** + * 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.Map; + +import org.apache.lucene.codecs.PostingsConsumer; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; + +public class TeePostingsConsumer extends PostingsConsumer { + PostingsConsumer mainConsumer; + Map consumers; + + public TeePostingsConsumer(PostingsConsumer mainConsumer, + Map consumers) { + this.mainConsumer = mainConsumer; + this.consumers = consumers; + } + + @Override + public synchronized void startDoc(int docID, int termDocFreq) throws IOException { + mainConsumer.startDoc(docID, termDocFreq); + for (PostingsConsumer c : consumers.values()) { + c.startDoc(docID, termDocFreq); + } + } + + @Override + public synchronized void addPosition(int position, BytesRef payload, int startOffset, + int endOffset) throws IOException { + mainConsumer.addPosition(position, payload, startOffset, endOffset); + for (PostingsConsumer c : consumers.values()) { + c.addPosition(position, payload, startOffset, endOffset); + } + } + + @Override + public synchronized void finishDoc() throws IOException { + mainConsumer.finishDoc(); + for (PostingsConsumer c : consumers.values()) { + c.finishDoc(); + } + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsFormat.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsFormat.java new file mode 100644 index 0000000..df8d32f --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeePostingsFormat.java @@ -0,0 +1,73 @@ +package org.apache.lucene.codecs.tee.impl; +/** + * 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.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.FieldsConsumer; +import org.apache.lucene.codecs.FieldsProducer; +import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.Directory; + +public class TeePostingsFormat extends PostingsFormat { + Codec main; + PostingsFormat mainFormat; + Map outputs; + + public TeePostingsFormat(Codec main, Map outputs) { + super(main.getName()); + this.main = main; + this.outputs = outputs; + this.mainFormat = main.postingsFormat(); + } + + + + @Override + public FieldsConsumer fieldsConsumer(SegmentWriteState state) + throws IOException { + FieldsConsumer mainConsumer = mainFormat.fieldsConsumer(state); + HashMap consumers = new HashMap(); + for (Entry e : outputs.entrySet()) { + SegmentWriteState s = new SegmentWriteState(state.infoStream, e.getKey(), + state.segmentName, state.fieldInfos, state.numDocs, state.termIndexInterval, + e.getValue(), state.segDeletes, state.context); + consumers.put(e.getKey(), e.getValue().postingsFormat().fieldsConsumer(s)); + } + return new TeeFieldsConsumer(mainConsumer, consumers); + } + + @Override + public FieldsProducer fieldsProducer(SegmentReadState state) + throws IOException { + return mainFormat.fieldsProducer(state); + } + + @Override + public void files(SegmentInfo segmentInfo, String segmentSuffix, + Set files) throws IOException { + mainFormat.files(segmentInfo, segmentSuffix, files); + } +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfosFormat.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfosFormat.java new file mode 100644 index 0000000..c6e5fab --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfosFormat.java @@ -0,0 +1,49 @@ +package org.apache.lucene.codecs.tee.impl; +/** + * 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 org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.SegmentInfosFormat; +import org.apache.lucene.codecs.SegmentInfosReader; +import org.apache.lucene.codecs.SegmentInfosWriter; +import org.apache.lucene.store.Directory; + +public class TeeSegmentInfosFormat extends SegmentInfosFormat { + Codec main; + Map outputs; + SegmentInfosFormat mainFormat; + + public TeeSegmentInfosFormat(Codec main, Map outputs) { + this.main = main; + this.outputs = outputs; + mainFormat = main.segmentInfosFormat(); + } + + @Override + public synchronized SegmentInfosReader getSegmentInfosReader() { + return mainFormat.getSegmentInfosReader(); + } + + @Override + public synchronized SegmentInfosWriter getSegmentInfosWriter() { + SegmentInfosWriter mainWriter = mainFormat.getSegmentInfosWriter(); + return new TeeSegmentInfosWriter(mainWriter, outputs); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfosWriter.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfosWriter.java new file mode 100644 index 0000000..85dd852 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeSegmentInfosWriter.java @@ -0,0 +1,77 @@ +package org.apache.lucene.codecs.tee.impl; +/** + * 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.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.SegmentInfosWriter; +import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexOutput; + +public class TeeSegmentInfosWriter extends SegmentInfosWriter { + SegmentInfosWriter mainWriter; + Map outputs; + + public TeeSegmentInfosWriter(SegmentInfosWriter mainWriter, + Map outputs) { + this.mainWriter = mainWriter; + this.outputs = outputs; + } + + @Override + public synchronized IndexOutput writeInfos(Directory dir, String segmentsFileName, + String codecID, SegmentInfos infos, IOContext context) + throws IOException { + Map outs = new HashMap(outputs.size() + 1); + outs.put(dir, mainWriter.writeInfos(dir, segmentsFileName, codecID, infos, context)); + Map writers = new HashMap(); + writers.put(dir, mainWriter); + for (Entry e : outputs.entrySet()) { + SegmentInfosWriter w = e.getValue().segmentInfosFormat().getSegmentInfosWriter(); + outs.put(e.getKey(), w.writeInfos(e.getKey(), segmentsFileName, e.getValue().getName(), infos, context)); + writers.put(e.getKey(), w); + } + return new TeeIndexOutput(mainWriter, outs, writers); + } + + @Override + public synchronized void prepareCommit(IndexOutput out) throws IOException { + assert out instanceof TeeIndexOutput; + Map outs = ((TeeIndexOutput)out).getOutputs(); + Map writers = ((TeeIndexOutput)out).getWriters(); + for (Entry e : writers.entrySet()) { + e.getValue().prepareCommit(outs.get(e.getKey())); + } + } + + @Override + public synchronized void finishCommit(IndexOutput out) throws IOException { + assert out instanceof TeeIndexOutput; + Map outs = ((TeeIndexOutput)out).getOutputs(); + Map writers = ((TeeIndexOutput)out).getWriters(); + for (Entry e : writers.entrySet()) { + e.getValue().finishCommit(outs.get(e.getKey())); + } + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsFormat.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsFormat.java new file mode 100644 index 0000000..c4ed0e8 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsFormat.java @@ -0,0 +1,68 @@ +package org.apache.lucene.codecs.tee.impl; +/** + * 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.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.StoredFieldsFormat; +import org.apache.lucene.codecs.StoredFieldsReader; +import org.apache.lucene.codecs.StoredFieldsWriter; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; + +public class TeeStoredFieldsFormat extends StoredFieldsFormat { + Codec main; + StoredFieldsFormat mainFormat; + Map outputs; + + public TeeStoredFieldsFormat(Codec main, Map outputs) { + this.main = main; + this.outputs = outputs; + mainFormat = main.storedFieldsFormat(); + } + + + @Override + public StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si, + FieldInfos fn, IOContext context) throws IOException { + return mainFormat.fieldsReader(directory, si, fn, context); + } + + @Override + public StoredFieldsWriter fieldsWriter(Directory directory, String segment, + IOContext context) throws IOException { + StoredFieldsWriter mainWriter = mainFormat.fieldsWriter(directory, segment, context); + HashMap writers = new HashMap(); + for (Entry e : outputs.entrySet()) { + writers.put(e.getKey(), e.getValue().storedFieldsFormat().fieldsWriter(e.getKey(), segment, context)); + } + return new TeeStoredFieldsWriter(mainWriter, writers); + } + + @Override + public void files(SegmentInfo info, Set files) throws IOException { + mainFormat.files(info, files); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsWriter.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsWriter.java new file mode 100644 index 0000000..d45c64a --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeStoredFieldsWriter.java @@ -0,0 +1,78 @@ +package org.apache.lucene.codecs.tee.impl; +/** + * 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.Map; + +import org.apache.lucene.codecs.StoredFieldsWriter; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.store.Directory; + +public class TeeStoredFieldsWriter extends StoredFieldsWriter { + StoredFieldsWriter mainWriter; + Map writers; + + public TeeStoredFieldsWriter(StoredFieldsWriter mainWriter, + Map writers) { + this.mainWriter = mainWriter; + this.writers = writers; + } + + @Override + public void close() throws IOException { + mainWriter.close(); + for (StoredFieldsWriter w : writers.values()) { + w.close(); + } + } + + @Override + public void startDocument(int numStoredFields) throws IOException { + mainWriter.startDocument(numStoredFields); + for (StoredFieldsWriter w : writers.values()) { + w.startDocument(numStoredFields); + } + } + + @Override + public void writeField(FieldInfo info, IndexableField field) + throws IOException { + mainWriter.writeField(info, field); + for (StoredFieldsWriter w : writers.values()) { + w.writeField(info, field); + } + } + + @Override + public void abort() { + mainWriter.abort(); + for (StoredFieldsWriter w : writers.values()) { + w.abort(); + } + } + + @Override + public void finish(int numDocs) throws IOException { + mainWriter.finish(numDocs); + for (StoredFieldsWriter w : writers.values()) { + w.finish(numDocs); + } + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsFormat.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsFormat.java new file mode 100644 index 0000000..a025107 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsFormat.java @@ -0,0 +1,68 @@ +package org.apache.lucene.codecs.tee.impl; +/** + * 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.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.TermVectorsFormat; +import org.apache.lucene.codecs.TermVectorsReader; +import org.apache.lucene.codecs.TermVectorsWriter; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; + +public class TeeTermVectorsFormat extends TermVectorsFormat { + Codec main; + TermVectorsFormat mainFormat; + Map outputs; + + public TeeTermVectorsFormat(Codec main, Map outputs) { + this.main = main; + this.outputs = outputs; + mainFormat = main.termVectorsFormat(); + } + + + @Override + public TermVectorsReader vectorsReader(Directory directory, + SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) + throws IOException { + return mainFormat.vectorsReader(directory, segmentInfo, fieldInfos, context); + } + + @Override + public synchronized TermVectorsWriter vectorsWriter(Directory directory, String segment, + IOContext context) throws IOException { + TermVectorsWriter mainWriter = mainFormat.vectorsWriter(directory, segment, context); + Map writers = new HashMap(); + for (Entry e : outputs.entrySet()) { + writers.put(e.getKey(), e.getValue().termVectorsFormat().vectorsWriter(e.getKey(), segment, context)); + } + return new TeeTermVectorsWriter(mainWriter, writers); + } + + @Override + public void files(SegmentInfo info, Set files) throws IOException { + mainFormat.files(info, files); + } +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsWriter.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsWriter.java new file mode 100644 index 0000000..e9b1f8d --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermVectorsWriter.java @@ -0,0 +1,101 @@ +package org.apache.lucene.codecs.tee.impl; +/** + * 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.Map; + +import org.apache.lucene.codecs.TermVectorsWriter; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; + +public class TeeTermVectorsWriter extends TermVectorsWriter { + TermVectorsWriter mainWriter; + Map writers; + + public TeeTermVectorsWriter(TermVectorsWriter mainWriter, + Map writers) { + this.mainWriter = mainWriter; + this.writers = writers; + } + + @Override + public void close() throws IOException { + mainWriter.close(); + for (TermVectorsWriter w : writers.values()) { + w.close(); + } + } + + @Override + public void startDocument(int numVectorFields) throws IOException { + mainWriter.startDocument(numVectorFields); + for (TermVectorsWriter w : writers.values()) { + w.startDocument(numVectorFields); + } + } + + @Override + public void startField(FieldInfo info, int numTerms, boolean positions, + boolean offsets) throws IOException { + mainWriter.startField(info, numTerms, positions, offsets); + for (TermVectorsWriter w : writers.values()) { + w.startField(info, numTerms, positions, offsets); + } + } + + @Override + public void startTerm(BytesRef term, int freq) throws IOException { + mainWriter.startTerm(term, freq); + for (TermVectorsWriter w : writers.values()) { + w.startTerm(term, freq); + } + } + + @Override + public void addPosition(int position, int startOffset, int endOffset) + throws IOException { + mainWriter.addPosition(position, startOffset, endOffset); + for (TermVectorsWriter w : writers.values()) { + w.addPosition(position, startOffset, endOffset); + } + } + + @Override + public void abort() { + mainWriter.abort(); + for (TermVectorsWriter w : writers.values()) { + w.abort(); + } + } + + @Override + public synchronized void finish(int numDocs) throws IOException { + mainWriter.finish(numDocs); + for (TermVectorsWriter w : writers.values()) { + w.finish(numDocs); + } + } + + @Override + public Comparator getComparator() throws IOException { + return mainWriter.getComparator(); + } + +} diff --git a/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermsConsumer.java b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermsConsumer.java new file mode 100644 index 0000000..6de3a77 --- /dev/null +++ b/lucene/contrib/misc/src/java/org/apache/lucene/codecs/tee/impl/TeeTermsConsumer.java @@ -0,0 +1,72 @@ +package org.apache.lucene.codecs.tee.impl; +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.lucene.codecs.PostingsConsumer; +import org.apache.lucene.codecs.TermStats; +import org.apache.lucene.codecs.TermsConsumer; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; + +public class TeeTermsConsumer extends TermsConsumer { + TermsConsumer main; + Map consumers; + + public TeeTermsConsumer(TermsConsumer main, + Map consumers) { + this.main = main; + this.consumers = consumers; + } + + @Override + public synchronized PostingsConsumer startTerm(BytesRef text) throws IOException { + PostingsConsumer mainPConsumer = main.startTerm(text); + HashMap pConsumers = new HashMap(); + for (Entry e : consumers.entrySet()) { + pConsumers.put(e.getKey(), e.getValue().startTerm(text)); + } + return new TeePostingsConsumer(mainPConsumer, pConsumers); + } + + @Override + public synchronized void finishTerm(BytesRef text, TermStats stats) throws IOException { + main.finishTerm(text, stats); + for (TermsConsumer tc : consumers.values()) { + tc.finishTerm(text, stats); + } + } + + @Override + public synchronized void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) + throws IOException { + main.finish(sumTotalTermFreq, sumDocFreq, docCount); + for (TermsConsumer tc : consumers.values()) { + tc.finish(sumTotalTermFreq, sumDocFreq, docCount); + } + } + + @Override + public Comparator getComparator() throws IOException { + return main.getComparator(); + } +} diff --git a/lucene/contrib/misc/src/test/org/apache/lucene/codecs/filtering/TestFilteringCodec.java b/lucene/contrib/misc/src/test/org/apache/lucene/codecs/filtering/TestFilteringCodec.java new file mode 100644 index 0000000..c2a15cc --- /dev/null +++ b/lucene/contrib/misc/src/test/org/apache/lucene/codecs/filtering/TestFilteringCodec.java @@ -0,0 +1,123 @@ +package org.apache.lucene.codecs.filtering; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; + +import org.apache.lucene.analysis.core.WhitespaceAnalyzer; +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.TermStats; +import org.apache.lucene.codecs.TermsConsumer; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.FieldType; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.Fields; +import org.apache.lucene.index.FieldsEnum; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.Terms; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.RAMDirectory; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.Version; + +public class TestFilteringCodec extends LuceneTestCase { + + BytesRef oneTerm = new BytesRef("one"); + + private class TestWriteFilter extends WriteFilter { + + @Override + public boolean addField(FieldInfo field) { + if (field.name.equals("w1")) { + return false; + } + return true; + } + + @Override + public boolean startTerm(FieldInfo info, BytesRef text) { + if (info.name.equals("w2") && text.equals(oneTerm)) { + return false; + } + return true; + } + + @Override + public void filterFinishTerm(TermsConsumer original, FieldInfo info, + BytesRef text, TermStats stats) throws IOException { + if (info.name.equals("w2") && text.equals(oneTerm)) { + return; + } + original.finishTerm(text, stats); + } + + @Override + public List filterStoredFields(List fields) { + Iterator it = fields.iterator(); + while (it.hasNext()) { + FilteredField ff = it.next(); + if (ff.info.name.equals("w1")) { + ff.field = null; + } + } + return fields; + } + } + + public void testBasicFiltering() throws Exception { + Codec codec = Codec.getDefault(); + boolean preFlex = codec.getName().equals("Lucene3x"); + assumeFalse("FilteringCodec doesn't work with Lucene3x codec", preFlex); + FilteringCodec filtering = new FilteringCodec(codec, new TestWriteFilter()); + IndexWriterConfig cfg = new IndexWriterConfig(Version.LUCENE_40, + new WhitespaceAnalyzer(Version.LUCENE_40)); + cfg.setCodec(filtering); + Directory d = new RAMDirectory(); + IndexWriter iw = new IndexWriter(d, cfg); + Document doc = new Document(); + FieldType ft = new FieldType(); + ft.setIndexed(true); + ft.setTokenized(true); + ft.setStored(true); + doc.add(new Field("w1", "throw away", ft)); + doc.add(new Field("w2", "one two three", ft)); + iw.addDocument(doc); + iw.close(); + // XXX this is ugly ... + Codec.setDefault(filtering); + DirectoryReader r = DirectoryReader.open(d); + assertEquals(1, r.numDocs()); + doc = r.document(0); + assertNull("should 'throw away'", doc.get("w1")); + assertEquals("one two three", doc.get("w2")); + Fields fields = MultiFields.getFields(r); + FieldsEnum fe = fields.iterator(); + String field; + TermsEnum te = null; + while ((field = fe.next()) != null) { + //System.out.println("field: " + field); + if (!preFlex) { + assertFalse("w1".equals(field)); // should be filtered out completely + } + Terms terms = fe.terms(); + te = terms.iterator(te); + BytesRef term; + int cnt = 0; + while ((term = te.next()) != null) { + cnt++; + if ("w2".equals(field)) { + assertFalse(term.equals(oneTerm)); + } + } + if (preFlex && "w1".equals(field)) { + assertEquals(0, cnt); // should be no terms + } + } + } +} diff --git a/lucene/contrib/misc/src/test/org/apache/lucene/codecs/tee/TestTeeCodec.java b/lucene/contrib/misc/src/test/org/apache/lucene/codecs/tee/TestTeeCodec.java new file mode 100644 index 0000000..1afd196 --- /dev/null +++ b/lucene/contrib/misc/src/test/org/apache/lucene/codecs/tee/TestTeeCodec.java @@ -0,0 +1,335 @@ +package org.apache.lucene.codecs.tee; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; + +import org.apache.lucene.analysis.core.WhitespaceAnalyzer; +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.FieldType; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.IndexableFieldType; +import org.apache.lucene.index.LogMergePolicy; +import org.apache.lucene.index.MergePolicy; +import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.SerialMergeScheduler; +import org.apache.lucene.index.Terms; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.TieredMergePolicy; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.RAMDirectory; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.Version; +import org.junit.Test; + +public class TestTeeCodec extends LuceneTestCase { + Map outputs = new HashMap(); + List dirs = new ArrayList(); + Directory mainDir; + Codec mainCodec; + TeeCodec tee; + IndexWriterConfig cfg; + + public void setUp() throws Exception { + super.setUp(); + outputs.clear(); + dirs.clear(); + mainCodec = Codec.getDefault(); + boolean preFlex = mainCodec.getName().equals("Lucene3x"); + assumeFalse("TeeCodec doesn't work with Lucene3x codec", preFlex); + mainDir = new RAMDirectory(); + for (int i = 0; i < 10; i++) { + Directory d = new RAMDirectory(); + dirs.add(d); + outputs.put(d, mainCodec); + } + tee = new TeeCodec(mainCodec, outputs); + cfg = new IndexWriterConfig(Version.LUCENE_40, + new WhitespaceAnalyzer(Version.LUCENE_40)); + cfg.setCodec(tee); + cfg.setMergeScheduler(new SerialMergeScheduler()); + MergePolicy mp = cfg.getMergePolicy(); + if (mp instanceof LogMergePolicy) { + ((LogMergePolicy)mp).setUseCompoundFile(false); + } else if (mp instanceof TieredMergePolicy) { + ((TieredMergePolicy)mp).setUseCompoundFile(false); + } else { + fail("Unsupported merge policy: " + mp.getClass().getName()); + } + } + + @Test + public void testCodec() throws Exception { + Directory wrappedDir = new TeeDirectory(mainDir, dirs, + new HashSet(Arrays.asList( + IndexFileNames.SEGMENTS_GEN, + "_nrm.cfs", "_nrm.cfe", // compound norms + ".len" // SimpleText norms + )), false); + IndexWriter w = new IndexWriter(wrappedDir, cfg); + Document doc = new Document(); + FieldType ft = new FieldType(); + ft.setStored(true); + ft.setIndexed(true); + ft.setTokenized(true); + ft.setStoreTermVectors(true); + ft.setStoreTermVectorOffsets(true); + ft.setStoreTermVectorPositions(true); + doc.add(new Field("f1", "one two three", ft)); + doc.add(new Field("f2", "two three four five", ft)); + w.addDocument(doc); + doc = new Document(); + doc.add(new Field("f1", "six seven eight", ft)); + doc.add(new Field("f2", "eight nine ten", ft)); + doc.add(new Field("f3", "eleven twelve", ft)); + w.addDocument(doc); + w.close(); + for (Directory d : dirs) { +// for (String s : d.listAll()) { +// System.out.println(s + "\t" + d.fileLength(s)); +// } + DirectoryReader r = DirectoryReader.open(d); + // check doc 0 + doc = r.document(0); + assertEquals("one two three", doc.get("f1")); + assertEquals("two three four five", doc.get("f2")); + // check doc 1 + doc = r.document(1); + assertEquals("six seven eight", doc.get("f1")); + assertEquals("eight nine ten", doc.get("f2")); + assertEquals("eleven twelve", doc.get("f3")); + // check terms + Terms terms = MultiFields.getTerms(r, "f1"); + assertEquals(6, terms.getUniqueTermCount()); + TermsEnum te = terms.iterator(null); + assertTrue(te.seekExact(new BytesRef("one"), true)); + assertTrue(te.seekExact(new BytesRef("two"), true)); + assertTrue(te.seekExact(new BytesRef("three"), true)); + assertTrue(te.seekExact(new BytesRef("six"), true)); + assertTrue(te.seekExact(new BytesRef("seven"), true)); + assertTrue(te.seekExact(new BytesRef("eight"), true)); + terms = MultiFields.getTerms(r, "f2"); + assertEquals(7, terms.getUniqueTermCount()); + te = terms.iterator(null); + assertTrue(te.seekExact(new BytesRef("two"), true)); + assertTrue(te.seekExact(new BytesRef("three"), true)); + assertTrue(te.seekExact(new BytesRef("four"), true)); + assertTrue(te.seekExact(new BytesRef("five"), true)); + assertTrue(te.seekExact(new BytesRef("eight"), true)); + assertTrue(te.seekExact(new BytesRef("nine"), true)); + assertTrue(te.seekExact(new BytesRef("ten"), true)); + terms = MultiFields.getTerms(r, "f3"); + assertEquals(2, terms.getUniqueTermCount()); + te = terms.iterator(null); + assertTrue(te.seekExact(new BytesRef("eleven"), true)); + assertTrue(te.seekExact(new BytesRef("twelve"), true)); + r.close(); + } + } + + @Test + public void testMerging() throws Exception { + int CNT = 1280; + cfg.setMaxBufferedDocs(10); + MergePolicy mp = cfg.getMergePolicy(); + if (mp instanceof LogMergePolicy) { + ((LogMergePolicy)mp).setMergeFactor(2); + } + Directory wrappedDir = new TeeDirectory(mainDir, dirs, + new HashSet(Arrays.asList( + IndexFileNames.SEGMENTS_GEN, + "_nrm.cfs", "_nrm.cfe", // compound norms + ".len" // SimpleText norms + )), false); + IndexWriter w = new IndexWriter(wrappedDir, cfg); + FieldType ft = new FieldType(); + ft.setStored(true); + ft.setIndexed(true); + ft.setTokenized(true); + ft.setStoreTermVectors(true); + ft.setStoreTermVectorOffsets(true); + ft.setStoreTermVectorPositions(true); + for (int i = 0; i < CNT; i++) { + Document doc = new Document(); + doc.add(new Field("f1", "one " + i, ft)); + w.addDocument(doc); + } + w.close(); + List readers = new ArrayList(dirs.size()); + for (Directory d : dirs) { + DirectoryReader r = DirectoryReader.open(d); + assertEquals("should be " + CNT + " docs", CNT, r.maxDoc()); + assertEquals("should be no deletes", 0, r.numDeletedDocs()); + readers.add(r); + // check terms + Terms terms = MultiFields.getTerms(r, "f1"); + TermsEnum te = terms.iterator(null); + assertTrue(te.seekExact(new BytesRef("one"), true)); + assertEquals(CNT, te.docFreq()); + for (int i = 0; i < CNT; i++) { + assertTrue(te.seekExact(new BytesRef("" + i), true)); + assertEquals(1, te.docFreq()); + } + } + for (int i = 0; i < CNT; i++) { + // check stored fields + Document doc = readers.get(0).document(i); + for (int j = 1; j < readers.size(); j++) { + Document newDoc = readers.get(j).document(i); + assertDocEquals(doc, newDoc); + } + } + } + + private void assertDocEquals(Document one, Document two) throws Exception { + assertNotNull("doc one", one); + assertNotNull("doc two", two); + List fieldsOne = one.getFields(); + List fieldsTwo = two.getFields(); + assertEquals("should have the same number of fields", fieldsOne.size(), fieldsTwo.size()); + for (IndexableField f1 : fieldsOne) { + IndexableField f2 = two.getField(f1.name()); + assertNotNull("should have field " + f1.name(), f2); + assertEquals("boosts differ", f1.boost(), f2.boost(), 0.00001); + IndexableFieldType t1 = f1.fieldType(); + IndexableFieldType t2 = f2.fieldType(); + assertTrue("field types differ", + t1.indexed() == t2.indexed() && + t1.omitNorms() == t2.omitNorms() && + t1.stored() == t2.stored() && + t1.storeTermVectorOffsets() == t2.storeTermVectorOffsets() && + t1.storeTermVectorPositions() == t2.storeTermVectorPositions() && + t1.storeTermVectors() == t2.storeTermVectors() && + t1.tokenized() == t2.tokenized() && + t1.docValueType() == t2.docValueType() && + t1.indexOptions() == t2.indexOptions()); + assertEquals("binary values differ", f1.binaryValue(), f2.binaryValue()); + assertEquals("string values differ", f1.stringValue(), f2.stringValue()); + assertEquals("numeric values differ", f1.numericValue(), f2.numericValue()); + } + } + + @Test + public void testTeeDirectory() throws Exception { + Directory wrappedDir = new TeeDirectory(mainDir, dirs, null, false); + cfg.setCodec(Codec.getDefault()); + IndexWriter w = new IndexWriter(wrappedDir, cfg); + FieldType ft = new FieldType(); + ft.setStored(true); + ft.setIndexed(true); + ft.setTokenized(true); + ft.setStoreTermVectors(true); + ft.setStoreTermVectorOffsets(true); + ft.setStoreTermVectorPositions(true); + int CNT = 100; + for (int i = 0; i < CNT; i++) { + Document doc = new Document(); + doc.add(new Field("f1", "one " + i, ft)); + w.addDocument(doc); + } + w.close(); + List readers = new ArrayList(dirs.size()); + for (Directory d : dirs) { + DirectoryReader r = DirectoryReader.open(d); + assertEquals("should be " + CNT + " docs", CNT, r.maxDoc()); + assertEquals("should be no deletes", 0, r.numDeletedDocs()); + readers.add(r); + // check terms + Terms terms = MultiFields.getTerms(r, "f1"); + TermsEnum te = terms.iterator(null); + assertTrue(te.seekExact(new BytesRef("one"), true)); + assertEquals(CNT, te.docFreq()); + for (int i = 0; i < CNT; i++) { + assertTrue(te.seekExact(new BytesRef("" + i), true)); + assertEquals(1, te.docFreq()); + } + } + for (int i = 0; i < CNT; i++) { + // check stored fields + Document doc = readers.get(0).document(i); + for (int j = 1; j < readers.size(); j++) { + Document newDoc = readers.get(j).document(i); + assertDocEquals(doc, newDoc); + } + } + } + + @Test + public void testInitialSync() throws Exception { + _testSync(false); + _testSync(true); + } + + private void cleanDir(Directory d) throws IOException { + String[] names = d.listAll(); + for (String name : names) { + d.deleteFile(name); + } + } + + private void _testSync(boolean initialSync) throws Exception { + // clean directories + cleanDir(mainDir); + for (Directory d : dirs) { + cleanDir(d); + } + // create initial files in the mainDir + for (int i = 0; i < 10; i++) { + IndexOutput out = mainDir.createOutput("initial" + i, IOContext.DEFAULT); + for (int j = 0; j < 10; j++) { + out.writeInt(j); + } + out.close(); + } + // now create a tee + Directory wrappedDir = new TeeDirectory(mainDir, dirs, null, initialSync); + // write some other junk + for (int i = 0; i < 10; i++) { + IndexOutput out = wrappedDir.createOutput("tee" + i, IOContext.DEFAULT); + for (int j = 0; j < 10; j++) { + out.writeInt(j); + } + out.close(); + } + for (Directory d : dirs) { + // make sure the initial files are absent/present depending on initialSync + for (int i = 0; i < 10; i++) { + if (initialSync) { + assertTrue("initial files should be mirrored", d.fileExists("initial" + i)); + // make sure they have the right content + IndexInput in = d.openInput("initial" + i, IOContext.READ); + for (int j = 0; j < 10; j++) { + int v = in.readInt(); + assertEquals("values read from file should be the same", v, j); + } + in.close(); + } else { + assertFalse("initial files should not be mirrored", d.fileExists("initial" + i)); + } + assertTrue("tee files should be there", d.fileExists("tee" + i)); + // make sure they have the right content + IndexInput in = d.openInput("tee" + i, IOContext.READ); + for (int j = 0; j < 10; j++) { + int v = in.readInt(); + assertEquals("values read from file should be the same", v, j); + } + in.close(); + } + } + } + +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PerDocConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/PerDocConsumer.java index dccd6a3..e477464 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/PerDocConsumer.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/PerDocConsumer.java @@ -71,7 +71,8 @@ public abstract class PerDocConsumer implements Closeable { * the default override this method accordingly. *

*/ - protected DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info) throws IOException { + //nocommit + public DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info) throws IOException { return reader.docValues(info.name); } @@ -83,7 +84,8 @@ public abstract class PerDocConsumer implements Closeable { * the default override this method accordingly. *

*/ - protected boolean canMerge(FieldInfo info) { + //nocommit + public boolean canMerge(FieldInfo info) { return info.hasDocValues(); } @@ -95,7 +97,8 @@ public abstract class PerDocConsumer implements Closeable { * the default override this method accordingly. *

*/ - protected Type getDocValuesType(FieldInfo info) { + //nocommit + public Type getDocValuesType(FieldInfo info) { return info.getDocValuesType(); } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsFormat.java index 8bf729e..f9c4630 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsFormat.java @@ -84,19 +84,22 @@ public class Lucene40NormsFormat extends NormsFormat { super(state, segmentSuffix); } + // nocommit @Override - protected DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info) + public DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info) throws IOException { return reader.normValues(info.name); } + // nocommit @Override - protected boolean canMerge(FieldInfo info) { + public boolean canMerge(FieldInfo info) { return info.normsPresent(); } + // nocommit @Override - protected Type getDocValuesType(FieldInfo info) { + public Type getDocValuesType(FieldInfo info) { return info.getNormType(); } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java index 086e770..8bf07d9 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java @@ -86,19 +86,22 @@ public class SimpleTextNormsConsumer extends PerDocConsumer { } } + // nocommit @Override - protected DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info) + public DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info) throws IOException { return reader.normValues(info.name); } + // nocommit @Override - protected boolean canMerge(FieldInfo info) { + public boolean canMerge(FieldInfo info) { return info.normsPresent(); } + // nocommit @Override - protected Type getDocValuesType(FieldInfo info) { + public Type getDocValuesType(FieldInfo info) { return info.getNormType(); } diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWNormsConsumer.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWNormsConsumer.java index 2bb5482..e9408d7 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWNormsConsumer.java +++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWNormsConsumer.java @@ -79,15 +79,20 @@ class PreFlexRWNormsConsumer extends PerDocConsumer { } } + // nocommit @Override - protected boolean canMerge(FieldInfo info) { + public boolean canMerge(FieldInfo info) { return info.normsPresent(); } + // nocommit @Override - protected Type getDocValuesType(FieldInfo info) { + public Type getDocValuesType(FieldInfo info) { return info.getNormType(); } + + //nocommit + // how is this actually working when it does not override getValuesForMerge?! @Override public DocValuesConsumer addValuesField(Type type, FieldInfo fieldInfo)